From 7f5d3f6ee480fd6ca15f962c66b3a2e1d0658360 Mon Sep 17 00:00:00 2001 From: Xinhao Xu <84456268+xxhZs@users.noreply.github.com> Date: Wed, 1 Nov 2023 11:53:44 +0800 Subject: [PATCH 01/17] feat(sink): Support big query sink (#12873) --- Cargo.lock | 54 +++ integration_tests/big-query-sink/README.md | 36 ++ .../append-only-sql/create_mv.sql | 7 + .../append-only-sql/create_sink.sql | 29 ++ .../append-only-sql/create_source.sql | 18 + .../big-query-sink/docker-compose.yml | 49 ++ src/connector/Cargo.toml | 2 + src/connector/src/sink/big_query.rs | 418 ++++++++++++++++++ src/connector/src/sink/encoder/json.rs | 20 +- src/connector/src/sink/encoder/mod.rs | 5 + src/connector/src/sink/mod.rs | 4 + 11 files changed, 639 insertions(+), 3 deletions(-) create mode 100644 integration_tests/big-query-sink/README.md create mode 100644 integration_tests/big-query-sink/append-only-sql/create_mv.sql create mode 100644 integration_tests/big-query-sink/append-only-sql/create_sink.sql create mode 100644 integration_tests/big-query-sink/append-only-sql/create_source.sql create mode 100644 integration_tests/big-query-sink/docker-compose.yml create mode 100644 src/connector/src/sink/big_query.rs diff --git a/Cargo.lock b/Cargo.lock index b811327e55a82..213760cbaa635 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3450,6 +3450,29 @@ dependencies = [ "byteorder", ] +[[package]] +name = "gcp-bigquery-client" +version = "0.17.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "576b349942a1b96327b1b2e50d271d3d54dc9669eb38271c830916168e753820" +dependencies = [ + "async-stream", + "async-trait", + "dyn-clone", + "hyper", + "hyper-rustls 0.24.1", + "log", + "reqwest", + "serde", + "serde_json", + "thiserror", + "time", + "tokio", + "tokio-stream", + "url", + "yup-oauth2", +] + [[package]] name = "generator" version = "0.7.5" @@ -3885,7 +3908,9 @@ dependencies = [ "futures-util", "http", "hyper", + "log", "rustls 0.21.7", + "rustls-native-certs", "tokio", "tokio-rustls 0.24.1", ] @@ -7459,6 +7484,7 @@ dependencies = [ "enum-as-inner", "futures", "futures-async-stream", + "gcp-bigquery-client", "glob", "google-cloud-pubsub", "http", @@ -7516,6 +7542,7 @@ dependencies = [ "url", "urlencoding", "workspace-hack", + "yup-oauth2", ] [[package]] @@ -11329,6 +11356,33 @@ version = "0.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "09041cd90cf85f7f8b2df60c646f853b7f535ce68f85244eb6731cf89fa498ec" +[[package]] +name = "yup-oauth2" +version = "8.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "364ca376b5c04d9b2be9693054e3e0d2d146b363819d0f9a10c6ee66e4c8406b" +dependencies = [ + "anyhow", + "async-trait", + "base64 0.13.1", + "futures", + "http", + "hyper", + "hyper-rustls 0.24.1", + "itertools 0.10.5", + "log", + "percent-encoding", + "rustls 0.21.7", + "rustls-pemfile", + "seahash", + "serde", + "serde_json", + "time", + "tokio", + "tower-service", + "url", +] + [[package]] name = "zerocopy" version = "0.6.4" diff --git a/integration_tests/big-query-sink/README.md b/integration_tests/big-query-sink/README.md new file mode 100644 index 0000000000000..42d4fdc793266 --- /dev/null +++ b/integration_tests/big-query-sink/README.md @@ -0,0 +1,36 @@ +# Demo: Sinking to Bigquery + +In this demo, we want to showcase how RisingWave is able to sink data to Bigquery. + +1. Launch the cluster: + +```sh +docker-compose up -d +``` + +The cluster contains a RisingWave cluster and its necessary dependencies, a datagen that generates the data. + +3. Create the Bigquery table in Bigquery + +```sql +CREATE table '${project_id}'.'${dataset_id}'.'${table_id}'( + user_id int, + target_id string, + event_timestamp datetime +); +``` + +4. Execute the SQL queries in sequence: + +- append-only/create_source.sql +- append-only/create_mv.sql +- append-only/create_sink.sql + + 1. We need to obtain the JSON file for Google Cloud service accounts, which can be configured here: https://console.cloud.google.com/iam-admin/serviceaccounts. + 2. Because BigQuery has limited support for updates and deletes, we currently only support 'append only' + 3. Regarding file path, we can choose between S3 and local files, and the specific SQL statement is in the 'create_sink.sql'. + +Run the following query +```sql +select user_id, count(*) from demo.demo_bhv_table group by user_id; +``` diff --git a/integration_tests/big-query-sink/append-only-sql/create_mv.sql b/integration_tests/big-query-sink/append-only-sql/create_mv.sql new file mode 100644 index 0000000000000..0a803f8a2762d --- /dev/null +++ b/integration_tests/big-query-sink/append-only-sql/create_mv.sql @@ -0,0 +1,7 @@ +CREATE MATERIALIZED VIEW bhv_mv AS +SELECT + user_id, + target_id, + event_timestamp +FROM + user_behaviors; \ No newline at end of file diff --git a/integration_tests/big-query-sink/append-only-sql/create_sink.sql b/integration_tests/big-query-sink/append-only-sql/create_sink.sql new file mode 100644 index 0000000000000..c5dd9d9d48725 --- /dev/null +++ b/integration_tests/big-query-sink/append-only-sql/create_sink.sql @@ -0,0 +1,29 @@ +-- create sink with local file +CREATE SINK bhv_big_query_sink +FROM + bhv_mv WITH ( + connector = 'bigquery', + type = 'append-only', + bigquery.local.path= '${bigquery_service_account_json_path}', + bigquery.project= '${project_id}', + bigquery.dataset= '${dataset_id}', + bigquery.table= '${table_id}', + force_append_only='true' +); + + +-- create sink with s3 file +CREATE SINK bhv_big_query_sink +FROM + bhv_mv WITH ( + connector = 'bigquery', + type = 'append-only', + bigquery.s3.path= '${s3_service_account_json_path}', + bigquery.project= '${project_id}', + bigquery.dataset= '${dataset_id}', + bigquery.table= '${table_id}', + access_key = '${aws_access_key}', + secret_access = '${aws_secret_access}', + region = '${aws_region}', + force_append_only='true', +); \ No newline at end of file diff --git a/integration_tests/big-query-sink/append-only-sql/create_source.sql b/integration_tests/big-query-sink/append-only-sql/create_source.sql new file mode 100644 index 0000000000000..c28c10f3616da --- /dev/null +++ b/integration_tests/big-query-sink/append-only-sql/create_source.sql @@ -0,0 +1,18 @@ +CREATE table user_behaviors ( + user_id int, + target_id VARCHAR, + target_type VARCHAR, + event_timestamp TIMESTAMP, + behavior_type VARCHAR, + parent_target_type VARCHAR, + parent_target_id VARCHAR, + PRIMARY KEY(user_id) +) WITH ( + connector = 'datagen', + fields.user_id.kind = 'sequence', + fields.user_id.start = '1', + fields.user_id.end = '1000', + fields.user_name.kind = 'random', + fields.user_name.length = '10', + datagen.rows.per.second = '10' +) FORMAT PLAIN ENCODE JSON; \ No newline at end of file diff --git a/integration_tests/big-query-sink/docker-compose.yml b/integration_tests/big-query-sink/docker-compose.yml new file mode 100644 index 0000000000000..e002b72065bf1 --- /dev/null +++ b/integration_tests/big-query-sink/docker-compose.yml @@ -0,0 +1,49 @@ +--- +version: "3" +services: + compactor-0: + extends: + file: ../../docker/docker-compose.yml + service: compactor-0 + compute-node-0: + extends: + file: ../../docker/docker-compose.yml + service: compute-node-0 + etcd-0: + extends: + file: ../../docker/docker-compose.yml + service: etcd-0 + frontend-node-0: + extends: + file: ../../docker/docker-compose.yml + service: frontend-node-0 + grafana-0: + extends: + file: ../../docker/docker-compose.yml + service: grafana-0 + meta-node-0: + extends: + file: ../../docker/docker-compose.yml + service: meta-node-0 + minio-0: + extends: + file: ../../docker/docker-compose.yml + service: minio-0 + prometheus-0: + extends: + file: ../../docker/docker-compose.yml + service: prometheus-0 +volumes: + compute-node-0: + external: false + etcd-0: + external: false + grafana-0: + external: false + minio-0: + external: false + prometheus-0: + external: false + message_queue: + external: false +name: risingwave-compose \ No newline at end of file diff --git a/src/connector/Cargo.toml b/src/connector/Cargo.toml index abd9617493e77..883303ae72e63 100644 --- a/src/connector/Cargo.toml +++ b/src/connector/Cargo.toml @@ -49,6 +49,7 @@ easy-ext = "1" enum-as-inner = "0.6" futures = { version = "0.3", default-features = false, features = ["alloc"] } futures-async-stream = { workspace = true } +gcp-bigquery-client = "0.17.1" glob = "0.3" google-cloud-pubsub = "0.20" http = "0.2" @@ -131,6 +132,7 @@ tracing = "0.1" tracing-futures = { version = "0.2", features = ["futures-03"] } url = "2" urlencoding = "2" +yup-oauth2 = "8.3" [target.'cfg(not(madsim))'.dependencies] workspace-hack = { path = "../workspace-hack" } diff --git a/src/connector/src/sink/big_query.rs b/src/connector/src/sink/big_query.rs new file mode 100644 index 0000000000000..4c540b2954233 --- /dev/null +++ b/src/connector/src/sink/big_query.rs @@ -0,0 +1,418 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use core::mem; +use std::collections::HashMap; +use std::sync::Arc; + +use anyhow::anyhow; +use async_trait::async_trait; +use gcp_bigquery_client::model::query_request::QueryRequest; +use gcp_bigquery_client::model::table_data_insert_all_request::TableDataInsertAllRequest; +use gcp_bigquery_client::model::table_data_insert_all_request_rows::TableDataInsertAllRequestRows; +use gcp_bigquery_client::Client; +use risingwave_common::array::{Op, StreamChunk}; +use risingwave_common::buffer::Bitmap; +use risingwave_common::catalog::Schema; +use risingwave_common::types::DataType; +use serde_derive::{Deserialize, Serialize}; +use serde_json::Value; +use serde_with::serde_as; +use url::Url; +use yup_oauth2::ServiceAccountKey; + +use super::encoder::{JsonEncoder, RowEncoder, TimestampHandlingMode}; +use super::writer::LogSinkerOf; +use super::{SinkError, SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SINK_TYPE_UPSERT}; +use crate::aws_auth::AwsAuthProps; +use crate::aws_utils::load_file_descriptor_from_s3; +use crate::sink::writer::SinkWriterExt; +use crate::sink::{ + DummySinkCommitCoordinator, Result, Sink, SinkParam, SinkWriter, SinkWriterParam, +}; + +pub const BIGQUERY_SINK: &str = "bigquery"; +const BIGQUERY_INSERT_MAX_NUMS: usize = 1024; + +#[derive(Deserialize, Serialize, Debug, Clone)] +pub struct BigQueryCommon { + #[serde(rename = "bigquery.local.path")] + pub local_path: Option, + #[serde(rename = "bigquery.s3.path")] + pub s3_path: Option, + #[serde(rename = "bigquery.project")] + pub project: String, + #[serde(rename = "bigquery.dataset")] + pub dataset: String, + #[serde(rename = "bigquery.table")] + pub table: String, + #[serde(flatten)] + /// required keys refer to [`crate::aws_utils::AWS_DEFAULT_CONFIG`] + pub s3_credentials: HashMap, +} + +impl BigQueryCommon { + pub(crate) async fn build_client(&self) -> Result { + let service_account = if let Some(local_path) = &self.local_path { + let auth_json = std::fs::read_to_string(local_path) + .map_err(|err| SinkError::BigQuery(anyhow::anyhow!(err)))?; + serde_json::from_str::(&auth_json) + .map_err(|err| SinkError::BigQuery(anyhow::anyhow!(err)))? + } else if let Some(s3_path) = &self.s3_path { + let url = + Url::parse(s3_path).map_err(|err| SinkError::BigQuery(anyhow::anyhow!(err)))?; + let auth_json = load_file_descriptor_from_s3( + &url, + &AwsAuthProps::from_pairs( + self.s3_credentials + .iter() + .map(|(k, v)| (k.as_str(), v.as_str())), + ), + ) + .await + .map_err(|err| SinkError::BigQuery(anyhow::anyhow!(err)))?; + serde_json::from_slice::(&auth_json) + .map_err(|err| SinkError::BigQuery(anyhow::anyhow!(err)))? + } else { + return Err(SinkError::BigQuery(anyhow::anyhow!("`bigquery.local.path` and `bigquery.s3.path` set at least one, configure as needed."))); + }; + let client: Client = Client::from_service_account_key(service_account, false) + .await + .map_err(|err| SinkError::BigQuery(anyhow::anyhow!(err)))?; + Ok(client) + } +} + +#[serde_as] +#[derive(Clone, Debug, Deserialize)] +pub struct BigQueryConfig { + #[serde(flatten)] + pub common: BigQueryCommon, + + pub r#type: String, // accept "append-only" or "upsert" +} +impl BigQueryConfig { + pub fn from_hashmap(properties: HashMap) -> Result { + let config = + serde_json::from_value::(serde_json::to_value(properties).unwrap()) + .map_err(|e| SinkError::Config(anyhow!(e)))?; + if config.r#type != SINK_TYPE_APPEND_ONLY && config.r#type != SINK_TYPE_UPSERT { + return Err(SinkError::Config(anyhow!( + "`{}` must be {}, or {}", + SINK_TYPE_OPTION, + SINK_TYPE_APPEND_ONLY, + SINK_TYPE_UPSERT + ))); + } + Ok(config) + } +} + +#[derive(Debug)] +pub struct BigQuerySink { + pub config: BigQueryConfig, + schema: Schema, + pk_indices: Vec, + is_append_only: bool, +} + +impl BigQuerySink { + pub fn new( + config: BigQueryConfig, + schema: Schema, + pk_indices: Vec, + is_append_only: bool, + ) -> Result { + Ok(Self { + config, + schema, + pk_indices, + is_append_only, + }) + } +} + +impl BigQuerySink { + fn check_column_name_and_type( + &self, + big_query_columns_desc: HashMap, + ) -> Result<()> { + let rw_fields_name = self.schema.fields(); + if big_query_columns_desc.is_empty() { + return Err(SinkError::BigQuery(anyhow::anyhow!( + "Cannot find table in bigquery" + ))); + } + if rw_fields_name.len().ne(&big_query_columns_desc.len()) { + return Err(SinkError::BigQuery(anyhow::anyhow!("The length of the RisingWave column {} must be equal to the length of the bigquery column {}",rw_fields_name.len(),big_query_columns_desc.len()))); + } + + for i in rw_fields_name { + let value = big_query_columns_desc.get(&i.name).ok_or_else(|| { + SinkError::BigQuery(anyhow::anyhow!( + "Column name don't find in bigquery, risingwave is {:?} ", + i.name + )) + })?; + let data_type_string = Self::get_string_and_check_support_from_datatype(&i.data_type)?; + if data_type_string.ne(value) { + return Err(SinkError::BigQuery(anyhow::anyhow!( + "Column type don't match, column name is {:?}. bigquery type is {:?} risingwave type is {:?} ",i.name,value,data_type_string + ))); + }; + } + Ok(()) + } + + fn get_string_and_check_support_from_datatype(rw_data_type: &DataType) -> Result { + match rw_data_type { + DataType::Boolean => Ok("BOOL".to_owned()), + DataType::Int16 => Ok("INT64".to_owned()), + DataType::Int32 => Ok("INT64".to_owned()), + DataType::Int64 => Ok("INT64".to_owned()), + DataType::Float32 => Err(SinkError::BigQuery(anyhow::anyhow!( + "Bigquery cannot support real" + ))), + DataType::Float64 => Ok("FLOAT64".to_owned()), + DataType::Decimal => Ok("NUMERIC".to_owned()), + DataType::Date => Ok("DATE".to_owned()), + DataType::Varchar => Ok("STRING".to_owned()), + DataType::Time => Err(SinkError::BigQuery(anyhow::anyhow!( + "Bigquery cannot support Time" + ))), + DataType::Timestamp => Ok("DATETIME".to_owned()), + DataType::Timestamptz => Ok("TIMESTAMP".to_owned()), + DataType::Interval => Ok("INTERVAL".to_owned()), + DataType::Struct(structs) => { + let mut elements_vec = vec![]; + for (name, datatype) in structs.iter() { + let element_string = + Self::get_string_and_check_support_from_datatype(datatype)?; + elements_vec.push(format!("{} {}", name, element_string)); + } + Ok(format!("STRUCT<{}>", elements_vec.join(", "))) + } + DataType::List(l) => { + let element_string = Self::get_string_and_check_support_from_datatype(l.as_ref())?; + Ok(format!("ARRAY<{}>", element_string)) + } + DataType::Bytea => Ok("BYTES".to_owned()), + DataType::Jsonb => Ok("JSON".to_owned()), + DataType::Serial => Ok("INT64".to_owned()), + DataType::Int256 => Err(SinkError::BigQuery(anyhow::anyhow!( + "Bigquery cannot support Int256" + ))), + } + } +} + +impl Sink for BigQuerySink { + type Coordinator = DummySinkCommitCoordinator; + type LogSinker = LogSinkerOf; + + const SINK_NAME: &'static str = BIGQUERY_SINK; + + async fn new_log_sinker(&self, writer_param: SinkWriterParam) -> Result { + Ok(BigQuerySinkWriter::new( + self.config.clone(), + self.schema.clone(), + self.pk_indices.clone(), + self.is_append_only, + ) + .await? + .into_log_sinker(writer_param.sink_metrics)) + } + + async fn validate(&self) -> Result<()> { + if !self.is_append_only { + return Err(SinkError::Config(anyhow!( + "BigQuery sink don't support upsert" + ))); + } + + let client = self.config.common.build_client().await?; + let mut rs = client + .job() + .query( + &self.config.common.project, + QueryRequest::new(format!( + "SELECT column_name, data_type FROM `{}.{}.INFORMATION_SCHEMA.COLUMNS` WHERE table_name = '{}'" + ,self.config.common.project,self.config.common.dataset,self.config.common.table, + )), + ) + .await.map_err(|e| SinkError::BigQuery(e.into()))?; + let mut big_query_schema = HashMap::default(); + while rs.next_row() { + big_query_schema.insert( + rs.get_string_by_name("column_name") + .map_err(|e| SinkError::BigQuery(e.into()))? + .ok_or_else(|| { + SinkError::BigQuery(anyhow::anyhow!("Cannot find column_name")) + })?, + rs.get_string_by_name("data_type") + .map_err(|e| SinkError::BigQuery(e.into()))? + .ok_or_else(|| { + SinkError::BigQuery(anyhow::anyhow!("Cannot find column_name")) + })?, + ); + } + + self.check_column_name_and_type(big_query_schema)?; + Ok(()) + } +} + +pub struct BigQuerySinkWriter { + pub config: BigQueryConfig, + schema: Schema, + pk_indices: Vec, + client: Client, + is_append_only: bool, + insert_request: TableDataInsertAllRequest, + row_encoder: JsonEncoder, +} + +impl TryFrom for BigQuerySink { + type Error = SinkError; + + fn try_from(param: SinkParam) -> std::result::Result { + let schema = param.schema(); + let config = BigQueryConfig::from_hashmap(param.properties)?; + BigQuerySink::new( + config, + schema, + param.downstream_pk, + param.sink_type.is_append_only(), + ) + } +} + +impl BigQuerySinkWriter { + pub async fn new( + config: BigQueryConfig, + schema: Schema, + pk_indices: Vec, + is_append_only: bool, + ) -> Result { + let client = config.common.build_client().await?; + Ok(Self { + config, + schema: schema.clone(), + pk_indices, + client, + is_append_only, + insert_request: TableDataInsertAllRequest::new(), + row_encoder: JsonEncoder::new_with_big_query( + schema, + None, + TimestampHandlingMode::String, + ), + }) + } + + async fn append_only(&mut self, chunk: StreamChunk) -> Result<()> { + let mut insert_vec = Vec::with_capacity(chunk.capacity()); + for (op, row) in chunk.rows() { + if op != Op::Insert { + return Err(SinkError::BigQuery(anyhow::anyhow!( + "BigQuery sink don't support upsert" + ))); + } + insert_vec.push(TableDataInsertAllRequestRows { + insert_id: None, + json: Value::Object(self.row_encoder.encode(row)?), + }) + } + self.insert_request + .add_rows(insert_vec) + .map_err(|e| SinkError::BigQuery(e.into()))?; + if self.insert_request.len().ge(&BIGQUERY_INSERT_MAX_NUMS) { + self.insert_data().await?; + } + Ok(()) + } + + async fn insert_data(&mut self) -> Result<()> { + if !self.insert_request.is_empty() { + let insert_request = + mem::replace(&mut self.insert_request, TableDataInsertAllRequest::new()); + self.client + .tabledata() + .insert_all( + &self.config.common.project, + &self.config.common.dataset, + &self.config.common.table, + insert_request, + ) + .await + .map_err(|e| SinkError::BigQuery(e.into()))?; + } + Ok(()) + } +} + +#[async_trait] +impl SinkWriter for BigQuerySinkWriter { + async fn write_batch(&mut self, chunk: StreamChunk) -> Result<()> { + if self.is_append_only { + self.append_only(chunk).await + } else { + Err(SinkError::BigQuery(anyhow::anyhow!( + "BigQuery sink don't support upsert" + ))) + } + } + + async fn begin_epoch(&mut self, _epoch: u64) -> Result<()> { + Ok(()) + } + + async fn abort(&mut self) -> Result<()> { + Ok(()) + } + + async fn barrier(&mut self, _is_checkpoint: bool) -> Result<()> { + self.insert_data().await + } + + async fn update_vnode_bitmap(&mut self, _vnode_bitmap: Arc) -> Result<()> { + Ok(()) + } +} + +#[cfg(test)] +mod test { + use risingwave_common::types::{DataType, StructType}; + + use crate::sink::big_query::BigQuerySink; + + #[tokio::test] + async fn test_type_check() { + let big_query_type_string = "ARRAY, v2 STRUCT>>"; + let rw_datatype = DataType::List(Box::new(DataType::Struct(StructType::new(vec![ + ("v1".to_owned(), DataType::List(Box::new(DataType::Int64))), + ( + "v2".to_owned(), + DataType::Struct(StructType::new(vec![ + ("v1".to_owned(), DataType::Int64), + ("v2".to_owned(), DataType::Int64), + ])), + ), + ])))); + assert_eq!( + BigQuerySink::get_string_and_check_support_from_datatype(&rw_datatype).unwrap(), + big_query_type_string + ); + } +} diff --git a/src/connector/src/sink/encoder/json.rs b/src/connector/src/sink/encoder/json.rs index 12176b491d6ab..c8bfc489c70c9 100644 --- a/src/connector/src/sink/encoder/json.rs +++ b/src/connector/src/sink/encoder/json.rs @@ -77,6 +77,20 @@ impl JsonEncoder { ..self } } + + pub fn new_with_big_query( + schema: Schema, + col_indices: Option>, + timestamp_handling_mode: TimestampHandlingMode, + ) -> Self { + Self { + schema, + col_indices, + timestamp_handling_mode, + custom_json_type: CustomJsonType::Bigquery, + kafka_connect: None, + } + } } impl RowEncoder for JsonEncoder { @@ -187,7 +201,7 @@ fn datum_to_json_object( } json!(v_string) } - CustomJsonType::None => { + CustomJsonType::None | CustomJsonType::Bigquery => { json!(v.to_text()) } }, @@ -204,7 +218,7 @@ fn datum_to_json_object( } (DataType::Date, ScalarRefImpl::Date(v)) => match custom_json_type { CustomJsonType::None => json!(v.0.num_days_from_ce()), - CustomJsonType::Doris(_) => { + CustomJsonType::Bigquery | CustomJsonType::Doris(_) => { let a = v.0.format("%Y-%m-%d").to_string(); json!(a) } @@ -259,7 +273,7 @@ fn datum_to_json_object( ArrayError::internal(format!("Json to string err{:?}", err)) })?) } - CustomJsonType::None => { + CustomJsonType::None | CustomJsonType::Bigquery => { let mut map = Map::with_capacity(st.len()); for (sub_datum_ref, sub_field) in struct_ref.iter_fields_ref().zip_eq_debug( st.iter() diff --git a/src/connector/src/sink/encoder/mod.rs b/src/connector/src/sink/encoder/mod.rs index b55fd534d5eb3..a58824b6fa92e 100644 --- a/src/connector/src/sink/encoder/mod.rs +++ b/src/connector/src/sink/encoder/mod.rs @@ -91,7 +91,12 @@ pub enum TimestampHandlingMode { #[derive(Clone)] pub enum CustomJsonType { + // Doris's json need date is string. + // The internal order of the struct should follow the insertion order. + // The decimal needs verification and calibration. Doris(HashMap), + // Bigquery's json need date is string. + Bigquery, None, } diff --git a/src/connector/src/sink/mod.rs b/src/connector/src/sink/mod.rs index 6afd08778cd96..fc590d2fa6935 100644 --- a/src/connector/src/sink/mod.rs +++ b/src/connector/src/sink/mod.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +pub mod big_query; pub mod blackhole; pub mod boxed; pub mod catalog; @@ -78,6 +79,7 @@ macro_rules! for_all_sinks { { ElasticSearch, $crate::sink::remote::ElasticSearchSink }, { Cassandra, $crate::sink::remote::CassandraSink }, { Doris, $crate::sink::doris::DorisSink }, + { BigQuery, $crate::sink::big_query::BigQuerySink }, { Test, $crate::sink::test_sink::TestSink } } $(,$arg)* @@ -393,6 +395,8 @@ pub enum SinkError { Pulsar(anyhow::Error), #[error("Internal error: {0}")] Internal(anyhow::Error), + #[error("BigQuery error: {0}")] + BigQuery(anyhow::Error), } impl From for SinkError { From 7122d6cf9768642fea3cd15f96ed3982d05502f9 Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Wed, 1 Nov 2023 12:31:02 +0800 Subject: [PATCH 02/17] fix(test): use rate limit instead in background ddl test (#13179) --- .../executor/backfill/no_shuffle_backfill.rs | 16 ---------- src/stream/src/executor/flow_control.rs | 8 +++-- src/tests/simulation/src/background_ddl.toml | 4 +-- src/tests/simulation/src/cluster.rs | 6 ++-- .../recovery/background_ddl.rs | 31 +++++++++++-------- 5 files changed, 28 insertions(+), 37 deletions(-) diff --git a/src/stream/src/executor/backfill/no_shuffle_backfill.rs b/src/stream/src/executor/backfill/no_shuffle_backfill.rs index 45a0d81b968ab..97a9da0ff6a99 100644 --- a/src/stream/src/executor/backfill/no_shuffle_backfill.rs +++ b/src/stream/src/executor/backfill/no_shuffle_backfill.rs @@ -146,14 +146,6 @@ where let pk_order = self.upstream_table.pk_serializer().get_order_types(); - #[cfg(madsim)] - let snapshot_read_delay = if let Ok(v) = std::env::var("RW_BACKFILL_SNAPSHOT_READ_DELAY") - && let Ok(v) = v.parse::() { - v - } else { - 0 - }; - let upstream_table_id = self.upstream_table.table_id().table_id; let mut upstream = self.upstream.execute(); @@ -303,14 +295,6 @@ where break 'backfill_loop; } Some(chunk) => { - #[cfg(madsim)] - { - tokio::time::sleep(std::time::Duration::from_millis( - snapshot_read_delay as u64, - )) - .await; - } - // Raise the current position. // As snapshot read streams are ordered by pk, so we can // just use the last row to update `current_pos`. diff --git a/src/stream/src/executor/flow_control.rs b/src/stream/src/executor/flow_control.rs index 4da40a95623bf..c48fdc8a4392f 100644 --- a/src/stream/src/executor/flow_control.rs +++ b/src/stream/src/executor/flow_control.rs @@ -54,10 +54,12 @@ impl FlowControlExecutor { let msg = msg?; match msg { Message::Chunk(chunk) => { + let Some(n) = NonZeroU32::new(chunk.cardinality() as u32) else { + // Handle case where chunk is empty + continue; + }; if let Some(rate_limiter) = &rate_limiter { - let result = rate_limiter - .until_n_ready(NonZeroU32::new(chunk.cardinality() as u32).unwrap()) - .await; + let result = rate_limiter.until_n_ready(n).await; if let Err(InsufficientCapacity(n)) = result { tracing::error!( "Rate Limit {:?} smaller than chunk cardinality {n}", diff --git a/src/tests/simulation/src/background_ddl.toml b/src/tests/simulation/src/background_ddl.toml index f0bf41f804739..5bd34329156ca 100644 --- a/src/tests/simulation/src/background_ddl.toml +++ b/src/tests/simulation/src/background_ddl.toml @@ -2,8 +2,8 @@ telemetry_enabled = false metrics_level = "Disabled" -#[streaming.developer] -#stream_chunk_size = 1 +[streaming.developer] +stream_chunk_size = 1 [system] barrier_interval_ms = 1000 diff --git a/src/tests/simulation/src/cluster.rs b/src/tests/simulation/src/cluster.rs index a9ba7a657bf67..202e2f63ed2da 100644 --- a/src/tests/simulation/src/cluster.rs +++ b/src/tests/simulation/src/cluster.rs @@ -151,10 +151,10 @@ impl Configuration { Configuration { config_path: ConfigPath::Temp(config_path.into()), - frontend_nodes: 1, - compute_nodes: 1, + frontend_nodes: 2, + compute_nodes: 3, meta_nodes: 3, - compactor_nodes: 1, + compactor_nodes: 2, compute_node_cores: 2, etcd_timeout_rate: 0.0, etcd_data_path: None, diff --git a/src/tests/simulation/tests/integration_tests/recovery/background_ddl.rs b/src/tests/simulation/tests/integration_tests/recovery/background_ddl.rs index 67b447aba9df8..b4849c877f8ec 100644 --- a/src/tests/simulation/tests/integration_tests/recovery/background_ddl.rs +++ b/src/tests/simulation/tests/integration_tests/recovery/background_ddl.rs @@ -21,9 +21,13 @@ use risingwave_simulation::utils::AssertResult; use tokio::time::sleep; const CREATE_TABLE: &str = "CREATE TABLE t(v1 int);"; -const SEED_TABLE: &str = "INSERT INTO t SELECT generate_series FROM generate_series(1, 100000);"; +const SEED_TABLE: &str = "INSERT INTO t SELECT generate_series FROM generate_series(1, 500);"; const SET_BACKGROUND_DDL: &str = "SET BACKGROUND_DDL=true;"; +const SET_RATE_LIMIT_2: &str = "SET STREAMING_RATE_LIMIT=2;"; +const SET_RATE_LIMIT_1: &str = "SET STREAMING_RATE_LIMIT=1;"; +const RESET_RATE_LIMIT: &str = "SET STREAMING_RATE_LIMIT=0;"; const CREATE_MV1: &str = "CREATE MATERIALIZED VIEW mv1 as SELECT * FROM t;"; +const WAIT: &str = "WAIT;"; async fn kill_cn_and_wait_recover(cluster: &Cluster) { // Kill it again @@ -69,6 +73,13 @@ async fn cancel_stream_jobs(session: &mut Session) -> Result> { Ok(ids) } +fn init_logger() { + let _ = tracing_subscriber::fmt() + .with_env_filter(tracing_subscriber::EnvFilter::from_default_env()) + .with_ansi(false) + .try_init(); +} + #[tokio::test] async fn test_background_mv_barrier_recovery() -> Result<()> { let mut cluster = Cluster::start(Configuration::for_scale()).await?; @@ -133,22 +144,17 @@ async fn test_background_mv_barrier_recovery() -> Result<()> { #[tokio::test] async fn test_background_ddl_cancel() -> Result<()> { - env::set_var("RW_BACKFILL_SNAPSHOT_READ_DELAY", "100"); async fn create_mv(session: &mut Session) -> Result<()> { session.run(CREATE_MV1).await?; sleep(Duration::from_secs(2)).await; Ok(()) } - // FIXME: See if we can use rate limit instead. - use std::env; - tracing_subscriber::fmt() - .with_env_filter(tracing_subscriber::EnvFilter::from_default_env()) - .with_ansi(false) - .init(); - let mut cluster = Cluster::start(Configuration::for_scale()).await?; + init_logger(); + let mut cluster = Cluster::start(Configuration::for_background_ddl()).await?; let mut session = cluster.start_session(); session.run(CREATE_TABLE).await?; session.run(SEED_TABLE).await?; + session.run(SET_RATE_LIMIT_2).await?; session.run(SET_BACKGROUND_DDL).await?; for _ in 0..5 { @@ -157,6 +163,7 @@ async fn test_background_ddl_cancel() -> Result<()> { assert_eq!(ids.len(), 1); } + session.run(SET_RATE_LIMIT_1).await?; create_mv(&mut session).await?; // Test cancel after kill cn @@ -176,16 +183,14 @@ async fn test_background_ddl_cancel() -> Result<()> { assert_eq!(ids.len(), 1); // Make sure MV can be created after all these cancels + session.run(RESET_RATE_LIMIT).await?; create_mv(&mut session).await?; - kill_and_wait_recover(&cluster).await; - // Wait for job to finish - session.run("WAIT;").await?; + session.run(WAIT).await?; session.run("DROP MATERIALIZED VIEW mv1").await?; session.run("DROP TABLE t").await?; - env::remove_var("RW_BACKFILL_SNAPSHOT_READ_DELAY"); Ok(()) } From b7195f85aee70c318f908268c2e53d899a8045af Mon Sep 17 00:00:00 2001 From: Richard Chien Date: Wed, 1 Nov 2023 13:57:17 +0800 Subject: [PATCH 03/17] refactor(test): reorganize over window e2e tests (#13186) Signed-off-by: Richard Chien --- e2e_test/batch/over_window/generated | 1 + e2e_test/batch/over_window/main.slt.part | 6 +- e2e_test/batch/over_window/over_window | 1 - .../over_window/special_cases/mod.slt.part | 1 - e2e_test/over_window/gen.py | 20 +++-- .../batch/agg_in_win_func/mod.slt.part | 80 +++++++++++++++++ .../batch/{ => basic}/cross_check.slt.part | 0 .../generated/batch/{ => basic}/mod.slt.part | 86 +------------------ .../{create.slt.part => basic/setup.slt.part} | 30 ------- .../teardown.slt.part} | 9 -- .../batch/expr_in_win_func/mod.slt.part | 71 +++++++++++++++ .../over_window/generated/batch/main.slt.part | 10 +++ .../opt_agg_then_join.slt.part/mod.slt.part} | 42 ++++++++- .../generated/batch/rank_func/mod.slt.part | 86 +++++++++++++++++++ .../streaming/agg_in_win_func/mod.slt.part | 80 +++++++++++++++++ .../{ => basic}/cross_check.slt.part | 0 .../streaming/{ => basic}/mod.slt.part | 86 +------------------ .../{create.slt.part => basic/setup.slt.part} | 30 ------- .../teardown.slt.part} | 9 -- .../streaming/expr_in_win_func/mod.slt.part | 71 +++++++++++++++ .../generated/streaming/main.slt.part | 10 +++ .../opt_agg_then_join.slt.part/mod.slt.part | 74 ++++++++++++++++ .../streaming/rank_func/mod.slt.part | 86 +++++++++++++++++++ .../templates/agg_in_win_func/mod.slt.part | 78 +++++++++++++++++ .../{ => basic}/cross_check.slt.part | 0 .../templates/{ => basic}/mod.slt.part | 86 +------------------ .../{create.slt.part => basic/setup.slt.part} | 30 ------- .../teardown.slt.part} | 9 -- .../templates/expr_in_win_func/mod.slt.part | 69 +++++++++++++++ e2e_test/over_window/templates/main.slt.part | 8 ++ .../opt_agg_then_join.slt.part/mod.slt.part} | 10 ++- .../templates/rank_func/mod.slt.part | 84 ++++++++++++++++++ e2e_test/streaming/over_window/generated | 1 + e2e_test/streaming/over_window/main.slt | 6 +- e2e_test/streaming/over_window/over_window | 1 - .../over_window/special_cases/mod.slt.part | 1 - 36 files changed, 882 insertions(+), 390 deletions(-) create mode 120000 e2e_test/batch/over_window/generated delete mode 120000 e2e_test/batch/over_window/over_window delete mode 100644 e2e_test/batch/over_window/special_cases/mod.slt.part create mode 100644 e2e_test/over_window/generated/batch/agg_in_win_func/mod.slt.part rename e2e_test/over_window/generated/batch/{ => basic}/cross_check.slt.part (100%) rename e2e_test/over_window/generated/batch/{ => basic}/mod.slt.part (68%) rename e2e_test/over_window/generated/batch/{create.slt.part => basic/setup.slt.part} (79%) rename e2e_test/over_window/generated/batch/{drop.slt.part => basic/teardown.slt.part} (77%) create mode 100644 e2e_test/over_window/generated/batch/expr_in_win_func/mod.slt.part create mode 100644 e2e_test/over_window/generated/batch/main.slt.part rename e2e_test/{batch/over_window/special_cases/to_agg_then_join.slt.part => over_window/generated/batch/opt_agg_then_join.slt.part/mod.slt.part} (54%) create mode 100644 e2e_test/over_window/generated/batch/rank_func/mod.slt.part create mode 100644 e2e_test/over_window/generated/streaming/agg_in_win_func/mod.slt.part rename e2e_test/over_window/generated/streaming/{ => basic}/cross_check.slt.part (100%) rename e2e_test/over_window/generated/streaming/{ => basic}/mod.slt.part (68%) rename e2e_test/over_window/generated/streaming/{create.slt.part => basic/setup.slt.part} (78%) rename e2e_test/over_window/generated/streaming/{drop.slt.part => basic/teardown.slt.part} (75%) create mode 100644 e2e_test/over_window/generated/streaming/expr_in_win_func/mod.slt.part create mode 100644 e2e_test/over_window/generated/streaming/main.slt.part create mode 100644 e2e_test/over_window/generated/streaming/opt_agg_then_join.slt.part/mod.slt.part create mode 100644 e2e_test/over_window/generated/streaming/rank_func/mod.slt.part create mode 100644 e2e_test/over_window/templates/agg_in_win_func/mod.slt.part rename e2e_test/over_window/templates/{ => basic}/cross_check.slt.part (100%) rename e2e_test/over_window/templates/{ => basic}/mod.slt.part (68%) rename e2e_test/over_window/templates/{create.slt.part => basic/setup.slt.part} (78%) rename e2e_test/over_window/templates/{drop.slt.part => basic/teardown.slt.part} (72%) create mode 100644 e2e_test/over_window/templates/expr_in_win_func/mod.slt.part create mode 100644 e2e_test/over_window/templates/main.slt.part rename e2e_test/{streaming/over_window/special_cases/to_agg_then_join.slt.part => over_window/templates/opt_agg_then_join.slt.part/mod.slt.part} (86%) create mode 100644 e2e_test/over_window/templates/rank_func/mod.slt.part create mode 120000 e2e_test/streaming/over_window/generated delete mode 120000 e2e_test/streaming/over_window/over_window delete mode 100644 e2e_test/streaming/over_window/special_cases/mod.slt.part diff --git a/e2e_test/batch/over_window/generated b/e2e_test/batch/over_window/generated new file mode 120000 index 0000000000000..32dfe679fac64 --- /dev/null +++ b/e2e_test/batch/over_window/generated @@ -0,0 +1 @@ +../../over_window/generated/batch/ \ No newline at end of file diff --git a/e2e_test/batch/over_window/main.slt.part b/e2e_test/batch/over_window/main.slt.part index 6e277352059c4..dca370c0306ee 100644 --- a/e2e_test/batch/over_window/main.slt.part +++ b/e2e_test/batch/over_window/main.slt.part @@ -1,5 +1 @@ -statement ok -SET RW_IMPLICIT_FLUSH TO true; - -include ./special_cases/mod.slt.part -include ./over_window/mod.slt.part +include ./generated/main.slt.part diff --git a/e2e_test/batch/over_window/over_window b/e2e_test/batch/over_window/over_window deleted file mode 120000 index f3208bd4500bc..0000000000000 --- a/e2e_test/batch/over_window/over_window +++ /dev/null @@ -1 +0,0 @@ -../../over_window/generated/batch \ No newline at end of file diff --git a/e2e_test/batch/over_window/special_cases/mod.slt.part b/e2e_test/batch/over_window/special_cases/mod.slt.part deleted file mode 100644 index 265ed4966619f..0000000000000 --- a/e2e_test/batch/over_window/special_cases/mod.slt.part +++ /dev/null @@ -1 +0,0 @@ -include ./to_agg_then_join.slt.part diff --git a/e2e_test/over_window/gen.py b/e2e_test/over_window/gen.py index bd30d998d83fc..af4bec4790c5b 100755 --- a/e2e_test/over_window/gen.py +++ b/e2e_test/over_window/gen.py @@ -25,19 +25,27 @@ shutil.rmtree(mode_dir) os.makedirs(mode_dir, exist_ok=True) -for file in os.listdir(templates_dir): - if not file.endswith(".slt") and not file.endswith(".slt.part"): - continue - print(f"Generating `{file}`...") +def render(filepath: str): + relpath = path.relpath(filepath, templates_dir) + print(f"Rendering `{relpath}`...") - with open(path.join(templates_dir, file), "r") as f: + with open(path.join(templates_dir, relpath), "r") as f: tpl = Template(f.read()) for mode, context in contexts.items(): - out_file = path.join(generated_dir, mode, file) + out_file = path.join(generated_dir, mode, relpath) + os.makedirs(path.dirname(out_file), exist_ok=True) with open(out_file, "w") as f: f.write(file_head + "\n\n") f.write(tpl.safe_substitute(context)) + +for dirpath, dirnames, filenames in os.walk(templates_dir): + for filename in filenames: + if not filename.endswith(".slt") and not filename.endswith(".slt.part"): + continue + render(path.join(dirpath, filename)) + + print("Done.") diff --git a/e2e_test/over_window/generated/batch/agg_in_win_func/mod.slt.part b/e2e_test/over_window/generated/batch/agg_in_win_func/mod.slt.part new file mode 100644 index 0000000000000..ff53636e393ac --- /dev/null +++ b/e2e_test/over_window/generated/batch/agg_in_win_func/mod.slt.part @@ -0,0 +1,80 @@ +# This file is generated by `gen.py`. Do not edit it manually! + +# Test aggregate function calls as window function args/PARTITION BY/ORDER BY. + +statement ok +create table t ( + id int + , p1 int + , p2 int + , time int + , v1 int + , v2 int +); + +statement ok +create view v as +select + p1, p2 + , row_number() over (partition by p1 order by p2) as out1 + , sum(sum(v2)) over (partition by p1, avg(time) order by max(v1), p2) as out2 +from t +group by p1, p2; + +statement ok +insert into t values + (100001, 100, 200, 1, 701, 805) +, (100002, 100, 200, 2, 700, 806) +, (100003, 100, 208, 2, 723, 807) +, (100004, 103, 200, 2, 702, 808); + +query iiii +select * from v order by p1, p2; +---- +100 200 1 1611 +100 208 2 807 +103 200 1 808 + +statement ok +insert into t values + (100005, 100, 200, 3, 717, 810) +, (100006, 105, 204, 5, 703, 828); + +query iiii +select * from v order by p1, p2; +---- +100 200 1 2421 +100 208 2 3228 +103 200 1 808 +105 204 1 828 + +statement ok +update t set v1 = 799 where id = 100002; -- value change + +statement ok +update t set p2 = 200 where id = 100003; -- partition change + +statement ok +update t set "time" = 1 where id = 100005; -- order change + +query iiiiiii +select * from v order by p1, p2; +---- +100 200 1 3228 +103 200 1 808 +105 204 1 828 + +statement ok +delete from t where time = 2; + +query iiii +select * from v order by p1, p2; +---- +100 200 1 1615 +105 204 1 828 + +statement ok +drop view v; + +statement ok +drop table t; diff --git a/e2e_test/over_window/generated/batch/cross_check.slt.part b/e2e_test/over_window/generated/batch/basic/cross_check.slt.part similarity index 100% rename from e2e_test/over_window/generated/batch/cross_check.slt.part rename to e2e_test/over_window/generated/batch/basic/cross_check.slt.part diff --git a/e2e_test/over_window/generated/batch/mod.slt.part b/e2e_test/over_window/generated/batch/basic/mod.slt.part similarity index 68% rename from e2e_test/over_window/generated/batch/mod.slt.part rename to e2e_test/over_window/generated/batch/basic/mod.slt.part index 2c7778fd46aff..a8c74b16bf790 100644 --- a/e2e_test/over_window/generated/batch/mod.slt.part +++ b/e2e_test/over_window/generated/batch/basic/mod.slt.part @@ -1,6 +1,8 @@ # This file is generated by `gen.py`. Do not edit it manually! -include ./create.slt.part +# Test basic functionality of general batch and streaming over window. + +include ./setup.slt.part statement ok insert into t values @@ -33,21 +35,6 @@ select * from v_c order by id; 100003 100 208 2 723 807 723 NULL NULL NULL NULL 100004 103 200 2 702 808 702 NULL NULL NULL NULL -query II -select * from v_d order by id; ----- -100001 100 200 1 701 805 1 2 -100002 100 200 2 700 806 2 3 -100003 100 208 2 723 807 3 1 -100004 103 200 2 702 808 1 1 - -query iiii -select * from v_e order by p1; ----- -100 200 1 1611 -100 208 2 807 -103 200 1 808 - include ./cross_check.slt.part statement ok @@ -85,24 +72,6 @@ select * from v_c order by id; 100005 100 200 3 717 810 717 700 700 NULL NULL 100006 105 204 5 703 828 703 NULL NULL NULL NULL -query II -select * from v_d order by id; ----- -100001 100 200 1 701 805 1 2 -100002 100 200 2 700 806 2 3 -100003 100 208 2 723 807 3 1 -100004 103 200 2 702 808 1 1 -100005 100 200 3 717 810 4 4 -100006 105 204 5 703 828 1 1 - -query iiii -select * from v_e order by p1, p2; ----- -100 200 1 2421 -100 208 2 3228 -103 200 1 808 -105 204 1 828 - include ./cross_check.slt.part statement ok @@ -144,33 +113,6 @@ select * from v_c order by id; 100005 100 200 1 717 810 717 723 701 806 806 100006 105 204 5 703 828 703 NULL NULL NULL NULL -query iiiiiii -select * from v_d order by id; ----- -100001 100 200 1 701 805 1 1 -100002 100 200 2 799 806 3 2 -100003 100 200 2 723 807 4 3 -100004 103 200 2 702 808 1 1 -100005 100 200 1 717 810 2 4 -100006 105 204 5 703 828 1 1 - -query iiiiiii -select * from v_e order by p1; ----- -100 200 1 3228 -103 200 1 808 -105 204 1 828 - -query iiiiiiiiii -select * from v_expr order by id; ----- -100001 100 200 1 701 805 805 0 701 NULL 1402 -100002 100 200 2 799 806 806 0 701 703 1446 -100003 100 200 2 723 807 807 0 701 801 1446 -100004 103 200 2 702 808 808 0 702 NULL 1404 -100005 100 200 1 717 810 810 0 701 725 1434 -100006 105 204 5 703 828 828 0 703 NULL 1406 - include ./cross_check.slt.part statement ok @@ -197,26 +139,6 @@ select * from v_c order by id; 100005 100 200 1 717 810 717 701 701 NULL NULL 100006 105 204 5 703 828 703 NULL NULL NULL NULL -query iiiiiii -select * from v_d order by id; ----- -100001 100 200 1 701 805 1 1 -100005 100 200 1 717 810 2 2 -100006 105 204 5 703 828 1 1 - -query iiii -select * from v_e order by p1; ----- -100 200 1 1615 -105 204 1 828 - -query iiiiiiiiii -select * from v_expr order by id; ----- -100001 100 200 1 701 805 805 0 701 NULL 1402 -100005 100 200 1 717 810 810 0 701 703 1434 -100006 105 204 5 703 828 828 0 703 NULL 1406 - include ./cross_check.slt.part -include ./drop.slt.part +include ./teardown.slt.part diff --git a/e2e_test/over_window/generated/batch/create.slt.part b/e2e_test/over_window/generated/batch/basic/setup.slt.part similarity index 79% rename from e2e_test/over_window/generated/batch/create.slt.part rename to e2e_test/over_window/generated/batch/basic/setup.slt.part index 5f4b5e1152804..2ffc1b055334d 100644 --- a/e2e_test/over_window/generated/batch/create.slt.part +++ b/e2e_test/over_window/generated/batch/basic/setup.slt.part @@ -40,25 +40,6 @@ select , lead(v2, 2) over (partition by p1, p2 order by v1, v2) as out9 from t; -# row_number -statement ok -create view v_d as -select - * - , row_number() over (partition by p1 order by time, id) as out10 - , row_number() over (partition by p1 order by p2 desc, id) as out11 -from t; - -# over + agg -statement ok -create view v_e as -select - p1, p2 - , row_number() over (partition by p1 order by p2) as out12 - , sum(sum(v2)) over (partition by p1, avg(time) order by max(v1), p2) as out13 -from t -group by p1, p2; - statement ok create view v_a_b as select @@ -103,14 +84,3 @@ select , lead(v2, 1) over (partition by p1, p2 order by time, id) as out8 , lead(v2, 2) over (partition by p1, p2 order by v1, v2) as out9 from t; - -statement ok -create view v_expr as -select - * - , t.v2 as out1 - , 0 as out2 - , first_value(v1) over (partition by p1, p2 order by time, id rows 3 preceding) as out3 - , lag(v1 + 2, 0 + 1) over (partition by p1 - 1 order by id) as out4 - , min(v1 * 2) over (partition by p1, p2 order by time + 1, id rows between current row and unbounded following) as out5 -from t; diff --git a/e2e_test/over_window/generated/batch/drop.slt.part b/e2e_test/over_window/generated/batch/basic/teardown.slt.part similarity index 77% rename from e2e_test/over_window/generated/batch/drop.slt.part rename to e2e_test/over_window/generated/batch/basic/teardown.slt.part index 435ffd46433e7..97f416dd49c9e 100644 --- a/e2e_test/over_window/generated/batch/drop.slt.part +++ b/e2e_test/over_window/generated/batch/basic/teardown.slt.part @@ -9,12 +9,6 @@ drop view v_b; statement ok drop view v_c; -statement ok -drop view v_d; - -statement ok -drop view v_e; - statement ok drop view v_a_b; @@ -27,8 +21,5 @@ drop view v_a_c; statement ok drop view v_a_b_c; -statement ok -drop view v_expr; - statement ok drop table t; diff --git a/e2e_test/over_window/generated/batch/expr_in_win_func/mod.slt.part b/e2e_test/over_window/generated/batch/expr_in_win_func/mod.slt.part new file mode 100644 index 0000000000000..653881de2c920 --- /dev/null +++ b/e2e_test/over_window/generated/batch/expr_in_win_func/mod.slt.part @@ -0,0 +1,71 @@ +# This file is generated by `gen.py`. Do not edit it manually! + +# Test expressions as window function args/PARTITION BY/ORDER BY. + +statement ok +create table t ( + id int + , p1 int + , p2 int + , time int + , v1 int + , v2 int +); + +statement ok +create view v as +select + * + , t.v2 as out1 + , 0 as out2 + , first_value(v1) over (partition by p1, p2 order by time, id rows 3 preceding) as out3 + , lag(v1 + 2, 0 + 1) over (partition by p1 - 1 order by id) as out4 + , min(v1 * 2) over (partition by p1, p2 order by time + 1, id rows between current row and unbounded following) as out5 +from t; + +statement ok +insert into t values + (100001, 100, 200, 1, 701, 805) +, (100002, 100, 200, 2, 700, 806) +, (100003, 100, 208, 2, 723, 807) +, (100004, 103, 200, 2, 702, 808); + +statement ok +insert into t values + (100005, 100, 200, 3, 717, 810) +, (100006, 105, 204, 5, 703, 828); + +statement ok +update t set v1 = 799 where id = 100002; -- value change + +statement ok +update t set p2 = 200 where id = 100003; -- partition change + +statement ok +update t set "time" = 1 where id = 100005; -- order change + +query iiiiiiiiii +select * from v order by id; +---- +100001 100 200 1 701 805 805 0 701 NULL 1402 +100002 100 200 2 799 806 806 0 701 703 1446 +100003 100 200 2 723 807 807 0 701 801 1446 +100004 103 200 2 702 808 808 0 702 NULL 1404 +100005 100 200 1 717 810 810 0 701 725 1434 +100006 105 204 5 703 828 828 0 703 NULL 1406 + +statement ok +delete from t where time = 2; + +query iiiiiiiiii +select * from v order by id; +---- +100001 100 200 1 701 805 805 0 701 NULL 1402 +100005 100 200 1 717 810 810 0 701 703 1434 +100006 105 204 5 703 828 828 0 703 NULL 1406 + +statement ok +drop view v; + +statement ok +drop table t; diff --git a/e2e_test/over_window/generated/batch/main.slt.part b/e2e_test/over_window/generated/batch/main.slt.part new file mode 100644 index 0000000000000..9f0ad1baeffe3 --- /dev/null +++ b/e2e_test/over_window/generated/batch/main.slt.part @@ -0,0 +1,10 @@ +# This file is generated by `gen.py`. Do not edit it manually! + +statement ok +SET RW_IMPLICIT_FLUSH TO true; + +include ./basic/mod.slt.part +include ./rank_func/mod.slt.part +include ./expr_in_win_func/mod.slt.part +include ./agg_in_win_func/mod.slt.part +include ./opt_agg_then_join/mod.slt.part diff --git a/e2e_test/batch/over_window/special_cases/to_agg_then_join.slt.part b/e2e_test/over_window/generated/batch/opt_agg_then_join.slt.part/mod.slt.part similarity index 54% rename from e2e_test/batch/over_window/special_cases/to_agg_then_join.slt.part rename to e2e_test/over_window/generated/batch/opt_agg_then_join.slt.part/mod.slt.part index e2545f9d69ed3..023946c6d4327 100644 --- a/e2e_test/batch/over_window/special_cases/to_agg_then_join.slt.part +++ b/e2e_test/over_window/generated/batch/opt_agg_then_join.slt.part/mod.slt.part @@ -1,9 +1,14 @@ -statement ok -SET RW_IMPLICIT_FLUSH TO true; +# This file is generated by `gen.py`. Do not edit it manually! + +# Test cases that should be optimized to Agg + Join. statement ok create table t(x int, y int); +statement ok +create view v as +select *, sum(y / x) OVER (PARTITION BY x) as a, count(x) OVER (PARTITION BY y) as b from t; + statement ok insert into t values (1, 1), (1, 2), (1, 3), (1, 4), (1, 5), @@ -13,7 +18,7 @@ insert into t values (1, 3), (2, 3), (3, 3); query IIII rowsort -select *, sum(y / x) OVER (PARTITION BY x) as a, count(x) OVER (PARTITION BY y) as b from t; +select * from v; ---- 1 1 18 3 1 2 18 3 @@ -34,5 +39,36 @@ select *, sum(y / x) OVER (PARTITION BY x) as a, count(x) OVER (PARTITION BY y) 3 4 4 3 3 5 4 3 +statement ok +insert into t values (1, 6), (2, 8), (3, 12); + +query IIII rowsort +select * from v; +---- +1 1 24 3 +1 2 24 3 +1 3 24 6 +1 3 24 6 +1 4 24 3 +1 5 24 3 +1 6 24 1 +2 1 11 3 +2 2 11 3 +2 3 11 6 +2 3 11 6 +2 4 11 3 +2 5 11 3 +2 8 11 1 +3 1 8 3 +3 12 8 1 +3 2 8 3 +3 3 8 6 +3 3 8 6 +3 4 8 3 +3 5 8 3 + +statement ok +drop view v; + statement ok drop table t; diff --git a/e2e_test/over_window/generated/batch/rank_func/mod.slt.part b/e2e_test/over_window/generated/batch/rank_func/mod.slt.part new file mode 100644 index 0000000000000..58e434bd11362 --- /dev/null +++ b/e2e_test/over_window/generated/batch/rank_func/mod.slt.part @@ -0,0 +1,86 @@ +# This file is generated by `gen.py`. Do not edit it manually! + +# Test rank window functions including `row_number`, `rank`, `dense_rank`. + +statement ok +create table t ( + id int + , p1 int + , p2 int + , time int + , v1 int + , v2 int +); + +statement ok +create view v as +select + * + , row_number() over (partition by p1 order by time, id) as out1 + , row_number() over (partition by p1 order by p2 desc, id) as out2 +from t; + +statement ok +insert into t values + (100001, 100, 200, 1, 701, 805) +, (100002, 100, 200, 2, 700, 806) +, (100003, 100, 208, 2, 723, 807) +, (100004, 103, 200, 2, 702, 808); + +query II +select * from v order by id; +---- +100001 100 200 1 701 805 1 2 +100002 100 200 2 700 806 2 3 +100003 100 208 2 723 807 3 1 +100004 103 200 2 702 808 1 1 + +statement ok +insert into t values + (100005, 100, 200, 3, 717, 810) +, (100006, 105, 204, 5, 703, 828); + +query II +select * from v order by id; +---- +100001 100 200 1 701 805 1 2 +100002 100 200 2 700 806 2 3 +100003 100 208 2 723 807 3 1 +100004 103 200 2 702 808 1 1 +100005 100 200 3 717 810 4 4 +100006 105 204 5 703 828 1 1 + +statement ok +update t set v1 = 799 where id = 100002; -- value change + +statement ok +update t set p2 = 200 where id = 100003; -- partition change + +statement ok +update t set "time" = 1 where id = 100005; -- order change + +query iiiiiii +select * from v order by id; +---- +100001 100 200 1 701 805 1 1 +100002 100 200 2 799 806 3 2 +100003 100 200 2 723 807 4 3 +100004 103 200 2 702 808 1 1 +100005 100 200 1 717 810 2 4 +100006 105 204 5 703 828 1 1 + +statement ok +delete from t where time = 2; + +query iiiiiii +select * from v order by id; +---- +100001 100 200 1 701 805 1 1 +100005 100 200 1 717 810 2 2 +100006 105 204 5 703 828 1 1 + +statement ok +drop view v; + +statement ok +drop table t; diff --git a/e2e_test/over_window/generated/streaming/agg_in_win_func/mod.slt.part b/e2e_test/over_window/generated/streaming/agg_in_win_func/mod.slt.part new file mode 100644 index 0000000000000..1cf5cb3951ef0 --- /dev/null +++ b/e2e_test/over_window/generated/streaming/agg_in_win_func/mod.slt.part @@ -0,0 +1,80 @@ +# This file is generated by `gen.py`. Do not edit it manually! + +# Test aggregate function calls as window function args/PARTITION BY/ORDER BY. + +statement ok +create table t ( + id int + , p1 int + , p2 int + , time int + , v1 int + , v2 int +); + +statement ok +create materialized view v as +select + p1, p2 + , row_number() over (partition by p1 order by p2) as out1 + , sum(sum(v2)) over (partition by p1, avg(time) order by max(v1), p2) as out2 +from t +group by p1, p2; + +statement ok +insert into t values + (100001, 100, 200, 1, 701, 805) +, (100002, 100, 200, 2, 700, 806) +, (100003, 100, 208, 2, 723, 807) +, (100004, 103, 200, 2, 702, 808); + +query iiii +select * from v order by p1, p2; +---- +100 200 1 1611 +100 208 2 807 +103 200 1 808 + +statement ok +insert into t values + (100005, 100, 200, 3, 717, 810) +, (100006, 105, 204, 5, 703, 828); + +query iiii +select * from v order by p1, p2; +---- +100 200 1 2421 +100 208 2 3228 +103 200 1 808 +105 204 1 828 + +statement ok +update t set v1 = 799 where id = 100002; -- value change + +statement ok +update t set p2 = 200 where id = 100003; -- partition change + +statement ok +update t set "time" = 1 where id = 100005; -- order change + +query iiiiiii +select * from v order by p1, p2; +---- +100 200 1 3228 +103 200 1 808 +105 204 1 828 + +statement ok +delete from t where time = 2; + +query iiii +select * from v order by p1, p2; +---- +100 200 1 1615 +105 204 1 828 + +statement ok +drop materialized view v; + +statement ok +drop table t; diff --git a/e2e_test/over_window/generated/streaming/cross_check.slt.part b/e2e_test/over_window/generated/streaming/basic/cross_check.slt.part similarity index 100% rename from e2e_test/over_window/generated/streaming/cross_check.slt.part rename to e2e_test/over_window/generated/streaming/basic/cross_check.slt.part diff --git a/e2e_test/over_window/generated/streaming/mod.slt.part b/e2e_test/over_window/generated/streaming/basic/mod.slt.part similarity index 68% rename from e2e_test/over_window/generated/streaming/mod.slt.part rename to e2e_test/over_window/generated/streaming/basic/mod.slt.part index 2c7778fd46aff..a8c74b16bf790 100644 --- a/e2e_test/over_window/generated/streaming/mod.slt.part +++ b/e2e_test/over_window/generated/streaming/basic/mod.slt.part @@ -1,6 +1,8 @@ # This file is generated by `gen.py`. Do not edit it manually! -include ./create.slt.part +# Test basic functionality of general batch and streaming over window. + +include ./setup.slt.part statement ok insert into t values @@ -33,21 +35,6 @@ select * from v_c order by id; 100003 100 208 2 723 807 723 NULL NULL NULL NULL 100004 103 200 2 702 808 702 NULL NULL NULL NULL -query II -select * from v_d order by id; ----- -100001 100 200 1 701 805 1 2 -100002 100 200 2 700 806 2 3 -100003 100 208 2 723 807 3 1 -100004 103 200 2 702 808 1 1 - -query iiii -select * from v_e order by p1; ----- -100 200 1 1611 -100 208 2 807 -103 200 1 808 - include ./cross_check.slt.part statement ok @@ -85,24 +72,6 @@ select * from v_c order by id; 100005 100 200 3 717 810 717 700 700 NULL NULL 100006 105 204 5 703 828 703 NULL NULL NULL NULL -query II -select * from v_d order by id; ----- -100001 100 200 1 701 805 1 2 -100002 100 200 2 700 806 2 3 -100003 100 208 2 723 807 3 1 -100004 103 200 2 702 808 1 1 -100005 100 200 3 717 810 4 4 -100006 105 204 5 703 828 1 1 - -query iiii -select * from v_e order by p1, p2; ----- -100 200 1 2421 -100 208 2 3228 -103 200 1 808 -105 204 1 828 - include ./cross_check.slt.part statement ok @@ -144,33 +113,6 @@ select * from v_c order by id; 100005 100 200 1 717 810 717 723 701 806 806 100006 105 204 5 703 828 703 NULL NULL NULL NULL -query iiiiiii -select * from v_d order by id; ----- -100001 100 200 1 701 805 1 1 -100002 100 200 2 799 806 3 2 -100003 100 200 2 723 807 4 3 -100004 103 200 2 702 808 1 1 -100005 100 200 1 717 810 2 4 -100006 105 204 5 703 828 1 1 - -query iiiiiii -select * from v_e order by p1; ----- -100 200 1 3228 -103 200 1 808 -105 204 1 828 - -query iiiiiiiiii -select * from v_expr order by id; ----- -100001 100 200 1 701 805 805 0 701 NULL 1402 -100002 100 200 2 799 806 806 0 701 703 1446 -100003 100 200 2 723 807 807 0 701 801 1446 -100004 103 200 2 702 808 808 0 702 NULL 1404 -100005 100 200 1 717 810 810 0 701 725 1434 -100006 105 204 5 703 828 828 0 703 NULL 1406 - include ./cross_check.slt.part statement ok @@ -197,26 +139,6 @@ select * from v_c order by id; 100005 100 200 1 717 810 717 701 701 NULL NULL 100006 105 204 5 703 828 703 NULL NULL NULL NULL -query iiiiiii -select * from v_d order by id; ----- -100001 100 200 1 701 805 1 1 -100005 100 200 1 717 810 2 2 -100006 105 204 5 703 828 1 1 - -query iiii -select * from v_e order by p1; ----- -100 200 1 1615 -105 204 1 828 - -query iiiiiiiiii -select * from v_expr order by id; ----- -100001 100 200 1 701 805 805 0 701 NULL 1402 -100005 100 200 1 717 810 810 0 701 703 1434 -100006 105 204 5 703 828 828 0 703 NULL 1406 - include ./cross_check.slt.part -include ./drop.slt.part +include ./teardown.slt.part diff --git a/e2e_test/over_window/generated/streaming/create.slt.part b/e2e_test/over_window/generated/streaming/basic/setup.slt.part similarity index 78% rename from e2e_test/over_window/generated/streaming/create.slt.part rename to e2e_test/over_window/generated/streaming/basic/setup.slt.part index 4334fb1cdd30e..cc46c4066f0f4 100644 --- a/e2e_test/over_window/generated/streaming/create.slt.part +++ b/e2e_test/over_window/generated/streaming/basic/setup.slt.part @@ -40,25 +40,6 @@ select , lead(v2, 2) over (partition by p1, p2 order by v1, v2) as out9 from t; -# row_number -statement ok -create materialized view v_d as -select - * - , row_number() over (partition by p1 order by time, id) as out10 - , row_number() over (partition by p1 order by p2 desc, id) as out11 -from t; - -# over + agg -statement ok -create materialized view v_e as -select - p1, p2 - , row_number() over (partition by p1 order by p2) as out12 - , sum(sum(v2)) over (partition by p1, avg(time) order by max(v1), p2) as out13 -from t -group by p1, p2; - statement ok create materialized view v_a_b as select @@ -103,14 +84,3 @@ select , lead(v2, 1) over (partition by p1, p2 order by time, id) as out8 , lead(v2, 2) over (partition by p1, p2 order by v1, v2) as out9 from t; - -statement ok -create materialized view v_expr as -select - * - , t.v2 as out1 - , 0 as out2 - , first_value(v1) over (partition by p1, p2 order by time, id rows 3 preceding) as out3 - , lag(v1 + 2, 0 + 1) over (partition by p1 - 1 order by id) as out4 - , min(v1 * 2) over (partition by p1, p2 order by time + 1, id rows between current row and unbounded following) as out5 -from t; diff --git a/e2e_test/over_window/generated/streaming/drop.slt.part b/e2e_test/over_window/generated/streaming/basic/teardown.slt.part similarity index 75% rename from e2e_test/over_window/generated/streaming/drop.slt.part rename to e2e_test/over_window/generated/streaming/basic/teardown.slt.part index e6c4fcfaad244..2089fefcac249 100644 --- a/e2e_test/over_window/generated/streaming/drop.slt.part +++ b/e2e_test/over_window/generated/streaming/basic/teardown.slt.part @@ -9,12 +9,6 @@ drop materialized view v_b; statement ok drop materialized view v_c; -statement ok -drop materialized view v_d; - -statement ok -drop materialized view v_e; - statement ok drop materialized view v_a_b; @@ -27,8 +21,5 @@ drop materialized view v_a_c; statement ok drop materialized view v_a_b_c; -statement ok -drop materialized view v_expr; - statement ok drop table t; diff --git a/e2e_test/over_window/generated/streaming/expr_in_win_func/mod.slt.part b/e2e_test/over_window/generated/streaming/expr_in_win_func/mod.slt.part new file mode 100644 index 0000000000000..89584947e8c33 --- /dev/null +++ b/e2e_test/over_window/generated/streaming/expr_in_win_func/mod.slt.part @@ -0,0 +1,71 @@ +# This file is generated by `gen.py`. Do not edit it manually! + +# Test expressions as window function args/PARTITION BY/ORDER BY. + +statement ok +create table t ( + id int + , p1 int + , p2 int + , time int + , v1 int + , v2 int +); + +statement ok +create materialized view v as +select + * + , t.v2 as out1 + , 0 as out2 + , first_value(v1) over (partition by p1, p2 order by time, id rows 3 preceding) as out3 + , lag(v1 + 2, 0 + 1) over (partition by p1 - 1 order by id) as out4 + , min(v1 * 2) over (partition by p1, p2 order by time + 1, id rows between current row and unbounded following) as out5 +from t; + +statement ok +insert into t values + (100001, 100, 200, 1, 701, 805) +, (100002, 100, 200, 2, 700, 806) +, (100003, 100, 208, 2, 723, 807) +, (100004, 103, 200, 2, 702, 808); + +statement ok +insert into t values + (100005, 100, 200, 3, 717, 810) +, (100006, 105, 204, 5, 703, 828); + +statement ok +update t set v1 = 799 where id = 100002; -- value change + +statement ok +update t set p2 = 200 where id = 100003; -- partition change + +statement ok +update t set "time" = 1 where id = 100005; -- order change + +query iiiiiiiiii +select * from v order by id; +---- +100001 100 200 1 701 805 805 0 701 NULL 1402 +100002 100 200 2 799 806 806 0 701 703 1446 +100003 100 200 2 723 807 807 0 701 801 1446 +100004 103 200 2 702 808 808 0 702 NULL 1404 +100005 100 200 1 717 810 810 0 701 725 1434 +100006 105 204 5 703 828 828 0 703 NULL 1406 + +statement ok +delete from t where time = 2; + +query iiiiiiiiii +select * from v order by id; +---- +100001 100 200 1 701 805 805 0 701 NULL 1402 +100005 100 200 1 717 810 810 0 701 703 1434 +100006 105 204 5 703 828 828 0 703 NULL 1406 + +statement ok +drop materialized view v; + +statement ok +drop table t; diff --git a/e2e_test/over_window/generated/streaming/main.slt.part b/e2e_test/over_window/generated/streaming/main.slt.part new file mode 100644 index 0000000000000..9f0ad1baeffe3 --- /dev/null +++ b/e2e_test/over_window/generated/streaming/main.slt.part @@ -0,0 +1,10 @@ +# This file is generated by `gen.py`. Do not edit it manually! + +statement ok +SET RW_IMPLICIT_FLUSH TO true; + +include ./basic/mod.slt.part +include ./rank_func/mod.slt.part +include ./expr_in_win_func/mod.slt.part +include ./agg_in_win_func/mod.slt.part +include ./opt_agg_then_join/mod.slt.part diff --git a/e2e_test/over_window/generated/streaming/opt_agg_then_join.slt.part/mod.slt.part b/e2e_test/over_window/generated/streaming/opt_agg_then_join.slt.part/mod.slt.part new file mode 100644 index 0000000000000..5a131ddd2029a --- /dev/null +++ b/e2e_test/over_window/generated/streaming/opt_agg_then_join.slt.part/mod.slt.part @@ -0,0 +1,74 @@ +# This file is generated by `gen.py`. Do not edit it manually! + +# Test cases that should be optimized to Agg + Join. + +statement ok +create table t(x int, y int); + +statement ok +create materialized view v as +select *, sum(y / x) OVER (PARTITION BY x) as a, count(x) OVER (PARTITION BY y) as b from t; + +statement ok +insert into t values + (1, 1), (1, 2), (1, 3), (1, 4), (1, 5), + (2, 1), (2, 2), (2, 3), (2, 4), (2, 5), + (3, 1), (3, 2), (3, 3), (3, 4), (3, 5), + -- ties + (1, 3), (2, 3), (3, 3); + +query IIII rowsort +select * from v; +---- +1 1 18 3 +1 2 18 3 +1 3 18 6 +1 3 18 6 +1 4 18 3 +1 5 18 3 +2 1 7 3 +2 2 7 3 +2 3 7 6 +2 3 7 6 +2 4 7 3 +2 5 7 3 +3 1 4 3 +3 2 4 3 +3 3 4 6 +3 3 4 6 +3 4 4 3 +3 5 4 3 + +statement ok +insert into t values (1, 6), (2, 8), (3, 12); + +query IIII rowsort +select * from v; +---- +1 1 24 3 +1 2 24 3 +1 3 24 6 +1 3 24 6 +1 4 24 3 +1 5 24 3 +1 6 24 1 +2 1 11 3 +2 2 11 3 +2 3 11 6 +2 3 11 6 +2 4 11 3 +2 5 11 3 +2 8 11 1 +3 1 8 3 +3 12 8 1 +3 2 8 3 +3 3 8 6 +3 3 8 6 +3 4 8 3 +3 5 8 3 + +statement ok +drop materialized view v; + +statement ok +drop table t; diff --git a/e2e_test/over_window/generated/streaming/rank_func/mod.slt.part b/e2e_test/over_window/generated/streaming/rank_func/mod.slt.part new file mode 100644 index 0000000000000..1a22f59053e9d --- /dev/null +++ b/e2e_test/over_window/generated/streaming/rank_func/mod.slt.part @@ -0,0 +1,86 @@ +# This file is generated by `gen.py`. Do not edit it manually! + +# Test rank window functions including `row_number`, `rank`, `dense_rank`. + +statement ok +create table t ( + id int + , p1 int + , p2 int + , time int + , v1 int + , v2 int +); + +statement ok +create materialized view v as +select + * + , row_number() over (partition by p1 order by time, id) as out1 + , row_number() over (partition by p1 order by p2 desc, id) as out2 +from t; + +statement ok +insert into t values + (100001, 100, 200, 1, 701, 805) +, (100002, 100, 200, 2, 700, 806) +, (100003, 100, 208, 2, 723, 807) +, (100004, 103, 200, 2, 702, 808); + +query II +select * from v order by id; +---- +100001 100 200 1 701 805 1 2 +100002 100 200 2 700 806 2 3 +100003 100 208 2 723 807 3 1 +100004 103 200 2 702 808 1 1 + +statement ok +insert into t values + (100005, 100, 200, 3, 717, 810) +, (100006, 105, 204, 5, 703, 828); + +query II +select * from v order by id; +---- +100001 100 200 1 701 805 1 2 +100002 100 200 2 700 806 2 3 +100003 100 208 2 723 807 3 1 +100004 103 200 2 702 808 1 1 +100005 100 200 3 717 810 4 4 +100006 105 204 5 703 828 1 1 + +statement ok +update t set v1 = 799 where id = 100002; -- value change + +statement ok +update t set p2 = 200 where id = 100003; -- partition change + +statement ok +update t set "time" = 1 where id = 100005; -- order change + +query iiiiiii +select * from v order by id; +---- +100001 100 200 1 701 805 1 1 +100002 100 200 2 799 806 3 2 +100003 100 200 2 723 807 4 3 +100004 103 200 2 702 808 1 1 +100005 100 200 1 717 810 2 4 +100006 105 204 5 703 828 1 1 + +statement ok +delete from t where time = 2; + +query iiiiiii +select * from v order by id; +---- +100001 100 200 1 701 805 1 1 +100005 100 200 1 717 810 2 2 +100006 105 204 5 703 828 1 1 + +statement ok +drop materialized view v; + +statement ok +drop table t; diff --git a/e2e_test/over_window/templates/agg_in_win_func/mod.slt.part b/e2e_test/over_window/templates/agg_in_win_func/mod.slt.part new file mode 100644 index 0000000000000..742fe034b9eb0 --- /dev/null +++ b/e2e_test/over_window/templates/agg_in_win_func/mod.slt.part @@ -0,0 +1,78 @@ +# Test aggregate function calls as window function args/PARTITION BY/ORDER BY. + +statement ok +create table t ( + id int + , p1 int + , p2 int + , time int + , v1 int + , v2 int +); + +statement ok +create $view_type v as +select + p1, p2 + , row_number() over (partition by p1 order by p2) as out1 + , sum(sum(v2)) over (partition by p1, avg(time) order by max(v1), p2) as out2 +from t +group by p1, p2; + +statement ok +insert into t values + (100001, 100, 200, 1, 701, 805) +, (100002, 100, 200, 2, 700, 806) +, (100003, 100, 208, 2, 723, 807) +, (100004, 103, 200, 2, 702, 808); + +query iiii +select * from v order by p1, p2; +---- +100 200 1 1611 +100 208 2 807 +103 200 1 808 + +statement ok +insert into t values + (100005, 100, 200, 3, 717, 810) +, (100006, 105, 204, 5, 703, 828); + +query iiii +select * from v order by p1, p2; +---- +100 200 1 2421 +100 208 2 3228 +103 200 1 808 +105 204 1 828 + +statement ok +update t set v1 = 799 where id = 100002; -- value change + +statement ok +update t set p2 = 200 where id = 100003; -- partition change + +statement ok +update t set "time" = 1 where id = 100005; -- order change + +query iiiiiii +select * from v order by p1, p2; +---- +100 200 1 3228 +103 200 1 808 +105 204 1 828 + +statement ok +delete from t where time = 2; + +query iiii +select * from v order by p1, p2; +---- +100 200 1 1615 +105 204 1 828 + +statement ok +drop $view_type v; + +statement ok +drop table t; diff --git a/e2e_test/over_window/templates/cross_check.slt.part b/e2e_test/over_window/templates/basic/cross_check.slt.part similarity index 100% rename from e2e_test/over_window/templates/cross_check.slt.part rename to e2e_test/over_window/templates/basic/cross_check.slt.part diff --git a/e2e_test/over_window/templates/mod.slt.part b/e2e_test/over_window/templates/basic/mod.slt.part similarity index 68% rename from e2e_test/over_window/templates/mod.slt.part rename to e2e_test/over_window/templates/basic/mod.slt.part index 1b1b86a0d40d3..421f5a911f468 100644 --- a/e2e_test/over_window/templates/mod.slt.part +++ b/e2e_test/over_window/templates/basic/mod.slt.part @@ -1,4 +1,6 @@ -include ./create.slt.part +# Test basic functionality of general batch and streaming over window. + +include ./setup.slt.part statement ok insert into t values @@ -31,21 +33,6 @@ select * from v_c order by id; 100003 100 208 2 723 807 723 NULL NULL NULL NULL 100004 103 200 2 702 808 702 NULL NULL NULL NULL -query II -select * from v_d order by id; ----- -100001 100 200 1 701 805 1 2 -100002 100 200 2 700 806 2 3 -100003 100 208 2 723 807 3 1 -100004 103 200 2 702 808 1 1 - -query iiii -select * from v_e order by p1; ----- -100 200 1 1611 -100 208 2 807 -103 200 1 808 - include ./cross_check.slt.part statement ok @@ -83,24 +70,6 @@ select * from v_c order by id; 100005 100 200 3 717 810 717 700 700 NULL NULL 100006 105 204 5 703 828 703 NULL NULL NULL NULL -query II -select * from v_d order by id; ----- -100001 100 200 1 701 805 1 2 -100002 100 200 2 700 806 2 3 -100003 100 208 2 723 807 3 1 -100004 103 200 2 702 808 1 1 -100005 100 200 3 717 810 4 4 -100006 105 204 5 703 828 1 1 - -query iiii -select * from v_e order by p1, p2; ----- -100 200 1 2421 -100 208 2 3228 -103 200 1 808 -105 204 1 828 - include ./cross_check.slt.part statement ok @@ -142,33 +111,6 @@ select * from v_c order by id; 100005 100 200 1 717 810 717 723 701 806 806 100006 105 204 5 703 828 703 NULL NULL NULL NULL -query iiiiiii -select * from v_d order by id; ----- -100001 100 200 1 701 805 1 1 -100002 100 200 2 799 806 3 2 -100003 100 200 2 723 807 4 3 -100004 103 200 2 702 808 1 1 -100005 100 200 1 717 810 2 4 -100006 105 204 5 703 828 1 1 - -query iiiiiii -select * from v_e order by p1; ----- -100 200 1 3228 -103 200 1 808 -105 204 1 828 - -query iiiiiiiiii -select * from v_expr order by id; ----- -100001 100 200 1 701 805 805 0 701 NULL 1402 -100002 100 200 2 799 806 806 0 701 703 1446 -100003 100 200 2 723 807 807 0 701 801 1446 -100004 103 200 2 702 808 808 0 702 NULL 1404 -100005 100 200 1 717 810 810 0 701 725 1434 -100006 105 204 5 703 828 828 0 703 NULL 1406 - include ./cross_check.slt.part statement ok @@ -195,26 +137,6 @@ select * from v_c order by id; 100005 100 200 1 717 810 717 701 701 NULL NULL 100006 105 204 5 703 828 703 NULL NULL NULL NULL -query iiiiiii -select * from v_d order by id; ----- -100001 100 200 1 701 805 1 1 -100005 100 200 1 717 810 2 2 -100006 105 204 5 703 828 1 1 - -query iiii -select * from v_e order by p1; ----- -100 200 1 1615 -105 204 1 828 - -query iiiiiiiiii -select * from v_expr order by id; ----- -100001 100 200 1 701 805 805 0 701 NULL 1402 -100005 100 200 1 717 810 810 0 701 703 1434 -100006 105 204 5 703 828 828 0 703 NULL 1406 - include ./cross_check.slt.part -include ./drop.slt.part +include ./teardown.slt.part diff --git a/e2e_test/over_window/templates/create.slt.part b/e2e_test/over_window/templates/basic/setup.slt.part similarity index 78% rename from e2e_test/over_window/templates/create.slt.part rename to e2e_test/over_window/templates/basic/setup.slt.part index 7ac749e459b02..d989d50029430 100644 --- a/e2e_test/over_window/templates/create.slt.part +++ b/e2e_test/over_window/templates/basic/setup.slt.part @@ -38,25 +38,6 @@ select , lead(v2, 2) over (partition by p1, p2 order by v1, v2) as out9 from t; -# row_number -statement ok -create $view_type v_d as -select - * - , row_number() over (partition by p1 order by time, id) as out10 - , row_number() over (partition by p1 order by p2 desc, id) as out11 -from t; - -# over + agg -statement ok -create $view_type v_e as -select - p1, p2 - , row_number() over (partition by p1 order by p2) as out12 - , sum(sum(v2)) over (partition by p1, avg(time) order by max(v1), p2) as out13 -from t -group by p1, p2; - statement ok create $view_type v_a_b as select @@ -101,14 +82,3 @@ select , lead(v2, 1) over (partition by p1, p2 order by time, id) as out8 , lead(v2, 2) over (partition by p1, p2 order by v1, v2) as out9 from t; - -statement ok -create $view_type v_expr as -select - * - , t.v2 as out1 - , 0 as out2 - , first_value(v1) over (partition by p1, p2 order by time, id rows 3 preceding) as out3 - , lag(v1 + 2, 0 + 1) over (partition by p1 - 1 order by id) as out4 - , min(v1 * 2) over (partition by p1, p2 order by time + 1, id rows between current row and unbounded following) as out5 -from t; diff --git a/e2e_test/over_window/templates/drop.slt.part b/e2e_test/over_window/templates/basic/teardown.slt.part similarity index 72% rename from e2e_test/over_window/templates/drop.slt.part rename to e2e_test/over_window/templates/basic/teardown.slt.part index def8e92379878..89915395703ad 100644 --- a/e2e_test/over_window/templates/drop.slt.part +++ b/e2e_test/over_window/templates/basic/teardown.slt.part @@ -7,12 +7,6 @@ drop $view_type v_b; statement ok drop $view_type v_c; -statement ok -drop $view_type v_d; - -statement ok -drop $view_type v_e; - statement ok drop $view_type v_a_b; @@ -25,8 +19,5 @@ drop $view_type v_a_c; statement ok drop $view_type v_a_b_c; -statement ok -drop $view_type v_expr; - statement ok drop table t; diff --git a/e2e_test/over_window/templates/expr_in_win_func/mod.slt.part b/e2e_test/over_window/templates/expr_in_win_func/mod.slt.part new file mode 100644 index 0000000000000..bdc71711c631b --- /dev/null +++ b/e2e_test/over_window/templates/expr_in_win_func/mod.slt.part @@ -0,0 +1,69 @@ +# Test expressions as window function args/PARTITION BY/ORDER BY. + +statement ok +create table t ( + id int + , p1 int + , p2 int + , time int + , v1 int + , v2 int +); + +statement ok +create $view_type v as +select + * + , t.v2 as out1 + , 0 as out2 + , first_value(v1) over (partition by p1, p2 order by time, id rows 3 preceding) as out3 + , lag(v1 + 2, 0 + 1) over (partition by p1 - 1 order by id) as out4 + , min(v1 * 2) over (partition by p1, p2 order by time + 1, id rows between current row and unbounded following) as out5 +from t; + +statement ok +insert into t values + (100001, 100, 200, 1, 701, 805) +, (100002, 100, 200, 2, 700, 806) +, (100003, 100, 208, 2, 723, 807) +, (100004, 103, 200, 2, 702, 808); + +statement ok +insert into t values + (100005, 100, 200, 3, 717, 810) +, (100006, 105, 204, 5, 703, 828); + +statement ok +update t set v1 = 799 where id = 100002; -- value change + +statement ok +update t set p2 = 200 where id = 100003; -- partition change + +statement ok +update t set "time" = 1 where id = 100005; -- order change + +query iiiiiiiiii +select * from v order by id; +---- +100001 100 200 1 701 805 805 0 701 NULL 1402 +100002 100 200 2 799 806 806 0 701 703 1446 +100003 100 200 2 723 807 807 0 701 801 1446 +100004 103 200 2 702 808 808 0 702 NULL 1404 +100005 100 200 1 717 810 810 0 701 725 1434 +100006 105 204 5 703 828 828 0 703 NULL 1406 + +statement ok +delete from t where time = 2; + +query iiiiiiiiii +select * from v order by id; +---- +100001 100 200 1 701 805 805 0 701 NULL 1402 +100005 100 200 1 717 810 810 0 701 703 1434 +100006 105 204 5 703 828 828 0 703 NULL 1406 + +statement ok +drop $view_type v; + +statement ok +drop table t; diff --git a/e2e_test/over_window/templates/main.slt.part b/e2e_test/over_window/templates/main.slt.part new file mode 100644 index 0000000000000..00dfac5101eee --- /dev/null +++ b/e2e_test/over_window/templates/main.slt.part @@ -0,0 +1,8 @@ +statement ok +SET RW_IMPLICIT_FLUSH TO true; + +include ./basic/mod.slt.part +include ./rank_func/mod.slt.part +include ./expr_in_win_func/mod.slt.part +include ./agg_in_win_func/mod.slt.part +include ./opt_agg_then_join/mod.slt.part diff --git a/e2e_test/streaming/over_window/special_cases/to_agg_then_join.slt.part b/e2e_test/over_window/templates/opt_agg_then_join.slt.part/mod.slt.part similarity index 86% rename from e2e_test/streaming/over_window/special_cases/to_agg_then_join.slt.part rename to e2e_test/over_window/templates/opt_agg_then_join.slt.part/mod.slt.part index aa428234dcb6d..75aa93b2f4f1f 100644 --- a/e2e_test/streaming/over_window/special_cases/to_agg_then_join.slt.part +++ b/e2e_test/over_window/templates/opt_agg_then_join.slt.part/mod.slt.part @@ -1,8 +1,10 @@ +# Test cases that should be optimized to Agg + Join. + statement ok create table t(x int, y int); statement ok -create materialized view mv as +create $view_type v as select *, sum(y / x) OVER (PARTITION BY x) as a, count(x) OVER (PARTITION BY y) as b from t; statement ok @@ -14,7 +16,7 @@ insert into t values (1, 3), (2, 3), (3, 3); query IIII rowsort -select * from mv; +select * from v; ---- 1 1 18 3 1 2 18 3 @@ -39,7 +41,7 @@ statement ok insert into t values (1, 6), (2, 8), (3, 12); query IIII rowsort -select * from mv; +select * from v; ---- 1 1 24 3 1 2 24 3 @@ -64,7 +66,7 @@ select * from mv; 3 5 8 3 statement ok -drop materialized view mv; +drop $view_type v; statement ok drop table t; diff --git a/e2e_test/over_window/templates/rank_func/mod.slt.part b/e2e_test/over_window/templates/rank_func/mod.slt.part new file mode 100644 index 0000000000000..b654819f8e0f0 --- /dev/null +++ b/e2e_test/over_window/templates/rank_func/mod.slt.part @@ -0,0 +1,84 @@ +# Test rank window functions including `row_number`, `rank`, `dense_rank`. + +statement ok +create table t ( + id int + , p1 int + , p2 int + , time int + , v1 int + , v2 int +); + +statement ok +create $view_type v as +select + * + , row_number() over (partition by p1 order by time, id) as out1 + , row_number() over (partition by p1 order by p2 desc, id) as out2 +from t; + +statement ok +insert into t values + (100001, 100, 200, 1, 701, 805) +, (100002, 100, 200, 2, 700, 806) +, (100003, 100, 208, 2, 723, 807) +, (100004, 103, 200, 2, 702, 808); + +query II +select * from v order by id; +---- +100001 100 200 1 701 805 1 2 +100002 100 200 2 700 806 2 3 +100003 100 208 2 723 807 3 1 +100004 103 200 2 702 808 1 1 + +statement ok +insert into t values + (100005, 100, 200, 3, 717, 810) +, (100006, 105, 204, 5, 703, 828); + +query II +select * from v order by id; +---- +100001 100 200 1 701 805 1 2 +100002 100 200 2 700 806 2 3 +100003 100 208 2 723 807 3 1 +100004 103 200 2 702 808 1 1 +100005 100 200 3 717 810 4 4 +100006 105 204 5 703 828 1 1 + +statement ok +update t set v1 = 799 where id = 100002; -- value change + +statement ok +update t set p2 = 200 where id = 100003; -- partition change + +statement ok +update t set "time" = 1 where id = 100005; -- order change + +query iiiiiii +select * from v order by id; +---- +100001 100 200 1 701 805 1 1 +100002 100 200 2 799 806 3 2 +100003 100 200 2 723 807 4 3 +100004 103 200 2 702 808 1 1 +100005 100 200 1 717 810 2 4 +100006 105 204 5 703 828 1 1 + +statement ok +delete from t where time = 2; + +query iiiiiii +select * from v order by id; +---- +100001 100 200 1 701 805 1 1 +100005 100 200 1 717 810 2 2 +100006 105 204 5 703 828 1 1 + +statement ok +drop $view_type v; + +statement ok +drop table t; diff --git a/e2e_test/streaming/over_window/generated b/e2e_test/streaming/over_window/generated new file mode 120000 index 0000000000000..747efa5c83183 --- /dev/null +++ b/e2e_test/streaming/over_window/generated @@ -0,0 +1 @@ +../../over_window/generated/streaming/ \ No newline at end of file diff --git a/e2e_test/streaming/over_window/main.slt b/e2e_test/streaming/over_window/main.slt index 6e277352059c4..dca370c0306ee 100644 --- a/e2e_test/streaming/over_window/main.slt +++ b/e2e_test/streaming/over_window/main.slt @@ -1,5 +1 @@ -statement ok -SET RW_IMPLICIT_FLUSH TO true; - -include ./special_cases/mod.slt.part -include ./over_window/mod.slt.part +include ./generated/main.slt.part diff --git a/e2e_test/streaming/over_window/over_window b/e2e_test/streaming/over_window/over_window deleted file mode 120000 index 2efb728950ba5..0000000000000 --- a/e2e_test/streaming/over_window/over_window +++ /dev/null @@ -1 +0,0 @@ -../../over_window/generated/streaming \ No newline at end of file diff --git a/e2e_test/streaming/over_window/special_cases/mod.slt.part b/e2e_test/streaming/over_window/special_cases/mod.slt.part deleted file mode 100644 index 265ed4966619f..0000000000000 --- a/e2e_test/streaming/over_window/special_cases/mod.slt.part +++ /dev/null @@ -1 +0,0 @@ -include ./to_agg_then_join.slt.part From edf314ee7e8a6e1cc253301c757849835d5b1adf Mon Sep 17 00:00:00 2001 From: xiangjinwu <17769960+xiangjinwu@users.noreply.github.com> Date: Wed, 1 Nov 2023 15:16:49 +0800 Subject: [PATCH 04/17] fix(sink): json `timestamptz` should contain UTC suffix `Z` (#13109) --- src/connector/src/sink/encoder/json.rs | 58 ++++++++++++++++--- src/connector/src/sink/encoder/mod.rs | 33 ++++++++++- .../src/sink/formatter/debezium_json.rs | 19 +++++- src/connector/src/sink/formatter/mod.rs | 21 +++++-- src/connector/src/sink/kafka.rs | 9 ++- src/connector/src/sink/nats.rs | 8 ++- src/connector/src/sink/remote.rs | 16 ++++- src/frontend/src/handler/create_sink.rs | 7 ++- 8 files changed, 147 insertions(+), 24 deletions(-) diff --git a/src/connector/src/sink/encoder/json.rs b/src/connector/src/sink/encoder/json.rs index c8bfc489c70c9..7da859b8e9045 100644 --- a/src/connector/src/sink/encoder/json.rs +++ b/src/connector/src/sink/encoder/json.rs @@ -29,7 +29,7 @@ use serde_json::{json, Map, Value}; use super::{ CustomJsonType, KafkaConnectParams, KafkaConnectParamsRef, Result, RowEncoder, SerTo, - TimestampHandlingMode, + TimestampHandlingMode, TimestamptzHandlingMode, }; use crate::sink::SinkError; @@ -37,6 +37,7 @@ pub struct JsonEncoder { schema: Schema, col_indices: Option>, timestamp_handling_mode: TimestampHandlingMode, + timestamptz_handling_mode: TimestamptzHandlingMode, custom_json_type: CustomJsonType, kafka_connect: Option, } @@ -46,11 +47,13 @@ impl JsonEncoder { schema: Schema, col_indices: Option>, timestamp_handling_mode: TimestampHandlingMode, + timestamptz_handling_mode: TimestamptzHandlingMode, ) -> Self { Self { schema, col_indices, timestamp_handling_mode, + timestamptz_handling_mode, custom_json_type: CustomJsonType::None, kafka_connect: None, } @@ -66,6 +69,7 @@ impl JsonEncoder { schema, col_indices, timestamp_handling_mode, + timestamptz_handling_mode: TimestamptzHandlingMode::UtcWithoutSuffix, custom_json_type: CustomJsonType::Doris(map), kafka_connect: None, } @@ -87,6 +91,7 @@ impl JsonEncoder { schema, col_indices, timestamp_handling_mode, + timestamptz_handling_mode: TimestamptzHandlingMode::UtcString, custom_json_type: CustomJsonType::Bigquery, kafka_connect: None, } @@ -118,6 +123,7 @@ impl RowEncoder for JsonEncoder { field, row.datum_at(*idx), self.timestamp_handling_mode, + self.timestamptz_handling_mode, &self.custom_json_type, ) .map_err(|e| SinkError::Encode(e.to_string()))?; @@ -152,6 +158,7 @@ fn datum_to_json_object( field: &Field, datum: DatumRef<'_>, timestamp_handling_mode: TimestampHandlingMode, + timestamptz_handling_mode: TimestamptzHandlingMode, custom_json_type: &CustomJsonType, ) -> ArrayResult { let scalar_ref = match datum { @@ -205,13 +212,20 @@ fn datum_to_json_object( json!(v.to_text()) } }, - (DataType::Timestamptz, ScalarRefImpl::Timestamptz(v)) => { - // risingwave's timestamp with timezone is stored in UTC and does not maintain the - // timezone info and the time is in microsecond. - let parsed = v.to_datetime_utc().naive_utc(); - let v = parsed.format("%Y-%m-%d %H:%M:%S%.6f").to_string(); - json!(v) - } + (DataType::Timestamptz, ScalarRefImpl::Timestamptz(v)) => match timestamptz_handling_mode { + TimestamptzHandlingMode::UtcString => { + let parsed = v.to_datetime_utc(); + let v = parsed.to_rfc3339_opts(chrono::SecondsFormat::Micros, true); + json!(v) + } + TimestamptzHandlingMode::UtcWithoutSuffix => { + let parsed = v.to_datetime_utc().naive_utc(); + let v = parsed.format("%Y-%m-%d %H:%M:%S%.6f").to_string(); + json!(v) + } + TimestamptzHandlingMode::Micro => json!(v.timestamp_micros()), + TimestamptzHandlingMode::Milli => json!(v.timestamp_millis()), + }, (DataType::Time, ScalarRefImpl::Time(v)) => { // todo: just ignore the nanos part to avoid leap second complex json!(v.0.num_seconds_from_midnight() as i64 * 1000) @@ -246,6 +260,7 @@ fn datum_to_json_object( &inner_field, sub_datum_ref, timestamp_handling_mode, + timestamptz_handling_mode, custom_json_type, )?; vec.push(value); @@ -265,6 +280,7 @@ fn datum_to_json_object( &sub_field, sub_datum_ref, timestamp_handling_mode, + timestamptz_handling_mode, custom_json_type, )?; map.insert(sub_field.name.clone(), value); @@ -283,6 +299,7 @@ fn datum_to_json_object( &sub_field, sub_datum_ref, timestamp_handling_mode, + timestamptz_handling_mode, custom_json_type, )?; map.insert(sub_field.name.clone(), value); @@ -399,6 +416,7 @@ mod tests { }, Some(ScalarImpl::Bool(false).as_scalar_ref_impl()), TimestampHandlingMode::String, + TimestamptzHandlingMode::UtcString, &CustomJsonType::None, ) .unwrap(); @@ -411,6 +429,7 @@ mod tests { }, Some(ScalarImpl::Int16(16).as_scalar_ref_impl()), TimestampHandlingMode::String, + TimestamptzHandlingMode::UtcString, &CustomJsonType::None, ) .unwrap(); @@ -423,6 +442,7 @@ mod tests { }, Some(ScalarImpl::Int64(std::i64::MAX).as_scalar_ref_impl()), TimestampHandlingMode::String, + TimestamptzHandlingMode::UtcString, &CustomJsonType::None, ) .unwrap(); @@ -440,6 +460,21 @@ mod tests { }, Some(ScalarImpl::Timestamptz(tstz_inner).as_scalar_ref_impl()), TimestampHandlingMode::String, + TimestamptzHandlingMode::UtcString, + &CustomJsonType::None, + ) + .unwrap(); + assert_eq!(tstz_value, "2018-01-26T18:30:09.453000Z"); + + let tstz_inner = "2018-01-26T18:30:09.453Z".parse().unwrap(); + let tstz_value = datum_to_json_object( + &Field { + data_type: DataType::Timestamptz, + ..mock_field.clone() + }, + Some(ScalarImpl::Timestamptz(tstz_inner).as_scalar_ref_impl()), + TimestampHandlingMode::String, + TimestamptzHandlingMode::UtcWithoutSuffix, &CustomJsonType::None, ) .unwrap(); @@ -455,6 +490,7 @@ mod tests { .as_scalar_ref_impl(), ), TimestampHandlingMode::Milli, + TimestamptzHandlingMode::UtcString, &CustomJsonType::None, ) .unwrap(); @@ -470,6 +506,7 @@ mod tests { .as_scalar_ref_impl(), ), TimestampHandlingMode::String, + TimestamptzHandlingMode::UtcString, &CustomJsonType::None, ) .unwrap(); @@ -486,6 +523,7 @@ mod tests { .as_scalar_ref_impl(), ), TimestampHandlingMode::String, + TimestamptzHandlingMode::UtcString, &CustomJsonType::None, ) .unwrap(); @@ -501,6 +539,7 @@ mod tests { .as_scalar_ref_impl(), ), TimestampHandlingMode::String, + TimestamptzHandlingMode::UtcString, &CustomJsonType::None, ) .unwrap(); @@ -516,6 +555,7 @@ mod tests { }, Some(ScalarImpl::Decimal(Decimal::try_from(1.1111111).unwrap()).as_scalar_ref_impl()), TimestampHandlingMode::String, + TimestamptzHandlingMode::UtcString, &CustomJsonType::Doris(map), ) .unwrap(); @@ -528,6 +568,7 @@ mod tests { }, Some(ScalarImpl::Date(Date::from_ymd_uncheck(2010, 10, 10)).as_scalar_ref_impl()), TimestampHandlingMode::String, + TimestamptzHandlingMode::UtcString, &CustomJsonType::Doris(HashMap::default()), ) .unwrap(); @@ -550,6 +591,7 @@ mod tests { }, Some(ScalarRefImpl::Struct(StructRef::ValueRef { val: &value })), TimestampHandlingMode::String, + TimestamptzHandlingMode::UtcString, &CustomJsonType::Doris(HashMap::default()), ) .unwrap(); diff --git a/src/connector/src/sink/encoder/mod.rs b/src/connector/src/sink/encoder/mod.rs index a58824b6fa92e..f557b9295c514 100644 --- a/src/connector/src/sink/encoder/mod.rs +++ b/src/connector/src/sink/encoder/mod.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::HashMap; +use std::collections::{BTreeMap, HashMap}; use std::sync::Arc; use risingwave_common::catalog::Schema; @@ -89,6 +89,37 @@ pub enum TimestampHandlingMode { String, } +#[derive(Clone, Copy, Default)] +pub enum TimestamptzHandlingMode { + #[default] + UtcString, + UtcWithoutSuffix, + Micro, + Milli, +} + +impl TimestamptzHandlingMode { + pub const FRONTEND_DEFAULT: &'static str = "utc_string"; + pub const OPTION_KEY: &'static str = "timestamptz.handling.mode"; + + pub fn from_options(options: &BTreeMap) -> Result { + match options.get(Self::OPTION_KEY).map(std::ops::Deref::deref) { + Some(Self::FRONTEND_DEFAULT) => Ok(Self::UtcString), + Some("utc_without_suffix") => Ok(Self::UtcWithoutSuffix), + Some("micro") => Ok(Self::Micro), + Some("milli") => Ok(Self::Milli), + Some(v) => Err(super::SinkError::Config(anyhow::anyhow!( + "unrecognized {} value {}", + Self::OPTION_KEY, + v + ))), + // This is not a good default. We just have to select it when no option is provided + // for compatibility with old version. + None => Ok(Self::UtcWithoutSuffix), + } + } +} + #[derive(Clone)] pub enum CustomJsonType { // Doris's json need date is string. diff --git a/src/connector/src/sink/formatter/debezium_json.rs b/src/connector/src/sink/formatter/debezium_json.rs index ce98daab88756..a40789f7d9c95 100644 --- a/src/connector/src/sink/formatter/debezium_json.rs +++ b/src/connector/src/sink/formatter/debezium_json.rs @@ -20,7 +20,9 @@ use serde_json::{json, Map, Value}; use tracing::warn; use super::{Result, SinkFormatter, StreamChunk}; -use crate::sink::encoder::{JsonEncoder, RowEncoder, TimestampHandlingMode}; +use crate::sink::encoder::{ + JsonEncoder, RowEncoder, TimestampHandlingMode, TimestamptzHandlingMode, +}; use crate::tri; const DEBEZIUM_NAME_FIELD_PREFIX: &str = "RisingWave"; @@ -63,8 +65,14 @@ impl DebeziumJsonFormatter { schema.clone(), Some(pk_indices.clone()), TimestampHandlingMode::Milli, + TimestamptzHandlingMode::UtcString, + ); + let val_encoder = JsonEncoder::new( + schema.clone(), + None, + TimestampHandlingMode::Milli, + TimestamptzHandlingMode::UtcString, ); - let val_encoder = JsonEncoder::new(schema.clone(), None, TimestampHandlingMode::Milli); Self { schema, pk_indices, @@ -360,7 +368,12 @@ mod tests { }, ]); - let encoder = JsonEncoder::new(schema.clone(), None, TimestampHandlingMode::Milli); + let encoder = JsonEncoder::new( + schema.clone(), + None, + TimestampHandlingMode::Milli, + TimestamptzHandlingMode::UtcString, + ); let json_chunk = chunk_to_json(chunk, &encoder).unwrap(); let schema_json = schema_to_json(&schema, "test_db", "test_table"); assert_eq!( diff --git a/src/connector/src/sink/formatter/mod.rs b/src/connector/src/sink/formatter/mod.rs index 1e165268300fa..9b8cb953372b4 100644 --- a/src/connector/src/sink/formatter/mod.rs +++ b/src/connector/src/sink/formatter/mod.rs @@ -28,7 +28,7 @@ pub use upsert::UpsertFormatter; use super::catalog::{SinkEncode, SinkFormat, SinkFormatDesc}; use super::encoder::template::TemplateEncoder; -use super::encoder::KafkaConnectParams; +use super::encoder::{KafkaConnectParams, TimestamptzHandlingMode}; use super::redis::{KEY_FORMAT, VALUE_FORMAT}; use crate::sink::encoder::{ AvroEncoder, AvroHeader, JsonEncoder, ProtoEncoder, TimestampHandlingMode, @@ -91,6 +91,7 @@ impl SinkFormatterImpl { format_desc.encode, ))) }; + let timestamptz_mode = TimestamptzHandlingMode::from_options(&format_desc.options)?; match format_desc.format { SinkFormat::AppendOnly => { @@ -99,13 +100,18 @@ impl SinkFormatterImpl { schema.clone(), Some(pk_indices.clone()), TimestampHandlingMode::Milli, + timestamptz_mode, ) }); match format_desc.encode { SinkEncode::Json => { - let val_encoder = - JsonEncoder::new(schema, None, TimestampHandlingMode::Milli); + let val_encoder = JsonEncoder::new( + schema, + None, + TimestampHandlingMode::Milli, + timestamptz_mode, + ); let formatter = AppendOnlyFormatter::new(key_encoder, val_encoder); Ok(SinkFormatterImpl::AppendOnlyJson(formatter)) } @@ -164,9 +170,14 @@ impl SinkFormatterImpl { schema.clone(), Some(pk_indices), TimestampHandlingMode::Milli, + timestamptz_mode, + ); + let mut val_encoder = JsonEncoder::new( + schema, + None, + TimestampHandlingMode::Milli, + timestamptz_mode, ); - let mut val_encoder = - JsonEncoder::new(schema, None, TimestampHandlingMode::Milli); if let Some(s) = format_desc.options.get("schemas.enable") { match s.to_lowercase().parse::() { diff --git a/src/connector/src/sink/kafka.rs b/src/connector/src/sink/kafka.rs index 07709f182dc47..e0ff1f67471df 100644 --- a/src/connector/src/sink/kafka.rs +++ b/src/connector/src/sink/kafka.rs @@ -564,7 +564,7 @@ mod test { use risingwave_common::types::DataType; use super::*; - use crate::sink::encoder::{JsonEncoder, TimestampHandlingMode}; + use crate::sink::encoder::{JsonEncoder, TimestampHandlingMode, TimestamptzHandlingMode}; use crate::sink::formatter::AppendOnlyFormatter; #[test] @@ -729,7 +729,12 @@ mod test { SinkFormatterImpl::AppendOnlyJson(AppendOnlyFormatter::new( // We do not specify primary key for this schema None, - JsonEncoder::new(schema, None, TimestampHandlingMode::Milli), + JsonEncoder::new( + schema, + None, + TimestampHandlingMode::Milli, + TimestamptzHandlingMode::UtcString, + ), )), ) .await diff --git a/src/connector/src/sink/nats.rs b/src/connector/src/sink/nats.rs index 2f810eed786a9..496f96eb66731 100644 --- a/src/connector/src/sink/nats.rs +++ b/src/connector/src/sink/nats.rs @@ -24,6 +24,7 @@ use serde_with::serde_as; use tokio_retry::strategy::{jitter, ExponentialBackoff}; use tokio_retry::Retry; +use super::encoder::TimestamptzHandlingMode; use super::utils::chunk_to_json; use super::{DummySinkCommitCoordinator, SinkWriterParam}; use crate::common::NatsCommon; @@ -138,7 +139,12 @@ impl NatsSinkWriter { config: config.clone(), context, schema: schema.clone(), - json_encoder: JsonEncoder::new(schema, None, TimestampHandlingMode::Milli), + json_encoder: JsonEncoder::new( + schema, + None, + TimestampHandlingMode::Milli, + TimestamptzHandlingMode::UtcWithoutSuffix, + ), }) } diff --git a/src/connector/src/sink/remote.rs b/src/connector/src/sink/remote.rs index 0b5948646637b..228da05a48108 100644 --- a/src/connector/src/sink/remote.rs +++ b/src/connector/src/sink/remote.rs @@ -57,7 +57,7 @@ use tracing::warn; use super::encoder::{JsonEncoder, RowEncoder}; use crate::sink::coordinate::CoordinatedSinkWriter; -use crate::sink::encoder::TimestampHandlingMode; +use crate::sink::encoder::{TimestampHandlingMode, TimestamptzHandlingMode}; use crate::sink::log_store::{LogReader, LogStoreReadItem, TruncateOffset}; use crate::sink::writer::{LogSinkerOf, SinkWriter, SinkWriterExt}; use crate::sink::{ @@ -412,7 +412,12 @@ impl RemoteSinkWriterInner { batch_id: 0, stream_handle, payload_format: connector_params.sink_payload_format, - json_encoder: JsonEncoder::new(schema, None, TimestampHandlingMode::String), + json_encoder: JsonEncoder::new( + schema, + None, + TimestampHandlingMode::String, + TimestamptzHandlingMode::UtcWithoutSuffix, + ), sink_metrics, _phantom: PhantomData, }) @@ -451,7 +456,12 @@ impl RemoteSinkWriterInner { properties, epoch: None, batch_id: 0, - json_encoder: JsonEncoder::new(schema, None, TimestampHandlingMode::String), + json_encoder: JsonEncoder::new( + schema, + None, + TimestampHandlingMode::String, + TimestamptzHandlingMode::UtcWithoutSuffix, + ), stream_handle, payload_format: SinkPayloadFormat::Json, sink_metrics: SinkMetrics::for_test(), diff --git a/src/frontend/src/handler/create_sink.rs b/src/frontend/src/handler/create_sink.rs index e4081fbee4fcf..a4db476b6274a 100644 --- a/src/frontend/src/handler/create_sink.rs +++ b/src/frontend/src/handler/create_sink.rs @@ -230,6 +230,7 @@ pub async fn handle_create_sink( /// which transforms sqlparser AST `SourceSchemaV2` into `StreamSourceInfo`. fn bind_sink_format_desc(value: ConnectorSchema) -> Result { use risingwave_connector::sink::catalog::{SinkEncode, SinkFormat}; + use risingwave_connector::sink::encoder::TimestamptzHandlingMode; use risingwave_sqlparser::ast::{Encode as E, Format as F}; let format = match value.format { @@ -249,7 +250,11 @@ fn bind_sink_format_desc(value: ConnectorSchema) -> Result { return Err(ErrorCode::BindError(format!("sink encode unsupported: {e}")).into()) } }; - let options = WithOptions::try_from(value.row_options.as_slice())?.into_inner(); + let mut options = WithOptions::try_from(value.row_options.as_slice())?.into_inner(); + + options + .entry(TimestamptzHandlingMode::OPTION_KEY.to_owned()) + .or_insert(TimestamptzHandlingMode::FRONTEND_DEFAULT.to_owned()); Ok(SinkFormatDesc { format, From 82c99264a0fd8d2b3667443511a5113523af557f Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Wed, 1 Nov 2023 15:22:14 +0800 Subject: [PATCH 05/17] fix(sqlsmith): use `encoded` string as input to `decode` (#13191) --- src/tests/sqlsmith/src/sql_gen/functions.rs | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/tests/sqlsmith/src/sql_gen/functions.rs b/src/tests/sqlsmith/src/sql_gen/functions.rs index 01cbb0604d262..863cf293d5d26 100644 --- a/src/tests/sqlsmith/src/sql_gen/functions.rs +++ b/src/tests/sqlsmith/src/sql_gen/functions.rs @@ -129,6 +129,11 @@ impl<'a, R: Rng> SqlGenerator<'a, R> { input_string, Expr::Value(Value::SingleQuotedString(encoding.to_string())), ]; + let encoded_string = Expr::Function(make_simple_func("encode", &args)); + let args = vec![ + encoded_string, + Expr::Value(Value::SingleQuotedString(encoding.to_string())), + ]; Expr::Function(make_simple_func("decode", &args)) } From 1b65be299e388927f1d884baeef4f555fa74572d Mon Sep 17 00:00:00 2001 From: Runji Wang Date: Wed, 1 Nov 2023 16:06:59 +0800 Subject: [PATCH 06/17] fix: `hash_agg` bench (#13193) Signed-off-by: Runji Wang --- src/stream/benches/stream_hash_agg.rs | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/stream/benches/stream_hash_agg.rs b/src/stream/benches/stream_hash_agg.rs index 62c45421c7f60..0a2ac22cac99c 100644 --- a/src/stream/benches/stream_hash_agg.rs +++ b/src/stream/benches/stream_hash_agg.rs @@ -29,6 +29,8 @@ use risingwave_stream::executor::test_utils::*; use risingwave_stream::executor::{BoxedExecutor, PkIndices}; use tokio::runtime::Runtime; +risingwave_expr_impl::enable!(); + fn bench_hash_agg(c: &mut Criterion) { let mut group = c.benchmark_group("Q17"); group.sample_size(10); From 675bdca9ac91d9ac2c29f3d12984b46830e317c6 Mon Sep 17 00:00:00 2001 From: zwang28 <70626450+zwang28@users.noreply.github.com> Date: Wed, 1 Nov 2023 16:19:59 +0800 Subject: [PATCH 07/17] refactor(meta): adapt metadata backup to metadata model V2 (#13040) Co-authored-by: zwang28 --- Cargo.lock | 3 +- src/meta/model_v2/src/compaction_config.rs | 14 +- src/meta/model_v2/src/compaction_status.rs | 12 +- src/meta/model_v2/src/compaction_task.rs | 23 +- .../model_v2/src/hummock_pinned_snapshot.rs | 19 +- .../model_v2/src/hummock_pinned_version.rs | 19 +- .../model_v2/src/hummock_version_delta.rs | 47 ++- .../model_v2/src/hummock_version_stats.rs | 26 +- src/meta/model_v2/src/lib.rs | 7 + src/meta/src/backup_restore/backup_manager.rs | 21 +- .../backup_restore/meta_snapshot_builder.rs | 21 +- .../meta_snapshot_builder_v2.rs | 122 ++++++++ src/meta/src/backup_restore/mod.rs | 1 + src/meta/src/backup_restore/restore.rs | 20 +- src/meta/src/hummock/compaction/mod.rs | 4 +- src/meta/src/hummock/model/ext/hummock.rs | 239 +++++++++++++++ src/meta/src/hummock/model/ext/mod.rs | 17 ++ src/meta/src/hummock/model/mod.rs | 1 + src/meta/src/hummock/vacuum.rs | 6 +- src/prost/build.rs | 18 ++ src/storage/backup/Cargo.toml | 2 + src/storage/backup/src/error.rs | 8 + src/storage/backup/src/lib.rs | 2 + src/storage/backup/src/meta_snapshot.rs | 260 ++--------------- src/storage/backup/src/meta_snapshot_v1.rs | 275 ++++++++++++++++++ src/storage/backup/src/meta_snapshot_v2.rs | 103 +++++++ src/storage/backup/src/storage.rs | 59 ++-- .../hummock_test/src/compactor_tests.rs | 8 +- src/storage/hummock_test/src/test_utils.rs | 2 +- src/storage/src/hummock/backup_reader.rs | 34 +-- src/workspace-hack/Cargo.toml | 1 - 31 files changed, 1041 insertions(+), 353 deletions(-) create mode 100644 src/meta/src/backup_restore/meta_snapshot_builder_v2.rs create mode 100644 src/meta/src/hummock/model/ext/hummock.rs create mode 100644 src/meta/src/hummock/model/ext/mod.rs create mode 100644 src/storage/backup/src/meta_snapshot_v1.rs create mode 100644 src/storage/backup/src/meta_snapshot_v2.rs diff --git a/Cargo.lock b/Cargo.lock index 213760cbaa635..568d22f0ae4e1 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -7081,12 +7081,14 @@ version = "1.3.0-alpha" dependencies = [ "anyhow", "async-trait", + "bincode 1.3.3", "bytes", "itertools 0.11.0", "parking_lot 0.12.1", "prost 0.12.1", "risingwave_common", "risingwave_hummock_sdk", + "risingwave_meta_model_v2", "risingwave_object_store", "risingwave_pb", "serde", @@ -11191,7 +11193,6 @@ dependencies = [ "clap_builder", "combine", "crossbeam-epoch", - "crossbeam-queue", "crossbeam-utils", "deranged", "digest", diff --git a/src/meta/model_v2/src/compaction_config.rs b/src/meta/model_v2/src/compaction_config.rs index 6f8345734586e..5236934411d95 100644 --- a/src/meta/model_v2/src/compaction_config.rs +++ b/src/meta/model_v2/src/compaction_config.rs @@ -12,18 +12,24 @@ // See the License for the specific language governing permissions and // limitations under the License. +use risingwave_pb::hummock::CompactionConfig as PbCompactionConfig; use sea_orm::entity::prelude::*; +use sea_orm::FromJsonQueryResult; +use serde::{Deserialize, Serialize}; -#[derive(Clone, Debug, PartialEq, DeriveEntityModel, Eq)] +use crate::CompactionGroupId; + +#[derive(Clone, Debug, PartialEq, DeriveEntityModel, Eq, Serialize, Deserialize, Default)] #[sea_orm(table_name = "compaction_config")] pub struct Model { #[sea_orm(primary_key, auto_increment = false)] - pub compaction_group_id: i64, - #[sea_orm(column_type = "JsonBinary", nullable)] - pub config: Option, + pub compaction_group_id: CompactionGroupId, + pub config: CompactionConfig, } #[derive(Copy, Clone, Debug, EnumIter, DeriveRelation)] pub enum Relation {} impl ActiveModelBehavior for ActiveModel {} + +crate::derive_from_json_struct!(CompactionConfig, PbCompactionConfig); diff --git a/src/meta/model_v2/src/compaction_status.rs b/src/meta/model_v2/src/compaction_status.rs index 5872463395066..da3f47ac8713b 100644 --- a/src/meta/model_v2/src/compaction_status.rs +++ b/src/meta/model_v2/src/compaction_status.rs @@ -12,18 +12,24 @@ // See the License for the specific language governing permissions and // limitations under the License. +use risingwave_pb::hummock::LevelHandler as PbLevelHandler; use sea_orm::entity::prelude::*; +use sea_orm::FromJsonQueryResult; +use serde::{Deserialize, Serialize}; + +use crate::CompactionGroupId; #[derive(Clone, Debug, PartialEq, DeriveEntityModel, Eq)] #[sea_orm(table_name = "compaction_status")] pub struct Model { #[sea_orm(primary_key, auto_increment = false)] - pub compaction_group_id: i64, - #[sea_orm(column_type = "JsonBinary", nullable)] - pub status: Option, + pub compaction_group_id: CompactionGroupId, + pub status: LevelHandlers, } #[derive(Copy, Clone, Debug, EnumIter, DeriveRelation)] pub enum Relation {} impl ActiveModelBehavior for ActiveModel {} + +crate::derive_from_json_struct!(LevelHandlers, Vec); diff --git a/src/meta/model_v2/src/compaction_task.rs b/src/meta/model_v2/src/compaction_task.rs index d3211b96d9a65..ede8ad8cfde33 100644 --- a/src/meta/model_v2/src/compaction_task.rs +++ b/src/meta/model_v2/src/compaction_task.rs @@ -12,19 +12,34 @@ // See the License for the specific language governing permissions and // limitations under the License. +use risingwave_pb::hummock::{CompactTask as PbCompactTask, CompactTaskAssignment}; use sea_orm::entity::prelude::*; +use sea_orm::FromJsonQueryResult; +use serde::{Deserialize, Serialize}; + +use crate::{CompactionTaskId, WorkerId}; #[derive(Clone, Debug, PartialEq, DeriveEntityModel, Eq)] #[sea_orm(table_name = "compaction_task")] pub struct Model { #[sea_orm(primary_key, auto_increment = false)] - pub id: i64, - #[sea_orm(column_type = "JsonBinary")] - pub task: Json, - pub context_id: i32, + pub id: CompactionTaskId, + pub task: CompactionTask, + pub context_id: WorkerId, } #[derive(Copy, Clone, Debug, EnumIter, DeriveRelation)] pub enum Relation {} impl ActiveModelBehavior for ActiveModel {} + +crate::derive_from_json_struct!(CompactionTask, PbCompactTask); + +impl From for CompactTaskAssignment { + fn from(value: Model) -> Self { + Self { + compact_task: Some(value.task.0), + context_id: value.context_id, + } + } +} diff --git a/src/meta/model_v2/src/hummock_pinned_snapshot.rs b/src/meta/model_v2/src/hummock_pinned_snapshot.rs index 170f35dd5d358..4d9a860c576f3 100644 --- a/src/meta/model_v2/src/hummock_pinned_snapshot.rs +++ b/src/meta/model_v2/src/hummock_pinned_snapshot.rs @@ -12,17 +12,30 @@ // See the License for the specific language governing permissions and // limitations under the License. +use risingwave_pb::hummock::HummockPinnedSnapshot; use sea_orm::entity::prelude::*; +use serde::{Deserialize, Serialize}; -#[derive(Clone, Debug, PartialEq, DeriveEntityModel, Eq)] +use crate::{Epoch, WorkerId}; + +#[derive(Clone, Debug, PartialEq, DeriveEntityModel, Eq, Serialize, Deserialize, Default)] #[sea_orm(table_name = "hummock_pinned_snapshot")] pub struct Model { #[sea_orm(primary_key, auto_increment = false)] - pub context_id: i32, - pub min_pinned_snapshot: i64, + pub context_id: WorkerId, + pub min_pinned_snapshot: Epoch, } #[derive(Copy, Clone, Debug, EnumIter, DeriveRelation)] pub enum Relation {} impl ActiveModelBehavior for ActiveModel {} + +impl From for HummockPinnedSnapshot { + fn from(value: Model) -> Self { + Self { + context_id: value.context_id, + minimal_pinned_snapshot: value.min_pinned_snapshot, + } + } +} diff --git a/src/meta/model_v2/src/hummock_pinned_version.rs b/src/meta/model_v2/src/hummock_pinned_version.rs index 6e2f34a5f735e..2f7ad9149d384 100644 --- a/src/meta/model_v2/src/hummock_pinned_version.rs +++ b/src/meta/model_v2/src/hummock_pinned_version.rs @@ -12,17 +12,30 @@ // See the License for the specific language governing permissions and // limitations under the License. +use risingwave_pb::hummock::HummockPinnedVersion; use sea_orm::entity::prelude::*; +use serde::{Deserialize, Serialize}; -#[derive(Clone, Debug, PartialEq, DeriveEntityModel, Eq)] +use crate::{HummockVersionId, WorkerId}; + +#[derive(Clone, Debug, PartialEq, DeriveEntityModel, Eq, Serialize, Deserialize, Default)] #[sea_orm(table_name = "hummock_pinned_version")] pub struct Model { #[sea_orm(primary_key, auto_increment = false)] - pub context_id: i32, - pub min_pinned_id: i64, + pub context_id: WorkerId, + pub min_pinned_id: HummockVersionId, } #[derive(Copy, Clone, Debug, EnumIter, DeriveRelation)] pub enum Relation {} impl ActiveModelBehavior for ActiveModel {} + +impl From for HummockPinnedVersion { + fn from(value: Model) -> Self { + Self { + context_id: value.context_id, + min_pinned_id: value.min_pinned_id, + } + } +} diff --git a/src/meta/model_v2/src/hummock_version_delta.rs b/src/meta/model_v2/src/hummock_version_delta.rs index 100dd82eafe94..4ce516f175343 100644 --- a/src/meta/model_v2/src/hummock_version_delta.rs +++ b/src/meta/model_v2/src/hummock_version_delta.rs @@ -12,24 +12,53 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::collections::HashMap; + +use risingwave_pb::hummock::{GroupDelta as PbGroupDelta, HummockVersionDelta}; use sea_orm::entity::prelude::*; +use sea_orm::FromJsonQueryResult; +use serde::{Deserialize, Serialize}; + +use crate::{CompactionGroupId, Epoch, HummockSstableObjectId, HummockVersionId}; -#[derive(Clone, Debug, PartialEq, DeriveEntityModel, Eq)] +#[derive(Clone, Debug, PartialEq, DeriveEntityModel, Eq, Serialize, Deserialize, Default)] #[sea_orm(table_name = "hummock_version_delta")] pub struct Model { #[sea_orm(primary_key, auto_increment = false)] - pub id: i64, - pub prev_id: i64, - #[sea_orm(column_type = "JsonBinary", nullable)] - pub group_deltas: Option, - pub max_committed_epoch: i64, - pub safe_epoch: i64, + pub id: HummockVersionId, + pub prev_id: HummockVersionId, + pub group_deltas: GroupDeltas, + pub max_committed_epoch: Epoch, + pub safe_epoch: Epoch, pub trivial_move: bool, - #[sea_orm(column_type = "JsonBinary", nullable)] - pub gc_object_ids: Option, + pub gc_object_ids: SstableObjectIds, } #[derive(Copy, Clone, Debug, EnumIter, DeriveRelation)] pub enum Relation {} impl ActiveModelBehavior for ActiveModel {} + +crate::derive_from_json_struct!(SstableObjectIds, Vec); + +crate::derive_from_json_struct!(GroupDeltas, HashMap>); + +impl From for HummockVersionDelta { + fn from(value: Model) -> Self { + use risingwave_pb::hummock::hummock_version_delta::GroupDeltas as PbGroupDeltas; + Self { + id: value.id, + prev_id: value.prev_id, + group_deltas: value + .group_deltas + .0 + .into_iter() + .map(|(cg_id, group_deltas)| (cg_id, PbGroupDeltas { group_deltas })) + .collect(), + max_committed_epoch: value.max_committed_epoch, + safe_epoch: value.safe_epoch, + trivial_move: value.trivial_move, + gc_object_ids: value.gc_object_ids.0, + } + } +} diff --git a/src/meta/model_v2/src/hummock_version_stats.rs b/src/meta/model_v2/src/hummock_version_stats.rs index 1a7e990df405a..fc16f29e33891 100644 --- a/src/meta/model_v2/src/hummock_version_stats.rs +++ b/src/meta/model_v2/src/hummock_version_stats.rs @@ -12,18 +12,36 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::collections::HashMap; + +use risingwave_pb::hummock::{HummockVersionStats, TableStats as PbTableStats}; use sea_orm::entity::prelude::*; +use sea_orm::FromJsonQueryResult; +use serde::{Deserialize, Serialize}; + +use crate::HummockVersionId; -#[derive(Clone, Debug, PartialEq, DeriveEntityModel, Eq)] +#[derive(Clone, Debug, PartialEq, DeriveEntityModel, Eq, Serialize, Deserialize, Default)] #[sea_orm(table_name = "hummock_version_stats")] pub struct Model { #[sea_orm(primary_key, auto_increment = false)] - pub id: i64, - #[sea_orm(column_type = "JsonBinary")] - pub stats: Json, + pub id: HummockVersionId, + pub stats: TableStats, } #[derive(Copy, Clone, Debug, EnumIter, DeriveRelation)] pub enum Relation {} impl ActiveModelBehavior for ActiveModel {} + +#[derive(Clone, Debug, PartialEq, Eq, FromJsonQueryResult, Serialize, Deserialize, Default)] +pub struct TableStats(pub HashMap); + +impl From for HummockVersionStats { + fn from(value: Model) -> Self { + Self { + hummock_version_id: value.id, + table_stats: value.stats.0, + } + } +} diff --git a/src/meta/model_v2/src/lib.rs b/src/meta/model_v2/src/lib.rs index 5b593300d51c1..ba2e114d5060d 100644 --- a/src/meta/model_v2/src/lib.rs +++ b/src/meta/model_v2/src/lib.rs @@ -61,6 +61,11 @@ pub type ViewId = ObjectId; pub type FunctionId = ObjectId; pub type ConnectionId = ObjectId; pub type UserId = u32; +pub type HummockVersionId = u64; +pub type Epoch = u64; +pub type CompactionGroupId = u64; +pub type CompactionTaskId = u64; +pub type HummockSstableObjectId = u64; pub type FragmentId = u32; @@ -121,6 +126,8 @@ macro_rules! derive_from_json_struct { }; } +pub(crate) use derive_from_json_struct; + derive_from_json_struct!(I32Array, Vec); derive_from_json_struct!(U32Array, Vec); diff --git a/src/meta/src/backup_restore/backup_manager.rs b/src/meta/src/backup_restore/backup_manager.rs index 819ea02e36346..2e957cca0a9ba 100644 --- a/src/meta/src/backup_restore/backup_manager.rs +++ b/src/meta/src/backup_restore/backup_manager.rs @@ -18,7 +18,7 @@ use std::time::Instant; use arc_swap::ArcSwap; use risingwave_backup::error::BackupError; -use risingwave_backup::storage::{BoxedMetaSnapshotStorage, ObjectStoreMetaSnapshotStorage}; +use risingwave_backup::storage::{MetaSnapshotStorage, ObjectStoreMetaSnapshotStorage}; use risingwave_backup::{MetaBackupJobId, MetaSnapshotId, MetaSnapshotManifest}; use risingwave_common::bail; use risingwave_hummock_sdk::HummockSstableObjectId; @@ -28,7 +28,7 @@ use risingwave_pb::backup_service::{BackupJobStatus, MetaBackupManifestId}; use risingwave_pb::meta::subscribe_response::{Info, Operation}; use tokio::task::JoinHandle; -use crate::backup_restore::meta_snapshot_builder::MetaSnapshotBuilder; +use crate::backup_restore::meta_snapshot_builder; use crate::backup_restore::metrics::BackupManagerMetrics; use crate::hummock::{HummockManagerRef, HummockVersionSafePoint}; use crate::manager::{IdCategory, LocalNotification, MetaSrvEnv}; @@ -66,7 +66,7 @@ type StoreConfig = (String, String); pub struct BackupManager { env: MetaSrvEnv, hummock_manager: HummockManagerRef, - backup_store: ArcSwap<(BoxedMetaSnapshotStorage, StoreConfig)>, + backup_store: ArcSwap<(ObjectStoreMetaSnapshotStorage, StoreConfig)>, /// Tracks the running backup job. Concurrent jobs is not supported. running_job_handle: tokio::sync::Mutex>, metrics: BackupManagerMetrics, @@ -143,7 +143,7 @@ impl BackupManager { env: MetaSrvEnv, hummock_manager: HummockManagerRef, meta_metrics: Arc, - backup_store: (BoxedMetaSnapshotStorage, StoreConfig), + backup_store: (ObjectStoreMetaSnapshotStorage, StoreConfig), ) -> Self { Self { env, @@ -169,13 +169,13 @@ impl BackupManager { } #[cfg(test)] - pub fn for_test(env: MetaSrvEnv, hummock_manager: HummockManagerRef) -> Self { + pub async fn for_test(env: MetaSrvEnv, hummock_manager: HummockManagerRef) -> Self { Self::with_store( env, hummock_manager, Arc::new(MetaMetrics::default()), ( - Box::::default(), + risingwave_backup::storage::unused().await, StoreConfig::default(), ), ) @@ -334,8 +334,9 @@ impl BackupWorker { fn start(self, job_id: u64) -> JoinHandle<()> { let backup_manager_clone = self.backup_manager.clone(); let job = async move { - let mut snapshot_builder = - MetaSnapshotBuilder::new(backup_manager_clone.env.meta_store_ref()); + let mut snapshot_builder = meta_snapshot_builder::MetaSnapshotV1Builder::new( + backup_manager_clone.env.meta_store_ref(), + ); // Reuse job id as snapshot id. let hummock_manager = backup_manager_clone.hummock_manager.clone(); snapshot_builder @@ -364,8 +365,8 @@ impl BackupWorker { async fn create_snapshot_store( config: &StoreConfig, metric: Arc, -) -> MetaResult { +) -> MetaResult { let object_store = Arc::new(parse_remote_object_store(&config.0, metric, "Meta Backup").await); let store = ObjectStoreMetaSnapshotStorage::new(&config.1, object_store).await?; - Ok(Box::new(store)) + Ok(store) } diff --git a/src/meta/src/backup_restore/meta_snapshot_builder.rs b/src/meta/src/backup_restore/meta_snapshot_builder.rs index ef98c1158fd2a..0933eb4d16255 100644 --- a/src/meta/src/backup_restore/meta_snapshot_builder.rs +++ b/src/meta/src/backup_restore/meta_snapshot_builder.rs @@ -17,7 +17,7 @@ use std::future::Future; use anyhow::anyhow; use risingwave_backup::error::{BackupError, BackupResult}; -use risingwave_backup::meta_snapshot::{ClusterMetadata, MetaSnapshot}; +use risingwave_backup::meta_snapshot_v1::{ClusterMetadata, MetaSnapshotV1}; use risingwave_backup::MetaSnapshotId; use risingwave_hummock_sdk::compaction_group::hummock_version_ext::HummockVersionUpdateExt; use risingwave_pb::catalog::{ @@ -33,15 +33,15 @@ use crate::storage::{MetaStore, Snapshot, DEFAULT_COLUMN_FAMILY}; const VERSION: u32 = 1; -pub struct MetaSnapshotBuilder { - snapshot: MetaSnapshot, +pub struct MetaSnapshotV1Builder { + snapshot: MetaSnapshotV1, meta_store: S, } -impl MetaSnapshotBuilder { +impl MetaSnapshotV1Builder { pub fn new(meta_store: S) -> Self { Self { - snapshot: MetaSnapshot::default(), + snapshot: MetaSnapshotV1::default(), meta_store, } } @@ -146,7 +146,7 @@ impl MetaSnapshotBuilder { Ok(()) } - pub fn finish(self) -> BackupResult { + pub fn finish(self) -> BackupResult { // Any sanity check goes here. Ok(self.snapshot) } @@ -168,16 +168,19 @@ mod tests { use assert_matches::assert_matches; use itertools::Itertools; use risingwave_backup::error::BackupError; - use risingwave_backup::meta_snapshot::MetaSnapshot; + use risingwave_backup::meta_snapshot_v1::MetaSnapshotV1; use risingwave_common::error::ToErrorStr; use risingwave_common::system_param::system_params_for_test; use risingwave_pb::hummock::{HummockVersion, HummockVersionStats}; - use crate::backup_restore::meta_snapshot_builder::MetaSnapshotBuilder; + use crate::backup_restore::meta_snapshot_builder; use crate::manager::model::SystemParamsModel; use crate::model::{ClusterId, MetadataModel}; use crate::storage::{MemStore, MetaStore, DEFAULT_COLUMN_FAMILY}; + type MetaSnapshot = MetaSnapshotV1; + type MetaSnapshotBuilder = meta_snapshot_builder::MetaSnapshotV1Builder; + #[tokio::test] async fn test_snapshot_builder() { let meta_store = MemStore::new(); @@ -244,7 +247,7 @@ mod tests { .await .unwrap(); let snapshot = builder.finish().unwrap(); - let encoded = snapshot.encode(); + let encoded = snapshot.encode().unwrap(); let decoded = MetaSnapshot::decode(&encoded).unwrap(); assert_eq!(snapshot, decoded); assert_eq!(snapshot.id, 1); diff --git a/src/meta/src/backup_restore/meta_snapshot_builder_v2.rs b/src/meta/src/backup_restore/meta_snapshot_builder_v2.rs new file mode 100644 index 0000000000000..95a9acac1fdb6 --- /dev/null +++ b/src/meta/src/backup_restore/meta_snapshot_builder_v2.rs @@ -0,0 +1,122 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#![expect(dead_code, reason = "WIP")] + +use std::future::Future; + +use itertools::Itertools; +use risingwave_backup::error::{BackupError, BackupResult}; +use risingwave_backup::meta_snapshot_v2::{MetaSnapshotV2, MetadataV2}; +use risingwave_backup::MetaSnapshotId; +use risingwave_hummock_sdk::compaction_group::hummock_version_ext::HummockVersionUpdateExt; +use risingwave_meta_model_v2 as model_v2; +use risingwave_pb::hummock::{HummockVersion, HummockVersionDelta}; +use sea_orm::{EntityTrait, QueryOrder, TransactionTrait}; + +use crate::controller::SqlMetaStore; + +const VERSION: u32 = 2; + +pub struct MetaSnapshotV2Builder { + snapshot: MetaSnapshotV2, + meta_store: SqlMetaStore, +} + +impl MetaSnapshotV2Builder { + pub fn new(meta_store: SqlMetaStore) -> Self { + Self { + snapshot: MetaSnapshotV2::default(), + meta_store, + } + } + + pub async fn build>( + &mut self, + id: MetaSnapshotId, + hummock_version_builder: D, + ) -> BackupResult<()> { + self.snapshot.format_version = VERSION; + self.snapshot.id = id; + // Get `hummock_version` before `meta_store_snapshot`. + // We have ensure the required delta logs for replay is available, see + // `HummockManager::delete_version_deltas`. + let hummock_version = hummock_version_builder.await; + let txn = self + .meta_store + .conn + .begin_with_config( + Some(sea_orm::IsolationLevel::Serializable), + Some(sea_orm::AccessMode::ReadOnly), + ) + .await + .map_err(|e| BackupError::MetaStorage(e.into()))?; + let version_deltas = model_v2::prelude::HummockVersionDelta::find() + .order_by_asc(model_v2::hummock_version_delta::Column::Id) + .all(&txn) + .await + .map_err(|e| BackupError::MetaStorage(e.into()))? + .into_iter() + .map_into::(); + let hummock_version = { + let mut redo_state = hummock_version; + let mut max_log_id = None; + for version_delta in version_deltas { + if version_delta.prev_id == redo_state.id { + redo_state.apply_version_delta(&version_delta); + } + max_log_id = Some(version_delta.id); + } + if let Some(max_log_id) = max_log_id { + if max_log_id != redo_state.id { + return Err(BackupError::Other(anyhow::anyhow!(format!( + "inconsistent hummock version: expected {}, actual {}", + max_log_id, redo_state.id + )))); + } + } + redo_state + }; + let version_stats = model_v2::prelude::HummockVersionStats::find_by_id(hummock_version.id) + .one(&txn) + .await + .map_err(|e| BackupError::MetaStorage(e.into()))? + .unwrap_or_else(|| { + panic!("version stats for version {} not found", hummock_version.id) + }); + let compaction_configs = model_v2::prelude::CompactionConfig::find() + .all(&txn) + .await + .map_err(|e| BackupError::MetaStorage(e.into()))?; + + // TODO: other metadata + let cluster_id = "TODO".to_string(); + + txn.commit() + .await + .map_err(|e| BackupError::MetaStorage(e.into()))?; + self.snapshot.metadata = MetadataV2 { + cluster_id, + hummock_version, + version_stats, + compaction_configs, + }; + Ok(()) + } + + pub fn finish(self) -> BackupResult { + // Any sanity check goes here. + Ok(self.snapshot) + } +} diff --git a/src/meta/src/backup_restore/mod.rs b/src/meta/src/backup_restore/mod.rs index 4115929f01729..0dfe5b3442415 100644 --- a/src/meta/src/backup_restore/mod.rs +++ b/src/meta/src/backup_restore/mod.rs @@ -16,6 +16,7 @@ mod backup_manager; pub use backup_manager::*; mod error; mod meta_snapshot_builder; +mod meta_snapshot_builder_v2; mod metrics; mod restore; mod utils; diff --git a/src/meta/src/backup_restore/restore.rs b/src/meta/src/backup_restore/restore.rs index ab4696e62f9bd..48dd6fc0cb664 100644 --- a/src/meta/src/backup_restore/restore.rs +++ b/src/meta/src/backup_restore/restore.rs @@ -16,8 +16,8 @@ use std::sync::Arc; use itertools::Itertools; use risingwave_backup::error::{BackupError, BackupResult}; -use risingwave_backup::meta_snapshot::MetaSnapshot; -use risingwave_backup::storage::MetaSnapshotStorageRef; +use risingwave_backup::meta_snapshot_v1::MetaSnapshotV1; +use risingwave_backup::storage::{MetaSnapshotStorage, MetaSnapshotStorageRef}; use risingwave_common::config::MetaBackend; use risingwave_hummock_sdk::version_checkpoint_path; use risingwave_object_store::object::object_metrics::ObjectStoreMetrics; @@ -137,7 +137,7 @@ async fn restore_cluster_id( async fn restore_default_cf( meta_store: &S, - snapshot: &MetaSnapshot, + snapshot: &MetaSnapshotV1, ) -> BackupResult<()> { if !meta_store.list_cf(DEFAULT_COLUMN_FAMILY).await?.is_empty() { return Err(BackupError::NonemptyMetaStorage); @@ -150,7 +150,10 @@ async fn restore_default_cf( Ok(()) } -async fn restore_metadata(meta_store: S, snapshot: MetaSnapshot) -> BackupResult<()> { +async fn restore_metadata( + meta_store: S, + snapshot: MetaSnapshotV1, +) -> BackupResult<()> { restore_default_cf(&meta_store, &snapshot).await?; restore_metadata_model(&meta_store, &[snapshot.metadata.version_stats]).await?; restore_metadata_model( @@ -216,7 +219,7 @@ async fn restore_impl( target_id ))); } - let mut target_snapshot = backup_store.get(target_id).await?; + let mut target_snapshot: MetaSnapshotV1 = backup_store.get(target_id).await?; tracing::info!( "snapshot {} before rewrite:\n{}", target_id, @@ -236,7 +239,7 @@ async fn restore_impl( // - Value is memcomparable. // - Keys of newest_snapshot is a superset of that of target_snapshot. if newest_id > target_id { - let newest_snapshot = backup_store.get(newest_id).await?; + let newest_snapshot: MetaSnapshotV1 = backup_store.get(newest_id).await?; for (k, v) in &target_snapshot.metadata.default_cf { let newest_v = newest_snapshot .metadata @@ -287,7 +290,8 @@ mod tests { use std::collections::HashMap; use itertools::Itertools; - use risingwave_backup::meta_snapshot::{ClusterMetadata, MetaSnapshot}; + use risingwave_backup::meta_snapshot_v1::{ClusterMetadata, MetaSnapshotV1}; + use risingwave_backup::storage::MetaSnapshotStorage; use risingwave_common::config::{MetaBackend, SystemConfig}; use risingwave_pb::hummock::{HummockVersion, HummockVersionStats}; use risingwave_pb::meta::SystemParams; @@ -300,6 +304,8 @@ mod tests { use crate::model::MetadataModel; use crate::storage::{MetaStore, DEFAULT_COLUMN_FAMILY}; + type MetaSnapshot = MetaSnapshotV1; + fn get_restore_opts() -> RestoreOpts { RestoreOpts { meta_snapshot_id: 1, diff --git a/src/meta/src/hummock/compaction/mod.rs b/src/meta/src/hummock/compaction/mod.rs index a056414034243..aa4d16a4e1dcf 100644 --- a/src/meta/src/hummock/compaction/mod.rs +++ b/src/meta/src/hummock/compaction/mod.rs @@ -23,7 +23,7 @@ use risingwave_pb::hummock::compact_task::{self, TaskStatus, TaskType}; mod picker; pub mod selector; -use std::collections::{HashMap, HashSet}; +use std::collections::{BTreeMap, HashMap, HashSet}; use std::fmt::{Debug, Formatter}; use std::sync::Arc; @@ -145,7 +145,7 @@ impl CompactStatus { compression_algorithm, target_file_size: ret.target_file_size, compaction_filter_mask: 0, - table_options: HashMap::default(), + table_options: BTreeMap::default(), current_epoch_time: 0, target_sub_level_id: ret.input.target_sub_level_id, task_type: ret.compaction_task_type as i32, diff --git a/src/meta/src/hummock/model/ext/hummock.rs b/src/meta/src/hummock/model/ext/hummock.rs new file mode 100644 index 0000000000000..c2834480cca40 --- /dev/null +++ b/src/meta/src/hummock/model/ext/hummock.rs @@ -0,0 +1,239 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use itertools::Itertools; +use risingwave_meta_model_v2::compaction_config::CompactionConfig; +use risingwave_meta_model_v2::compaction_status::LevelHandlers; +use risingwave_meta_model_v2::compaction_task::CompactionTask; +use risingwave_meta_model_v2::hummock_version_delta::{GroupDeltas, SstableObjectIds}; +use risingwave_meta_model_v2::{ + compaction_config, compaction_status, compaction_task, hummock_pinned_snapshot, + hummock_pinned_version, hummock_version_delta, +}; +use risingwave_pb::hummock::{ + CompactTaskAssignment, HummockPinnedSnapshot, HummockPinnedVersion, HummockVersionDelta, +}; +use sea_orm::sea_query::OnConflict; +use sea_orm::ActiveValue::Set; +use sea_orm::EntityTrait; + +use crate::hummock::compaction::CompactStatus; +use crate::hummock::model::CompactionGroup; +use crate::model::{MetadataModelError, MetadataModelResult, Transactional}; +use crate::storage::MetaStoreError; + +pub type Transaction = sea_orm::DatabaseTransaction; + +impl From for MetadataModelError { + fn from(err: sea_orm::DbErr) -> Self { + MetadataModelError::MetaStoreError(MetaStoreError::Internal(err.into())) + } +} + +// TODO: reduce boilerplate code + +#[async_trait::async_trait] +impl Transactional for CompactionGroup { + async fn upsert_in_transaction(&self, trx: &mut Transaction) -> MetadataModelResult<()> { + let m = compaction_config::ActiveModel { + compaction_group_id: Set(self.group_id), + config: Set(CompactionConfig((*self.compaction_config).to_owned())), + }; + compaction_config::Entity::insert(m) + .on_conflict( + OnConflict::column(compaction_config::Column::CompactionGroupId) + .update_columns([compaction_config::Column::Config]) + .to_owned(), + ) + .exec(trx) + .await?; + Ok(()) + } + + async fn delete_in_transaction(&self, trx: &mut Transaction) -> MetadataModelResult<()> { + compaction_config::Entity::delete_by_id(self.group_id) + .exec(trx) + .await?; + Ok(()) + } +} + +#[async_trait::async_trait] +impl Transactional for CompactStatus { + async fn upsert_in_transaction(&self, trx: &mut Transaction) -> MetadataModelResult<()> { + let m = compaction_status::ActiveModel { + compaction_group_id: Set(self.compaction_group_id), + status: Set(LevelHandlers( + self.level_handlers.iter().map_into().collect(), + )), + }; + compaction_status::Entity::insert(m) + .on_conflict( + OnConflict::column(compaction_status::Column::CompactionGroupId) + .update_columns([compaction_status::Column::Status]) + .to_owned(), + ) + .exec(trx) + .await?; + Ok(()) + } + + async fn delete_in_transaction(&self, trx: &mut Transaction) -> MetadataModelResult<()> { + compaction_status::Entity::delete_by_id(self.compaction_group_id) + .exec(trx) + .await?; + Ok(()) + } +} + +#[async_trait::async_trait] +impl Transactional for CompactTaskAssignment { + async fn upsert_in_transaction(&self, trx: &mut Transaction) -> MetadataModelResult<()> { + let task = self.compact_task.to_owned().unwrap(); + let m = compaction_task::ActiveModel { + id: Set(task.task_id), + context_id: Set(self.context_id), + task: Set(CompactionTask(task)), + }; + compaction_task::Entity::insert(m) + .on_conflict( + OnConflict::column(compaction_task::Column::Id) + .update_columns([ + compaction_task::Column::ContextId, + compaction_task::Column::Task, + ]) + .to_owned(), + ) + .exec(trx) + .await?; + Ok(()) + } + + async fn delete_in_transaction(&self, trx: &mut Transaction) -> MetadataModelResult<()> { + compaction_task::Entity::delete_by_id(self.compact_task.as_ref().unwrap().task_id) + .exec(trx) + .await?; + Ok(()) + } +} + +#[async_trait::async_trait] +impl Transactional for HummockPinnedVersion { + async fn upsert_in_transaction(&self, trx: &mut Transaction) -> MetadataModelResult<()> { + let m = hummock_pinned_version::ActiveModel { + context_id: Set(self.context_id), + min_pinned_id: Set(self.min_pinned_id), + }; + hummock_pinned_version::Entity::insert(m) + .on_conflict( + OnConflict::column(hummock_pinned_version::Column::ContextId) + .update_columns([hummock_pinned_version::Column::MinPinnedId]) + .to_owned(), + ) + .exec(trx) + .await?; + Ok(()) + } + + async fn delete_in_transaction(&self, trx: &mut Transaction) -> MetadataModelResult<()> { + hummock_pinned_version::Entity::delete_by_id(self.context_id) + .exec(trx) + .await?; + Ok(()) + } +} + +#[async_trait::async_trait] +impl Transactional for HummockPinnedSnapshot { + async fn upsert_in_transaction(&self, trx: &mut Transaction) -> MetadataModelResult<()> { + let m = hummock_pinned_snapshot::ActiveModel { + context_id: Set(self.context_id), + min_pinned_snapshot: Set(self.minimal_pinned_snapshot), + }; + hummock_pinned_snapshot::Entity::insert(m) + .on_conflict( + OnConflict::column(hummock_pinned_snapshot::Column::ContextId) + .update_columns([hummock_pinned_snapshot::Column::MinPinnedSnapshot]) + .to_owned(), + ) + .exec(trx) + .await?; + Ok(()) + } + + async fn delete_in_transaction(&self, trx: &mut Transaction) -> MetadataModelResult<()> { + hummock_pinned_snapshot::Entity::delete_by_id(self.context_id) + .exec(trx) + .await?; + Ok(()) + } +} + +#[async_trait::async_trait] +impl Transactional for HummockVersionDelta { + async fn upsert_in_transaction(&self, trx: &mut Transaction) -> MetadataModelResult<()> { + let m = hummock_version_delta::ActiveModel { + id: Set(self.id), + prev_id: Set(self.prev_id), + group_deltas: Set(GroupDeltas( + self.group_deltas + .iter() + .map(|(k, v)| (*k, v.group_deltas.to_owned())) + .collect(), + )), + max_committed_epoch: Set(self.max_committed_epoch), + safe_epoch: Set(self.safe_epoch), + trivial_move: Set(self.trivial_move), + gc_object_ids: Set(SstableObjectIds(self.gc_object_ids.to_owned())), + }; + hummock_version_delta::Entity::insert(m) + .on_conflict( + OnConflict::column(hummock_version_delta::Column::Id) + .update_columns([ + hummock_version_delta::Column::PrevId, + hummock_version_delta::Column::GroupDeltas, + hummock_version_delta::Column::MaxCommittedEpoch, + hummock_version_delta::Column::SafeEpoch, + hummock_version_delta::Column::TrivialMove, + hummock_version_delta::Column::GcObjectIds, + ]) + .to_owned(), + ) + .exec(trx) + .await?; + Ok(()) + } + + async fn delete_in_transaction(&self, trx: &mut Transaction) -> MetadataModelResult<()> { + hummock_version_delta::Entity::delete_by_id(self.id) + .exec(trx) + .await?; + Ok(()) + } +} + +impl From for CompactionGroup { + fn from(value: compaction_config::Model) -> Self { + Self::new(value.compaction_group_id, value.config.0) + } +} + +impl From for CompactStatus { + fn from(value: compaction_status::Model) -> Self { + Self { + compaction_group_id: value.compaction_group_id, + level_handlers: value.status.0.iter().map_into().collect(), + } + } +} diff --git a/src/meta/src/hummock/model/ext/mod.rs b/src/meta/src/hummock/model/ext/mod.rs new file mode 100644 index 0000000000000..d22d06d86d93b --- /dev/null +++ b/src/meta/src/hummock/model/ext/mod.rs @@ -0,0 +1,17 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +mod hummock; + +pub use hummock::*; diff --git a/src/meta/src/hummock/model/mod.rs b/src/meta/src/hummock/model/mod.rs index 66c12d90836b9..5c9880c6d1244 100644 --- a/src/meta/src/hummock/model/mod.rs +++ b/src/meta/src/hummock/model/mod.rs @@ -15,6 +15,7 @@ mod compact_task_assignment; mod compaction_group_config; mod compaction_status; +pub mod ext; mod pinned_snapshot; mod pinned_version; mod version_delta; diff --git a/src/meta/src/hummock/vacuum.rs b/src/meta/src/hummock/vacuum.rs index 992deb5e636ce..c17591a52c149 100644 --- a/src/meta/src/hummock/vacuum.rs +++ b/src/meta/src/hummock/vacuum.rs @@ -219,10 +219,8 @@ mod tests { let (env, hummock_manager, _cluster_manager, worker_node) = setup_compute_env(80).await; let context_id = worker_node.id; let compactor_manager = hummock_manager.compactor_manager_ref_for_test(); - let backup_manager = Arc::new(BackupManager::for_test( - env.clone(), - hummock_manager.clone(), - )); + let backup_manager = + Arc::new(BackupManager::for_test(env.clone(), hummock_manager.clone()).await); let vacuum = Arc::new(VacuumManager::new( env, hummock_manager.clone(), diff --git a/src/prost/build.rs b/src/prost/build.rs index e446518b7a6f1..dcc4257e627b4 100644 --- a/src/prost/build.rs +++ b/src/prost/build.rs @@ -62,6 +62,7 @@ fn main() -> Result<(), Box> { let btree_map_paths = [ ".monitor_service.StackTraceResponse", ".plan_common.ExternalTableDesc", + ".hummock.CompactTask", ]; // Build protobuf structs. @@ -115,6 +116,23 @@ fn main() -> Result<(), Box> { .type_attribute("plan_common.ColumnDesc", "#[derive(Eq, Hash)]") .type_attribute("common.ColumnOrder", "#[derive(Eq, Hash)]") .type_attribute("common.OrderType", "#[derive(Eq, Hash)]") + // Eq is required to derive `FromJsonQueryResult` for models in risingwave_meta_model_v2. + .type_attribute("hummock.TableStats", "#[derive(Eq)]") + .type_attribute("hummock.SstableInfo", "#[derive(Eq)]") + .type_attribute("hummock.KeyRange", "#[derive(Eq)]") + .type_attribute("hummock.CompactionConfig", "#[derive(Eq)]") + .type_attribute("hummock.GroupDelta.delta_type", "#[derive(Eq)]") + .type_attribute("hummock.IntraLevelDelta", "#[derive(Eq)]") + .type_attribute("hummock.GroupConstruct", "#[derive(Eq)]") + .type_attribute("hummock.GroupDestroy", "#[derive(Eq)]") + .type_attribute("hummock.GroupMetaChange", "#[derive(Eq)]") + .type_attribute("hummock.GroupTableChange", "#[derive(Eq)]") + .type_attribute("hummock.GroupDelta", "#[derive(Eq)]") + .type_attribute("hummock.LevelHandler.RunningCompactTask", "#[derive(Eq)]") + .type_attribute("hummock.LevelHandler", "#[derive(Eq)]") + .type_attribute("hummock.TableOption", "#[derive(Eq)]") + .type_attribute("hummock.InputLevel", "#[derive(Eq)]") + .type_attribute("hummock.CompactTask", "#[derive(Eq)]") // =================== .out_dir(out_dir.as_path()) .compile(&protos, &[proto_dir.to_string()]) diff --git a/src/storage/backup/Cargo.toml b/src/storage/backup/Cargo.toml index f4f66927c33d4..61e98f584eb91 100644 --- a/src/storage/backup/Cargo.toml +++ b/src/storage/backup/Cargo.toml @@ -16,12 +16,14 @@ normal = ["workspace-hack"] [dependencies] anyhow = "1" async-trait = "0.1" +bincode = "1.3" bytes = { version = "1", features = ["serde"] } itertools = "0.11" parking_lot = { version = "0.12", features = ["arc_lock"] } prost = { workspace = true } risingwave_common = { workspace = true } risingwave_hummock_sdk = { workspace = true } +risingwave_meta_model_v2 = { workspace = true } risingwave_object_store = { workspace = true } risingwave_pb = { workspace = true } serde = { version = "1", features = ["derive"] } diff --git a/src/storage/backup/src/error.rs b/src/storage/backup/src/error.rs index 2ed162bae9e12..4cc1faaa12070 100644 --- a/src/storage/backup/src/error.rs +++ b/src/storage/backup/src/error.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use bincode::Error; use risingwave_common::error::BoxedError; use thiserror::Error; @@ -56,3 +57,10 @@ pub enum BackupError { #[error(transparent)] Other(#[from] anyhow::Error), } + +impl From for BackupError { + fn from(value: Error) -> Self { + // TODO: match error + BackupError::Other(value.into()) + } +} diff --git a/src/storage/backup/src/lib.rs b/src/storage/backup/src/lib.rs index 1daacbf691c0d..af3c989995b9a 100644 --- a/src/storage/backup/src/lib.rs +++ b/src/storage/backup/src/lib.rs @@ -29,6 +29,8 @@ pub mod error; pub mod meta_snapshot; +pub mod meta_snapshot_v1; +pub mod meta_snapshot_v2; pub mod storage; use std::collections::HashSet; diff --git a/src/storage/backup/src/meta_snapshot.rs b/src/storage/backup/src/meta_snapshot.rs index 38d595be4f10a..c42fcc5d5851f 100644 --- a/src/storage/backup/src/meta_snapshot.rs +++ b/src/storage/backup/src/meta_snapshot.rs @@ -12,39 +12,43 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::HashMap; use std::fmt::{Display, Formatter}; use bytes::{Buf, BufMut}; -use itertools::Itertools; -use risingwave_common::util::iter_util::ZipEqFast; -use risingwave_pb::catalog::{ - Connection, Database, Function, Index, Schema, Sink, Source, Table, View, -}; -use risingwave_pb::hummock::{CompactionGroup, HummockVersion, HummockVersionStats}; -use risingwave_pb::meta::{SystemParams, TableFragments}; -use risingwave_pb::user::UserInfo; +use risingwave_pb::hummock::HummockVersion; -use crate::error::{BackupError, BackupResult}; +use crate::error::BackupResult; use crate::{xxhash64_checksum, xxhash64_verify, MetaSnapshotId}; +pub trait Metadata: Display + Send + Sync { + fn encode_to(&self, buf: &mut Vec) -> BackupResult<()>; + + fn decode(buf: &[u8]) -> BackupResult + where + Self: Sized; + + fn hummock_version_ref(&self) -> &HummockVersion; + + fn hummock_version(self) -> HummockVersion; +} + #[derive(Debug, Default, Clone, PartialEq)] -pub struct MetaSnapshot { +pub struct MetaSnapshot { pub format_version: u32, pub id: MetaSnapshotId, /// Snapshot of meta store. - pub metadata: ClusterMetadata, + pub metadata: T, } -impl MetaSnapshot { - pub fn encode(&self) -> Vec { +impl MetaSnapshot { + pub fn encode(&self) -> BackupResult> { let mut buf = vec![]; buf.put_u32_le(self.format_version); buf.put_u64_le(self.id); - self.metadata.encode_to(&mut buf); + self.metadata.encode_to(&mut buf)?; let checksum = xxhash64_checksum(&buf); buf.put_u64_le(checksum); - buf + Ok(buf) } pub fn decode(mut buf: &[u8]) -> BackupResult { @@ -52,231 +56,27 @@ impl MetaSnapshot { xxhash64_verify(&buf[..buf.len() - 8], checksum)?; let format_version = buf.get_u32_le(); let id = buf.get_u64_le(); - let metadata = ClusterMetadata::decode(buf)?; + let metadata = T::decode(buf)?; Ok(Self { format_version, id, metadata, }) } + + pub fn decode_format_version(mut buf: &[u8]) -> BackupResult { + let checksum = (&buf[buf.len() - 8..]).get_u64_le(); + xxhash64_verify(&buf[..buf.len() - 8], checksum)?; + let format_version = buf.get_u32_le(); + Ok(format_version) + } } -impl Display for MetaSnapshot { +impl Display for MetaSnapshot { fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { writeln!(f, "format_version: {}", self.format_version)?; writeln!(f, "id: {}", self.id)?; - writeln!(f, "default_cf:")?; - for (k, v) in &self.metadata.default_cf { - let key = String::from_utf8(k.clone()).unwrap(); - writeln!(f, "{} {:x?}", key, v)?; - } - writeln!(f, "hummock_version:")?; - writeln!(f, "{:#?}", self.metadata.hummock_version)?; - writeln!(f, "version_stats:")?; - writeln!(f, "{:#?}", self.metadata.version_stats)?; - writeln!(f, "compaction_groups:")?; - writeln!(f, "{:#?}", self.metadata.compaction_groups)?; - writeln!(f, "database:")?; - writeln!(f, "{:#?}", self.metadata.database)?; - writeln!(f, "schema:")?; - writeln!(f, "{:#?}", self.metadata.schema)?; - writeln!(f, "table:")?; - writeln!(f, "{:#?}", self.metadata.table)?; - writeln!(f, "index:")?; - writeln!(f, "{:#?}", self.metadata.index)?; - writeln!(f, "sink:")?; - writeln!(f, "{:#?}", self.metadata.sink)?; - writeln!(f, "source:")?; - writeln!(f, "{:#?}", self.metadata.source)?; - writeln!(f, "view:")?; - writeln!(f, "{:#?}", self.metadata.view)?; - writeln!(f, "connection:")?; - writeln!(f, "{:#?}", self.metadata.connection)?; - writeln!(f, "table_fragments:")?; - writeln!(f, "{:#?}", self.metadata.table_fragments)?; - writeln!(f, "user_info:")?; - writeln!(f, "{:#?}", self.metadata.user_info)?; - writeln!(f, "function:")?; - writeln!(f, "{:#?}", self.metadata.function)?; - writeln!(f, "system_param:")?; - writeln!(f, "{:#?}", self.metadata.system_param)?; - writeln!(f, "cluster_id:")?; - writeln!(f, "{:#?}", self.metadata.cluster_id)?; + writeln!(f, "{}", self.metadata)?; Ok(()) } } - -/// For backward compatibility, never remove fields and only append new field. -#[derive(Debug, Default, Clone, PartialEq)] -pub struct ClusterMetadata { - /// Unlike other metadata that has implemented `MetadataModel`, - /// DEFAULT_COLUMN_FAMILY stores various single row metadata, e.g. id offset and epoch offset. - /// So we use `default_cf` stores raw KVs for them. - pub default_cf: HashMap, Vec>, - pub hummock_version: HummockVersion, - pub version_stats: HummockVersionStats, - pub compaction_groups: Vec, - pub database: Vec, - pub schema: Vec, - pub table: Vec, - pub index: Vec, - pub sink: Vec, - pub source: Vec, - pub view: Vec, - pub table_fragments: Vec, - pub user_info: Vec, - pub function: Vec, - pub connection: Vec, - pub system_param: SystemParams, - pub cluster_id: String, -} - -impl ClusterMetadata { - pub fn encode_to(&self, buf: &mut Vec) { - let default_cf_keys = self.default_cf.keys().collect_vec(); - let default_cf_values = self.default_cf.values().collect_vec(); - Self::encode_prost_message_list(&default_cf_keys, buf); - Self::encode_prost_message_list(&default_cf_values, buf); - Self::encode_prost_message(&self.hummock_version, buf); - Self::encode_prost_message(&self.version_stats, buf); - Self::encode_prost_message_list(&self.compaction_groups.iter().collect_vec(), buf); - Self::encode_prost_message_list(&self.table_fragments.iter().collect_vec(), buf); - Self::encode_prost_message_list(&self.user_info.iter().collect_vec(), buf); - Self::encode_prost_message_list(&self.database.iter().collect_vec(), buf); - Self::encode_prost_message_list(&self.schema.iter().collect_vec(), buf); - Self::encode_prost_message_list(&self.table.iter().collect_vec(), buf); - Self::encode_prost_message_list(&self.index.iter().collect_vec(), buf); - Self::encode_prost_message_list(&self.sink.iter().collect_vec(), buf); - Self::encode_prost_message_list(&self.source.iter().collect_vec(), buf); - Self::encode_prost_message_list(&self.view.iter().collect_vec(), buf); - Self::encode_prost_message_list(&self.function.iter().collect_vec(), buf); - Self::encode_prost_message_list(&self.connection.iter().collect_vec(), buf); - Self::encode_prost_message(&self.system_param, buf); - Self::encode_prost_message(&self.cluster_id, buf); - } - - pub fn decode(mut buf: &[u8]) -> BackupResult { - let default_cf_keys: Vec> = Self::decode_prost_message_list(&mut buf)?; - let default_cf_values: Vec> = Self::decode_prost_message_list(&mut buf)?; - let default_cf = default_cf_keys - .into_iter() - .zip_eq_fast(default_cf_values.into_iter()) - .collect(); - let hummock_version = Self::decode_prost_message(&mut buf)?; - let version_stats = Self::decode_prost_message(&mut buf)?; - let compaction_groups: Vec = Self::decode_prost_message_list(&mut buf)?; - let table_fragments: Vec = Self::decode_prost_message_list(&mut buf)?; - let user_info: Vec = Self::decode_prost_message_list(&mut buf)?; - let database: Vec = Self::decode_prost_message_list(&mut buf)?; - let schema: Vec = Self::decode_prost_message_list(&mut buf)?; - let table: Vec
= Self::decode_prost_message_list(&mut buf)?; - let index: Vec = Self::decode_prost_message_list(&mut buf)?; - let sink: Vec = Self::decode_prost_message_list(&mut buf)?; - let source: Vec = Self::decode_prost_message_list(&mut buf)?; - let view: Vec = Self::decode_prost_message_list(&mut buf)?; - let function: Vec = Self::decode_prost_message_list(&mut buf)?; - let connection: Vec = Self::decode_prost_message_list(&mut buf)?; - let system_param: SystemParams = Self::decode_prost_message(&mut buf)?; - let cluster_id: String = Self::decode_prost_message(&mut buf)?; - - Ok(Self { - default_cf, - hummock_version, - version_stats, - compaction_groups, - database, - schema, - table, - index, - sink, - source, - view, - table_fragments, - user_info, - function, - connection, - system_param, - cluster_id, - }) - } - - fn encode_prost_message(message: &impl prost::Message, buf: &mut Vec) { - let encoded_message = message.encode_to_vec(); - buf.put_u32_le(encoded_message.len() as u32); - buf.put_slice(&encoded_message); - } - - fn decode_prost_message(buf: &mut &[u8]) -> BackupResult - where - T: prost::Message + Default, - { - let len = buf.get_u32_le() as usize; - let v = buf[..len].to_vec(); - buf.advance(len); - T::decode(v.as_slice()).map_err(|e| BackupError::Decoding(e.into())) - } - - fn encode_prost_message_list(messages: &[&impl prost::Message], buf: &mut Vec) { - buf.put_u32_le(messages.len() as u32); - for message in messages { - Self::encode_prost_message(*message, buf); - } - } - - fn decode_prost_message_list(buf: &mut &[u8]) -> BackupResult> - where - T: prost::Message + Default, - { - let vec_len = buf.get_u32_le() as usize; - let mut result = vec![]; - for _ in 0..vec_len { - let v: T = Self::decode_prost_message(buf)?; - result.push(v); - } - Ok(result) - } -} - -#[cfg(test)] -mod tests { - use risingwave_pb::hummock::{CompactionGroup, TableStats}; - - use crate::meta_snapshot::{ClusterMetadata, MetaSnapshot}; - - #[test] - fn test_snapshot_encoding_decoding() { - let mut metadata = ClusterMetadata::default(); - metadata.hummock_version.id = 321; - let raw = MetaSnapshot { - format_version: 0, - id: 123, - metadata, - }; - let encoded = raw.encode(); - let decoded = MetaSnapshot::decode(&encoded).unwrap(); - assert_eq!(raw, decoded); - } - - #[test] - fn test_metadata_encoding_decoding() { - let mut buf = vec![]; - let mut raw = ClusterMetadata::default(); - raw.default_cf.insert(vec![0, 1, 2], vec![3, 4, 5]); - raw.hummock_version.id = 1; - raw.version_stats.hummock_version_id = 10; - raw.version_stats.table_stats.insert( - 200, - TableStats { - total_key_count: 1000, - ..Default::default() - }, - ); - raw.compaction_groups.push(CompactionGroup { - id: 3000, - ..Default::default() - }); - raw.encode_to(&mut buf); - let decoded = ClusterMetadata::decode(buf.as_slice()).unwrap(); - assert_eq!(raw, decoded); - } -} diff --git a/src/storage/backup/src/meta_snapshot_v1.rs b/src/storage/backup/src/meta_snapshot_v1.rs new file mode 100644 index 0000000000000..731107723c2f4 --- /dev/null +++ b/src/storage/backup/src/meta_snapshot_v1.rs @@ -0,0 +1,275 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::collections::HashMap; +use std::fmt::{Display, Formatter}; + +use bytes::{Buf, BufMut}; +use itertools::Itertools; +use risingwave_common::util::iter_util::ZipEqFast; +use risingwave_pb::catalog::{ + Connection, Database, Function, Index, Schema, Sink, Source, Table, View, +}; +use risingwave_pb::hummock::{CompactionGroup, HummockVersion, HummockVersionStats}; +use risingwave_pb::meta::{SystemParams, TableFragments}; +use risingwave_pb::user::UserInfo; + +use crate::error::{BackupError, BackupResult}; +use crate::meta_snapshot::{MetaSnapshot, Metadata}; + +/// TODO: remove `ClusterMetadata` and even the trait, after applying model v2. + +pub type MetaSnapshotV1 = MetaSnapshot; + +impl Display for ClusterMetadata { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + writeln!(f, "default_cf:")?; + for (k, v) in &self.default_cf { + let key = String::from_utf8(k.clone()).unwrap(); + writeln!(f, "{} {:x?}", key, v)?; + } + writeln!(f, "hummock_version:")?; + writeln!(f, "{:#?}", self.hummock_version)?; + writeln!(f, "version_stats:")?; + writeln!(f, "{:#?}", self.version_stats)?; + writeln!(f, "compaction_groups:")?; + writeln!(f, "{:#?}", self.compaction_groups)?; + writeln!(f, "database:")?; + writeln!(f, "{:#?}", self.database)?; + writeln!(f, "schema:")?; + writeln!(f, "{:#?}", self.schema)?; + writeln!(f, "table:")?; + writeln!(f, "{:#?}", self.table)?; + writeln!(f, "index:")?; + writeln!(f, "{:#?}", self.index)?; + writeln!(f, "sink:")?; + writeln!(f, "{:#?}", self.sink)?; + writeln!(f, "source:")?; + writeln!(f, "{:#?}", self.source)?; + writeln!(f, "view:")?; + writeln!(f, "{:#?}", self.view)?; + writeln!(f, "connection:")?; + writeln!(f, "{:#?}", self.connection)?; + writeln!(f, "table_fragments:")?; + writeln!(f, "{:#?}", self.table_fragments)?; + writeln!(f, "user_info:")?; + writeln!(f, "{:#?}", self.user_info)?; + writeln!(f, "function:")?; + writeln!(f, "{:#?}", self.function)?; + writeln!(f, "system_param:")?; + writeln!(f, "{:#?}", self.system_param)?; + writeln!(f, "cluster_id:")?; + writeln!(f, "{:#?}", self.cluster_id)?; + Ok(()) + } +} + +impl Metadata for ClusterMetadata { + fn encode_to(&self, buf: &mut Vec) -> BackupResult<()> { + self.encode_to(buf) + } + + fn decode(buf: &[u8]) -> BackupResult + where + Self: Sized, + { + ClusterMetadata::decode(buf) + } + + fn hummock_version_ref(&self) -> &HummockVersion { + &self.hummock_version + } + + fn hummock_version(self) -> HummockVersion { + self.hummock_version + } +} + +/// For backward compatibility, never remove fields and only append new field. +#[derive(Debug, Default, Clone, PartialEq)] +pub struct ClusterMetadata { + /// Unlike other metadata that has implemented `MetadataModel`, + /// DEFAULT_COLUMN_FAMILY stores various single row metadata, e.g. id offset and epoch offset. + /// So we use `default_cf` stores raw KVs for them. + pub default_cf: HashMap, Vec>, + pub hummock_version: HummockVersion, + pub version_stats: HummockVersionStats, + pub compaction_groups: Vec, + pub database: Vec, + pub schema: Vec, + pub table: Vec
, + pub index: Vec, + pub sink: Vec, + pub source: Vec, + pub view: Vec, + pub table_fragments: Vec, + pub user_info: Vec, + pub function: Vec, + pub connection: Vec, + pub system_param: SystemParams, + pub cluster_id: String, +} + +impl ClusterMetadata { + pub fn encode_to(&self, buf: &mut Vec) -> BackupResult<()> { + let default_cf_keys = self.default_cf.keys().collect_vec(); + let default_cf_values = self.default_cf.values().collect_vec(); + Self::encode_prost_message_list(&default_cf_keys, buf); + Self::encode_prost_message_list(&default_cf_values, buf); + Self::encode_prost_message(&self.hummock_version, buf); + Self::encode_prost_message(&self.version_stats, buf); + Self::encode_prost_message_list(&self.compaction_groups.iter().collect_vec(), buf); + Self::encode_prost_message_list(&self.table_fragments.iter().collect_vec(), buf); + Self::encode_prost_message_list(&self.user_info.iter().collect_vec(), buf); + Self::encode_prost_message_list(&self.database.iter().collect_vec(), buf); + Self::encode_prost_message_list(&self.schema.iter().collect_vec(), buf); + Self::encode_prost_message_list(&self.table.iter().collect_vec(), buf); + Self::encode_prost_message_list(&self.index.iter().collect_vec(), buf); + Self::encode_prost_message_list(&self.sink.iter().collect_vec(), buf); + Self::encode_prost_message_list(&self.source.iter().collect_vec(), buf); + Self::encode_prost_message_list(&self.view.iter().collect_vec(), buf); + Self::encode_prost_message_list(&self.function.iter().collect_vec(), buf); + Self::encode_prost_message_list(&self.connection.iter().collect_vec(), buf); + Self::encode_prost_message(&self.system_param, buf); + Self::encode_prost_message(&self.cluster_id, buf); + Ok(()) + } + + pub fn decode(mut buf: &[u8]) -> BackupResult { + let default_cf_keys: Vec> = Self::decode_prost_message_list(&mut buf)?; + let default_cf_values: Vec> = Self::decode_prost_message_list(&mut buf)?; + let default_cf = default_cf_keys + .into_iter() + .zip_eq_fast(default_cf_values.into_iter()) + .collect(); + let hummock_version = Self::decode_prost_message(&mut buf)?; + let version_stats = Self::decode_prost_message(&mut buf)?; + let compaction_groups: Vec = Self::decode_prost_message_list(&mut buf)?; + let table_fragments: Vec = Self::decode_prost_message_list(&mut buf)?; + let user_info: Vec = Self::decode_prost_message_list(&mut buf)?; + let database: Vec = Self::decode_prost_message_list(&mut buf)?; + let schema: Vec = Self::decode_prost_message_list(&mut buf)?; + let table: Vec
= Self::decode_prost_message_list(&mut buf)?; + let index: Vec = Self::decode_prost_message_list(&mut buf)?; + let sink: Vec = Self::decode_prost_message_list(&mut buf)?; + let source: Vec = Self::decode_prost_message_list(&mut buf)?; + let view: Vec = Self::decode_prost_message_list(&mut buf)?; + let function: Vec = Self::decode_prost_message_list(&mut buf)?; + let connection: Vec = Self::decode_prost_message_list(&mut buf)?; + let system_param: SystemParams = Self::decode_prost_message(&mut buf)?; + let cluster_id: String = Self::decode_prost_message(&mut buf)?; + + Ok(Self { + default_cf, + hummock_version, + version_stats, + compaction_groups, + database, + schema, + table, + index, + sink, + source, + view, + table_fragments, + user_info, + function, + connection, + system_param, + cluster_id, + }) + } + + fn encode_prost_message(message: &impl prost::Message, buf: &mut Vec) { + let encoded_message = message.encode_to_vec(); + buf.put_u32_le(encoded_message.len() as u32); + buf.put_slice(&encoded_message); + } + + fn decode_prost_message(buf: &mut &[u8]) -> BackupResult + where + T: prost::Message + Default, + { + let len = buf.get_u32_le() as usize; + let v = buf[..len].to_vec(); + buf.advance(len); + T::decode(v.as_slice()).map_err(|e| BackupError::Decoding(e.into())) + } + + fn encode_prost_message_list(messages: &[&impl prost::Message], buf: &mut Vec) { + buf.put_u32_le(messages.len() as u32); + for message in messages { + Self::encode_prost_message(*message, buf); + } + } + + fn decode_prost_message_list(buf: &mut &[u8]) -> BackupResult> + where + T: prost::Message + Default, + { + let vec_len = buf.get_u32_le() as usize; + let mut result = vec![]; + for _ in 0..vec_len { + let v: T = Self::decode_prost_message(buf)?; + result.push(v); + } + Ok(result) + } +} + +#[cfg(test)] +mod tests { + use risingwave_pb::hummock::{CompactionGroup, TableStats}; + + use crate::meta_snapshot_v1::{ClusterMetadata, MetaSnapshotV1}; + + type MetaSnapshot = MetaSnapshotV1; + + #[test] + fn test_snapshot_encoding_decoding() { + let mut metadata = ClusterMetadata::default(); + metadata.hummock_version.id = 321; + let raw = MetaSnapshot { + format_version: 0, + id: 123, + metadata, + }; + let encoded = raw.encode().unwrap(); + let decoded = MetaSnapshot::decode(&encoded).unwrap(); + assert_eq!(raw, decoded); + } + + #[test] + fn test_metadata_encoding_decoding() { + let mut buf = vec![]; + let mut raw = ClusterMetadata::default(); + raw.default_cf.insert(vec![0, 1, 2], vec![3, 4, 5]); + raw.hummock_version.id = 1; + raw.version_stats.hummock_version_id = 10; + raw.version_stats.table_stats.insert( + 200, + TableStats { + total_key_count: 1000, + ..Default::default() + }, + ); + raw.compaction_groups.push(CompactionGroup { + id: 3000, + ..Default::default() + }); + raw.encode_to(&mut buf).unwrap(); + let decoded = ClusterMetadata::decode(buf.as_slice()).unwrap(); + assert_eq!(raw, decoded); + } +} diff --git a/src/storage/backup/src/meta_snapshot_v2.rs b/src/storage/backup/src/meta_snapshot_v2.rs new file mode 100644 index 0000000000000..092cfd083783c --- /dev/null +++ b/src/storage/backup/src/meta_snapshot_v2.rs @@ -0,0 +1,103 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::fmt::{Display, Formatter}; + +use bytes::{Buf, BufMut}; +use risingwave_meta_model_v2 as model_v2; +use risingwave_pb::hummock::HummockVersion; +use serde::{Deserialize, Serialize}; + +use crate::meta_snapshot::{MetaSnapshot, Metadata}; +use crate::BackupResult; + +pub type MetaSnapshotV2 = MetaSnapshot; + +#[derive(Default)] +pub struct MetadataV2 { + pub cluster_id: String, + pub hummock_version: HummockVersion, + pub version_stats: model_v2::hummock_version_stats::Model, + pub compaction_configs: Vec, + // TODO other metadata +} + +impl Display for MetadataV2 { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + writeln!(f, "cluster_id:")?; + writeln!(f, "{:#?}", self.cluster_id)?; + writeln!(f, "hummock_version:")?; + writeln!(f, "{:#?}", self.hummock_version)?; + writeln!(f, "version_stats:")?; + writeln!(f, "{:#?}", self.version_stats)?; + writeln!(f, "compaction_configs:")?; + writeln!(f, "{:#?}", self.compaction_configs)?; + // TODO: other metadata + Ok(()) + } +} + +impl Metadata for MetadataV2 { + fn encode_to(&self, buf: &mut Vec) -> BackupResult<()> { + put_with_len_prefix(buf, &self.cluster_id)?; + put_with_len_prefix(buf, &self.hummock_version)?; + put_with_len_prefix(buf, &self.version_stats)?; + put_with_len_prefix(buf, &self.compaction_configs)?; + // TODO: other metadata + Ok(()) + } + + fn decode(mut buf: &[u8]) -> BackupResult + where + Self: Sized, + { + let cluster_id = get_with_len_prefix(&mut buf)?; + let hummock_version = get_with_len_prefix(&mut buf)?; + let version_stats = get_with_len_prefix(&mut buf)?; + let compaction_configs = get_with_len_prefix(&mut buf)?; + // TODO: other metadata + Ok(Self { + cluster_id, + hummock_version, + version_stats, + compaction_configs, + }) + } + + fn hummock_version_ref(&self) -> &HummockVersion { + &self.hummock_version + } + + fn hummock_version(self) -> HummockVersion { + self.hummock_version + } +} + +fn put_with_len_prefix(buf: &mut Vec, data: &T) -> Result<(), bincode::Error> { + let b = bincode::serialize(data)?; + buf.put_u32_le( + b.len() + .try_into() + .unwrap_or_else(|_| panic!("cannot convert {} into u32", b.len())), + ); + buf.put_slice(&b); + Ok(()) +} + +fn get_with_len_prefix<'a, T: Deserialize<'a>>(buf: &mut &'a [u8]) -> Result { + let len = buf.get_u32_le() as usize; + let d = bincode::deserialize(&buf[..len])?; + buf.advance(len); + Ok(d) +} diff --git a/src/storage/backup/src/storage.rs b/src/storage/backup/src/storage.rs index 85583e6a9b267..5c325809f2650 100644 --- a/src/storage/backup/src/storage.rs +++ b/src/storage/backup/src/storage.rs @@ -16,23 +16,25 @@ use std::collections::HashSet; use std::sync::Arc; use itertools::Itertools; -use risingwave_object_store::object::{ObjectError, ObjectStoreRef}; +use risingwave_object_store::object::object_metrics::ObjectStoreMetrics; +use risingwave_object_store::object::{ + InMemObjectStore, MonitoredObjectStore, ObjectError, ObjectStoreImpl, ObjectStoreRef, +}; -use crate::meta_snapshot::MetaSnapshot; +use crate::meta_snapshot::{MetaSnapshot, Metadata}; use crate::{ BackupError, BackupResult, MetaSnapshotId, MetaSnapshotManifest, MetaSnapshotMetadata, }; -pub type MetaSnapshotStorageRef = Arc; -pub type BoxedMetaSnapshotStorage = Box; +pub type MetaSnapshotStorageRef = Arc; #[async_trait::async_trait] pub trait MetaSnapshotStorage: 'static + Sync + Send { /// Creates a snapshot. - async fn create(&self, snapshot: &MetaSnapshot) -> BackupResult<()>; + async fn create(&self, snapshot: &MetaSnapshot) -> BackupResult<()>; /// Gets a snapshot by id. - async fn get(&self, id: MetaSnapshotId) -> BackupResult; + async fn get(&self, id: MetaSnapshotId) -> BackupResult>; /// Gets local snapshot manifest. fn manifest(&self) -> Arc; @@ -110,9 +112,9 @@ impl ObjectStoreMetaSnapshotStorage { #[async_trait::async_trait] impl MetaSnapshotStorage for ObjectStoreMetaSnapshotStorage { - async fn create(&self, snapshot: &MetaSnapshot) -> BackupResult<()> { + async fn create(&self, snapshot: &MetaSnapshot) -> BackupResult<()> { let path = self.get_snapshot_path(snapshot.id); - self.store.upload(&path, snapshot.encode().into()).await?; + self.store.upload(&path, snapshot.encode()?.into()).await?; // update manifest last let mut new_manifest = (**self.manifest.read()).clone(); @@ -121,13 +123,13 @@ impl MetaSnapshotStorage for ObjectStoreMetaSnapshotStorage { .snapshot_metadata .push(MetaSnapshotMetadata::new( snapshot.id, - &snapshot.metadata.hummock_version, + snapshot.metadata.hummock_version_ref(), )); self.update_manifest(new_manifest).await?; Ok(()) } - async fn get(&self, id: MetaSnapshotId) -> BackupResult { + async fn get(&self, id: MetaSnapshotId) -> BackupResult> { let path = self.get_snapshot_path(id); let data = self.store.read(&path, ..).await?; MetaSnapshot::decode(&data) @@ -172,30 +174,15 @@ impl From for BackupError { } } -#[derive(Default)] -pub struct DummyMetaSnapshotStorage { - manifest: Arc, -} - -#[async_trait::async_trait] -impl MetaSnapshotStorage for DummyMetaSnapshotStorage { - async fn create(&self, _snapshot: &MetaSnapshot) -> BackupResult<()> { - panic!("should not create from DummyBackupStorage") - } - - async fn get(&self, _id: MetaSnapshotId) -> BackupResult { - panic!("should not get from DummyBackupStorage") - } - - fn manifest(&self) -> Arc { - self.manifest.clone() - } - - async fn refresh_manifest(&self) -> BackupResult<()> { - Ok(()) - } - - async fn delete(&self, _ids: &[MetaSnapshotId]) -> BackupResult<()> { - panic!("should not delete from DummyBackupStorage") - } +// #[cfg(test)] +pub async fn unused() -> ObjectStoreMetaSnapshotStorage { + ObjectStoreMetaSnapshotStorage::new( + "", + Arc::new(ObjectStoreImpl::InMem(MonitoredObjectStore::new( + InMemObjectStore::new(), + Arc::new(ObjectStoreMetrics::unused()), + ))), + ) + .await + .unwrap() } diff --git a/src/storage/hummock_test/src/compactor_tests.rs b/src/storage/hummock_test/src/compactor_tests.rs index 34b23a9b79774..cd921b3725369 100644 --- a/src/storage/hummock_test/src/compactor_tests.rs +++ b/src/storage/hummock_test/src/compactor_tests.rs @@ -15,7 +15,7 @@ #[cfg(test)] pub(crate) mod tests { - use std::collections::{BTreeSet, HashMap, VecDeque}; + use std::collections::{BTreeMap, BTreeSet, VecDeque}; use std::ops::Bound; use std::sync::atomic::AtomicU32; use std::sync::Arc; @@ -280,7 +280,7 @@ pub(crate) mod tests { let compaction_filter_flag = CompactionFilterFlag::TTL; compact_task.watermark = (TEST_WATERMARK * 1000) << 16; compact_task.compaction_filter_mask = compaction_filter_flag.bits(); - compact_task.table_options = HashMap::from([( + compact_task.table_options = BTreeMap::from([( 0, TableOption { retention_seconds: 64, @@ -952,7 +952,7 @@ pub(crate) mod tests { let compaction_filter_flag = CompactionFilterFlag::STATE_CLEAN | CompactionFilterFlag::TTL; compact_task.compaction_filter_mask = compaction_filter_flag.bits(); let retention_seconds_expire_second = 1; - compact_task.table_options = HashMap::from_iter([( + compact_task.table_options = BTreeMap::from_iter([( existing_table_id, TableOption { retention_seconds: retention_seconds_expire_second, @@ -1157,8 +1157,6 @@ pub(crate) mod tests { let compaction_filter_flag = CompactionFilterFlag::STATE_CLEAN | CompactionFilterFlag::TTL; compact_task.compaction_filter_mask = compaction_filter_flag.bits(); - // compact_task.table_options = - // HashMap::from_iter([(existing_table_id, TableOption { ttl: 0 })]); compact_task.current_epoch_time = epoch; // 3. compact diff --git a/src/storage/hummock_test/src/test_utils.rs b/src/storage/hummock_test/src/test_utils.rs index cd713e3977777..831d3bfc6b198 100644 --- a/src/storage/hummock_test/src/test_utils.rs +++ b/src/storage/hummock_test/src/test_utils.rs @@ -72,7 +72,7 @@ pub async fn prepare_first_valid_version( let (tx, mut rx) = unbounded_channel(); let notification_client = get_notification_client_for_test(env, hummock_manager_ref.clone(), worker_node.clone()); - let backup_manager = BackupReader::unused(); + let backup_manager = BackupReader::unused().await; let write_limiter = WriteLimiter::unused(); let observer_manager = ObserverManager::new( notification_client, diff --git a/src/storage/src/hummock/backup_reader.rs b/src/storage/src/hummock/backup_reader.rs index 1a79a87e9d5ec..b9fa4a6258bbe 100644 --- a/src/storage/src/hummock/backup_reader.rs +++ b/src/storage/src/hummock/backup_reader.rs @@ -22,11 +22,9 @@ use arc_swap::ArcSwap; use futures::future::Shared; use futures::FutureExt; use risingwave_backup::error::BackupError; -use risingwave_backup::meta_snapshot::MetaSnapshot; -use risingwave_backup::storage::{ - BoxedMetaSnapshotStorage, DummyMetaSnapshotStorage, ObjectStoreMetaSnapshotStorage, -}; -use risingwave_backup::MetaSnapshotId; +use risingwave_backup::meta_snapshot::{MetaSnapshot, Metadata}; +use risingwave_backup::storage::{MetaSnapshotStorage, ObjectStoreMetaSnapshotStorage}; +use risingwave_backup::{meta_snapshot_v1, MetaSnapshotId}; use risingwave_common::system_param::local_manager::SystemParamsReaderRef; use risingwave_object_store::object::object_metrics::ObjectStoreMetrics; use risingwave_object_store::object::parse_remote_object_store; @@ -42,7 +40,9 @@ type VersionHolder = ( tokio::sync::mpsc::UnboundedReceiver, ); -async fn create_snapshot_store(config: &StoreConfig) -> StorageResult { +async fn create_snapshot_store( + config: &StoreConfig, +) -> StorageResult { let backup_object_store = Arc::new( parse_remote_object_store( &config.0, @@ -51,8 +51,7 @@ async fn create_snapshot_store(config: &StoreConfig) -> StorageResult>, inflight_request: parking_lot::Mutex>, - store: ArcSwap<(BoxedMetaSnapshotStorage, StoreConfig)>, + store: ArcSwap<(ObjectStoreMetaSnapshotStorage, StoreConfig)>, refresh_tx: tokio::sync::mpsc::UnboundedSender, } @@ -80,7 +79,7 @@ impl BackupReader { Ok(Self::with_store((store, config))) } - fn with_store(store: (BoxedMetaSnapshotStorage, StoreConfig)) -> BackupReaderRef { + fn with_store(store: (ObjectStoreMetaSnapshotStorage, StoreConfig)) -> BackupReaderRef { let (refresh_tx, refresh_rx) = tokio::sync::mpsc::unbounded_channel(); let instance = Arc::new(Self { store: ArcSwap::from_pointee(store), @@ -92,9 +91,9 @@ impl BackupReader { instance } - pub fn unused() -> BackupReaderRef { + pub async fn unused() -> BackupReaderRef { Self::with_store(( - Box::::default(), + risingwave_backup::storage::unused().await, StoreConfig::default(), )) } @@ -193,9 +192,10 @@ impl BackupReader { } else { let this = self.clone(); let f = async move { - let snapshot = current_store.0.get(snapshot_id).await.map_err(|e| { - format!("failed to get meta snapshot {}. {}", snapshot_id, e) - })?; + let snapshot: meta_snapshot_v1::MetaSnapshotV1 = + current_store.0.get(snapshot_id).await.map_err(|e| { + format!("failed to get meta snapshot {}. {}", snapshot_id, e) + })?; let version_holder = build_version_holder(snapshot); let version_clone = version_holder.0.clone(); this.versions.write().insert(snapshot_id, version_holder); @@ -244,9 +244,9 @@ impl BackupReader { } } -fn build_version_holder(s: MetaSnapshot) -> VersionHolder { +fn build_version_holder(s: MetaSnapshot) -> VersionHolder { let (tx, rx) = tokio::sync::mpsc::unbounded_channel(); - (PinnedVersion::new(s.metadata.hummock_version, tx), rx) + (PinnedVersion::new(s.metadata.hummock_version(), tx), rx) } impl From for StorageError { diff --git a/src/workspace-hack/Cargo.toml b/src/workspace-hack/Cargo.toml index 4ad9af21bf929..9f83f40008d2e 100644 --- a/src/workspace-hack/Cargo.toml +++ b/src/workspace-hack/Cargo.toml @@ -35,7 +35,6 @@ clap = { version = "4", features = ["cargo", "derive", "env"] } clap_builder = { version = "4", default-features = false, features = ["cargo", "color", "env", "help", "std", "suggestions", "usage"] } combine = { version = "4", features = ["tokio"] } crossbeam-epoch = { version = "0.9" } -crossbeam-queue = { version = "0.3" } crossbeam-utils = { version = "0.8" } deranged = { version = "0.3", default-features = false, features = ["powerfmt", "serde", "std"] } digest = { version = "0.10", features = ["mac", "oid", "std"] } From 75b676db73f452051b39696f7c6d1371109da365 Mon Sep 17 00:00:00 2001 From: William Wen <44139337+wenym1@users.noreply.github.com> Date: Wed, 1 Nov 2023 18:05:54 +0800 Subject: [PATCH 08/17] fix(sink): spawn blocking in remote validation (#13194) --- src/connector/src/sink/remote.rs | 81 +++++++++++++++++--------------- 1 file changed, 44 insertions(+), 37 deletions(-) diff --git a/src/connector/src/sink/remote.rs b/src/connector/src/sink/remote.rs index 228da05a48108..79a8f1760ced6 100644 --- a/src/connector/src/sink/remote.rs +++ b/src/connector/src/sink/remote.rs @@ -52,6 +52,7 @@ use risingwave_rpc_client::{ }; use tokio::sync::mpsc; use tokio::sync::mpsc::{Receiver, Sender}; +use tokio::task::spawn_blocking; use tokio_stream::wrappers::ReceiverStream; use tracing::warn; @@ -152,47 +153,53 @@ impl Sink for RemoteSink { } }).try_collect()?; - let mut env = JVM - .get_or_init()? - .attach_current_thread() - .map_err(|err| SinkError::Internal(err.into()))?; - let validate_sink_request = ValidateSinkRequest { - sink_param: Some(self.param.to_proto()), - }; - let validate_sink_request_bytes = env - .byte_array_from_slice(&Message::encode_to_vec(&validate_sink_request)) - .map_err(|err| SinkError::Internal(err.into()))?; + let jvm = JVM.get_or_init()?; + let sink_param = self.param.to_proto(); + + spawn_blocking(move || { + let mut env = jvm + .attach_current_thread() + .map_err(|err| SinkError::Internal(err.into()))?; + let validate_sink_request = ValidateSinkRequest { + sink_param: Some(sink_param), + }; + let validate_sink_request_bytes = env + .byte_array_from_slice(&Message::encode_to_vec(&validate_sink_request)) + .map_err(|err| SinkError::Internal(err.into()))?; + + let response = env + .call_static_method( + "com/risingwave/connector/JniSinkValidationHandler", + "validate", + "([B)[B", + &[JValue::Object(&validate_sink_request_bytes)], + ) + .map_err(|err| SinkError::Internal(err.into()))?; + + let validate_sink_response_bytes = match response { + JValueOwned::Object(o) => unsafe { JByteArray::from_raw(o.into_raw()) }, + _ => unreachable!(), + }; - let response = env - .call_static_method( - "com/risingwave/connector/JniSinkValidationHandler", - "validate", - "([B)[B", - &[JValue::Object(&validate_sink_request_bytes)], + let validate_sink_response: ValidateSinkResponse = Message::decode( + risingwave_jni_core::to_guarded_slice(&validate_sink_response_bytes, &mut env) + .map_err(|err| SinkError::Internal(err.into()))? + .deref(), ) .map_err(|err| SinkError::Internal(err.into()))?; - let validate_sink_response_bytes = match response { - JValueOwned::Object(o) => unsafe { JByteArray::from_raw(o.into_raw()) }, - _ => unreachable!(), - }; - - let validate_sink_response: ValidateSinkResponse = Message::decode( - risingwave_jni_core::to_guarded_slice(&validate_sink_response_bytes, &mut env) - .map_err(|err| SinkError::Internal(err.into()))? - .deref(), - ) - .map_err(|err| SinkError::Internal(err.into()))?; - - validate_sink_response.error.map_or_else( - || Ok(()), // If there is no error message, return Ok here. - |err| { - Err(SinkError::Remote(anyhow!(format!( - "sink cannot pass validation: {}", - err.error_message - )))) - }, - ) + validate_sink_response.error.map_or_else( + || Ok(()), // If there is no error message, return Ok here. + |err| { + Err(SinkError::Remote(anyhow!(format!( + "sink cannot pass validation: {}", + err.error_message + )))) + }, + ) + }) + .await + .map_err(|e| anyhow!("unable to validate: {:?}", e))? } } From 5ab109bbebcdca08e4143d85a9a526b1e375316f Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 1 Nov 2023 18:28:39 +0800 Subject: [PATCH 09/17] chore(deps): Bump org.elasticsearch:elasticsearch from 7.17.10 to 7.17.13 in /java (#13156) Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- java/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/java/pom.xml b/java/pom.xml index 4b815f9512eb5..83a1426411490 100644 --- a/java/pom.xml +++ b/java/pom.xml @@ -75,7 +75,7 @@ 2.13.5 3.3.1 3.3.3 - 7.17.10 + 7.17.13 4.15.0 From 2a6c4aaeda77a3e9cc6a4231bfaac1443f99c737 Mon Sep 17 00:00:00 2001 From: xxchan Date: Wed, 1 Nov 2023 19:04:11 +0800 Subject: [PATCH 10/17] chore: bump rustix 0.38.11->0.38.21 (#13199) --- Cargo.lock | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 568d22f0ae4e1..8fef1708dc5fa 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -4162,7 +4162,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "cb0889898416213fab133e1d33a0e5858a48177452750691bde3666d0fdbaf8b" dependencies = [ "hermit-abi", - "rustix 0.38.11", + "rustix 0.38.21", "windows-sys 0.48.0", ] @@ -4494,9 +4494,9 @@ checksum = "ef53942eb7bf7ff43a617b3e2c1c4a5ecf5944a7c1bc12d7ee39bbb15e5c1519" [[package]] name = "linux-raw-sys" -version = "0.4.5" +version = "0.4.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "57bcfdad1b858c2db7c38303a6d2ad4dfaf5eb53dfeb0910128b2c26d6158503" +checksum = "da2479e8c062e40bf0066ffa0bc823de0a9368974af99c9f6df941d2c231e03f" [[package]] name = "local_stats_alloc" @@ -8567,14 +8567,14 @@ dependencies = [ [[package]] name = "rustix" -version = "0.38.11" +version = "0.38.21" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c0c3dde1fc030af041adc40e79c0e7fbcf431dd24870053d187d7c66e4b87453" +checksum = "2b426b0506e5d50a7d8dafcf2e81471400deb602392c7dd110815afb4eaf02a3" dependencies = [ "bitflags 2.4.0", "errno", "libc", - "linux-raw-sys 0.4.5", + "linux-raw-sys 0.4.10", "windows-sys 0.48.0", ] @@ -9928,7 +9928,7 @@ dependencies = [ "cfg-if", "fastrand 2.0.0", "redox_syscall 0.3.5", - "rustix 0.38.11", + "rustix 0.38.21", "windows-sys 0.48.0", ] @@ -10955,7 +10955,7 @@ dependencies = [ "either", "home", "once_cell", - "rustix 0.38.11", + "rustix 0.38.21", ] [[package]] From 8ab2f266bc3e8b596b536a441aad48ee9aea6c88 Mon Sep 17 00:00:00 2001 From: Richard Chien Date: Wed, 1 Nov 2023 19:15:22 +0800 Subject: [PATCH 11/17] feat(over window): support general `rank` and `dense_rank` window functions (#13183) Signed-off-by: Richard Chien --- .../generated/batch/rank_func/mod.slt.part | 89 ++--- .../streaming/rank_func/mod.slt.part | 89 ++--- .../templates/rank_func/mod.slt.part | 89 ++--- .../templates/rank_func/row_number_old.part | 82 +++++ proto/batch_plan.proto | 1 + src/batch/src/executor/sort_over_window.rs | 31 +- .../core/src/window_function/state/mod.rs | 6 +- .../core/src/window_function/state/rank.rs | 331 ++++++++++++++++++ .../src/window_function/state/row_number.rs | 160 --------- .../testdata/input/over_window_function.yaml | 14 +- .../testdata/output/over_window_function.yaml | 91 +++-- .../optimizer/plan_node/batch_over_window.rs | 34 +- .../plan_node/logical_over_window.rs | 47 +-- .../rule/over_window_to_topn_rule.rs | 7 +- 14 files changed, 644 insertions(+), 427 deletions(-) create mode 100644 e2e_test/over_window/templates/rank_func/row_number_old.part create mode 100644 src/expr/core/src/window_function/state/rank.rs delete mode 100644 src/expr/core/src/window_function/state/row_number.rs diff --git a/e2e_test/over_window/generated/batch/rank_func/mod.slt.part b/e2e_test/over_window/generated/batch/rank_func/mod.slt.part index 58e434bd11362..e9c750bfd3548 100644 --- a/e2e_test/over_window/generated/batch/rank_func/mod.slt.part +++ b/e2e_test/over_window/generated/batch/rank_func/mod.slt.part @@ -2,82 +2,57 @@ # Test rank window functions including `row_number`, `rank`, `dense_rank`. +include ./row_number_old.slt.part + statement ok -create table t ( - id int - , p1 int - , p2 int - , time int - , v1 int - , v2 int -); +create table t (id int, score int); statement ok create view v as select * - , row_number() over (partition by p1 order by time, id) as out1 - , row_number() over (partition by p1 order by p2 desc, id) as out2 + , row_number() over (partition by 0::int order by score desc, id) as r1 + , rank() over (partition by 0::int order by score desc) as r2 + , dense_rank() over (partition by 0::int order by score desc) as r3 from t; statement ok insert into t values - (100001, 100, 200, 1, 701, 805) -, (100002, 100, 200, 2, 700, 806) -, (100003, 100, 208, 2, 723, 807) -, (100004, 103, 200, 2, 702, 808); - -query II -select * from v order by id; ----- -100001 100 200 1 701 805 1 2 -100002 100 200 2 700 806 2 3 -100003 100 208 2 723 807 3 1 -100004 103 200 2 702 808 1 1 - -statement ok -insert into t values - (100005, 100, 200, 3, 717, 810) -, (100006, 105, 204, 5, 703, 828); - -query II + (10001, 95) +, (10002, 90) +, (10003, 80) +, (10004, 95) +, (10005, 90) +, (10006, 90) +, (10007, 96) +; + +query iiIII select * from v order by id; ---- -100001 100 200 1 701 805 1 2 -100002 100 200 2 700 806 2 3 -100003 100 208 2 723 807 3 1 -100004 103 200 2 702 808 1 1 -100005 100 200 3 717 810 4 4 -100006 105 204 5 703 828 1 1 +10001 95 2 2 2 +10002 90 4 4 3 +10003 80 7 7 4 +10004 95 3 2 2 +10005 90 5 4 3 +10006 90 6 4 3 +10007 96 1 1 1 statement ok -update t set v1 = 799 where id = 100002; -- value change - -statement ok -update t set p2 = 200 where id = 100003; -- partition change - -statement ok -update t set "time" = 1 where id = 100005; -- order change - -query iiiiiii -select * from v order by id; ----- -100001 100 200 1 701 805 1 1 -100002 100 200 2 799 806 3 2 -100003 100 200 2 723 807 4 3 -100004 103 200 2 702 808 1 1 -100005 100 200 1 717 810 2 4 -100006 105 204 5 703 828 1 1 +update t set score = 96 where id = 10001; statement ok -delete from t where time = 2; +delete from t where id = 10006; -query iiiiiii +query iiIII select * from v order by id; ---- -100001 100 200 1 701 805 1 1 -100005 100 200 1 717 810 2 2 -100006 105 204 5 703 828 1 1 +10001 96 1 1 1 +10002 90 4 4 3 +10003 80 6 6 4 +10004 95 3 3 2 +10005 90 5 4 3 +10007 96 2 1 1 statement ok drop view v; diff --git a/e2e_test/over_window/generated/streaming/rank_func/mod.slt.part b/e2e_test/over_window/generated/streaming/rank_func/mod.slt.part index 1a22f59053e9d..18500279af36c 100644 --- a/e2e_test/over_window/generated/streaming/rank_func/mod.slt.part +++ b/e2e_test/over_window/generated/streaming/rank_func/mod.slt.part @@ -2,82 +2,57 @@ # Test rank window functions including `row_number`, `rank`, `dense_rank`. +include ./row_number_old.slt.part + statement ok -create table t ( - id int - , p1 int - , p2 int - , time int - , v1 int - , v2 int -); +create table t (id int, score int); statement ok create materialized view v as select * - , row_number() over (partition by p1 order by time, id) as out1 - , row_number() over (partition by p1 order by p2 desc, id) as out2 + , row_number() over (partition by 0::int order by score desc, id) as r1 + , rank() over (partition by 0::int order by score desc) as r2 + , dense_rank() over (partition by 0::int order by score desc) as r3 from t; statement ok insert into t values - (100001, 100, 200, 1, 701, 805) -, (100002, 100, 200, 2, 700, 806) -, (100003, 100, 208, 2, 723, 807) -, (100004, 103, 200, 2, 702, 808); - -query II -select * from v order by id; ----- -100001 100 200 1 701 805 1 2 -100002 100 200 2 700 806 2 3 -100003 100 208 2 723 807 3 1 -100004 103 200 2 702 808 1 1 - -statement ok -insert into t values - (100005, 100, 200, 3, 717, 810) -, (100006, 105, 204, 5, 703, 828); - -query II + (10001, 95) +, (10002, 90) +, (10003, 80) +, (10004, 95) +, (10005, 90) +, (10006, 90) +, (10007, 96) +; + +query iiIII select * from v order by id; ---- -100001 100 200 1 701 805 1 2 -100002 100 200 2 700 806 2 3 -100003 100 208 2 723 807 3 1 -100004 103 200 2 702 808 1 1 -100005 100 200 3 717 810 4 4 -100006 105 204 5 703 828 1 1 +10001 95 2 2 2 +10002 90 4 4 3 +10003 80 7 7 4 +10004 95 3 2 2 +10005 90 5 4 3 +10006 90 6 4 3 +10007 96 1 1 1 statement ok -update t set v1 = 799 where id = 100002; -- value change - -statement ok -update t set p2 = 200 where id = 100003; -- partition change - -statement ok -update t set "time" = 1 where id = 100005; -- order change - -query iiiiiii -select * from v order by id; ----- -100001 100 200 1 701 805 1 1 -100002 100 200 2 799 806 3 2 -100003 100 200 2 723 807 4 3 -100004 103 200 2 702 808 1 1 -100005 100 200 1 717 810 2 4 -100006 105 204 5 703 828 1 1 +update t set score = 96 where id = 10001; statement ok -delete from t where time = 2; +delete from t where id = 10006; -query iiiiiii +query iiIII select * from v order by id; ---- -100001 100 200 1 701 805 1 1 -100005 100 200 1 717 810 2 2 -100006 105 204 5 703 828 1 1 +10001 96 1 1 1 +10002 90 4 4 3 +10003 80 6 6 4 +10004 95 3 3 2 +10005 90 5 4 3 +10007 96 2 1 1 statement ok drop materialized view v; diff --git a/e2e_test/over_window/templates/rank_func/mod.slt.part b/e2e_test/over_window/templates/rank_func/mod.slt.part index b654819f8e0f0..9b8fedc0352e2 100644 --- a/e2e_test/over_window/templates/rank_func/mod.slt.part +++ b/e2e_test/over_window/templates/rank_func/mod.slt.part @@ -1,81 +1,56 @@ # Test rank window functions including `row_number`, `rank`, `dense_rank`. +include ./row_number_old.slt.part + statement ok -create table t ( - id int - , p1 int - , p2 int - , time int - , v1 int - , v2 int -); +create table t (id int, score int); statement ok create $view_type v as select * - , row_number() over (partition by p1 order by time, id) as out1 - , row_number() over (partition by p1 order by p2 desc, id) as out2 + , row_number() over (partition by 0::int order by score desc, id) as r1 + , rank() over (partition by 0::int order by score desc) as r2 + , dense_rank() over (partition by 0::int order by score desc) as r3 from t; statement ok insert into t values - (100001, 100, 200, 1, 701, 805) -, (100002, 100, 200, 2, 700, 806) -, (100003, 100, 208, 2, 723, 807) -, (100004, 103, 200, 2, 702, 808); - -query II -select * from v order by id; ----- -100001 100 200 1 701 805 1 2 -100002 100 200 2 700 806 2 3 -100003 100 208 2 723 807 3 1 -100004 103 200 2 702 808 1 1 - -statement ok -insert into t values - (100005, 100, 200, 3, 717, 810) -, (100006, 105, 204, 5, 703, 828); - -query II + (10001, 95) +, (10002, 90) +, (10003, 80) +, (10004, 95) +, (10005, 90) +, (10006, 90) +, (10007, 96) +; + +query iiIII select * from v order by id; ---- -100001 100 200 1 701 805 1 2 -100002 100 200 2 700 806 2 3 -100003 100 208 2 723 807 3 1 -100004 103 200 2 702 808 1 1 -100005 100 200 3 717 810 4 4 -100006 105 204 5 703 828 1 1 +10001 95 2 2 2 +10002 90 4 4 3 +10003 80 7 7 4 +10004 95 3 2 2 +10005 90 5 4 3 +10006 90 6 4 3 +10007 96 1 1 1 statement ok -update t set v1 = 799 where id = 100002; -- value change - -statement ok -update t set p2 = 200 where id = 100003; -- partition change - -statement ok -update t set "time" = 1 where id = 100005; -- order change - -query iiiiiii -select * from v order by id; ----- -100001 100 200 1 701 805 1 1 -100002 100 200 2 799 806 3 2 -100003 100 200 2 723 807 4 3 -100004 103 200 2 702 808 1 1 -100005 100 200 1 717 810 2 4 -100006 105 204 5 703 828 1 1 +update t set score = 96 where id = 10001; statement ok -delete from t where time = 2; +delete from t where id = 10006; -query iiiiiii +query iiIII select * from v order by id; ---- -100001 100 200 1 701 805 1 1 -100005 100 200 1 717 810 2 2 -100006 105 204 5 703 828 1 1 +10001 96 1 1 1 +10002 90 4 4 3 +10003 80 6 6 4 +10004 95 3 3 2 +10005 90 5 4 3 +10007 96 2 1 1 statement ok drop $view_type v; diff --git a/e2e_test/over_window/templates/rank_func/row_number_old.part b/e2e_test/over_window/templates/rank_func/row_number_old.part new file mode 100644 index 0000000000000..8c23c0f4b4967 --- /dev/null +++ b/e2e_test/over_window/templates/rank_func/row_number_old.part @@ -0,0 +1,82 @@ +statement ok +create table t ( + id int + , p1 int + , p2 int + , time int + , v1 int + , v2 int +); + +statement ok +create $view_type v as +select + * + , row_number() over (partition by p1 order by time, id) as out1 + , row_number() over (partition by p1 order by p2 desc, id) as out2 +from t; + +statement ok +insert into t values + (100001, 100, 200, 1, 701, 805) +, (100002, 100, 200, 2, 700, 806) +, (100003, 100, 208, 2, 723, 807) +, (100004, 103, 200, 2, 702, 808); + +query II +select * from v order by id; +---- +100001 100 200 1 701 805 1 2 +100002 100 200 2 700 806 2 3 +100003 100 208 2 723 807 3 1 +100004 103 200 2 702 808 1 1 + +statement ok +insert into t values + (100005, 100, 200, 3, 717, 810) +, (100006, 105, 204, 5, 703, 828); + +query II +select * from v order by id; +---- +100001 100 200 1 701 805 1 2 +100002 100 200 2 700 806 2 3 +100003 100 208 2 723 807 3 1 +100004 103 200 2 702 808 1 1 +100005 100 200 3 717 810 4 4 +100006 105 204 5 703 828 1 1 + +statement ok +update t set v1 = 799 where id = 100002; -- value change + +statement ok +update t set p2 = 200 where id = 100003; -- partition change + +statement ok +update t set "time" = 1 where id = 100005; -- order change + +query iiiiiii +select * from v order by id; +---- +100001 100 200 1 701 805 1 1 +100002 100 200 2 799 806 3 2 +100003 100 200 2 723 807 4 3 +100004 103 200 2 702 808 1 1 +100005 100 200 1 717 810 2 4 +100006 105 204 5 703 828 1 1 + +statement ok +delete from t where time = 2; + +query iiiiiii +select * from v order by id; +---- +100001 100 200 1 701 805 1 1 +100005 100 200 1 717 810 2 2 +100006 105 204 5 703 828 1 1 + +statement ok +drop $view_type v; + +statement ok +drop table t; diff --git a/proto/batch_plan.proto b/proto/batch_plan.proto index 5e7c75383c60f..7cf813efa0163 100644 --- a/proto/batch_plan.proto +++ b/proto/batch_plan.proto @@ -280,6 +280,7 @@ message UnionNode {} message SortOverWindowNode { repeated expr.WindowFunction calls = 1; repeated uint32 partition_by = 2; + repeated common.ColumnOrder order_by = 3; } message PlanNode { diff --git a/src/batch/src/executor/sort_over_window.rs b/src/batch/src/executor/sort_over_window.rs index 21bfc8aa6b177..d04a359f2c555 100644 --- a/src/batch/src/executor/sort_over_window.rs +++ b/src/batch/src/executor/sort_over_window.rs @@ -19,6 +19,8 @@ use risingwave_common::error::{Result, RwError}; use risingwave_common::row::{OwnedRow, Row, RowExt}; use risingwave_common::util::chunk_coalesce::DataChunkBuilder; use risingwave_common::util::iter_util::ZipEqFast; +use risingwave_common::util::memcmp_encoding::{self, MemcmpEncoded}; +use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_expr::window_function::{ create_window_state, StateKey, WindowFuncCall, WindowStates, }; @@ -41,6 +43,8 @@ pub struct SortOverWindowExecutor { struct ExecutorInner { calls: Vec, partition_key_indices: Vec, + order_key_indices: Vec, + order_key_order_types: Vec, chunk_size: usize, } @@ -67,6 +71,12 @@ impl BoxedExecutorBuilder for SortOverWindowExecutor { .iter() .map(|i| *i as usize) .collect(); + let (order_key_indices, order_key_order_types) = node + .get_order_by() + .iter() + .map(ColumnOrder::from_protobuf) + .map(|o| (o.column_index, o.order_type)) + .unzip(); let mut schema = child.schema().clone(); calls.iter().for_each(|call| { @@ -82,6 +92,8 @@ impl BoxedExecutorBuilder for SortOverWindowExecutor { inner: ExecutorInner { calls, partition_key_indices, + order_key_indices, + order_key_order_types, chunk_size: source.context.get_config().developer.chunk_size, }, })) @@ -108,12 +120,19 @@ impl ExecutorInner { .project(&self.partition_key_indices) .into_owned_row() } -} -fn state_key_placeholder() -> StateKey { - StateKey { - order_key: vec![].into(), - pk: OwnedRow::empty().into(), + fn encode_order_key(&self, full_row: impl Row) -> Result { + Ok(memcmp_encoding::encode_row( + full_row.project(&self.order_key_indices), + &self.order_key_order_types, + )?) + } + + fn row_to_state_key(&self, full_row: impl Row) -> Result { + Ok(StateKey { + order_key: self.encode_order_key(full_row)?, + pk: OwnedRow::empty().into(), // we don't rely on the pk part in `WindowStates` + }) } } @@ -182,7 +201,7 @@ impl SortOverWindowExecutor { for (call, state) in this.calls.iter().zip_eq_fast(states.iter_mut()) { // TODO(rc): batch appending state.append( - state_key_placeholder(), // we don't rely on the state key in `WindowStates` + this.row_to_state_key(row)?, row.project(call.args.val_indices()) .into_owned_row() .as_inner() diff --git a/src/expr/core/src/window_function/state/mod.rs b/src/expr/core/src/window_function/state/mod.rs index 927f5aaf6e0c0..47d09bb31bd09 100644 --- a/src/expr/core/src/window_function/state/mod.rs +++ b/src/expr/core/src/window_function/state/mod.rs @@ -27,7 +27,7 @@ use crate::{ExprError, Result}; mod buffer; mod aggregate; -mod row_number; +mod rank; /// Unique and ordered identifier for a row in internal states. #[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Ord, EstimateSize)] @@ -113,7 +113,9 @@ pub fn create_window_state(call: &WindowFuncCall) -> Result Box::new(row_number::RowNumberState::new(call)), + RowNumber => Box::new(rank::RankState::::new(call)), + Rank => Box::new(rank::RankState::::new(call)), + DenseRank => Box::new(rank::RankState::::new(call)), Aggregate(_) => Box::new(aggregate::AggregateState::new(call)?), kind => { return Err(ExprError::UnsupportedFunction(format!( diff --git a/src/expr/core/src/window_function/state/rank.rs b/src/expr/core/src/window_function/state/rank.rs new file mode 100644 index 0000000000000..b4dc41560811a --- /dev/null +++ b/src/expr/core/src/window_function/state/rank.rs @@ -0,0 +1,331 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::marker::PhantomData; + +use risingwave_common::estimate_size::collections::VecDeque; +use risingwave_common::estimate_size::EstimateSize; +use risingwave_common::types::Datum; +use risingwave_common::util::memcmp_encoding::MemcmpEncoded; +use smallvec::SmallVec; + +use self::private::RankFuncCount; +use super::{StateEvictHint, StateKey, StatePos, WindowState}; +use crate::window_function::WindowFuncCall; +use crate::Result; + +mod private { + use super::*; + + pub trait RankFuncCount: Default + EstimateSize { + fn count(&mut self, curr_key: StateKey) -> i64; + } +} + +#[derive(Default, EstimateSize)] +pub struct RowNumber { + prev_rank: i64, +} + +impl RankFuncCount for RowNumber { + fn count(&mut self, _curr_key: StateKey) -> i64 { + let curr_rank = self.prev_rank + 1; + self.prev_rank = curr_rank; + curr_rank + } +} + +#[derive(EstimateSize)] +pub struct Rank { + prev_order_key: Option, + prev_rank: i64, + prev_pos_in_peer_group: i64, +} + +impl Default for Rank { + fn default() -> Self { + Self { + prev_order_key: None, + prev_rank: 0, + prev_pos_in_peer_group: 1, // first position in the fake starting peer group + } + } +} + +impl RankFuncCount for Rank { + fn count(&mut self, curr_key: StateKey) -> i64 { + let (curr_rank, curr_pos_in_group) = if let Some(prev_order_key) = self.prev_order_key.as_ref() && prev_order_key == &curr_key.order_key { + // current key is in the same peer group as the previous one + (self.prev_rank, self.prev_pos_in_peer_group + 1) + } else { + // starting a new peer group + (self.prev_rank + self.prev_pos_in_peer_group, 1) + }; + self.prev_order_key = Some(curr_key.order_key); + self.prev_rank = curr_rank; + self.prev_pos_in_peer_group = curr_pos_in_group; + curr_rank + } +} + +#[derive(Default, EstimateSize)] +pub struct DenseRank { + prev_order_key: Option, + prev_rank: i64, +} + +impl RankFuncCount for DenseRank { + fn count(&mut self, curr_key: StateKey) -> i64 { + let curr_rank = if let Some(prev_order_key) = self.prev_order_key.as_ref() && prev_order_key == &curr_key.order_key { + // current key is in the same peer group as the previous one + self.prev_rank + } else { + // starting a new peer group + self.prev_rank + 1 + }; + self.prev_order_key = Some(curr_key.order_key); + self.prev_rank = curr_rank; + curr_rank + } +} + +/// Generic state for rank window functions including `row_number`, `rank` and `dense_rank`. +#[derive(EstimateSize)] +pub struct RankState { + /// First state key of the partition. + first_key: Option, + /// State keys that are waiting to be outputted. + buffer: VecDeque, + /// Function-specific state. + func_state: RF, + _phantom: PhantomData, +} + +impl RankState { + pub fn new(_call: &WindowFuncCall) -> Self { + Self { + first_key: None, + buffer: Default::default(), + func_state: Default::default(), + _phantom: PhantomData, + } + } + + fn slide_inner(&mut self) -> (i64, StateEvictHint) { + let curr_key = self + .buffer + .pop_front() + .expect("should not slide forward when the current window is not ready"); + let rank = self.func_state.count(curr_key); + // can't evict any state key in EOWC mode, because we can't recover from previous output now + let evict_hint = StateEvictHint::CannotEvict( + self.first_key + .clone() + .expect("should have appended some rows"), + ); + (rank, evict_hint) + } +} + +impl WindowState for RankState { + fn append(&mut self, key: StateKey, _args: SmallVec<[Datum; 2]>) { + if self.first_key.is_none() { + self.first_key = Some(key.clone()); + } + self.buffer.push_back(key); + } + + fn curr_window(&self) -> StatePos<'_> { + let curr_key = self.buffer.front(); + StatePos { + key: curr_key, + is_ready: curr_key.is_some(), + } + } + + fn slide(&mut self) -> Result<(Datum, StateEvictHint)> { + let (rank, evict_hint) = self.slide_inner(); + Ok((Some(rank.into()), evict_hint)) + } + + fn slide_no_output(&mut self) -> Result { + let (_rank, evict_hint) = self.slide_inner(); + Ok(evict_hint) + } +} + +#[cfg(test)] +mod tests { + use risingwave_common::row::OwnedRow; + use risingwave_common::types::{DataType, ScalarImpl}; + use risingwave_common::util::memcmp_encoding; + use risingwave_common::util::sort_util::OrderType; + + use super::*; + use crate::aggregate::AggArgs; + use crate::window_function::{Frame, FrameBound, WindowFuncKind}; + + fn create_state_key(order: i64, pk: i64) -> StateKey { + StateKey { + order_key: memcmp_encoding::encode_value( + Some(ScalarImpl::from(order)), + OrderType::ascending(), + ) + .unwrap(), + pk: OwnedRow::new(vec![Some(pk.into())]).into(), + } + } + + #[test] + #[should_panic(expected = "should not slide forward when the current window is not ready")] + fn test_rank_state_bad_use() { + let call = WindowFuncCall { + kind: WindowFuncKind::RowNumber, + args: AggArgs::None, + return_type: DataType::Int64, + frame: Frame::rows( + FrameBound::UnboundedPreceding, + FrameBound::UnboundedFollowing, + ), + }; + let mut state = RankState::::new(&call); + assert!(state.curr_window().key.is_none()); + assert!(!state.curr_window().is_ready); + _ = state.slide() + } + + #[test] + fn test_row_number_state() { + let call = WindowFuncCall { + kind: WindowFuncKind::RowNumber, + args: AggArgs::None, + return_type: DataType::Int64, + frame: Frame::rows( + FrameBound::UnboundedPreceding, + FrameBound::UnboundedFollowing, + ), + }; + let mut state = RankState::::new(&call); + assert!(state.curr_window().key.is_none()); + assert!(!state.curr_window().is_ready); + state.append(create_state_key(1, 100), SmallVec::new()); + assert_eq!(state.curr_window().key.unwrap(), &create_state_key(1, 100)); + assert!(state.curr_window().is_ready); + let (output, evict_hint) = state.slide().unwrap(); + assert_eq!(output.unwrap(), 1i64.into()); + match evict_hint { + StateEvictHint::CannotEvict(state_key) => { + assert_eq!(state_key, create_state_key(1, 100)); + } + _ => unreachable!(), + } + assert!(!state.curr_window().is_ready); + state.append(create_state_key(2, 103), SmallVec::new()); + state.append(create_state_key(2, 102), SmallVec::new()); + assert_eq!(state.curr_window().key.unwrap(), &create_state_key(2, 103)); + let (output, evict_hint) = state.slide().unwrap(); + assert_eq!(output.unwrap(), 2i64.into()); + match evict_hint { + StateEvictHint::CannotEvict(state_key) => { + assert_eq!(state_key, create_state_key(1, 100)); + } + _ => unreachable!(), + } + assert_eq!(state.curr_window().key.unwrap(), &create_state_key(2, 102)); + let (output, _) = state.slide().unwrap(); + assert_eq!(output.unwrap(), 3i64.into()); + } + + #[test] + fn test_rank_state() { + let call = WindowFuncCall { + kind: WindowFuncKind::Rank, + args: AggArgs::None, + return_type: DataType::Int64, + frame: Frame::rows( + FrameBound::UnboundedPreceding, + FrameBound::UnboundedFollowing, + ), + }; + let mut state = RankState::::new(&call); + assert!(state.curr_window().key.is_none()); + assert!(!state.curr_window().is_ready); + state.append(create_state_key(1, 100), SmallVec::new()); + state.append(create_state_key(2, 103), SmallVec::new()); + state.append(create_state_key(2, 102), SmallVec::new()); + state.append(create_state_key(3, 106), SmallVec::new()); + state.append(create_state_key(3, 105), SmallVec::new()); + state.append(create_state_key(3, 104), SmallVec::new()); + state.append(create_state_key(8, 108), SmallVec::new()); + + let mut outputs = vec![]; + while state.curr_window().is_ready { + outputs.push(state.slide().unwrap().0) + } + + assert_eq!( + outputs, + vec![ + Some(1i64.into()), + Some(2i64.into()), + Some(2i64.into()), + Some(4i64.into()), + Some(4i64.into()), + Some(4i64.into()), + Some(7i64.into()) + ] + ); + } + + #[test] + fn test_dense_rank_state() { + let call = WindowFuncCall { + kind: WindowFuncKind::DenseRank, + args: AggArgs::None, + return_type: DataType::Int64, + frame: Frame::rows( + FrameBound::UnboundedPreceding, + FrameBound::UnboundedFollowing, + ), + }; + let mut state = RankState::::new(&call); + assert!(state.curr_window().key.is_none()); + assert!(!state.curr_window().is_ready); + state.append(create_state_key(1, 100), SmallVec::new()); + state.append(create_state_key(2, 103), SmallVec::new()); + state.append(create_state_key(2, 102), SmallVec::new()); + state.append(create_state_key(3, 106), SmallVec::new()); + state.append(create_state_key(3, 105), SmallVec::new()); + state.append(create_state_key(3, 104), SmallVec::new()); + state.append(create_state_key(8, 108), SmallVec::new()); + + let mut outputs = vec![]; + while state.curr_window().is_ready { + outputs.push(state.slide().unwrap().0) + } + + assert_eq!( + outputs, + vec![ + Some(1i64.into()), + Some(2i64.into()), + Some(2i64.into()), + Some(3i64.into()), + Some(3i64.into()), + Some(3i64.into()), + Some(4i64.into()) + ] + ); + } +} diff --git a/src/expr/core/src/window_function/state/row_number.rs b/src/expr/core/src/window_function/state/row_number.rs deleted file mode 100644 index 6a2759d69308c..0000000000000 --- a/src/expr/core/src/window_function/state/row_number.rs +++ /dev/null @@ -1,160 +0,0 @@ -// Copyright 2023 RisingWave Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use risingwave_common::estimate_size::collections::VecDeque; -use risingwave_common::estimate_size::EstimateSize; -use risingwave_common::types::Datum; -use smallvec::SmallVec; - -use super::{StateEvictHint, StateKey, StatePos, WindowState}; -use crate::window_function::WindowFuncCall; -use crate::Result; - -#[derive(EstimateSize)] -pub struct RowNumberState { - first_key: Option, - buffer: VecDeque, - curr_row_number: i64, -} - -impl RowNumberState { - pub fn new(_call: &WindowFuncCall) -> Self { - Self { - first_key: None, - buffer: Default::default(), - curr_row_number: 1, - } - } - - fn slide_inner(&mut self) -> StateEvictHint { - self.curr_row_number += 1; - self.buffer - .pop_front() - .expect("should not slide forward when the current window is not ready"); - // can't evict any state key in EOWC mode, because we can't recover from previous output now - StateEvictHint::CannotEvict( - self.first_key - .clone() - .expect("should have appended some rows"), - ) - } -} - -impl WindowState for RowNumberState { - fn append(&mut self, key: StateKey, _args: SmallVec<[Datum; 2]>) { - if self.first_key.is_none() { - self.first_key = Some(key.clone()); - } - self.buffer.push_back(key); - } - - fn curr_window(&self) -> StatePos<'_> { - let curr_key = self.buffer.front(); - StatePos { - key: curr_key, - is_ready: curr_key.is_some(), - } - } - - fn slide(&mut self) -> Result<(Datum, StateEvictHint)> { - let output = if self.curr_window().is_ready { - Some(self.curr_row_number.into()) - } else { - None - }; - let evict_hint = self.slide_inner(); - Ok((output, evict_hint)) - } - - fn slide_no_output(&mut self) -> Result { - Ok(self.slide_inner()) - } -} - -#[cfg(test)] -mod tests { - use risingwave_common::row::OwnedRow; - use risingwave_common::types::DataType; - - use super::*; - use crate::aggregate::AggArgs; - use crate::window_function::{Frame, FrameBound, WindowFuncKind}; - - fn create_state_key(pk: i64) -> StateKey { - StateKey { - order_key: vec![].into(), // doesn't matter here - pk: OwnedRow::new(vec![Some(pk.into())]).into(), - } - } - - #[test] - #[should_panic(expected = "should not slide forward when the current window is not ready")] - fn test_row_number_state_bad_use() { - let call = WindowFuncCall { - kind: WindowFuncKind::RowNumber, - args: AggArgs::None, - return_type: DataType::Int64, - frame: Frame::rows( - FrameBound::UnboundedPreceding, - FrameBound::UnboundedFollowing, - ), - }; - let mut state = RowNumberState::new(&call); - assert!(state.curr_window().key.is_none()); - assert!(!state.curr_window().is_ready); - _ = state.slide() - } - - #[test] - fn test_row_number_state() { - let call = WindowFuncCall { - kind: WindowFuncKind::RowNumber, - args: AggArgs::None, - return_type: DataType::Int64, - frame: Frame::rows( - FrameBound::UnboundedPreceding, - FrameBound::UnboundedFollowing, - ), - }; - let mut state = RowNumberState::new(&call); - assert!(state.curr_window().key.is_none()); - assert!(!state.curr_window().is_ready); - state.append(create_state_key(100), SmallVec::new()); - assert_eq!(state.curr_window().key.unwrap(), &create_state_key(100)); - assert!(state.curr_window().is_ready); - let (output, evict_hint) = state.slide().unwrap(); - assert_eq!(output.unwrap(), 1i64.into()); - match evict_hint { - StateEvictHint::CannotEvict(state_key) => { - assert_eq!(state_key, create_state_key(100)); - } - _ => unreachable!(), - } - assert!(!state.curr_window().is_ready); - state.append(create_state_key(103), SmallVec::new()); - state.append(create_state_key(102), SmallVec::new()); - assert_eq!(state.curr_window().key.unwrap(), &create_state_key(103)); - let (output, evict_hint) = state.slide().unwrap(); - assert_eq!(output.unwrap(), 2i64.into()); - match evict_hint { - StateEvictHint::CannotEvict(state_key) => { - assert_eq!(state_key, create_state_key(100)); - } - _ => unreachable!(), - } - assert_eq!(state.curr_window().key.unwrap(), &create_state_key(102)); - let (output, _) = state.slide().unwrap(); - assert_eq!(output.unwrap(), 3i64.into()); - } -} diff --git a/src/frontend/planner_test/tests/testdata/input/over_window_function.yaml b/src/frontend/planner_test/tests/testdata/input/over_window_function.yaml index c02b915c18c88..33c00048abc60 100644 --- a/src/frontend/planner_test/tests/testdata/input/over_window_function.yaml +++ b/src/frontend/planner_test/tests/testdata/input/over_window_function.yaml @@ -183,12 +183,12 @@ - id: multiple rank function calls sql: | create table t(x int); - select row_number() over(PARTITION BY x ORDER BY x), rank() over(ORDER BY x) from t; + select row_number() over(PARTITION BY x ORDER BY x), rank() over(PARTITION BY x ORDER BY x), dense_rank() over (PARTITION BY x ORDER BY x) from t; expected_outputs: - logical_plan - optimized_logical_plan_for_stream - - stream_error - - batch_error + - stream_plan + - batch_plan - id: row_number with valid over clause sql: | create table t(x int, y int); @@ -251,8 +251,8 @@ -- OFFSET for RANK() is not yet supported expected_outputs: - logical_plan - - stream_error - - batch_error + - stream_plan + - batch_plan # >>> TopN by row_number, with offset - sql: | create table t(x int, y int); @@ -471,6 +471,6 @@ expected_outputs: - logical_plan - optimized_logical_plan_for_stream - - stream_error + - stream_plan - optimized_logical_plan_for_batch - - batch_error + - batch_plan diff --git a/src/frontend/planner_test/tests/testdata/output/over_window_function.yaml b/src/frontend/planner_test/tests/testdata/output/over_window_function.yaml index 733a19f4ba05c..c54f2a9458e44 100644 --- a/src/frontend/planner_test/tests/testdata/output/over_window_function.yaml +++ b/src/frontend/planner_test/tests/testdata/output/over_window_function.yaml @@ -377,23 +377,35 @@ - id: multiple rank function calls sql: | create table t(x int); - select row_number() over(PARTITION BY x ORDER BY x), rank() over(ORDER BY x) from t; + select row_number() over(PARTITION BY x ORDER BY x), rank() over(PARTITION BY x ORDER BY x), dense_rank() over (PARTITION BY x ORDER BY x) from t; logical_plan: |- - LogicalProject { exprs: [row_number, rank] } - └─LogicalOverWindow { window_functions: [row_number() OVER(PARTITION BY t.x ORDER BY t.x ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), rank() OVER(ORDER BY t.x ASCROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } + LogicalProject { exprs: [row_number, rank, dense_rank] } + └─LogicalOverWindow { window_functions: [row_number() OVER(PARTITION BY t.x ORDER BY t.x ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), rank() OVER(PARTITION BY t.x ORDER BY t.x ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), dense_rank() OVER(PARTITION BY t.x ORDER BY t.x ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } └─LogicalProject { exprs: [t.x, t._row_id] } └─LogicalScan { table: t, columns: [t.x, t._row_id] } optimized_logical_plan_for_stream: |- - LogicalProject { exprs: [row_number, rank] } - └─LogicalOverWindow { window_functions: [rank() OVER(ORDER BY t.x ASCROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } - └─LogicalOverWindow { window_functions: [row_number() OVER(PARTITION BY t.x ORDER BY t.x ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } - └─LogicalScan { table: t, columns: [t.x] } - batch_error: |- - Feature is not yet implemented: `rank` and `dense_rank` function calls that don't match TopN pattern are not supported yet - Tracking issue: https://github.com/risingwavelabs/risingwave/issues/8965 - stream_error: |- - Feature is not yet implemented: `rank` and `dense_rank` function calls that don't match TopN pattern are not supported yet - Tracking issue: https://github.com/risingwavelabs/risingwave/issues/8965 + LogicalProject { exprs: [row_number, rank, dense_rank] } + └─LogicalOverWindow { window_functions: [dense_rank() OVER(PARTITION BY t.x ORDER BY t.x ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } + └─LogicalOverWindow { window_functions: [rank() OVER(PARTITION BY t.x ORDER BY t.x ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } + └─LogicalOverWindow { window_functions: [row_number() OVER(PARTITION BY t.x ORDER BY t.x ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } + └─LogicalScan { table: t, columns: [t.x] } + batch_plan: |- + BatchExchange { order: [], dist: Single } + └─BatchProject { exprs: [row_number, rank, dense_rank] } + └─BatchOverWindow { window_functions: [dense_rank() OVER(PARTITION BY t.x ORDER BY t.x ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } + └─BatchOverWindow { window_functions: [rank() OVER(PARTITION BY t.x ORDER BY t.x ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } + └─BatchOverWindow { window_functions: [row_number() OVER(PARTITION BY t.x ORDER BY t.x ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } + └─BatchExchange { order: [t.x ASC, t.x ASC], dist: HashShard(t.x) } + └─BatchSort { order: [t.x ASC, t.x ASC] } + └─BatchScan { table: t, columns: [t.x], distribution: SomeShard } + stream_plan: |- + StreamMaterialize { columns: [row_number, rank, dense_rank, t._row_id(hidden), t.x(hidden)], stream_key: [t._row_id, t.x], pk_columns: [t._row_id, t.x], pk_conflict: NoCheck } + └─StreamProject { exprs: [row_number, rank, dense_rank, t._row_id, t.x] } + └─StreamOverWindow { window_functions: [dense_rank() OVER(PARTITION BY t.x ORDER BY t.x ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } + └─StreamOverWindow { window_functions: [rank() OVER(PARTITION BY t.x ORDER BY t.x ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } + └─StreamOverWindow { window_functions: [row_number() OVER(PARTITION BY t.x ORDER BY t.x ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } + └─StreamExchange { dist: HashShard(t.x) } + └─StreamTableScan { table: t, columns: [t.x, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - id: row_number with valid over clause sql: | create table t(x int, y int); @@ -536,12 +548,21 @@ └─LogicalOverWindow { window_functions: [rank() OVER(PARTITION BY t.y ORDER BY t.x ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } └─LogicalProject { exprs: [t.x, t.y, t._row_id] } └─LogicalScan { table: t, columns: [t.x, t.y, t._row_id] } - batch_error: |- - Feature is not yet implemented: `rank` and `dense_rank` function calls that don't match TopN pattern are not supported yet - Tracking issue: https://github.com/risingwavelabs/risingwave/issues/8965 - stream_error: |- - Feature is not yet implemented: `rank` and `dense_rank` function calls that don't match TopN pattern are not supported yet - Tracking issue: https://github.com/risingwavelabs/risingwave/issues/8965 + batch_plan: |- + BatchExchange { order: [], dist: Single } + └─BatchProject { exprs: [t.x, t.y] } + └─BatchFilter { predicate: (rank <= 3:Int32) AND (rank > 1:Int32) } + └─BatchOverWindow { window_functions: [rank() OVER(PARTITION BY t.y ORDER BY t.x ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } + └─BatchExchange { order: [t.y ASC, t.x ASC], dist: HashShard(t.y) } + └─BatchSort { order: [t.y ASC, t.x ASC] } + └─BatchScan { table: t, columns: [t.x, t.y], distribution: SomeShard } + stream_plan: |- + StreamMaterialize { columns: [x, y, t._row_id(hidden)], stream_key: [t._row_id, y], pk_columns: [t._row_id, y], pk_conflict: NoCheck } + └─StreamProject { exprs: [t.x, t.y, t._row_id] } + └─StreamFilter { predicate: (rank <= 3:Int32) AND (rank > 1:Int32) } + └─StreamOverWindow { window_functions: [rank() OVER(PARTITION BY t.y ORDER BY t.x ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } + └─StreamExchange { dist: HashShard(t.y) } + └─StreamTableScan { table: t, columns: [t.x, t.y, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - sql: | create table t(x int, y int); select x, y from @@ -942,9 +963,29 @@ └─LogicalOverWindow { window_functions: [sum(t.x) OVER(PARTITION BY t.z ORDER BY t.y ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } └─LogicalOverWindow { window_functions: [rank() OVER(PARTITION BY t.x ORDER BY t.y ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } └─LogicalScan { table: t, columns: [t.x, t.y, t.z] } - batch_error: |- - Feature is not yet implemented: `rank` and `dense_rank` function calls that don't match TopN pattern are not supported yet - Tracking issue: https://github.com/risingwavelabs/risingwave/issues/8965 - stream_error: |- - Feature is not yet implemented: `rank` and `dense_rank` function calls that don't match TopN pattern are not supported yet - Tracking issue: https://github.com/risingwavelabs/risingwave/issues/8965 + batch_plan: |- + BatchExchange { order: [], dist: Single } + └─BatchOverWindow { window_functions: [rank() OVER(PARTITION BY t.y ORDER BY t.x ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } + └─BatchExchange { order: [t.y ASC, t.x ASC], dist: HashShard(t.y) } + └─BatchSort { order: [t.y ASC, t.x ASC] } + └─BatchOverWindow { window_functions: [row_number() OVER(PARTITION BY t.x ORDER BY t.y ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } + └─BatchExchange { order: [t.x ASC, t.y ASC], dist: HashShard(t.x) } + └─BatchSort { order: [t.x ASC, t.y ASC] } + └─BatchOverWindow { window_functions: [sum(t.x) OVER(PARTITION BY t.z ORDER BY t.y ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } + └─BatchExchange { order: [t.z ASC, t.y ASC], dist: HashShard(t.z) } + └─BatchSort { order: [t.z ASC, t.y ASC] } + └─BatchOverWindow { window_functions: [rank() OVER(PARTITION BY t.x ORDER BY t.y ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } + └─BatchExchange { order: [t.x ASC, t.y ASC], dist: HashShard(t.x) } + └─BatchSort { order: [t.x ASC, t.y ASC] } + └─BatchScan { table: t, columns: [t.x, t.y, t.z], distribution: SomeShard } + stream_plan: |- + StreamMaterialize { columns: [x, y, z, t._row_id(hidden), w0, w1, w2, w3], stream_key: [t._row_id, x, z, y], pk_columns: [t._row_id, x, z, y], pk_conflict: NoCheck } + └─StreamOverWindow { window_functions: [rank() OVER(PARTITION BY t.y ORDER BY t.x ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } + └─StreamExchange { dist: HashShard(t.y) } + └─StreamOverWindow { window_functions: [row_number() OVER(PARTITION BY t.x ORDER BY t.y ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } + └─StreamExchange { dist: HashShard(t.x) } + └─StreamOverWindow { window_functions: [sum(t.x) OVER(PARTITION BY t.z ORDER BY t.y ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } + └─StreamExchange { dist: HashShard(t.z) } + └─StreamOverWindow { window_functions: [rank() OVER(PARTITION BY t.x ORDER BY t.y ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } + └─StreamExchange { dist: HashShard(t.x) } + └─StreamTableScan { table: t, columns: [t.x, t.y, t.z, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } diff --git a/src/frontend/src/optimizer/plan_node/batch_over_window.rs b/src/frontend/src/optimizer/plan_node/batch_over_window.rs index 011de0dfb1459..6cabcc6a3cbc5 100644 --- a/src/frontend/src/optimizer/plan_node/batch_over_window.rs +++ b/src/frontend/src/optimizer/plan_node/batch_over_window.rs @@ -97,19 +97,29 @@ impl ToLocalBatch for BatchOverWindow { impl ToBatchPb for BatchOverWindow { fn to_batch_prost_body(&self) -> NodeBody { + let calls = self + .core + .window_functions() + .iter() + .map(PlanWindowFunction::to_protobuf) + .collect(); + let partition_by = self + .core + .partition_key_indices() + .into_iter() + .map(|idx| idx as _) + .collect(); + let order_by = self + .core + .order_key() + .iter() + .map(ColumnOrder::to_protobuf) + .collect(); + NodeBody::SortOverWindow(SortOverWindowNode { - calls: self - .core - .window_functions() - .iter() - .map(PlanWindowFunction::to_protobuf) - .collect(), - partition_by: self - .core - .partition_key_indices() - .into_iter() - .map(|idx| idx as _) - .collect(), + calls, + partition_by, + order_by, }) } } diff --git a/src/frontend/src/optimizer/plan_node/logical_over_window.rs b/src/frontend/src/optimizer/plan_node/logical_over_window.rs index 6193c072563c6..b9e58f9c9d6eb 100644 --- a/src/frontend/src/optimizer/plan_node/logical_over_window.rs +++ b/src/frontend/src/optimizer/plan_node/logical_over_window.rs @@ -382,21 +382,12 @@ impl LogicalOverWindow { let rewritten_selected_items = over_window_builder.rewrite_selected_items(select_exprs)?; for window_func in &window_functions { - if window_func.kind.is_rank() { - if window_func.order_by.sort_exprs.is_empty() { - return Err(ErrorCode::InvalidInputSyntax(format!( - "window rank function without order by: {:?}", - window_func - )) - .into()); - } - if window_func.kind == WindowFuncKind::DenseRank { - return Err(ErrorCode::NotImplemented( - format!("window rank function: {}", window_func.kind), - 4847.into(), - ) - .into()); - } + if window_func.kind.is_rank() && window_func.order_by.sort_exprs.is_empty() { + return Err(ErrorCode::InvalidInputSyntax(format!( + "window rank function without order by: {:?}", + window_func + )) + .into()); } } @@ -709,19 +700,6 @@ impl PredicatePushdown for LogicalOverWindow { impl ToBatch for LogicalOverWindow { fn to_batch(&self) -> Result { - if self - .window_functions() - .iter() - .any(|x| matches!(x.kind, WindowFuncKind::Rank | WindowFuncKind::DenseRank)) - { - return Err(ErrorCode::NotImplemented( - "`rank` and `dense_rank` function calls that don't match TopN pattern are not supported yet" - .to_string(), - 8965.into(), - ) - .into()); - } - if !self.core.funcs_have_same_partition_and_order() { return Err(ErrorCode::InvalidInputSyntax( "All window functions must have the same PARTITION BY and ORDER BY".to_string(), @@ -757,19 +735,6 @@ impl ToStream for LogicalOverWindow { fn to_stream(&self, ctx: &mut ToStreamContext) -> Result { use super::stream::prelude::*; - if self - .window_functions() - .iter() - .any(|x| matches!(x.kind, WindowFuncKind::Rank | WindowFuncKind::DenseRank)) - { - return Err(ErrorCode::NotImplemented( - "`rank` and `dense_rank` function calls that don't match TopN pattern are not supported yet" - .to_string(), - 8965.into(), - ) - .into()); - } - let stream_input = self.core.input.to_stream(ctx)?; if ctx.emit_on_window_close() { diff --git a/src/frontend/src/optimizer/rule/over_window_to_topn_rule.rs b/src/frontend/src/optimizer/rule/over_window_to_topn_rule.rs index 496a51d6d9f3d..8da05e9fc7581 100644 --- a/src/frontend/src/optimizer/rule/over_window_to_topn_rule.rs +++ b/src/frontend/src/optimizer/rule/over_window_to_topn_rule.rs @@ -65,8 +65,6 @@ impl Rule for OverWindowToTopNRule { // The filter is directly on top of the over window after predicate pushdown. let over_window = plan.as_logical_over_window()?; - // TODO(st1page): split the OverAgg if there is some part of window function can be - // rewritten to group topn if over_window.window_functions().len() != 1 { // Queries with multiple window function calls are not supported yet. return None; @@ -84,7 +82,10 @@ impl Rule for OverWindowToTopNRule { // Only `ROW_NUMBER` and `RANK` can be optimized to TopN now. WindowFuncKind::RowNumber => false, WindowFuncKind::Rank => true, - WindowFuncKind::DenseRank => unimplemented!("should be banned in planner"), + WindowFuncKind::DenseRank => { + ctx.warn_to_user("`dense_rank` is not supported in Top-N pattern, will fallback to inefficient implementation"); + return None; + } _ => unreachable!("window functions other than rank functions should not reach here"), }; From 6bd797ea5c46071258aeaabcf8e0608b16520806 Mon Sep 17 00:00:00 2001 From: Dylan Date: Wed, 1 Nov 2023 19:46:30 +0800 Subject: [PATCH 12/17] feat(optimizer): support value extract project rule (#13182) --- .../tests/testdata/input/subquery_expr.yaml | 6 ++ .../tests/testdata/output/cse_expr.yaml | 2 +- .../tests/testdata/output/explain.yaml | 2 +- .../tests/testdata/output/expr.yaml | 4 +- .../tests/testdata/output/subquery_expr.yaml | 21 +++++++ .../testdata/output/with_ordinality.yaml | 21 ++++--- .../src/optimizer/logical_optimization.rs | 10 +++ .../plan_visitor/plan_correlated_id_finder.rs | 4 ++ src/frontend/src/optimizer/rule/mod.rs | 3 + .../rule/values_extract_project_rule.rs | 61 +++++++++++++++++++ 10 files changed, 121 insertions(+), 13 deletions(-) create mode 100644 src/frontend/src/optimizer/rule/values_extract_project_rule.rs diff --git a/src/frontend/planner_test/tests/testdata/input/subquery_expr.yaml b/src/frontend/planner_test/tests/testdata/input/subquery_expr.yaml index 255a87f84099d..6e7949cff1ed7 100644 --- a/src/frontend/planner_test/tests/testdata/input/subquery_expr.yaml +++ b/src/frontend/planner_test/tests/testdata/input/subquery_expr.yaml @@ -129,3 +129,9 @@ expected_outputs: - logical_plan - batch_plan +- sql: | + create table b (b1 varchar); + select * from b cross join repeat(b1, 2); + expected_outputs: + - logical_plan + - batch_plan diff --git a/src/frontend/planner_test/tests/testdata/output/cse_expr.yaml b/src/frontend/planner_test/tests/testdata/output/cse_expr.yaml index 09e0e7872e7c7..eba0edf5b1f04 100644 --- a/src/frontend/planner_test/tests/testdata/output/cse_expr.yaml +++ b/src/frontend/planner_test/tests/testdata/output/cse_expr.yaml @@ -84,5 +84,5 @@ sql: | with t(v, arr) as (select 1, array[2, 3]) select v < all(arr), v < some(arr) from t; batch_plan: |- - BatchProject { exprs: [All((1:Int32 < $expr10060)) as $expr1, Some((1:Int32 < $expr10060)) as $expr2] } + BatchProject { exprs: [All((1:Int32 < $expr10063)) as $expr1, Some((1:Int32 < $expr10063)) as $expr2] } └─BatchValues { rows: [[1:Int32, ARRAY[2, 3]:List(Int32)]] } diff --git a/src/frontend/planner_test/tests/testdata/output/explain.yaml b/src/frontend/planner_test/tests/testdata/output/explain.yaml index 157736e53b21d..63899bc26c0e4 100644 --- a/src/frontend/planner_test/tests/testdata/output/explain.yaml +++ b/src/frontend/planner_test/tests/testdata/output/explain.yaml @@ -70,7 +70,7 @@ "stages": { "0": { "root": { - "plan_node_id": 10036, + "plan_node_id": 10037, "plan_node_type": "BatchValues", "schema": [ { diff --git a/src/frontend/planner_test/tests/testdata/output/expr.yaml b/src/frontend/planner_test/tests/testdata/output/expr.yaml index 0cfa66218ef79..020c6a3548ef5 100644 --- a/src/frontend/planner_test/tests/testdata/output/expr.yaml +++ b/src/frontend/planner_test/tests/testdata/output/expr.yaml @@ -450,7 +450,7 @@ └─LogicalProject { exprs: [Array(1:Int32) as $expr1] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: |- - BatchProject { exprs: [Some((1:Int32 < ArrayCat($expr10039, ARRAY[2]:List(Int32)))) as $expr1] } + BatchProject { exprs: [Some((1:Int32 < ArrayCat($expr10041, ARRAY[2]:List(Int32)))) as $expr1] } └─BatchNestedLoopJoin { type: LeftOuter, predicate: true, output: all } ├─BatchValues { rows: [[]] } └─BatchValues { rows: [[ARRAY[1]:List(Int32)]] } @@ -473,7 +473,7 @@ └─LogicalProject { exprs: [Array(1:Int32) as $expr1] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: |- - BatchProject { exprs: [All((1:Int32 < ArrayCat($expr10039, ARRAY[2]:List(Int32)))) as $expr1] } + BatchProject { exprs: [All((1:Int32 < ArrayCat($expr10041, ARRAY[2]:List(Int32)))) as $expr1] } └─BatchNestedLoopJoin { type: LeftOuter, predicate: true, output: all } ├─BatchValues { rows: [[]] } └─BatchValues { rows: [[ARRAY[1]:List(Int32)]] } diff --git a/src/frontend/planner_test/tests/testdata/output/subquery_expr.yaml b/src/frontend/planner_test/tests/testdata/output/subquery_expr.yaml index 1383c156a18f5..9c9bbb2551576 100644 --- a/src/frontend/planner_test/tests/testdata/output/subquery_expr.yaml +++ b/src/frontend/planner_test/tests/testdata/output/subquery_expr.yaml @@ -272,3 +272,24 @@ │ └─BatchScan { table: t1, columns: [t1.a], distribution: SomeShard } └─BatchExchange { order: [], dist: Single } └─BatchScan { table: t2, columns: [], distribution: SomeShard } +- sql: | + create table b (b1 varchar); + select * from b cross join repeat(b1, 2); + logical_plan: |- + LogicalProject { exprs: [b.b1, ] } + └─LogicalApply { type: Inner, on: true, correlated_id: 1 } + ├─LogicalScan { table: b, columns: [b.b1, b._row_id] } + └─LogicalValues { rows: [[Repeat(CorrelatedInputRef { index: 0, correlated_id: 1 }, 2:Int32)]], schema: Schema { fields: [:Varchar] } } + batch_plan: |- + BatchExchange { order: [], dist: Single } + └─BatchProject { exprs: [b.b1, Repeat(b.b1, 2:Int32) as $expr1] } + └─BatchHashJoin { type: Inner, predicate: b.b1 IS NOT DISTINCT FROM b.b1, output: all } + ├─BatchExchange { order: [], dist: HashShard(b.b1) } + │ └─BatchScan { table: b, columns: [b.b1], distribution: SomeShard } + └─BatchExchange { order: [], dist: HashShard(b.b1) } + └─BatchNestedLoopJoin { type: Inner, predicate: true, output: all } + ├─BatchExchange { order: [], dist: Single } + │ └─BatchHashAgg { group_key: [b.b1], aggs: [] } + │ └─BatchExchange { order: [], dist: HashShard(b.b1) } + │ └─BatchScan { table: b, columns: [b.b1], distribution: SomeShard } + └─BatchValues { rows: [[]] } diff --git a/src/frontend/planner_test/tests/testdata/output/with_ordinality.yaml b/src/frontend/planner_test/tests/testdata/output/with_ordinality.yaml index 867855fe7b44d..e614b0c20a36b 100644 --- a/src/frontend/planner_test/tests/testdata/output/with_ordinality.yaml +++ b/src/frontend/planner_test/tests/testdata/output/with_ordinality.yaml @@ -195,15 +195,18 @@ create table t(x int , arr int[]); select * from t, abs(x) WITH ORDINALITY; batch_plan: |- - BatchNestedLoopJoin { type: Inner, predicate: true, output: all } - ├─BatchExchange { order: [], dist: Single } - │ └─BatchHashJoin { type: Inner, predicate: t.x IS NOT DISTINCT FROM t.x, output: [t.x, t.arr] } - │ ├─BatchExchange { order: [], dist: HashShard(t.x) } - │ │ └─BatchScan { table: t, columns: [t.x, t.arr], distribution: SomeShard } - │ └─BatchHashAgg { group_key: [t.x], aggs: [] } - │ └─BatchExchange { order: [], dist: HashShard(t.x) } - │ └─BatchScan { table: t, columns: [t.x], distribution: SomeShard } - └─BatchValues { rows: [[Abs(CorrelatedInputRef { index: 0, correlated_id: 1 }), 1:Int64]] } + BatchExchange { order: [], dist: Single } + └─BatchProject { exprs: [t.x, t.arr, Abs(t.x) as $expr1, 1:Int64] } + └─BatchHashJoin { type: Inner, predicate: t.x IS NOT DISTINCT FROM t.x, output: all } + ├─BatchExchange { order: [], dist: HashShard(t.x) } + │ └─BatchScan { table: t, columns: [t.x, t.arr], distribution: SomeShard } + └─BatchExchange { order: [], dist: HashShard(t.x) } + └─BatchNestedLoopJoin { type: Inner, predicate: true, output: all } + ├─BatchExchange { order: [], dist: Single } + │ └─BatchHashAgg { group_key: [t.x], aggs: [] } + │ └─BatchExchange { order: [], dist: HashShard(t.x) } + │ └─BatchScan { table: t, columns: [t.x], distribution: SomeShard } + └─BatchValues { rows: [[]] } stream_error: |- Not supported: streaming nested-loop join HINT: The non-equal join in the query requires a nested-loop join executor, which could be very expensive to run. Consider rewriting the query to use dynamic filter as a substitute if possible. diff --git a/src/frontend/src/optimizer/logical_optimization.rs b/src/frontend/src/optimizer/logical_optimization.rs index b2047d7cae089..ce52e12486e94 100644 --- a/src/frontend/src/optimizer/logical_optimization.rs +++ b/src/frontend/src/optimizer/logical_optimization.rs @@ -118,6 +118,14 @@ static TABLE_FUNCTION_TO_PROJECT_SET: LazyLock = LazyLock::ne ) }); +static VALUES_EXTRACT_PROJECT: LazyLock = LazyLock::new(|| { + OptimizationStage::new( + "Values Extract Project", + vec![ValuesExtractProjectRule::create()], + ApplyOrder::TopDown, + ) +}); + static SIMPLE_UNNESTING: LazyLock = LazyLock::new(|| { OptimizationStage::new( "Simple Unnesting", @@ -423,6 +431,8 @@ impl LogicalOptimizer { plan = Self::predicate_pushdown(plan, explain_trace, ctx); // In order to unnest a table function, we need to convert it into a `project_set` first. plan = plan.optimize_by_rules(&TABLE_FUNCTION_TO_PROJECT_SET); + // In order to unnest values with correlated input ref, we need to extract project first. + plan = plan.optimize_by_rules(&VALUES_EXTRACT_PROJECT); // General Unnesting. // Translate Apply, push Apply down the plan and finally replace Apply with regular inner // join. diff --git a/src/frontend/src/optimizer/plan_visitor/plan_correlated_id_finder.rs b/src/frontend/src/optimizer/plan_visitor/plan_correlated_id_finder.rs index 7dd32e29b98bb..7ff5d0adb7c0a 100644 --- a/src/frontend/src/optimizer/plan_visitor/plan_correlated_id_finder.rs +++ b/src/frontend/src/optimizer/plan_visitor/plan_correlated_id_finder.rs @@ -129,6 +129,10 @@ impl ExprCorrelatedIdFinder { pub fn contains(&self, correlated_id: &CorrelatedId) -> bool { self.correlated_id_set.contains(correlated_id) } + + pub fn has_correlated_input_ref(&self) -> bool { + !self.correlated_id_set.is_empty() + } } impl ExprVisitor for ExprCorrelatedIdFinder { diff --git a/src/frontend/src/optimizer/rule/mod.rs b/src/frontend/src/optimizer/rule/mod.rs index 7867bb1bb54f9..723f8436d39e5 100644 --- a/src/frontend/src/optimizer/rule/mod.rs +++ b/src/frontend/src/optimizer/rule/mod.rs @@ -150,6 +150,8 @@ mod apply_hop_window_transpose_rule; pub use apply_hop_window_transpose_rule::*; mod agg_call_merge_rule; pub use agg_call_merge_rule::*; +mod values_extract_project_rule; +pub use values_extract_project_rule::*; #[macro_export] macro_rules! for_all_rules { @@ -215,6 +217,7 @@ macro_rules! for_all_rules { , { AggGroupBySimplifyRule } , { ApplyHopWindowTransposeRule } , { AggCallMergeRule } + , { ValuesExtractProjectRule } } }; } diff --git a/src/frontend/src/optimizer/rule/values_extract_project_rule.rs b/src/frontend/src/optimizer/rule/values_extract_project_rule.rs new file mode 100644 index 0000000000000..41b1528f282e8 --- /dev/null +++ b/src/frontend/src/optimizer/rule/values_extract_project_rule.rs @@ -0,0 +1,61 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use risingwave_common::catalog::{Field, Schema}; +use risingwave_common::types::DataType; + +use super::{BoxedRule, Rule}; +use crate::expr::{ExprImpl, ExprVisitor}; +use crate::optimizer::plan_node::generic::GenericPlanRef; +use crate::optimizer::plan_node::{LogicalProject, LogicalValues}; +use crate::optimizer::plan_visitor::ExprCorrelatedIdFinder; +use crate::optimizer::PlanRef; + +pub struct ValuesExtractProjectRule {} +impl Rule for ValuesExtractProjectRule { + fn apply(&self, plan: PlanRef) -> Option { + let old_values: &LogicalValues = plan.as_logical_values()?; + + let mut expr_correlated_id_finder = ExprCorrelatedIdFinder::default(); + + if old_values.rows().len() != 1 { + return None; + } + + old_values.rows()[0] + .iter() + .for_each(|expr| expr_correlated_id_finder.visit_expr(expr)); + + if !expr_correlated_id_finder.has_correlated_input_ref() { + return None; + } + + let new_values = LogicalValues::create( + vec![vec![ExprImpl::literal_bigint(1)]], + Schema::new(vec![Field::with_name(DataType::Int64, "$const")]), + old_values.ctx(), + ); + + Some(LogicalProject::create( + new_values, + old_values.rows()[0].clone(), + )) + } +} + +impl ValuesExtractProjectRule { + pub fn create() -> BoxedRule { + Box::new(ValuesExtractProjectRule {}) + } +} From fdfbf5306a29694e7d6556cd8498c6f782a6063f Mon Sep 17 00:00:00 2001 From: Chengyou Liu <35356271+cyliu0@users.noreply.github.com> Date: Thu, 2 Nov 2023 09:44:02 +0800 Subject: [PATCH 13/17] test(connector): add PG CDC data compatibility test (#13176) --- .../postgres-cdc/compatibility-pg.sql | 37 ++++ .../postgres-cdc/compatibility-rw.sql | 44 ++++ integration_tests/postgres-cdc/data_check | 2 +- .../postgres-cdc/docker-compose.yml | 11 +- .../postgres-cdc/postgresql-datatypes.yml | 86 ++++++++ .../scripts/compatibility/cli.py | 82 +++++++ .../scripts/compatibility/compatibility.py | 200 ++++++++++++++++++ 7 files changed, 458 insertions(+), 4 deletions(-) create mode 100644 integration_tests/postgres-cdc/compatibility-pg.sql create mode 100644 integration_tests/postgres-cdc/compatibility-rw.sql create mode 100644 integration_tests/postgres-cdc/postgresql-datatypes.yml create mode 100644 integration_tests/scripts/compatibility/cli.py create mode 100644 integration_tests/scripts/compatibility/compatibility.py diff --git a/integration_tests/postgres-cdc/compatibility-pg.sql b/integration_tests/postgres-cdc/compatibility-pg.sql new file mode 100644 index 0000000000000..6f96aa1e4c65c --- /dev/null +++ b/integration_tests/postgres-cdc/compatibility-pg.sql @@ -0,0 +1,37 @@ +DROP TABLE IF EXISTS postgres_all_types; +CREATE TABLE IF NOT EXISTS postgres_all_types( +c_boolean boolean, +c_smallint smallint, +c_integer integer, +c_bigint bigint, +c_decimal decimal, +c_real real, +c_double_precision double precision, +c_varchar varchar, +c_bytea bytea, +c_date date, +c_time time, +c_timestamp timestamp, +c_timestamptz timestamptz, +c_interval interval, +c_jsonb jsonb, +c_boolean_array boolean[], +c_smallint_array smallint[], +c_integer_array integer[], +c_bigint_array bigint[], +c_decimal_array decimal[], +c_real_array real[], +c_double_precision_array double precision[], +c_varchar_array varchar[], +c_bytea_array bytea[], +c_date_array date[], +c_time_array time[], +c_timestamp_array timestamp[], +c_timestamptz_array timestamptz[], +c_interval_array interval[], +c_jsonb_array jsonb[], +PRIMARY KEY (c_boolean,c_bigint,c_date) +); +INSERT INTO postgres_all_types VALUES ( False, 0, 0, 0, 0, 0, 0, '', '\x00', '0001-01-01', '00:00:00', '0001-01-01 00:00:00'::timestamp, '0001-01-01 00:00:00'::timestamptz, interval '0 second', '{}', array[]::boolean[], array[]::smallint[], array[]::integer[], array[]::bigint[], array[]::decimal[], array[]::real[], array[]::double precision[], array[]::varchar[], array[]::bytea[], array[]::date[], array[]::time[], array[]::timestamp[], array[]::timestamptz[], array[]::interval[], array[]::jsonb[]); +INSERT INTO postgres_all_types VALUES ( False, -32767, -2147483647, -9223372036854775807, -10.0, -9999.999999, -10000.0, '', '\x00', '0001-01-01', '00:00:00', '0001-01-01 00:00:00'::timestamp, '0001-01-01 00:00:00'::timestamptz, interval '0 second', '{}', array[False::boolean]::boolean[], array[-32767::smallint]::smallint[], array[-2147483647::integer]::integer[], array[-9223372036854775807::bigint]::bigint[], array[-10.0::decimal]::decimal[], array[-9999.999999::real]::real[], array[-10000.0::double precision]::double precision[], array[''::varchar]::varchar[], array['\x00'::bytea]::bytea[], array['0001-01-01'::date]::date[], array['00:00:00'::time]::time[], array['0001-01-01 00:00:00'::timestamp::timestamp]::timestamp[], array['0001-01-01 00:00:00'::timestamptz::timestamptz]::timestamptz[], array[interval '0 second'::interval]::interval[], array['{}'::jsonb]::jsonb[]); +INSERT INTO postgres_all_types VALUES ( True, 32767, 2147483647, 9223372036854775807, -10.0, 9999.999999, 10000.0, 'zzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzz', '\xffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff', '9999-12-31', '23:59:59', '9999-12-31 23:59:59'::timestamp, '9999-12-31 23:59:59'::timestamptz, interval '9990 year', '{"whatever":"meaningless"}', array[True::boolean]::boolean[], array[32767::smallint]::smallint[], array[2147483647::integer]::integer[], array[9223372036854775807::bigint]::bigint[], array[-10.0::decimal]::decimal[], array[9999.999999::real]::real[], array[10000.0::double precision]::double precision[], array['zzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzz'::varchar]::varchar[], array['\xffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff'::bytea]::bytea[], array['9999-12-31'::date]::date[], array['23:59:59'::time]::time[], array['9999-12-31 23:59:59'::timestamp::timestamp]::timestamp[], array['9999-12-31 23:59:59'::timestamptz::timestamptz]::timestamptz[], array[interval '9990 year'::interval]::interval[], array['{"whatever":"meaningless"}'::jsonb]::jsonb[]); diff --git a/integration_tests/postgres-cdc/compatibility-rw.sql b/integration_tests/postgres-cdc/compatibility-rw.sql new file mode 100644 index 0000000000000..7063d774bd1f1 --- /dev/null +++ b/integration_tests/postgres-cdc/compatibility-rw.sql @@ -0,0 +1,44 @@ +DROP TABLE IF EXISTS postgres_all_types; +CREATE TABLE IF NOT EXISTS postgres_all_types( +c_boolean boolean, +c_smallint smallint, +c_integer integer, +c_bigint bigint, +c_decimal decimal, +c_real real, +c_double_precision double precision, +c_varchar varchar, +c_bytea bytea, +c_date date, +c_time time, +c_timestamp timestamp, +c_timestamptz timestamptz, +c_interval interval, +c_jsonb jsonb, +c_boolean_array boolean[], +c_smallint_array smallint[], +c_integer_array integer[], +c_bigint_array bigint[], +c_decimal_array decimal[], +c_real_array real[], +c_double_precision_array double precision[], +c_varchar_array varchar[], +c_bytea_array bytea[], +c_date_array date[], +c_time_array time[], +c_timestamp_array timestamp[], +c_timestamptz_array timestamptz[], +c_interval_array interval[], +c_jsonb_array jsonb[], +PRIMARY KEY (c_boolean,c_bigint,c_date) +) WITH ( +connector = 'postgres-cdc', +hostname = 'postgres', +port = '5432', +username = 'myuser', +password = '123456', +database.name = 'mydb', +schema.name = 'public', +table.name = 'postgres_all_types', +slot.name = 'postgres_all_types' +); diff --git a/integration_tests/postgres-cdc/data_check b/integration_tests/postgres-cdc/data_check index 4e00aba632aaa..a4437610dfa7c 100644 --- a/integration_tests/postgres-cdc/data_check +++ b/integration_tests/postgres-cdc/data_check @@ -1 +1 @@ -person,city_population,nexmark_q8 \ No newline at end of file +person,city_population,nexmark_q8,postgres_all_types diff --git a/integration_tests/postgres-cdc/docker-compose.yml b/integration_tests/postgres-cdc/docker-compose.yml index 5031f9ef620c7..5b98b7ba2fcd2 100644 --- a/integration_tests/postgres-cdc/docker-compose.yml +++ b/integration_tests/postgres-cdc/docker-compose.yml @@ -58,20 +58,25 @@ services: command: - /bin/sh - -c - - "psql postgresql://myuser:123456@postgres:5432/mydb < postgres_prepare.sql" + - "psql postgresql://myuser:123456@postgres:5432/mydb < postgres_prepare.sql && + psql postgresql://myuser:123456@postgres:5432/mydb < compatibility-pg.sql && + sleep 5 && + psql postgresql://root:@frontend-node-0:4566/dev < compatibility-rw.sql" volumes: - "./postgres_prepare.sql:/postgres_prepare.sql" + - "./compatibility-pg.sql:/compatibility-pg.sql" + - "./compatibility-rw.sql:/compatibility-rw.sql" container_name: postgres_prepare restart: on-failure datagen_tpch: image: ghcr.io/risingwavelabs/go-tpc:v0.1 - depends_on: [postgres] + depends_on: [ postgres ] command: tpch prepare --sf 1 --threads 4 -d postgres -U myuser -p '123456' -H postgres -D mydb -P 5432 --conn-params sslmode=disable container_name: datagen_tpch restart: on-failure datagen_kafka: build: ../datagen - depends_on: [message_queue] + depends_on: [ message_queue ] command: - /bin/sh - -c diff --git a/integration_tests/postgres-cdc/postgresql-datatypes.yml b/integration_tests/postgres-cdc/postgresql-datatypes.yml new file mode 100644 index 0000000000000..ee067d241511d --- /dev/null +++ b/integration_tests/postgres-cdc/postgresql-datatypes.yml @@ -0,0 +1,86 @@ +pk_types: + - boolean + - bigint + - date +datatypes: + - name: boolean + aliases: + - bool + zero: false + minimum: false + maximum: true + rw_type: boolean + - name: smallint + zero: 0 + minimum: -32767 + maximum: 32767 + rw_type: + - name: integer + aliases: + - int + zero: 0 + minimum: -2147483647 + maximum: 2147483647 + - name: bigint + zero: 0 + minimum: -9223372036854775807 + maximum: 9223372036854775807 + - name: decimal + aliases: + - numeric + zero: 0 + minimum: -9.9999999999999999999999999999999 + maximum: -9.9999999999999999999999999999999 + - name: real + zero: 0 + minimum: -9999.999999 + maximum: 9999.999999 + - name: double precision + zero: 0 + minimum: -9999.99999999999999 + maximum: 9999.99999999999999 + - name: varchar + aliases: + - character varying + - string + zero: "''" + minimum: "''" + maximum_gen_py: "\"'{}'\".format('z'*65535)" + - name: bytea + zero: "'\\x00'" + minimum: "'\\x00'" + maximum_gen_py: "\"'{}'\".format('\\\\x'+'f'*65534)" + - name: date + zero: "'0001-01-01'" + minimum: "'0001-01-01'" + maximum: "'9999-12-31'" + - name: time + aliases: + - time without time zone + zero: "'00:00:00'" + minimum: "'00:00:00'" + maximum: "'23:59:59'" + - name: timestamp + aliases: + - timestamp without time zone + zero: "'0001-01-01 00:00:00'::timestamp" + minimum: "'0001-01-01 00:00:00'::timestamp" + maximum: "'9999-12-31 23:59:59'::timestamp" + - name: timestamptz + aliases: + - timestamp with time zone + zero: "'0001-01-01 00:00:00'::timestamptz" + minimum: "'0001-01-01 00:00:00'::timestamptz" + maximum: "'9999-12-31 23:59:59'::timestamptz" + - name: interval + zero: "interval '0 second'" + minimum: "interval '0 second'" + maximum: "interval '9990 year'" + - name: jsonb + zero: "'{}'" + minimum: "'{}'" + maximum: "'{\"whatever\":\"meaningless\"}'" + + + + diff --git a/integration_tests/scripts/compatibility/cli.py b/integration_tests/scripts/compatibility/cli.py new file mode 100644 index 0000000000000..772e4b967d6c7 --- /dev/null +++ b/integration_tests/scripts/compatibility/cli.py @@ -0,0 +1,82 @@ +import click +from compatibility import * + + +@click.group() +def cli(): + pass + + +@click.command() +@click.option("--datatype-file", default="./compatibility/risingwave-datatypes.yml", help="data type file") +@click.option("--database-type", default="postgres", help="database type") +def gen_select_sql(datatype_file: str, database_type: str): + database_type = database_type.lower() + with open(datatype_file) as f: + datatypes_map = yaml.safe_load(f) + datatype_list = [] + for data_type in datatypes_map["datatypes"]: + new_datatype = DataType(**data_type) + if database_type == "mysql": + new_datatype = MysqlDataType(**data_type) + datatype_list.append(new_datatype) + print(new_datatype.select_zero_sql()) + print(new_datatype.select_min_sql()) + print(new_datatype.select_max_sql()) + if data_type in ["postgres", "risingwave"]: + print(new_datatype.select_array_zero_sql()) + print(new_datatype.select_array_min_sql()) + print(new_datatype.select_array_max_sql()) + + +@click.command() +@click.option("--datatype-file", default="./compatibility/risingwave-datatypes.yml", help="data type file") +@click.option("--database-type", default="postgres", help="database type") +def gen_ddl_dml(datatype_file: str, database_type: str): + database_type = database_type.lower() + with open(datatype_file) as f: + datatypes_map = yaml.safe_load(f) + datatype_list = [] + for data_type in datatypes_map["datatypes"]: + new_datatype = DataType(**data_type) + if database_type == "mysql": + new_datatype = MysqlDataType(**data_type) + datatype_list.append(new_datatype) + table_sql_generator = TableSqlGenerator( + name='{}_all_types'.format(database_type), + enable_array=False, + enable_struct=False, + pk_types=datatypes_map.get("pk_types", []), + datatypes=datatype_list + ) + if database_type == "mysql": + pass + elif database_type == "postgres": + table_sql_generator = PostgresTableSqlGenerator( + name='{}_all_types'.format(database_type), + enable_array=True, + enable_struct=False, + pk_types=datatypes_map.get("pk_types", []), + datatypes=datatype_list + ) + elif database_type == "risingwave": + table_sql_generator = RisingwaveTableSqlGenerator( + name='{}_all_types'.format(database_type), + enable_array=True, + enable_struct=True, + pk_types=datatypes_map.get("pk_types", []), + datatypes=datatype_list + ) + + print(table_sql_generator.drop_table_sql()) + print(table_sql_generator.create_table_sql()) + print(table_sql_generator.insert_zero_sql()) + print(table_sql_generator.insert_min_sql()) + print(table_sql_generator.insert_max_sql()) + + +cli.add_command(gen_select_sql) +cli.add_command(gen_ddl_dml) + +if __name__ == '__main__': + cli() diff --git a/integration_tests/scripts/compatibility/compatibility.py b/integration_tests/scripts/compatibility/compatibility.py new file mode 100644 index 0000000000000..a7c67fcd02171 --- /dev/null +++ b/integration_tests/scripts/compatibility/compatibility.py @@ -0,0 +1,200 @@ +#!/usr/bin/python3 + +import yaml + + +class DataType: + def __init__(self, name: str, zero=None, minimum=None, maximum=None, maximum_gen_py="", null="null", aliases=None, + rw_type=None): + self.name = name + self.col_name = "c_" + self.name.replace(" ", "_") + self.array_col_name = self.col_name + "_array" + self.aliases = aliases + self.zero = zero + self.min = minimum + self.max = maximum + self.null = null + self.rw_type = rw_type + if maximum_gen_py != "": + exec("self.max={}".format(maximum_gen_py)) + + def cast(self, value): + return '{}::{}'.format(value, self.name) + + def array_cast(self, value): + return '{}::{}'.format(value, self.array_type()) + + def array_type(self): + return self.name + "[]" + + def array_zero(self): + return "array[]" + + def array_min(self): + return "array[{}]".format(self.cast(self.min)) + + def array_max(self): + return "array[{}]".format(self.cast(self.max)) + + def select_zero_sql(self): + return "SELECT {};".format(self.cast(self.zero)) + + def select_min_sql(self): + return "SELECT {};".format(self.cast(self.min)) + + def select_max_sql(self): + return "SELECT {};".format(self.cast(self.max)) + + def select_array_zero_sql(self): + return "SELECT {};".format(self.array_cast(self.array_zero())) + + def select_array_min_sql(self): + return "SELECT {}".format(self.array_cast(self.array_min())) + + def select_array_max_sql(self): + return "SELECT {}".format(self.array_cast(self.array_max())) + + +class MysqlDataType(DataType): + def cast(self, value): + return "CAST({} AS {})".format(value, self.name) + + +class TableSqlGenerator: + def __init__(self, name: str, enable_array: bool, enable_struct: bool, pk_types: list[str], + datatypes: list[DataType]): + self.table_name = name + self.pk_types = pk_types + self.datatypes = datatypes + self.enable_array = enable_array + self.enable_struct = enable_struct + self.null = "null" + + def struct_type(self): + pass + + def struct_values(self, value): + return 'ROW({})'.format(value) + + def create_table_sql(self): + prefix = "CREATE TABLE IF NOT EXISTS " + self.table_name + cols = "(\n" + pk_col_names = [] + for data_type in self.datatypes: + if cols != "(\n": + cols += ",\n" + cols += data_type.col_name + " " + data_type.name + if data_type.name in self.pk_types: + pk_col_names.append(data_type.col_name) + if self.enable_array: + for data_type in self.datatypes: + cols += ",\n" + cols += data_type.array_col_name + " " + data_type.array_type() + if self.enable_struct: + cols += ",\n" + cols += "c_struct {}".format(self.struct_type()) + if self.pk_types: + cols += ",\nPRIMARY KEY (" + ",".join(pk_col_names) + ")\n);" + return prefix + cols + + def drop_table_sql(self): + return "DROP TABLE IF EXISTS {};".format(self.table_name) + + def insert_null_sql(self, ): + prefix = "INSERT INTO " + self.table_name + "VALUES (" + cols = "" + for data_type in self.datatypes: + if cols != "": + cols += "," + cols += data_type.null + if self.enable_array: + for data_type in self.datatypes: + cols += "," + cols += data_type.null + if self.enable_struct: + cols += "," + self.null + return prefix + cols + ");" + + def zero_values(self): + cols = "" + for data_type in self.datatypes: + if cols != "": + cols += "," + cols = '{} {}'.format(cols, data_type.zero) + if self.enable_array: + for data_type in self.datatypes: + cols = '{}, {}'.format(cols, data_type.array_cast(data_type.array_zero())) + if self.enable_struct: + cols += "," + self.struct_values(cols) + return cols + + def min_values(self): + cols = "" + for data_type in self.datatypes: + if cols != "": + cols += "," + cols = '{} {}'.format(cols, data_type.min) + if self.enable_array: + for data_type in self.datatypes: + cols = '{}, {}'.format(cols, data_type.array_cast(data_type.array_min())) + if self.enable_struct: + cols += "," + self.struct_values(cols) + return cols + + def max_values(self): + cols = "" + for data_type in self.datatypes: + if cols != "": + cols += "," + cols = '{} {}'.format(cols, data_type.max) + if self.enable_array: + for data_type in self.datatypes: + cols = '{}, {}'.format(cols, data_type.array_cast(data_type.array_max())) + if self.enable_struct: + cols += "," + self.struct_values(cols) + return cols + + def insert_zero_sql(self): + prefix = "INSERT INTO " + self.table_name + " VALUES (" + cols = self.zero_values() + return prefix + cols + ");" + + def insert_min_sql(self): + prefix = "INSERT INTO " + self.table_name + " VALUES (" + cols = self.min_values() + return prefix + cols + ");" + + def insert_max_sql(self): + prefix = "INSERT INTO " + self.table_name + " VALUES (" + cols = self.max_values() + return prefix + cols + ");" + + +class RisingwaveTableSqlGenerator(TableSqlGenerator): + def struct_type(self): + cols = "" + for data_type in self.datatypes: + if cols != "": + cols += ",\n" + cols = cols + data_type.col_name + " " + data_type.name + if self.enable_array: + for data_type in self.datatypes: + cols += ",\n" + cols = cols + data_type.array_col_name + " " + data_type.array_type() + return "struct <\n{}\n>".format(cols) + + +class PostgresTableSqlGenerator(TableSqlGenerator): + def struct_type(self): + cols = "" + for data_type in self.datatypes: + if cols != "": + cols += ",\n" + cols = cols + data_type.col_name + " " + data_type.name + if self.enable_array: + for data_type in self.datatypes: + cols += ",\n" + cols = cols + data_type.array_col_name + " " + data_type.array_type() + print("DROP TYPE IF EXISTS struct;") + print("CREATE TYPE struct AS (\n{}\n);".format(cols)) + return "struct" From 41d82d2176d31a7192d44570e16c9b13931a1aec Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 2 Nov 2023 12:45:54 +0800 Subject: [PATCH 14/17] chore(deps): Bump procfs from 0.15.1 to 0.16.0 (#13203) Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- Cargo.lock | 24 +++++++++++++++++------- src/common/Cargo.toml | 2 +- src/storage/Cargo.toml | 2 +- 3 files changed, 19 insertions(+), 9 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 8fef1708dc5fa..ace586f107531 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -6376,15 +6376,25 @@ dependencies = [ [[package]] name = "procfs" -version = "0.15.1" +version = "0.16.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "943ca7f9f29bab5844ecd8fdb3992c5969b6622bb9609b9502fef9b4310e3f1f" +checksum = "731e0d9356b0c25f16f33b5be79b1c57b562f141ebfcdb0ad8ac2c13a24293b4" dependencies = [ - "bitflags 1.3.2", - "byteorder", + "bitflags 2.4.0", "hex", "lazy_static", - "rustix 0.36.16", + "procfs-core", + "rustix 0.38.21", +] + +[[package]] +name = "procfs-core" +version = "0.16.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2d3554923a69f4ce04c4a754260c338f505ce22642d3830e049a399fc2059a29" +dependencies = [ + "bitflags 2.4.0", + "hex", ] [[package]] @@ -7285,7 +7295,7 @@ dependencies = [ "postgres-types", "prehash", "pretty_assertions", - "procfs 0.15.1", + "procfs 0.16.0", "prometheus", "prost 0.12.1", "rand", @@ -8304,7 +8314,7 @@ dependencies = [ "nix 0.27.1", "num-integer", "parking_lot 0.12.1", - "procfs 0.15.1", + "procfs 0.16.0", "prometheus", "prost 0.12.1", "rand", diff --git a/src/common/Cargo.toml b/src/common/Cargo.toml index f44c0c9ba8a5d..c0d6305009d75 100644 --- a/src/common/Cargo.toml +++ b/src/common/Cargo.toml @@ -113,7 +113,7 @@ tower-layer = "0.3.2" tower-service = "0.3.2" [target.'cfg(target_os = "linux")'.dependencies] -procfs = { version = "0.15", default-features = false } +procfs = { version = "0.16", default-features = false } libc = "0.2" [target.'cfg(target_os = "macos")'.dependencies] diff --git a/src/storage/Cargo.toml b/src/storage/Cargo.toml index fc01eba294564..0a948c52dbaa6 100644 --- a/src/storage/Cargo.toml +++ b/src/storage/Cargo.toml @@ -73,7 +73,7 @@ xxhash-rust = { version = "0.8.7", features = ["xxh32", "xxh64"] } zstd = { version = "0.13", default-features = false } [target.'cfg(target_os = "linux")'.dependencies] -procfs = { version = "0.15", default-features = false } +procfs = { version = "0.16", default-features = false } libc = "0.2" nix = { version = "0.27", features = ["fs", "mman"] } From d92ace1d6b528a47278807054830095cb9ed49bd Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 2 Nov 2023 13:51:32 +0800 Subject: [PATCH 15/17] chore(deps): Bump gcp-bigquery-client from 0.17.1 to 0.18.0 (#13202) Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- Cargo.lock | 4 ++-- src/connector/Cargo.toml | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index ace586f107531..a0b3e08b46441 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3452,9 +3452,9 @@ dependencies = [ [[package]] name = "gcp-bigquery-client" -version = "0.17.1" +version = "0.18.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "576b349942a1b96327b1b2e50d271d3d54dc9669eb38271c830916168e753820" +checksum = "b0ce6fcbdaca0a4521a734f2bc7f2f6bd872fe40576e24f8bd0b05732c19a74f" dependencies = [ "async-stream", "async-trait", diff --git a/src/connector/Cargo.toml b/src/connector/Cargo.toml index 883303ae72e63..13eaa1fcb8785 100644 --- a/src/connector/Cargo.toml +++ b/src/connector/Cargo.toml @@ -49,7 +49,7 @@ easy-ext = "1" enum-as-inner = "0.6" futures = { version = "0.3", default-features = false, features = ["alloc"] } futures-async-stream = { workspace = true } -gcp-bigquery-client = "0.17.1" +gcp-bigquery-client = "0.18.0" glob = "0.3" google-cloud-pubsub = "0.20" http = "0.2" From 73dcbc2701a22530f698ed03ac1613bc70cab797 Mon Sep 17 00:00:00 2001 From: August Date: Thu, 2 Nov 2023 13:57:10 +0800 Subject: [PATCH 16/17] feat: introduce user and privilege functions based on sql meta store (#13189) --- .../migration/src/m20230908_072257_init.rs | 36 +- src/meta/model_v2/src/connection.rs | 8 +- src/meta/model_v2/src/database.rs | 6 +- src/meta/model_v2/src/function.rs | 18 +- src/meta/model_v2/src/lib.rs | 3 + src/meta/model_v2/src/schema.rs | 6 +- src/meta/model_v2/src/user.rs | 39 +- src/meta/model_v2/src/user_privilege.rs | 64 +- src/meta/model_v2/src/view.rs | 12 +- src/meta/model_v2/src/worker.rs | 12 +- src/meta/src/controller/catalog.rs | 57 +- src/meta/src/controller/cluster.rs | 35 +- src/meta/src/controller/mod.rs | 4 +- src/meta/src/controller/system_param.rs | 24 +- src/meta/src/controller/user.rs | 695 ++++++++++++++++++ src/meta/src/controller/utils.rs | 188 ++++- 16 files changed, 1102 insertions(+), 105 deletions(-) create mode 100644 src/meta/src/controller/user.rs diff --git a/src/meta/model_v2/migration/src/m20230908_072257_init.rs b/src/meta/model_v2/migration/src/m20230908_072257_init.rs index e2aaa0da59d51..9d14f8bac7201 100644 --- a/src/meta/model_v2/migration/src/m20230908_072257_init.rs +++ b/src/meta/model_v2/migration/src/m20230908_072257_init.rs @@ -124,13 +124,12 @@ impl MigrationTrait for Migration { .primary_key() .auto_increment(), ) - .col(ColumnDef::new(User::Name).string().not_null()) + .col(ColumnDef::new(User::Name).string().unique_key().not_null()) .col(ColumnDef::new(User::IsSuper).boolean().not_null()) .col(ColumnDef::new(User::CanCreateDb).boolean().not_null()) .col(ColumnDef::new(User::CanCreateUser).boolean().not_null()) .col(ColumnDef::new(User::CanLogin).boolean().not_null()) - .col(ColumnDef::new(User::AuthType).string()) - .col(ColumnDef::new(User::AuthValue).string()) + .col(ColumnDef::new(User::AuthInfo).json()) .to_owned(), ) .await?; @@ -197,6 +196,7 @@ impl MigrationTrait for Migration { .primary_key() .auto_increment(), ) + .col(ColumnDef::new(UserPrivilege::DependentId).integer()) .col(ColumnDef::new(UserPrivilege::UserId).integer().not_null()) .col(ColumnDef::new(UserPrivilege::Oid).integer().not_null()) .col( @@ -204,12 +204,20 @@ impl MigrationTrait for Migration { .integer() .not_null(), ) - .col(ColumnDef::new(UserPrivilege::Actions).string().not_null()) + .col(ColumnDef::new(UserPrivilege::Action).string().not_null()) .col( ColumnDef::new(UserPrivilege::WithGrantOption) .boolean() .not_null(), ) + .foreign_key( + &mut ForeignKey::create() + .name("FK_user_privilege_dependent_id") + .from(UserPrivilege::Table, UserPrivilege::DependentId) + .to(UserPrivilege::Table, UserPrivilege::Id) + .on_delete(ForeignKeyAction::Cascade) + .to_owned(), + ) .foreign_key( &mut ForeignKey::create() .name("FK_user_privilege_user_id") @@ -230,6 +238,7 @@ impl MigrationTrait for Migration { .name("FK_user_privilege_oid") .from(UserPrivilege::Table, UserPrivilege::Oid) .to(Object::Table, Object::Oid) + .on_delete(ForeignKeyAction::Cascade) .to_owned(), ) .to_owned(), @@ -651,6 +660,19 @@ impl MigrationTrait for Migration { .to_owned(), ) .await?; + manager + .create_index( + MigrationIndex::create() + .table(UserPrivilege::Table) + .name("idx_user_privilege_item") + .unique() + .col(UserPrivilege::UserId) + .col(UserPrivilege::Oid) + .col(UserPrivilege::Action) + .col(UserPrivilege::GrantedBy) + .to_owned(), + ) + .await?; // 4. initialize data. let insert_cluster_id = Query::insert() @@ -799,18 +821,18 @@ enum User { CanCreateDb, CanCreateUser, CanLogin, - AuthType, - AuthValue, + AuthInfo, } #[derive(DeriveIden)] enum UserPrivilege { Table, Id, + DependentId, UserId, Oid, GrantedBy, - Actions, + Action, WithGrantOption, } diff --git a/src/meta/model_v2/src/connection.rs b/src/meta/model_v2/src/connection.rs index 8cff6b2a6025b..a7a7e1ea33952 100644 --- a/src/meta/model_v2/src/connection.rs +++ b/src/meta/model_v2/src/connection.rs @@ -15,7 +15,7 @@ use risingwave_pb::catalog::connection::PbInfo; use risingwave_pb::catalog::PbConnection; use sea_orm::entity::prelude::*; -use sea_orm::ActiveValue; +use sea_orm::ActiveValue::Set; use crate::{ConnectionId, PrivateLinkService}; @@ -71,9 +71,9 @@ impl From for ActiveModel { }; Self { - connection_id: ActiveValue::Set(conn.id as _), - name: ActiveValue::Set(conn.name), - info: ActiveValue::Set(PrivateLinkService(private_link_srv)), + connection_id: Set(conn.id as _), + name: Set(conn.name), + info: Set(PrivateLinkService(private_link_srv)), } } } diff --git a/src/meta/model_v2/src/database.rs b/src/meta/model_v2/src/database.rs index 95ff3a8aee8e6..25f164c45805e 100644 --- a/src/meta/model_v2/src/database.rs +++ b/src/meta/model_v2/src/database.rs @@ -14,7 +14,7 @@ use risingwave_pb::catalog::PbDatabase; use sea_orm::entity::prelude::*; -use sea_orm::ActiveValue; +use sea_orm::ActiveValue::Set; use crate::DatabaseId; @@ -50,8 +50,8 @@ impl ActiveModelBehavior for ActiveModel {} impl From for ActiveModel { fn from(db: PbDatabase) -> Self { Self { - database_id: ActiveValue::Set(db.id), - name: ActiveValue::Set(db.name), + database_id: Set(db.id), + name: Set(db.name), } } } diff --git a/src/meta/model_v2/src/function.rs b/src/meta/model_v2/src/function.rs index 4126dddc0f5ee..c4774b177eabc 100644 --- a/src/meta/model_v2/src/function.rs +++ b/src/meta/model_v2/src/function.rs @@ -15,7 +15,7 @@ use risingwave_pb::catalog::function::Kind; use risingwave_pb::catalog::PbFunction; use sea_orm::entity::prelude::*; -use sea_orm::ActiveValue; +use sea_orm::ActiveValue::Set; use crate::{DataType, DataTypeArray, FunctionId}; @@ -77,14 +77,14 @@ impl From for FunctionKind { impl From for ActiveModel { fn from(function: PbFunction) -> Self { Self { - function_id: ActiveValue::Set(function.id as _), - name: ActiveValue::Set(function.name), - arg_types: ActiveValue::Set(DataTypeArray(function.arg_types)), - return_type: ActiveValue::Set(DataType(function.return_type.unwrap())), - language: ActiveValue::Set(function.language), - link: ActiveValue::Set(function.link), - identifier: ActiveValue::Set(function.identifier), - kind: ActiveValue::Set(function.kind.unwrap().into()), + function_id: Set(function.id as _), + name: Set(function.name), + arg_types: Set(DataTypeArray(function.arg_types)), + return_type: Set(DataType(function.return_type.unwrap())), + language: Set(function.language), + link: Set(function.link), + identifier: Set(function.identifier), + kind: Set(function.kind.unwrap().into()), } } } diff --git a/src/meta/model_v2/src/lib.rs b/src/meta/model_v2/src/lib.rs index ba2e114d5060d..0d0e373076703 100644 --- a/src/meta/model_v2/src/lib.rs +++ b/src/meta/model_v2/src/lib.rs @@ -61,6 +61,8 @@ pub type ViewId = ObjectId; pub type FunctionId = ObjectId; pub type ConnectionId = ObjectId; pub type UserId = u32; +pub type PrivilegeId = u32; + pub type HummockVersionId = u64; pub type Epoch = u64; pub type CompactionGroupId = u64; @@ -157,6 +159,7 @@ derive_from_json_struct!( PrivateLinkService, risingwave_pb::catalog::connection::PbPrivateLinkService ); +derive_from_json_struct!(AuthInfo, risingwave_pb::user::PbAuthInfo); derive_from_json_struct!(StreamNode, risingwave_pb::stream_plan::PbStreamNode); derive_from_json_struct!(Dispatchers, Vec); diff --git a/src/meta/model_v2/src/schema.rs b/src/meta/model_v2/src/schema.rs index 0af2d7fc020c9..6417c974d29fc 100644 --- a/src/meta/model_v2/src/schema.rs +++ b/src/meta/model_v2/src/schema.rs @@ -14,7 +14,7 @@ use risingwave_pb::catalog::PbSchema; use sea_orm::entity::prelude::*; -use sea_orm::ActiveValue; +use sea_orm::ActiveValue::Set; use crate::SchemaId; @@ -49,8 +49,8 @@ impl ActiveModelBehavior for ActiveModel {} impl From for ActiveModel { fn from(schema: PbSchema) -> Self { Self { - schema_id: ActiveValue::Set(schema.id), - name: ActiveValue::Set(schema.name), + schema_id: Set(schema.id), + name: Set(schema.name), } } } diff --git a/src/meta/model_v2/src/user.rs b/src/meta/model_v2/src/user.rs index e9cd36f75fb43..716e8aa309f4d 100644 --- a/src/meta/model_v2/src/user.rs +++ b/src/meta/model_v2/src/user.rs @@ -12,22 +12,25 @@ // See the License for the specific language governing permissions and // limitations under the License. +use risingwave_pb::user::PbUserInfo; use sea_orm::entity::prelude::*; +use sea_orm::ActiveValue::Set; +use sea_orm::NotSet; -use crate::UserId; +use crate::{AuthInfo, UserId}; #[derive(Clone, Debug, PartialEq, DeriveEntityModel, Eq)] #[sea_orm(table_name = "user")] pub struct Model { #[sea_orm(primary_key)] pub user_id: UserId, + #[sea_orm(unique)] pub name: String, pub is_super: bool, pub can_create_db: bool, pub can_create_user: bool, pub can_login: bool, - pub auth_type: Option, - pub auth_value: Option, + pub auth_info: Option, } #[derive(Copy, Clone, Debug, EnumIter, DeriveRelation)] @@ -43,3 +46,33 @@ impl Related for Entity { } impl ActiveModelBehavior for ActiveModel {} + +impl From for ActiveModel { + fn from(user: PbUserInfo) -> Self { + let user_id = if user.id == 0 { NotSet } else { Set(user.id) }; + Self { + user_id, + name: Set(user.name), + is_super: Set(user.is_super), + can_create_db: Set(user.can_create_db), + can_create_user: Set(user.can_create_user), + can_login: Set(user.can_login), + auth_info: Set(user.auth_info.map(AuthInfo)), + } + } +} + +impl From for PbUserInfo { + fn from(val: Model) -> Self { + PbUserInfo { + id: val.user_id, + name: val.name, + is_super: val.is_super, + can_create_db: val.can_create_db, + can_create_user: val.can_create_user, + can_login: val.can_login, + auth_info: val.auth_info.map(|x| x.into_inner()), + grant_privileges: vec![], // fill in later + } + } +} diff --git a/src/meta/model_v2/src/user_privilege.rs b/src/meta/model_v2/src/user_privilege.rs index 7e12af225ed02..f77c146cfa66b 100644 --- a/src/meta/model_v2/src/user_privilege.rs +++ b/src/meta/model_v2/src/user_privilege.rs @@ -12,19 +12,69 @@ // See the License for the specific language governing permissions and // limitations under the License. +use risingwave_pb::user::grant_privilege::PbAction; use sea_orm::entity::prelude::*; -use crate::{ObjectId, UserId}; +use crate::{ObjectId, PrivilegeId, UserId}; + +#[derive(Clone, Debug, Hash, PartialEq, Eq, EnumIter, DeriveActiveEnum)] +#[sea_orm(rs_type = "String", db_type = "String(None)")] +pub enum Action { + #[sea_orm(string_value = "INSERT")] + Insert, + #[sea_orm(string_value = "SELECT")] + Select, + #[sea_orm(string_value = "UPDATE")] + Update, + #[sea_orm(string_value = "DELETE")] + Delete, + #[sea_orm(string_value = "USAGE")] + Usage, + #[sea_orm(string_value = "CREATE")] + Create, + #[sea_orm(string_value = "CONNECT")] + Connect, +} + +impl From for Action { + fn from(action: PbAction) -> Self { + match action { + PbAction::Unspecified => unreachable!("unspecified action"), + PbAction::Insert => Self::Insert, + PbAction::Select => Self::Select, + PbAction::Update => Self::Update, + PbAction::Delete => Self::Delete, + PbAction::Usage => Self::Usage, + PbAction::Create => Self::Create, + PbAction::Connect => Self::Connect, + } + } +} + +impl From for PbAction { + fn from(action: Action) -> Self { + match action { + Action::Insert => Self::Insert, + Action::Select => Self::Select, + Action::Update => Self::Update, + Action::Delete => Self::Delete, + Action::Usage => Self::Usage, + Action::Create => Self::Create, + Action::Connect => Self::Connect, + } + } +} #[derive(Clone, Debug, PartialEq, DeriveEntityModel, Eq)] #[sea_orm(table_name = "user_privilege")] pub struct Model { #[sea_orm(primary_key)] - pub id: i32, + pub id: PrivilegeId, + pub dependent_id: Option, pub user_id: UserId, pub oid: ObjectId, pub granted_by: UserId, - pub actions: String, + pub action: Action, pub with_grant_option: bool, } @@ -54,6 +104,14 @@ pub enum Relation { on_delete = "Cascade" )] User1, + #[sea_orm( + belongs_to = "Entity", + from = "Column::DependentId", + to = "Column::Id", + on_update = "NoAction", + on_delete = "Cascade" + )] + SelfRef, } impl Related for Entity { diff --git a/src/meta/model_v2/src/view.rs b/src/meta/model_v2/src/view.rs index 0de9ea64a616e..4547c5de559ec 100644 --- a/src/meta/model_v2/src/view.rs +++ b/src/meta/model_v2/src/view.rs @@ -14,7 +14,7 @@ use risingwave_pb::catalog::PbView; use sea_orm::entity::prelude::*; -use sea_orm::ActiveValue; +use sea_orm::ActiveValue::Set; use crate::{FieldArray, Property, ViewId}; @@ -52,11 +52,11 @@ impl ActiveModelBehavior for ActiveModel {} impl From for ActiveModel { fn from(view: PbView) -> Self { Self { - view_id: ActiveValue::Set(view.id as _), - name: ActiveValue::Set(view.name), - properties: ActiveValue::Set(Property(view.properties)), - definition: ActiveValue::Set(view.sql), - columns: ActiveValue::Set(FieldArray(view.columns)), + view_id: Set(view.id as _), + name: Set(view.name), + properties: Set(Property(view.properties)), + definition: Set(view.sql), + columns: Set(FieldArray(view.columns)), } } } diff --git a/src/meta/model_v2/src/worker.rs b/src/meta/model_v2/src/worker.rs index d164fba62b41e..a18453c67ef5d 100644 --- a/src/meta/model_v2/src/worker.rs +++ b/src/meta/model_v2/src/worker.rs @@ -15,7 +15,7 @@ use risingwave_pb::common::worker_node::PbState; use risingwave_pb::common::{PbWorkerNode, PbWorkerType}; use sea_orm::entity::prelude::*; -use sea_orm::ActiveValue; +use sea_orm::ActiveValue::Set; use crate::{TransactionId, WorkerId}; @@ -91,11 +91,11 @@ impl From<&PbWorkerNode> for ActiveModel { fn from(worker: &PbWorkerNode) -> Self { let host = worker.host.clone().unwrap(); Self { - worker_id: ActiveValue::Set(worker.id), - worker_type: ActiveValue::Set(worker.r#type().into()), - host: ActiveValue::Set(host.host), - port: ActiveValue::Set(host.port), - status: ActiveValue::Set(worker.state().into()), + worker_id: Set(worker.id), + worker_type: Set(worker.r#type().into()), + host: Set(host.host), + port: Set(host.port), + status: Set(worker.state().into()), ..Default::default() } } diff --git a/src/meta/src/controller/catalog.rs b/src/meta/src/controller/catalog.rs index 0abfd5f4b354f..998af75bf3b3d 100644 --- a/src/meta/src/controller/catalog.rs +++ b/src/meta/src/controller/catalog.rs @@ -34,9 +34,10 @@ use risingwave_pb::meta::subscribe_response::{ Info as NotificationInfo, Operation as NotificationOperation, }; use risingwave_pb::meta::{PbRelation, PbRelationGroup}; +use sea_orm::ActiveValue::Set; use sea_orm::{ - ActiveModelTrait, ActiveValue, ColumnTrait, DatabaseConnection, DatabaseTransaction, - EntityTrait, QueryFilter, QuerySelect, TransactionTrait, + ActiveModelTrait, ColumnTrait, DatabaseConnection, DatabaseTransaction, EntityTrait, + QueryFilter, QuerySelect, TransactionTrait, }; use tokio::sync::RwLock; @@ -84,7 +85,7 @@ pub(crate) struct CatalogControllerInner { } impl CatalogController { - async fn notify_frontend( + pub(crate) async fn notify_frontend( &self, operation: NotificationOperation, info: NotificationInfo, @@ -95,7 +96,7 @@ impl CatalogController { .await } - async fn notify_frontend_relation_info( + pub(crate) async fn notify_frontend_relation_info( &self, operation: NotificationOperation, relation_info: PbRelationInfo, @@ -121,10 +122,10 @@ impl CatalogController { ) -> MetaResult { let active_db = object::ActiveModel { oid: Default::default(), - obj_type: ActiveValue::Set(obj_type), - owner_id: ActiveValue::Set(owner_id), - schema_id: ActiveValue::Set(schema_id), - database_id: ActiveValue::Set(database_id), + obj_type: Set(obj_type), + owner_id: Set(owner_id), + schema_id: Set(schema_id), + database_id: Set(database_id), initialized_at: Default::default(), created_at: Default::default(), }; @@ -139,7 +140,7 @@ impl CatalogController { let db_obj = Self::create_object(&txn, ObjectType::Database, owner_id, None, None).await?; let mut db: database::ActiveModel = db.into(); - db.database_id = ActiveValue::Set(db_obj.oid); + db.database_id = Set(db_obj.oid); let db = db.insert(&txn).await?; let mut schemas = vec![]; @@ -148,8 +149,8 @@ impl CatalogController { Self::create_object(&txn, ObjectType::Schema, owner_id, Some(db_obj.oid), None) .await?; let schema = schema::ActiveModel { - schema_id: ActiveValue::Set(schema_obj.oid), - name: ActiveValue::Set(schema_name.into()), + schema_id: Set(schema_obj.oid), + name: Set(schema_name.into()), }; let schema = schema.insert(&txn).await?; schemas.push(ObjectModel(schema, schema_obj).into()); @@ -256,7 +257,7 @@ impl CatalogController { ) .await?; let mut schema: schema::ActiveModel = schema.into(); - schema.schema_id = ActiveValue::Set(schema_obj.oid); + schema.schema_id = Set(schema_obj.oid); let schema = schema.insert(&txn).await?; txn.commit().await?; @@ -284,7 +285,7 @@ impl CatalogController { } let res = Object::delete(object::ActiveModel { - oid: ActiveValue::Set(schema_id), + oid: Set(schema_id), ..Default::default() }) .exec(&inner.db) @@ -473,8 +474,8 @@ impl CatalogController { // todo: shall we need to check existence of them Or let database handle it by FOREIGN KEY constraint. for obj_id in &pb_view.dependent_relations { object_dependency::ActiveModel { - oid: ActiveValue::Set(*obj_id), - used_by: ActiveValue::Set(view_obj.oid), + oid: Set(*obj_id), + used_by: Set(view_obj.oid), ..Default::default() } .insert(&txn) @@ -523,16 +524,16 @@ impl CatalogController { })?; column_desc.description = comment.description; table::ActiveModel { - table_id: ActiveValue::Set(comment.table_id), - columns: ActiveValue::Set(columns), + table_id: Set(comment.table_id), + columns: Set(columns), ..Default::default() } .update(&txn) .await? } else { table::ActiveModel { - table_id: ActiveValue::Set(comment.table_id), - description: ActiveValue::Set(comment.description), + table_id: Set(comment.table_id), + description: Set(comment.description), ..Default::default() } .update(&txn) @@ -746,9 +747,9 @@ impl CatalogController { relation.name = object_name.into(); relation.definition = alter_relation_rename(&relation.definition, object_name); let active_model = $table::ActiveModel { - $identity: ActiveValue::Set(relation.$identity), - name: ActiveValue::Set(object_name.into()), - definition: ActiveValue::Set(relation.definition.clone()), + $identity: Set(relation.$identity), + name: Set(object_name.into()), + definition: Set(relation.definition.clone()), ..Default::default() }; active_model.update(&txn).await?; @@ -777,8 +778,8 @@ impl CatalogController { // the name of index and its associated table is the same. let active_model = index::ActiveModel { - index_id: ActiveValue::Set(index.index_id), - name: ActiveValue::Set(object_name.into()), + index_id: Set(index.index_id), + name: Set(object_name.into()), ..Default::default() }; active_model.update(&txn).await?; @@ -803,8 +804,8 @@ impl CatalogController { relation.definition = alter_relation_rename_refs(&relation.definition, &old_name, object_name); let active_model = $table::ActiveModel { - $identity: ActiveValue::Set(relation.$identity), - definition: ActiveValue::Set(relation.definition.clone()), + $identity: Set(relation.$identity), + definition: Set(relation.definition.clone()), ..Default::default() }; active_model.update(&txn).await?; @@ -851,8 +852,6 @@ impl CatalogController { #[cfg(test)] #[cfg(not(madsim))] mod tests { - use risingwave_common::catalog::DEFAULT_SUPER_USER_ID; - use super::*; const TEST_DATABASE_ID: DatabaseId = 1; @@ -864,7 +863,7 @@ mod tests { let mgr = CatalogController::new(MetaSrvEnv::for_test().await)?; let db = PbDatabase { name: "test".to_string(), - owner: DEFAULT_SUPER_USER_ID, + owner: TEST_OWNER_ID, ..Default::default() }; mgr.create_database(db).await?; diff --git a/src/meta/src/controller/cluster.rs b/src/meta/src/controller/cluster.rs index 392a0def5d53f..50eb6cd5921a4 100644 --- a/src/meta/src/controller/cluster.rs +++ b/src/meta/src/controller/cluster.rs @@ -34,9 +34,10 @@ use risingwave_pb::meta::heartbeat_request; use risingwave_pb::meta::subscribe_response::{Info, Operation}; use risingwave_pb::meta::update_worker_node_schedulability_request::Schedulability; use sea_orm::prelude::Expr; +use sea_orm::ActiveValue::Set; use sea_orm::{ - ActiveModelTrait, ActiveValue, ColumnTrait, DatabaseConnection, EntityTrait, QueryFilter, - QuerySelect, TransactionTrait, + ActiveModelTrait, ColumnTrait, DatabaseConnection, EntityTrait, QueryFilter, QuerySelect, + TransactionTrait, }; use tokio::sync::oneshot::Sender; use tokio::sync::{RwLock, RwLockReadGuard}; @@ -536,9 +537,9 @@ impl ClusterControllerInner { let mut property: worker_property::ActiveModel = property.into(); // keep `is_unschedulable` unchanged. - property.is_streaming = ActiveValue::Set(add_property.is_streaming); - property.is_serving = ActiveValue::Set(add_property.is_serving); - property.parallel_unit_ids = ActiveValue::Set(I32Array(current_parallelism)); + property.is_streaming = Set(add_property.is_streaming); + property.is_serving = Set(add_property.is_serving); + property.parallel_unit_ids = Set(I32Array(current_parallelism)); WorkerProperty::update(property).exec(&txn).await?; txn.commit().await?; @@ -553,25 +554,25 @@ impl ClusterControllerInner { let worker = worker::ActiveModel { worker_id: Default::default(), - worker_type: ActiveValue::Set(r#type.into()), - host: ActiveValue::Set(host_address.host), - port: ActiveValue::Set(host_address.port), - status: ActiveValue::Set(WorkerStatus::Starting), - transaction_id: ActiveValue::Set(txn_id), + worker_type: Set(r#type.into()), + host: Set(host_address.host), + port: Set(host_address.port), + status: Set(WorkerStatus::Starting), + transaction_id: Set(txn_id), }; let insert_res = Worker::insert(worker).exec(&txn).await?; let worker_id = insert_res.last_insert_id as WorkerId; if r#type == PbWorkerType::ComputeNode { let property = worker_property::ActiveModel { - worker_id: ActiveValue::Set(worker_id), - parallel_unit_ids: ActiveValue::Set(I32Array(derive_parallel_units( + worker_id: Set(worker_id), + parallel_unit_ids: Set(I32Array(derive_parallel_units( *txn_id.as_ref().unwrap(), 0, add_property.worker_node_parallelism as _, ))), - is_streaming: ActiveValue::Set(add_property.is_streaming), - is_serving: ActiveValue::Set(add_property.is_streaming), - is_unschedulable: ActiveValue::Set(add_property.is_streaming), + is_streaming: Set(add_property.is_streaming), + is_serving: Set(add_property.is_streaming), + is_unschedulable: Set(add_property.is_streaming), }; WorkerProperty::insert(property).exec(&txn).await?; } @@ -588,8 +589,8 @@ impl ClusterControllerInner { pub async fn activate_worker(&self, worker_id: WorkerId) -> MetaResult { let worker = worker::ActiveModel { - worker_id: ActiveValue::Set(worker_id), - status: ActiveValue::Set(WorkerStatus::Running), + worker_id: Set(worker_id), + status: Set(WorkerStatus::Running), ..Default::default() }; diff --git a/src/meta/src/controller/mod.rs b/src/meta/src/controller/mod.rs index 7fe9de46e5742..37ca54dd1a18f 100644 --- a/src/meta/src/controller/mod.rs +++ b/src/meta/src/controller/mod.rs @@ -12,6 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. +#![expect(dead_code, reason = "WIP")] + use anyhow::anyhow; use risingwave_common::util::epoch::Epoch; use risingwave_meta_model_v2::{ @@ -28,12 +30,12 @@ use sea_orm::{DatabaseConnection, ModelTrait}; use crate::MetaError; -#[allow(dead_code)] pub mod catalog; pub mod cluster; pub mod fragment; pub mod rename; pub mod system_param; +pub mod user; pub mod utils; // todo: refine the error transform. diff --git a/src/meta/src/controller/system_param.rs b/src/meta/src/controller/system_param.rs index 5c9761a9a119d..e64ddf5b4ab85 100644 --- a/src/meta/src/controller/system_param.rs +++ b/src/meta/src/controller/system_param.rs @@ -25,7 +25,8 @@ use risingwave_meta_model_v2::prelude::SystemParameter; use risingwave_meta_model_v2::system_parameter; use risingwave_pb::meta::subscribe_response::{Info, Operation}; use risingwave_pb::meta::PbSystemParams; -use sea_orm::{ActiveModelTrait, ActiveValue, DatabaseConnection, EntityTrait, TransactionTrait}; +use sea_orm::ActiveValue::Set; +use sea_orm::{ActiveModelTrait, DatabaseConnection, EntityTrait, TransactionTrait}; use tokio::sync::oneshot::Sender; use tokio::sync::RwLock; use tokio::task::JoinHandle; @@ -83,10 +84,10 @@ macro_rules! impl_system_params_to_models { $( let value = params.$field.as_ref().unwrap().to_string(); models.push(system_parameter::ActiveModel { - name: ActiveValue::Set(key_of!($field).to_string()), - value: ActiveValue::Set(value), - is_mutable: ActiveValue::Set($is_mutable), - description: ActiveValue::Set(None), + name: Set(key_of!($field).to_string()), + value: Set(value), + is_mutable: Set($is_mutable), + description: Set(None), }); )* Ok(models) @@ -190,9 +191,8 @@ impl SystemParamsController { }; let mut params = params_guard.clone(); let mut param: system_parameter::ActiveModel = param.into(); - param.value = ActiveValue::Set( - set_system_param(&mut params, name, value).map_err(MetaError::system_param)?, - ); + param.value = + Set(set_system_param(&mut params, name, value).map_err(MetaError::system_param)?); param.update(&self.db).await?; *params_guard = params.clone(); @@ -281,10 +281,10 @@ mod tests { // insert deprecated params. let deprecated_param = system_parameter::ActiveModel { - name: ActiveValue::Set("deprecated_param".into()), - value: ActiveValue::Set("foo".into()), - is_mutable: ActiveValue::Set(true), - description: ActiveValue::Set(None), + name: Set("deprecated_param".into()), + value: Set("foo".into()), + is_mutable: Set(true), + description: Set(None), }; deprecated_param.insert(&system_param_ctl.db).await.unwrap(); diff --git a/src/meta/src/controller/user.rs b/src/meta/src/controller/user.rs new file mode 100644 index 0000000000000..76e2d6225ca8d --- /dev/null +++ b/src/meta/src/controller/user.rs @@ -0,0 +1,695 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::collections::{HashMap, HashSet}; + +use itertools::Itertools; +use risingwave_common::catalog::{DEFAULT_SUPER_USER, DEFAULT_SUPER_USER_FOR_PG}; +use risingwave_meta_model_v2::prelude::{Object, User, UserPrivilege}; +use risingwave_meta_model_v2::user_privilege::Action; +use risingwave_meta_model_v2::{object, user, user_privilege, AuthInfo, PrivilegeId, UserId}; +use risingwave_pb::meta::subscribe_response::{ + Info as NotificationInfo, Operation as NotificationOperation, +}; +use risingwave_pb::user::update_user_request::PbUpdateField; +use risingwave_pb::user::{PbGrantPrivilege, PbUserInfo}; +use sea_orm::sea_query::{OnConflict, SimpleExpr, Value}; +use sea_orm::ActiveValue::Set; +use sea_orm::{ + ActiveModelTrait, ColumnTrait, EntityTrait, IntoActiveModel, PaginatorTrait, QueryFilter, + QuerySelect, TransactionTrait, +}; + +use crate::controller::catalog::CatalogController; +use crate::controller::utils::{ + check_user_name_duplicate, ensure_privileges_not_referred, ensure_user_id, + extract_grant_obj_id, get_referring_privileges_cascade, get_user_privilege, + PartialUserPrivilege, +}; +use crate::manager::NotificationVersion; +use crate::{MetaError, MetaResult}; + +impl CatalogController { + async fn create_user(&self, pb_user: PbUserInfo) -> MetaResult { + let inner = self.inner.write().await; + let txn = inner.db.begin().await?; + check_user_name_duplicate(&pb_user.name, &txn).await?; + + let grant_privileges = pb_user.grant_privileges.clone(); + let user: user::ActiveModel = pb_user.into(); + let user = user.insert(&txn).await?; + + if !grant_privileges.is_empty() { + let mut privileges = vec![]; + for gp in &grant_privileges { + let id = extract_grant_obj_id(gp.get_object()?); + for action_with_opt in &gp.action_with_opts { + privileges.push(user_privilege::ActiveModel { + user_id: Set(user.user_id), + oid: Set(id), + granted_by: Set(action_with_opt.granted_by), + action: Set(action_with_opt.get_action()?.into()), + with_grant_option: Set(action_with_opt.with_grant_option), + ..Default::default() + }); + } + } + UserPrivilege::insert_many(privileges).exec(&txn).await?; + } + txn.commit().await?; + + let mut user_info: PbUserInfo = user.into(); + user_info.grant_privileges = grant_privileges; + let version = self + .notify_frontend( + NotificationOperation::Add, + NotificationInfo::User(user_info), + ) + .await; + + Ok(version) + } + + async fn update_user( + &self, + update_user: PbUserInfo, + update_fields: &[PbUpdateField], + ) -> MetaResult { + let inner = self.inner.write().await; + let rename_flag = update_fields + .iter() + .any(|&field| field == PbUpdateField::Rename); + if rename_flag { + check_user_name_duplicate(&update_user.name, &inner.db).await?; + } + + let user = User::find_by_id(update_user.id) + .one(&inner.db) + .await? + .ok_or_else(|| MetaError::catalog_id_not_found("user", update_user.id))?; + let mut user = user.into_active_model(); + update_fields.iter().for_each(|&field| match field { + PbUpdateField::Unspecified => unreachable!(), + PbUpdateField::Super => user.is_super = Set(update_user.is_super), + PbUpdateField::Login => user.can_login = Set(update_user.can_login), + PbUpdateField::CreateDb => user.can_create_db = Set(update_user.can_create_db), + PbUpdateField::CreateUser => user.can_create_user = Set(update_user.can_create_user), + PbUpdateField::AuthInfo => { + user.auth_info = Set(update_user.auth_info.clone().map(AuthInfo)) + } + PbUpdateField::Rename => user.name = Set(update_user.name.clone()), + }); + + let user = user.update(&inner.db).await?; + let mut user_info: PbUserInfo = user.into(); + user_info.grant_privileges = get_user_privilege(user_info.id, &inner.db).await?; + let version = self + .notify_frontend( + NotificationOperation::Update, + NotificationInfo::User(user_info), + ) + .await; + + Ok(version) + } + + #[cfg(test)] + pub async fn get_user(&self, id: UserId) -> MetaResult { + let inner = self.inner.read().await; + let user = User::find_by_id(id) + .one(&inner.db) + .await? + .ok_or_else(|| MetaError::catalog_id_not_found("user", id))?; + Ok(user) + } + + #[cfg(test)] + pub async fn get_user_by_name(&self, name: &str) -> MetaResult { + let inner = self.inner.read().await; + let user = User::find() + .filter(user::Column::Name.eq(name)) + .one(&inner.db) + .await? + .ok_or_else(|| anyhow::anyhow!("user {name} not found"))?; + Ok(user) + } + + async fn drop_user(&self, user_id: UserId) -> MetaResult { + let inner = self.inner.write().await; + let txn = inner.db.begin().await?; + let user = User::find_by_id(user_id) + .one(&txn) + .await? + .ok_or_else(|| MetaError::catalog_id_not_found("user", user_id))?; + if user.name == DEFAULT_SUPER_USER || user.name == DEFAULT_SUPER_USER_FOR_PG { + return Err(MetaError::permission_denied(format!( + "drop default super user {} is not allowed", + user.name + ))); + } + + // check if the user is the owner of any objects. + let count = Object::find() + .filter(object::Column::OwnerId.eq(user_id)) + .count(&txn) + .await?; + if count != 0 { + return Err(MetaError::permission_denied(format!( + "drop user {} is not allowed, because it owns {} objects", + user.name, count + ))); + } + + // check if the user granted any privileges to other users. + let count = UserPrivilege::find() + .filter(user_privilege::Column::GrantedBy.eq(user_id)) + .count(&txn) + .await?; + if count != 0 { + return Err(MetaError::permission_denied(format!( + "drop user {} is not allowed, because it granted {} privileges to others", + user.name, count + ))); + } + + let res = User::delete_by_id(user_id).exec(&txn).await?; + if res.rows_affected != 1 { + return Err(MetaError::catalog_id_not_found("user", user_id)); + } + txn.commit().await?; + + let version = self + .notify_frontend( + NotificationOperation::Delete, + NotificationInfo::User(PbUserInfo { + id: user_id, + ..Default::default() + }), + ) + .await; + + Ok(version) + } + + pub async fn grant_privilege( + &self, + user_ids: Vec, + new_grant_privileges: &[PbGrantPrivilege], + grantor: UserId, + ) -> MetaResult { + let inner = self.inner.write().await; + let txn = inner.db.begin().await?; + for user_id in &user_ids { + ensure_user_id(*user_id, &txn).await?; + } + + let mut privileges = vec![]; + for gp in new_grant_privileges { + let id = extract_grant_obj_id(gp.get_object()?); + for action_with_opt in &gp.action_with_opts { + privileges.push(user_privilege::ActiveModel { + oid: Set(id), + granted_by: Set(grantor), + action: Set(action_with_opt.get_action()?.into()), + with_grant_option: Set(action_with_opt.with_grant_option), + ..Default::default() + }); + } + } + + // check whether grantor has the privilege to grant the privilege. + let user = User::find_by_id(grantor) + .one(&txn) + .await? + .ok_or_else(|| MetaError::catalog_id_not_found("user", grantor))?; + if !user.is_super { + for privilege in &mut privileges { + let filter = user_privilege::Column::UserId + .eq(grantor) + .and(user_privilege::Column::Oid.eq(*privilege.oid.as_ref())) + .and(user_privilege::Column::Action.eq(privilege.action.as_ref().clone())) + .and(user_privilege::Column::WithGrantOption.eq(true)); + let privilege_id: Option = UserPrivilege::find() + .select_only() + .column(user_privilege::Column::Id) + .filter(filter) + .into_tuple() + .one(&txn) + .await?; + let Some(privilege_id) = privilege_id else { + return Err(MetaError::permission_denied(format!( + "user {} don't have privilege {:?} or grant option", + grantor, privilege.action, + ))); + }; + privilege.dependent_id = Set(Some(privilege_id)); + } + } + + // insert privileges + let user_privileges = user_ids.iter().flat_map(|user_id| { + privileges.iter().map(|p| { + let mut p = p.clone(); + p.user_id = Set(*user_id); + p + }) + }); + for privilege in user_privileges { + let mut on_conflict = OnConflict::columns([ + user_privilege::Column::UserId, + user_privilege::Column::Oid, + user_privilege::Column::Action, + user_privilege::Column::GrantedBy, + ]); + if *privilege.with_grant_option.as_ref() { + on_conflict.update_columns([user_privilege::Column::WithGrantOption]); + } else { + on_conflict.do_nothing(); + } + UserPrivilege::insert(privilege) + .on_conflict(on_conflict) + .exec(&txn) + .await?; + } + + let mut user_infos = vec![]; + for user_id in user_ids { + let user = User::find_by_id(user_id) + .one(&txn) + .await? + .ok_or_else(|| MetaError::catalog_id_not_found("user", user_id))?; + let mut user_info: PbUserInfo = user.into(); + user_info.grant_privileges = get_user_privilege(user_info.id, &txn).await?; + user_infos.push(user_info); + } + txn.commit().await?; + + let mut version = 0; + for info in user_infos { + version = self + .notify_frontend(NotificationOperation::Update, NotificationInfo::User(info)) + .await; + } + Ok(version) + } + + pub async fn revoke_privilege( + &self, + user_ids: Vec, + revoke_grant_privileges: &[PbGrantPrivilege], + granted_by: Option, + revoke_by: UserId, + revoke_grant_option: bool, + cascade: bool, + ) -> MetaResult { + let inner = self.inner.write().await; + let txn = inner.db.begin().await?; + for user_id in &user_ids { + ensure_user_id(*user_id, &txn).await?; + } + // check whether revoke has the privilege to grant the privilege. + let revoke_user = User::find_by_id(revoke_by) + .one(&txn) + .await? + .ok_or_else(|| MetaError::catalog_id_not_found("user", revoke_by))?; + + let granted_by = granted_by.unwrap_or(revoke_by); + // check whether user can revoke the privilege. + if !revoke_user.is_super && granted_by != revoke_by { + let granted_user_name: String = User::find_by_id(granted_by) + .select_only() + .column(user::Column::Name) + .into_tuple() + .one(&txn) + .await? + .ok_or_else(|| MetaError::catalog_id_not_found("user", granted_by))?; + return Err(MetaError::permission_denied(format!( + "user {} is not super, can't revoke privileges for {}", + revoke_user.name, granted_user_name + ))); + } + + let mut revoke_items = HashMap::new(); + for privilege in revoke_grant_privileges { + let obj = extract_grant_obj_id(privilege.get_object()?); + let actions = privilege + .action_with_opts + .iter() + .map(|ao| Action::from(ao.get_action().unwrap())) + .collect_vec(); + revoke_items.insert(obj, actions); + } + + let filter = if !revoke_user.is_super { + // ensure user have grant options. + for (obj, actions) in &revoke_items { + let owned_actions: HashSet = UserPrivilege::find() + .select_only() + .column(user_privilege::Column::Action) + .filter( + user_privilege::Column::UserId + .eq(granted_by) + .and(user_privilege::Column::Oid.eq(*obj)) + .and(user_privilege::Column::WithGrantOption.eq(true)), + ) + .into_tuple::() + .all(&txn) + .await? + .into_iter() + .collect(); + if actions.iter().any(|ac| !owned_actions.contains(ac)) { + return Err(MetaError::permission_denied(format!( + "user {} don't have privileges {:?} or grant option", + revoke_user.name, actions, + ))); + } + } + + user_privilege::Column::GrantedBy + .eq(granted_by) + .and(user_privilege::Column::UserId.is_in(user_ids.clone())) + } else { + user_privilege::Column::UserId.is_in(user_ids.clone()) + }; + let mut root_user_privileges: Vec = vec![]; + for (obj, actions) in &revoke_items { + let filter = filter + .clone() + .and(user_privilege::Column::Oid.eq(*obj)) + .and(user_privilege::Column::Action.is_in(actions.clone())); + root_user_privileges.extend( + UserPrivilege::find() + .select_only() + .columns([user_privilege::Column::Id, user_privilege::Column::UserId]) + .filter(filter) + .into_partial_model() + .all(&txn) + .await?, + ); + } + if root_user_privileges.is_empty() { + return Err(MetaError::invalid_parameter( + "no privilege to revoke".to_string(), + )); + } + + // check if the user granted any privileges to other users. + let root_privilege_ids = root_user_privileges.iter().map(|ur| ur.id).collect_vec(); + let (all_privilege_ids, to_update_user_ids): (_, HashSet) = if !cascade { + ensure_privileges_not_referred(root_privilege_ids.clone(), &txn).await?; + ( + root_privilege_ids.clone(), + root_user_privileges.iter().map(|ur| ur.user_id).collect(), + ) + } else { + let all_user_privileges = + get_referring_privileges_cascade(root_privilege_ids.clone(), &txn).await?; + ( + all_user_privileges.iter().map(|ur| ur.id).collect_vec(), + all_user_privileges.iter().map(|ur| ur.user_id).collect(), + ) + }; + + if revoke_grant_option { + UserPrivilege::update_many() + .col_expr( + user_privilege::Column::WithGrantOption, + SimpleExpr::Value(Value::Bool(Some(false))), + ) + .filter( + user_privilege::Column::Id + .is_in(all_privilege_ids) + .and(user_privilege::Column::WithGrantOption.eq(true)), + ) + .exec(&txn) + .await?; + } else { + // The dependent privileges will be deleted cascade. + UserPrivilege::delete_many() + .filter(user_privilege::Column::Id.is_in(root_privilege_ids)) + .exec(&txn) + .await?; + } + + let mut user_infos = vec![]; + for user_id in to_update_user_ids { + let user = User::find_by_id(user_id) + .one(&txn) + .await? + .ok_or_else(|| MetaError::catalog_id_not_found("user", user_id))?; + let mut user_info: PbUserInfo = user.into(); + user_info.grant_privileges = get_user_privilege(user_info.id, &txn).await?; + user_infos.push(user_info); + } + txn.commit().await?; + + let mut version = 0; + for info in user_infos { + version = self + .notify_frontend(NotificationOperation::Update, NotificationInfo::User(info)) + .await; + } + Ok(version) + } +} + +#[cfg(test)] +#[cfg(not(madsim))] +mod tests { + use risingwave_meta_model_v2::DatabaseId; + use risingwave_pb::user::grant_privilege::{PbAction, PbActionWithGrantOption, PbObject}; + + use super::*; + use crate::manager::MetaSrvEnv; + + const TEST_DATABASE_ID: DatabaseId = 1; + const TEST_ROOT_USER_ID: UserId = 1; + + fn make_test_user(name: &str) -> PbUserInfo { + PbUserInfo { + name: name.to_string(), + ..Default::default() + } + } + + fn make_privilege( + object: PbObject, + actions: &[PbAction], + with_grant_option: bool, + ) -> PbGrantPrivilege { + PbGrantPrivilege { + object: Some(object), + action_with_opts: actions + .iter() + .map(|&action| PbActionWithGrantOption { + action: action as _, + with_grant_option, + ..Default::default() + }) + .collect(), + } + } + + #[tokio::test] + async fn test_user_and_privilege() -> MetaResult<()> { + let mgr = CatalogController::new(MetaSrvEnv::for_test().await)?; + mgr.create_user(make_test_user("test_user_1")).await?; + mgr.create_user(make_test_user("test_user_2")).await?; + let user_1 = mgr.get_user_by_name("test_user_1").await?; + let user_2 = mgr.get_user_by_name("test_user_2").await?; + + assert!( + mgr.create_user(make_test_user("test_user_1")) + .await + .is_err(), + "user_1 already exists" + ); + mgr.update_user( + PbUserInfo { + id: user_1.user_id, + name: "test_user_1_new".to_string(), + ..Default::default() + }, + &[PbUpdateField::Rename], + ) + .await?; + let user_1 = mgr.get_user(user_1.user_id).await?; + assert_eq!(user_1.name, "test_user_1_new".to_string()); + + let conn_with_option = make_privilege( + PbObject::DatabaseId(TEST_DATABASE_ID), + &[PbAction::Connect], + true, + ); + let create_without_option = make_privilege( + PbObject::DatabaseId(TEST_DATABASE_ID), + &[PbAction::Create], + false, + ); + // ROOT grant CONN with grant option to user_1. + mgr.grant_privilege( + vec![user_1.user_id], + &[conn_with_option.clone()], + TEST_ROOT_USER_ID, + ) + .await?; + // ROOT grant CREATE without grant option to user_1. + mgr.grant_privilege( + vec![user_1.user_id], + &[create_without_option.clone()], + TEST_ROOT_USER_ID, + ) + .await?; + // user_1 grant CONN with grant option to user_2. + mgr.grant_privilege( + vec![user_2.user_id], + &[conn_with_option.clone()], + user_1.user_id, + ) + .await?; + // user_1 grant CREATE without grant option to user_2. + assert!( + mgr.grant_privilege( + vec![user_2.user_id], + &[create_without_option.clone()], + user_1.user_id + ) + .await + .is_err(), + "user_1 don't have grant option" + ); + + assert!( + mgr.drop_user(user_1.user_id).await.is_err(), + "user_1 can't be dropped" + ); + + let privilege_1 = get_user_privilege(user_1.user_id, &mgr.inner.read().await.db).await?; + assert_eq!(privilege_1.len(), 2); + assert!(privilege_1.iter().all(|gp| gp.object + == Some(PbObject::DatabaseId(TEST_DATABASE_ID)) + && gp.action_with_opts[0].granted_by == TEST_ROOT_USER_ID)); + + let privilege_2 = get_user_privilege(user_2.user_id, &mgr.inner.read().await.db).await?; + assert_eq!(privilege_2.len(), 1); + assert!(privilege_2.iter().all(|gp| gp.object + == Some(PbObject::DatabaseId(TEST_DATABASE_ID)) + && gp.action_with_opts[0].granted_by == user_1.user_id + && gp.action_with_opts[0].with_grant_option)); + + // revoke privilege for others by non-super user. + assert!( + mgr.revoke_privilege( + vec![user_1.user_id], + &[conn_with_option.clone()], + Some(TEST_ROOT_USER_ID), + user_2.user_id, + false, + false + ) + .await + .is_err(), + "user_2 can't revoke for user_1" + ); + + // revoke privilege without grant option. + assert!( + mgr.revoke_privilege( + vec![user_2.user_id], + &[create_without_option.clone()], + None, + user_1.user_id, + false, + false + ) + .await + .is_err(), + "user_2 don't have grant option for CREATE" + ); + + // revoke referred privilege in restrict mode. + assert!( + mgr.revoke_privilege( + vec![user_1.user_id], + &[conn_with_option.clone()], + None, + TEST_ROOT_USER_ID, + false, + false + ) + .await + .is_err(), + "permission deny in restrict mode, CONN granted to user_2" + ); + + // revoke non-referred privilege in restrict mode. + mgr.revoke_privilege( + vec![user_1.user_id], + &[create_without_option.clone()], + None, + TEST_ROOT_USER_ID, + false, + false, + ) + .await?; + + let privilege_1 = get_user_privilege(user_1.user_id, &mgr.inner.read().await.db).await?; + assert_eq!(privilege_1.len(), 1); + assert!(privilege_1.iter().all(|gp| gp.object + == Some(PbObject::DatabaseId(TEST_DATABASE_ID)) + && gp.action_with_opts[0].action == PbAction::Connect as i32)); + + // revoke grant option for referred privilege in cascade mode. + mgr.revoke_privilege( + vec![user_1.user_id], + &[conn_with_option.clone()], + None, + TEST_ROOT_USER_ID, + true, + true, + ) + .await?; + let privilege_1 = get_user_privilege(user_1.user_id, &mgr.inner.read().await.db).await?; + assert_eq!(privilege_1.len(), 1); + assert!(privilege_1.iter().all(|gp| gp.object + == Some(PbObject::DatabaseId(TEST_DATABASE_ID)) + && gp.action_with_opts[0].action == PbAction::Connect as i32 + && !gp.action_with_opts[0].with_grant_option)); + let privilege_2 = get_user_privilege(user_2.user_id, &mgr.inner.read().await.db).await?; + assert_eq!(privilege_2.len(), 1); + assert!(privilege_2.iter().all(|gp| gp.object + == Some(PbObject::DatabaseId(TEST_DATABASE_ID)) + && gp.action_with_opts[0].action == PbAction::Connect as i32 + && !gp.action_with_opts[0].with_grant_option)); + + // revoke referred privilege in cascade mode. + mgr.revoke_privilege( + vec![user_1.user_id], + &[conn_with_option.clone()], + None, + TEST_ROOT_USER_ID, + false, + true, + ) + .await?; + let privilege_1 = get_user_privilege(user_1.user_id, &mgr.inner.read().await.db).await?; + assert!(privilege_1.is_empty()); + let privilege_2 = get_user_privilege(user_2.user_id, &mgr.inner.read().await.db).await?; + assert!(privilege_2.is_empty()); + + mgr.drop_user(user_1.user_id).await?; + mgr.drop_user(user_2.user_id).await?; + Ok(()) + } +} diff --git a/src/meta/src/controller/utils.rs b/src/meta/src/controller/utils.rs index 2dbd89ac92423..9b5ee9e6a2c0b 100644 --- a/src/meta/src/controller/utils.rs +++ b/src/meta/src/controller/utils.rs @@ -17,10 +17,12 @@ use risingwave_meta_model_migration::WithQuery; use risingwave_meta_model_v2::object::ObjectType; use risingwave_meta_model_v2::prelude::*; use risingwave_meta_model_v2::{ - connection, function, index, object, object_dependency, schema, sink, source, table, view, - DataTypeArray, DatabaseId, ObjectId, SchemaId, UserId, + connection, function, index, object, object_dependency, schema, sink, source, table, user, + user_privilege, view, DataTypeArray, DatabaseId, ObjectId, PrivilegeId, SchemaId, UserId, }; use risingwave_pb::catalog::{PbConnection, PbFunction}; +use risingwave_pb::user::grant_privilege::{PbAction, PbActionWithGrantOption, PbObject}; +use risingwave_pb::user::PbGrantPrivilege; use sea_orm::sea_query::{ Alias, CommonTableExpression, Expr, Query, QueryStatementBuilder, SelectStatement, UnionType, WithClause, @@ -220,6 +222,22 @@ where Ok(()) } +/// `check_user_name_duplicate` checks whether the user is already existed in the cluster. +pub async fn check_user_name_duplicate(name: &str, db: &C) -> MetaResult<()> +where + C: ConnectionTrait, +{ + let count = User::find() + .filter(user::Column::Name.eq(name)) + .count(db) + .await?; + if count > 0 { + assert_eq!(count, 1); + return Err(MetaError::catalog_duplicated("user", name)); + } + Ok(()) +} + /// `check_relation_name_duplicate` checks whether the relation name is already used in the target namespace. pub async fn check_relation_name_duplicate( name: &str, @@ -354,3 +372,169 @@ where Ok(()) } + +/// `construct_privilege_dependency_query` constructs a query to find all privileges that are dependent on the given one. +/// +/// # Examples +/// +/// ``` +/// use risingwave_meta::controller::utils::construct_privilege_dependency_query; +/// use sea_orm::sea_query::*; +/// use sea_orm::*; +/// +/// let query = construct_privilege_dependency_query(vec![1, 2, 3]); +/// +/// assert_eq!( +/// query.to_string(MysqlQueryBuilder), +/// r#"WITH RECURSIVE `granted_privilege_ids` (`id`, `user_id`) AS (SELECT `id`, `user_id` FROM `user_privilege` WHERE `user_privilege`.`id` IN (1, 2, 3) UNION ALL (SELECT `user_privilege`.`id`, `user_privilege`.`user_id` FROM `user_privilege` INNER JOIN `granted_privilege_ids` ON `granted_privilege_ids`.`id` = `dependent_id`)) SELECT `id`, `user_id` FROM `granted_privilege_ids`"# +/// ); +/// assert_eq!( +/// query.to_string(PostgresQueryBuilder), +/// r#"WITH RECURSIVE "granted_privilege_ids" ("id", "user_id") AS (SELECT "id", "user_id" FROM "user_privilege" WHERE "user_privilege"."id" IN (1, 2, 3) UNION ALL (SELECT "user_privilege"."id", "user_privilege"."user_id" FROM "user_privilege" INNER JOIN "granted_privilege_ids" ON "granted_privilege_ids"."id" = "dependent_id")) SELECT "id", "user_id" FROM "granted_privilege_ids""# +/// ); +/// assert_eq!( +/// query.to_string(SqliteQueryBuilder), +/// r#"WITH RECURSIVE "granted_privilege_ids" ("id", "user_id") AS (SELECT "id", "user_id" FROM "user_privilege" WHERE "user_privilege"."id" IN (1, 2, 3) UNION ALL SELECT "user_privilege"."id", "user_privilege"."user_id" FROM "user_privilege" INNER JOIN "granted_privilege_ids" ON "granted_privilege_ids"."id" = "dependent_id") SELECT "id", "user_id" FROM "granted_privilege_ids""# +/// ); +/// ``` +pub fn construct_privilege_dependency_query(ids: Vec) -> WithQuery { + let cte_alias = Alias::new("granted_privilege_ids"); + let cte_return_privilege_alias = Alias::new("id"); + let cte_return_user_alias = Alias::new("user_id"); + + let mut base_query = SelectStatement::new() + .columns([user_privilege::Column::Id, user_privilege::Column::UserId]) + .from(UserPrivilege) + .and_where(user_privilege::Column::Id.is_in(ids)) + .to_owned(); + + let cte_referencing = Query::select() + .columns([ + (UserPrivilege, user_privilege::Column::Id), + (UserPrivilege, user_privilege::Column::UserId), + ]) + .from(UserPrivilege) + .inner_join( + cte_alias.clone(), + Expr::col((cte_alias.clone(), cte_return_privilege_alias.clone())) + .equals(user_privilege::Column::DependentId), + ) + .to_owned(); + + let common_table_expr = CommonTableExpression::new() + .query(base_query.union(UnionType::All, cte_referencing).to_owned()) + .columns([ + cte_return_privilege_alias.clone(), + cte_return_user_alias.clone(), + ]) + .table_name(cte_alias.clone()) + .to_owned(); + + SelectStatement::new() + .columns([cte_return_privilege_alias, cte_return_user_alias]) + .from(cte_alias.clone()) + .to_owned() + .with( + WithClause::new() + .recursive(true) + .cte(common_table_expr) + .to_owned(), + ) + .to_owned() +} + +#[derive(Clone, DerivePartialModel, FromQueryResult)] +#[sea_orm(entity = "UserPrivilege")] +pub struct PartialUserPrivilege { + pub id: PrivilegeId, + pub user_id: UserId, +} + +pub async fn get_referring_privileges_cascade( + ids: Vec, + db: &C, +) -> MetaResult> +where + C: ConnectionTrait, +{ + let query = construct_privilege_dependency_query(ids); + let (sql, values) = query.build_any(&*db.get_database_backend().get_query_builder()); + let privileges = PartialUserPrivilege::find_by_statement(Statement::from_sql_and_values( + db.get_database_backend(), + sql, + values, + )) + .all(db) + .await?; + + Ok(privileges) +} + +/// `ensure_privileges_not_referred` ensures that the privileges are not granted to any other users. +pub async fn ensure_privileges_not_referred(ids: Vec, db: &C) -> MetaResult<()> +where + C: ConnectionTrait, +{ + let count = UserPrivilege::find() + .filter(user_privilege::Column::DependentId.is_in(ids)) + .count(db) + .await?; + if count != 0 { + return Err(MetaError::permission_denied(format!( + "privileges granted to {} other ones.", + count + ))); + } + Ok(()) +} + +/// `get_user_privilege` returns the privileges of the given user. +pub async fn get_user_privilege(user_id: UserId, db: &C) -> MetaResult> +where + C: ConnectionTrait, +{ + let user_privileges = UserPrivilege::find() + .find_also_related(Object) + .filter(user_privilege::Column::UserId.eq(user_id)) + .all(db) + .await?; + Ok(user_privileges + .into_iter() + .map(|(privilege, object)| { + let object = object.unwrap(); + let obj = match object.obj_type { + ObjectType::Database => PbObject::DatabaseId(object.oid), + ObjectType::Schema => PbObject::SchemaId(object.oid), + ObjectType::Table => PbObject::TableId(object.oid), + ObjectType::Source => PbObject::SourceId(object.oid), + ObjectType::Sink => PbObject::SinkId(object.oid), + ObjectType::View => PbObject::ViewId(object.oid), + ObjectType::Function => PbObject::FunctionId(object.oid), + ObjectType::Index => unreachable!("index is not supported yet"), + ObjectType::Connection => unreachable!("connection is not supported yet"), + }; + PbGrantPrivilege { + action_with_opts: vec![PbActionWithGrantOption { + action: PbAction::from(privilege.action) as _, + with_grant_option: privilege.with_grant_option, + granted_by: privilege.granted_by, + }], + object: Some(obj), + } + }) + .collect()) +} + +// todo: remove it after migrated to sql backend. +pub fn extract_grant_obj_id(object: &PbObject) -> ObjectId { + match object { + PbObject::DatabaseId(id) + | PbObject::SchemaId(id) + | PbObject::TableId(id) + | PbObject::SourceId(id) + | PbObject::SinkId(id) + | PbObject::ViewId(id) + | PbObject::FunctionId(id) => *id, + _ => unreachable!("invalid object type: {:?}", object), + } +} From 801582bf8b3b7ef6d245b18ceb0f6377a77f9afa Mon Sep 17 00:00:00 2001 From: zwang28 <70626450+zwang28@users.noreply.github.com> Date: Thu, 2 Nov 2023 16:19:19 +0800 Subject: [PATCH 17/17] refactor(meta): adapt metadata restore to metadata model V2 (#13197) --- proto/backup_service.proto | 1 + src/meta/src/backup_restore/mod.rs | 1 + src/meta/src/backup_restore/restore.rs | 192 +++++------------ .../src/backup_restore/restore_impl/mod.rs | 32 +++ .../src/backup_restore/restore_impl/v1.rs | 202 ++++++++++++++++++ .../src/backup_restore/restore_impl/v2.rs | 96 +++++++++ src/meta/src/backup_restore/utils.rs | 4 + src/storage/backup/src/lib.rs | 6 +- src/storage/backup/src/storage.rs | 1 + src/storage/src/hummock/backup_reader.rs | 1 + 10 files changed, 395 insertions(+), 141 deletions(-) create mode 100644 src/meta/src/backup_restore/restore_impl/mod.rs create mode 100644 src/meta/src/backup_restore/restore_impl/v1.rs create mode 100644 src/meta/src/backup_restore/restore_impl/v2.rs diff --git a/proto/backup_service.proto b/proto/backup_service.proto index feca5f17b7dc3..c84b628e58550 100644 --- a/proto/backup_service.proto +++ b/proto/backup_service.proto @@ -45,6 +45,7 @@ message MetaSnapshotMetadata { uint64 hummock_version_id = 2; uint64 max_committed_epoch = 3; uint64 safe_epoch = 4; + optional uint32 format_version = 5; } service BackupService { diff --git a/src/meta/src/backup_restore/mod.rs b/src/meta/src/backup_restore/mod.rs index 0dfe5b3442415..fc33af5ef4628 100644 --- a/src/meta/src/backup_restore/mod.rs +++ b/src/meta/src/backup_restore/mod.rs @@ -19,6 +19,7 @@ mod meta_snapshot_builder; mod meta_snapshot_builder_v2; mod metrics; mod restore; +mod restore_impl; mod utils; pub use restore::*; diff --git a/src/meta/src/backup_restore/restore.rs b/src/meta/src/backup_restore/restore.rs index 48dd6fc0cb664..5474b43654017 100644 --- a/src/meta/src/backup_restore/restore.rs +++ b/src/meta/src/backup_restore/restore.rs @@ -14,22 +14,20 @@ use std::sync::Arc; -use itertools::Itertools; use risingwave_backup::error::{BackupError, BackupResult}; -use risingwave_backup::meta_snapshot_v1::MetaSnapshotV1; +use risingwave_backup::meta_snapshot::Metadata; use risingwave_backup::storage::{MetaSnapshotStorage, MetaSnapshotStorageRef}; +use risingwave_backup::MetaSnapshotId; use risingwave_common::config::MetaBackend; use risingwave_hummock_sdk::version_checkpoint_path; use risingwave_object_store::object::object_metrics::ObjectStoreMetrics; use risingwave_object_store::object::parse_remote_object_store; use risingwave_pb::hummock::{HummockVersion, HummockVersionCheckpoint}; +use crate::backup_restore::restore_impl::v1::{LoaderV1, WriterModelV1ToMetaStoreV1}; +use crate::backup_restore::restore_impl::v2::{LoaderV2, WriterModelV2ToMetaStoreV2}; +use crate::backup_restore::restore_impl::{Loader, Writer}; use crate::backup_restore::utils::{get_backup_store, get_meta_store, MetaStoreBackendImpl}; -use crate::dispatch_meta_store; -use crate::hummock::model::CompactionGroup; -use crate::manager::model::SystemParamsModel; -use crate::model::{ClusterId, MetadataModel, TableFragments}; -use crate::storage::{MetaStore, DEFAULT_COLUMN_FAMILY}; /// Command-line arguments for restore. #[derive(clap::Args, Debug, Clone)] @@ -98,99 +96,6 @@ async fn restore_hummock_version( Ok(()) } -async fn restore_metadata_model( - meta_store: &S, - metadata: &[T], -) -> BackupResult<()> { - if !T::list(meta_store).await?.is_empty() { - return Err(BackupError::NonemptyMetaStorage); - } - for d in metadata { - d.insert(meta_store).await?; - } - Ok(()) -} - -async fn restore_system_param_model( - meta_store: &S, - metadata: &[T], -) -> BackupResult<()> { - if T::get(meta_store).await?.is_some() { - return Err(BackupError::NonemptyMetaStorage); - } - for d in metadata { - d.insert(meta_store).await?; - } - Ok(()) -} - -async fn restore_cluster_id( - meta_store: &S, - cluster_id: ClusterId, -) -> BackupResult<()> { - if ClusterId::from_meta_store(meta_store).await?.is_some() { - return Err(BackupError::NonemptyMetaStorage); - } - cluster_id.put_at_meta_store(meta_store).await?; - Ok(()) -} - -async fn restore_default_cf( - meta_store: &S, - snapshot: &MetaSnapshotV1, -) -> BackupResult<()> { - if !meta_store.list_cf(DEFAULT_COLUMN_FAMILY).await?.is_empty() { - return Err(BackupError::NonemptyMetaStorage); - } - for (k, v) in &snapshot.metadata.default_cf { - meta_store - .put_cf(DEFAULT_COLUMN_FAMILY, k.clone(), v.clone()) - .await?; - } - Ok(()) -} - -async fn restore_metadata( - meta_store: S, - snapshot: MetaSnapshotV1, -) -> BackupResult<()> { - restore_default_cf(&meta_store, &snapshot).await?; - restore_metadata_model(&meta_store, &[snapshot.metadata.version_stats]).await?; - restore_metadata_model( - &meta_store, - &snapshot - .metadata - .compaction_groups - .into_iter() - .map(CompactionGroup::from_protobuf) - .collect_vec(), - ) - .await?; - restore_metadata_model( - &meta_store, - &snapshot - .metadata - .table_fragments - .into_iter() - .map(TableFragments::from_protobuf) - .collect_vec(), - ) - .await?; - restore_metadata_model(&meta_store, &snapshot.metadata.user_info).await?; - restore_metadata_model(&meta_store, &snapshot.metadata.database).await?; - restore_metadata_model(&meta_store, &snapshot.metadata.schema).await?; - restore_metadata_model(&meta_store, &snapshot.metadata.table).await?; - restore_metadata_model(&meta_store, &snapshot.metadata.index).await?; - restore_metadata_model(&meta_store, &snapshot.metadata.sink).await?; - restore_metadata_model(&meta_store, &snapshot.metadata.view).await?; - restore_metadata_model(&meta_store, &snapshot.metadata.source).await?; - restore_metadata_model(&meta_store, &snapshot.metadata.function).await?; - restore_metadata_model(&meta_store, &snapshot.metadata.connection).await?; - restore_system_param_model(&meta_store, &[snapshot.metadata.system_param]).await?; - restore_cluster_id(&meta_store, snapshot.metadata.cluster_id.into()).await?; - Ok(()) -} - /// Restores a meta store. /// Uses `meta_store` and `backup_store` if provided. /// Otherwise creates them based on `opts`. @@ -212,62 +117,69 @@ async fn restore_impl( Some(b) => b, }; let target_id = opts.meta_snapshot_id; - let snapshot_list = backup_store.manifest().snapshot_metadata.clone(); + let snapshot_list = &backup_store.manifest().snapshot_metadata; if !snapshot_list.iter().any(|m| m.id == target_id) { return Err(BackupError::Other(anyhow::anyhow!( "snapshot id {} not found", target_id ))); } - let mut target_snapshot: MetaSnapshotV1 = backup_store.get(target_id).await?; - tracing::info!( - "snapshot {} before rewrite:\n{}", - target_id, - target_snapshot - ); - let newest_id = snapshot_list - .into_iter() - .map(|m| m.id) - .max() - .expect("should exist"); - assert!(newest_id >= target_id); - // Always use newest snapshot's `default_cf` during restoring, in order not to corrupt shared - // data of snapshots. Otherwise, for example if we restore a older SST id generator, an - // existent SST in object store is at risk of being overwrote by the restored cluster. - // All these risky metadata are in `default_cf`, e.g. id generator, epoch. They must satisfy: - // - Value is monotonically non-decreasing. - // - Value is memcomparable. - // - Keys of newest_snapshot is a superset of that of target_snapshot. - if newest_id > target_id { - let newest_snapshot: MetaSnapshotV1 = backup_store.get(newest_id).await?; - for (k, v) in &target_snapshot.metadata.default_cf { - let newest_v = newest_snapshot - .metadata - .default_cf - .get(k) - .unwrap_or_else(|| panic!("violate superset requirement. key {:x?}", k)); - assert!(newest_v >= v, "violate monotonicity requirement"); + + let format_version = match snapshot_list.iter().find(|m| m.id == target_id) { + None => { + return Err(BackupError::Other(anyhow::anyhow!( + "snapshot id {} not found", + target_id + ))); + } + Some(s) => s.format_version, + }; + match &meta_store { + MetaStoreBackendImpl::Sql(m) => { + if format_version < 2 { + todo!("write model V1 to meta store V2"); + } else { + dispatch( + target_id, + &opts, + LoaderV2::new(backup_store), + WriterModelV2ToMetaStoreV2::new(m.to_owned()), + ) + .await?; + } + } + _ => { + assert!(format_version < 2, "format_version {}", format_version); + dispatch( + target_id, + &opts, + LoaderV1::new(backup_store), + WriterModelV1ToMetaStoreV1::new(meta_store), + ) + .await?; } - target_snapshot.metadata.default_cf = newest_snapshot.metadata.default_cf; - tracing::info!( - "snapshot {} after rewrite by snapshot {}:\n{}", - target_id, - newest_id, - target_snapshot, - ); } + + Ok(()) +} + +async fn dispatch, W: Writer, S: Metadata>( + target_id: MetaSnapshotId, + opts: &RestoreOpts, + loader: L, + writer: W, +) -> BackupResult<()> { + let target_snapshot = loader.load(target_id).await?; if opts.dry_run { return Ok(()); } restore_hummock_version( &opts.hummock_storage_url, &opts.hummock_storage_directory, - &target_snapshot.metadata.hummock_version, + target_snapshot.metadata.hummock_version_ref(), ) .await?; - dispatch_meta_store!(meta_store.clone(), store, { - restore_metadata(store.clone(), target_snapshot.clone()).await?; - }); + writer.write(target_snapshot).await?; Ok(()) } diff --git a/src/meta/src/backup_restore/restore_impl/mod.rs b/src/meta/src/backup_restore/restore_impl/mod.rs new file mode 100644 index 0000000000000..0d997c7af7cab --- /dev/null +++ b/src/meta/src/backup_restore/restore_impl/mod.rs @@ -0,0 +1,32 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use risingwave_backup::error::BackupResult; +use risingwave_backup::meta_snapshot::{MetaSnapshot, Metadata}; +use risingwave_backup::MetaSnapshotId; + +pub mod v1; +pub mod v2; + +/// `Loader` gets, validates and amends `MetaSnapshot`. +#[async_trait::async_trait] +pub trait Loader { + async fn load(&self, id: MetaSnapshotId) -> BackupResult>; +} + +/// `Writer` writes `MetaSnapshot` to meta store. +#[async_trait::async_trait] +pub trait Writer { + async fn write(&self, s: MetaSnapshot) -> BackupResult<()>; +} diff --git a/src/meta/src/backup_restore/restore_impl/v1.rs b/src/meta/src/backup_restore/restore_impl/v1.rs new file mode 100644 index 0000000000000..793ce411714c1 --- /dev/null +++ b/src/meta/src/backup_restore/restore_impl/v1.rs @@ -0,0 +1,202 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use itertools::Itertools; +use risingwave_backup::error::{BackupError, BackupResult}; +use risingwave_backup::meta_snapshot::MetaSnapshot; +use risingwave_backup::meta_snapshot_v1::{ClusterMetadata, MetaSnapshotV1}; +use risingwave_backup::storage::{MetaSnapshotStorage, MetaSnapshotStorageRef}; +use risingwave_backup::MetaSnapshotId; + +use crate::backup_restore::restore_impl::{Loader, Writer}; +use crate::backup_restore::utils::MetaStoreBackendImpl; +use crate::dispatch_meta_store; +use crate::hummock::model::CompactionGroup; +use crate::manager::model::SystemParamsModel; +use crate::model::{ClusterId, MetadataModel, TableFragments}; +use crate::storage::{MetaStore, DEFAULT_COLUMN_FAMILY}; + +pub struct LoaderV1 { + backup_store: MetaSnapshotStorageRef, +} + +impl LoaderV1 { + pub fn new(backup_store: MetaSnapshotStorageRef) -> Self { + Self { backup_store } + } +} + +#[async_trait::async_trait] +impl Loader for LoaderV1 { + async fn load(&self, target_id: MetaSnapshotId) -> BackupResult> { + let backup_store = &self.backup_store; + let snapshot_list = &backup_store.manifest().snapshot_metadata; + let mut target_snapshot: MetaSnapshotV1 = backup_store.get(target_id).await?; + tracing::info!( + "snapshot {} before rewrite:\n{}", + target_id, + target_snapshot + ); + let newest_id = snapshot_list + .iter() + .map(|m| m.id) + .max() + .expect("should exist"); + assert!( + newest_id >= target_id, + "newest_id={}, target_id={}", + newest_id, + target_id + ); + // Always use newest snapshot's `default_cf` during restoring, in order not to corrupt shared + // data of snapshots. Otherwise, for example if we restore a older SST id generator, an + // existent SST in object store is at risk of being overwrote by the restored cluster. + // All these risky metadata are in `default_cf`, e.g. id generator, epoch. They must satisfy: + // - Value is monotonically non-decreasing. + // - Value is memcomparable. + // - Keys of newest_snapshot is a superset of that of target_snapshot. + if newest_id > target_id { + let newest_snapshot: MetaSnapshotV1 = backup_store.get(newest_id).await?; + for (k, v) in &target_snapshot.metadata.default_cf { + let newest_v = newest_snapshot + .metadata + .default_cf + .get(k) + .unwrap_or_else(|| panic!("violate superset requirement. key {:x?}", k)); + assert!(newest_v >= v, "violate monotonicity requirement"); + } + target_snapshot.metadata.default_cf = newest_snapshot.metadata.default_cf; + tracing::info!( + "snapshot {} after rewrite by snapshot {}:\n{}", + target_id, + newest_id, + target_snapshot, + ); + } + Ok(target_snapshot) + } +} + +pub struct WriterModelV1ToMetaStoreV1 { + meta_store: MetaStoreBackendImpl, +} + +impl WriterModelV1ToMetaStoreV1 { + pub fn new(meta_store: MetaStoreBackendImpl) -> Self { + Self { meta_store } + } +} + +#[async_trait::async_trait] +impl Writer for WriterModelV1ToMetaStoreV1 { + async fn write(&self, target_snapshot: MetaSnapshot) -> BackupResult<()> { + dispatch_meta_store!(&self.meta_store, store, { + restore_metadata(store.clone(), target_snapshot.clone()).await?; + }); + Ok(()) + } +} + +async fn restore_metadata_model( + meta_store: &S, + metadata: &[T], +) -> BackupResult<()> { + if !T::list(meta_store).await?.is_empty() { + return Err(BackupError::NonemptyMetaStorage); + } + for d in metadata { + d.insert(meta_store).await?; + } + Ok(()) +} + +async fn restore_system_param_model( + meta_store: &S, + metadata: &[T], +) -> BackupResult<()> { + if T::get(meta_store).await?.is_some() { + return Err(BackupError::NonemptyMetaStorage); + } + for d in metadata { + d.insert(meta_store).await?; + } + Ok(()) +} + +async fn restore_cluster_id( + meta_store: &S, + cluster_id: ClusterId, +) -> BackupResult<()> { + if ClusterId::from_meta_store(meta_store).await?.is_some() { + return Err(BackupError::NonemptyMetaStorage); + } + cluster_id.put_at_meta_store(meta_store).await?; + Ok(()) +} + +async fn restore_default_cf( + meta_store: &S, + snapshot: &MetaSnapshotV1, +) -> BackupResult<()> { + if !meta_store.list_cf(DEFAULT_COLUMN_FAMILY).await?.is_empty() { + return Err(BackupError::NonemptyMetaStorage); + } + for (k, v) in &snapshot.metadata.default_cf { + meta_store + .put_cf(DEFAULT_COLUMN_FAMILY, k.clone(), v.clone()) + .await?; + } + Ok(()) +} + +async fn restore_metadata( + meta_store: S, + snapshot: MetaSnapshotV1, +) -> BackupResult<()> { + restore_default_cf(&meta_store, &snapshot).await?; + restore_metadata_model(&meta_store, &[snapshot.metadata.version_stats]).await?; + restore_metadata_model( + &meta_store, + &snapshot + .metadata + .compaction_groups + .into_iter() + .map(CompactionGroup::from_protobuf) + .collect_vec(), + ) + .await?; + restore_metadata_model( + &meta_store, + &snapshot + .metadata + .table_fragments + .into_iter() + .map(TableFragments::from_protobuf) + .collect_vec(), + ) + .await?; + restore_metadata_model(&meta_store, &snapshot.metadata.user_info).await?; + restore_metadata_model(&meta_store, &snapshot.metadata.database).await?; + restore_metadata_model(&meta_store, &snapshot.metadata.schema).await?; + restore_metadata_model(&meta_store, &snapshot.metadata.table).await?; + restore_metadata_model(&meta_store, &snapshot.metadata.index).await?; + restore_metadata_model(&meta_store, &snapshot.metadata.sink).await?; + restore_metadata_model(&meta_store, &snapshot.metadata.view).await?; + restore_metadata_model(&meta_store, &snapshot.metadata.source).await?; + restore_metadata_model(&meta_store, &snapshot.metadata.function).await?; + restore_metadata_model(&meta_store, &snapshot.metadata.connection).await?; + restore_system_param_model(&meta_store, &[snapshot.metadata.system_param]).await?; + restore_cluster_id(&meta_store, snapshot.metadata.cluster_id.into()).await?; + Ok(()) +} diff --git a/src/meta/src/backup_restore/restore_impl/v2.rs b/src/meta/src/backup_restore/restore_impl/v2.rs new file mode 100644 index 0000000000000..086ad70f38437 --- /dev/null +++ b/src/meta/src/backup_restore/restore_impl/v2.rs @@ -0,0 +1,96 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::iter; + +use risingwave_backup::error::{BackupError, BackupResult}; +use risingwave_backup::meta_snapshot::MetaSnapshot; +use risingwave_backup::meta_snapshot_v2::{MetaSnapshotV2, MetadataV2}; +use risingwave_backup::storage::{MetaSnapshotStorage, MetaSnapshotStorageRef}; +use risingwave_backup::MetaSnapshotId; + +use crate::backup_restore::restore_impl::{Loader, Writer}; +use crate::controller::SqlMetaStore; + +pub struct LoaderV2 { + backup_store: MetaSnapshotStorageRef, +} + +impl LoaderV2 { + pub fn new(backup_store: MetaSnapshotStorageRef) -> Self { + Self { backup_store } + } +} + +#[async_trait::async_trait] +impl Loader for LoaderV2 { + async fn load(&self, target_id: MetaSnapshotId) -> BackupResult> { + let target_snapshot: MetaSnapshotV2 = self.backup_store.get(target_id).await?; + tracing::info!( + "snapshot {} before rewrite:\n{}", + target_id, + target_snapshot + ); + todo!("validate and rewrite seq") + } +} + +pub struct WriterModelV2ToMetaStoreV2 { + meta_store: SqlMetaStore, +} + +impl WriterModelV2ToMetaStoreV2 { + pub fn new(meta_store: SqlMetaStore) -> Self { + Self { meta_store } + } +} + +#[async_trait::async_trait] +impl Writer for WriterModelV2ToMetaStoreV2 { + async fn write(&self, target_snapshot: MetaSnapshot) -> BackupResult<()> { + let metadata = target_snapshot.metadata; + let db = &self.meta_store.conn; + insert_models(iter::once(metadata.version_stats), db).await?; + insert_models(metadata.compaction_configs, db).await?; + todo!("write other metadata") + } +} + +async fn insert_models( + models: impl IntoIterator, + db: &impl sea_orm::ConnectionTrait, +) -> BackupResult<()> +where + S: sea_orm::ModelTrait + Sync + Send + Sized + sea_orm::IntoActiveModel, + A: sea_orm::ActiveModelTrait + sea_orm::ActiveModelBehavior + Send + Sync + From, + <::Entity as sea_orm::EntityTrait>::Model: + sea_orm::IntoActiveModel, +{ + use sea_orm::EntityTrait; + if ::Entity::find() + .one(db) + .await + .map_err(|e| BackupError::MetaStorage(e.into()))? + .is_some() + { + return Err(BackupError::NonemptyMetaStorage); + } + for m in models { + m.into_active_model() + .insert(db) + .await + .map_err(|e| BackupError::MetaStorage(e.into()))?; + } + Ok(()) +} diff --git a/src/meta/src/backup_restore/utils.rs b/src/meta/src/backup_restore/utils.rs index c17650fb2aead..83cd1095beb25 100644 --- a/src/meta/src/backup_restore/utils.rs +++ b/src/meta/src/backup_restore/utils.rs @@ -23,6 +23,7 @@ use risingwave_object_store::object::object_metrics::ObjectStoreMetrics; use risingwave_object_store::object::parse_remote_object_store; use crate::backup_restore::RestoreOpts; +use crate::controller::SqlMetaStore; use crate::storage::{EtcdMetaStore, MemStore, WrappedEtcdClient as EtcdClient}; use crate::MetaStoreBackend; @@ -30,6 +31,8 @@ use crate::MetaStoreBackend; pub enum MetaStoreBackendImpl { Etcd(EtcdMetaStore), Mem(MemStore), + #[expect(dead_code, reason = "WIP")] + Sql(SqlMetaStore), } #[macro_export] @@ -38,6 +41,7 @@ macro_rules! dispatch_meta_store { match $impl { MetaStoreBackendImpl::Etcd($store) => $body, MetaStoreBackendImpl::Mem($store) => $body, + MetaStoreBackendImpl::Sql(_) => panic!("not supported"), } }}; } diff --git a/src/storage/backup/src/lib.rs b/src/storage/backup/src/lib.rs index af3c989995b9a..a69c4a3479c51 100644 --- a/src/storage/backup/src/lib.rs +++ b/src/storage/backup/src/lib.rs @@ -56,10 +56,12 @@ pub struct MetaSnapshotMetadata { pub ssts: Vec, pub max_committed_epoch: u64, pub safe_epoch: u64, + #[serde(default)] + pub format_version: u32, } impl MetaSnapshotMetadata { - pub fn new(id: MetaSnapshotId, v: &HummockVersion) -> Self { + pub fn new(id: MetaSnapshotId, v: &HummockVersion, format_version: u32) -> Self { Self { id, hummock_version_id: v.id, @@ -68,6 +70,7 @@ impl MetaSnapshotMetadata { .collect_vec(), max_committed_epoch: v.max_committed_epoch, safe_epoch: v.safe_epoch, + format_version, } } } @@ -104,6 +107,7 @@ impl From<&MetaSnapshotMetadata> for PbMetaSnapshotMetadata { hummock_version_id: m.hummock_version_id, max_committed_epoch: m.max_committed_epoch, safe_epoch: m.safe_epoch, + format_version: Some(m.format_version), } } } diff --git a/src/storage/backup/src/storage.rs b/src/storage/backup/src/storage.rs index 5c325809f2650..3db89abfacf33 100644 --- a/src/storage/backup/src/storage.rs +++ b/src/storage/backup/src/storage.rs @@ -124,6 +124,7 @@ impl MetaSnapshotStorage for ObjectStoreMetaSnapshotStorage { .push(MetaSnapshotMetadata::new( snapshot.id, snapshot.metadata.hummock_version_ref(), + snapshot.format_version, )); self.update_manifest(new_manifest).await?; Ok(()) diff --git a/src/storage/src/hummock/backup_reader.rs b/src/storage/src/hummock/backup_reader.rs index b9fa4a6258bbe..acca6454c2fec 100644 --- a/src/storage/src/hummock/backup_reader.rs +++ b/src/storage/src/hummock/backup_reader.rs @@ -192,6 +192,7 @@ impl BackupReader { } else { let this = self.clone(); let f = async move { + // TODO: change to v2 let snapshot: meta_snapshot_v1::MetaSnapshotV1 = current_store.0.get(snapshot_id).await.map_err(|e| { format!("failed to get meta snapshot {}. {}", snapshot_id, e)