kszucs commented on code in PR #21110:
URL: https://github.com/apache/datafusion/pull/21110#discussion_r3039917878


##########
datafusion/core/tests/parquet/content_defined_chunking.rs:
##########
@@ -0,0 +1,197 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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.
+
+//! Tests for parquet content-defined chunking (CDC).
+//!
+//! These tests verify that CDC options are correctly wired through to the
+//! parquet writer by inspecting file metadata (compressed sizes, page
+//! boundaries) on the written files.
+
+use arrow::array::{Int32Array, StringArray};
+use arrow::datatypes::{DataType, Field, Schema};
+use arrow::record_batch::RecordBatch;
+use datafusion::prelude::{ParquetReadOptions, SessionContext};
+use datafusion_common::config::{CdcOptions, TableParquetOptions};
+use parquet::arrow::ArrowWriter;
+use parquet::arrow::arrow_reader::ArrowReaderMetadata;
+use parquet::file::properties::WriterProperties;
+use std::fs::File;
+use std::sync::Arc;
+use tempfile::NamedTempFile;
+
+/// Create a RecordBatch with enough data to exercise CDC chunking.
+fn make_test_batch(num_rows: usize) -> RecordBatch {
+    let ids: Vec<i32> = (0..num_rows as i32).collect();
+    // ~100 bytes per row to generate enough data for CDC page splits
+    let payloads: Vec<String> = (0..num_rows)
+        .map(|i| format!("row-{i:06}-payload-{}", "x".repeat(80)))
+        .collect();
+
+    let schema = Arc::new(Schema::new(vec![
+        Field::new("id", DataType::Int32, false),
+        Field::new("payload", DataType::Utf8, false),
+    ]));
+
+    RecordBatch::try_new(
+        schema,
+        vec![
+            Arc::new(Int32Array::from(ids)),
+            Arc::new(StringArray::from(payloads)),
+        ],
+    )
+    .unwrap()
+}
+
+/// Build WriterProperties from TableParquetOptions, exercising the same
+/// code path that DataFusion's parquet sink uses.
+fn writer_props(
+    opts: &mut TableParquetOptions,
+    schema: &Arc<Schema>,
+) -> WriterProperties {
+    opts.arrow_schema(schema);
+    parquet::file::properties::WriterPropertiesBuilder::try_from(
+        opts as &TableParquetOptions,
+    )
+    .unwrap()
+    .build()
+}
+
+/// Write a batch to a temp parquet file and return the file handle.
+fn write_parquet_file(batch: &RecordBatch, props: WriterProperties) -> 
NamedTempFile {
+    let tmp = tempfile::Builder::new()
+        .suffix(".parquet")
+        .tempfile()
+        .unwrap();
+    let mut writer =
+        ArrowWriter::try_new(tmp.reopen().unwrap(), batch.schema(), 
Some(props)).unwrap();
+    writer.write(batch).unwrap();
+    writer.close().unwrap();
+    tmp
+}
+
+/// Read parquet metadata from a file.
+fn read_metadata(file: &NamedTempFile) -> 
parquet::file::metadata::ParquetMetaData {
+    let f = File::open(file.path()).unwrap();
+    let reader_meta = ArrowReaderMetadata::load(&f, 
Default::default()).unwrap();
+    reader_meta.metadata().as_ref().clone()
+}
+
+/// Write parquet with CDC enabled, read it back via DataFusion, and verify
+/// the data round-trips correctly.
+#[tokio::test]
+async fn cdc_data_round_trip() {
+    let batch = make_test_batch(5000);
+
+    let mut opts = TableParquetOptions::default();
+    opts.global.use_content_defined_chunking = Some(CdcOptions::default());
+    let props = writer_props(&mut opts, &batch.schema());
+
+    let tmp = write_parquet_file(&batch, props);
+
+    // Read back via DataFusion and verify row count
+    let ctx = SessionContext::new();
+    ctx.register_parquet(
+        "data",
+        tmp.path().to_str().unwrap(),
+        ParquetReadOptions::default(),
+    )
+    .await
+    .unwrap();
+
+    let result = ctx
+        .sql("SELECT COUNT(*), MIN(id), MAX(id) FROM data")
+        .await
+        .unwrap()
+        .collect()
+        .await
+        .unwrap();
+
+    let row = &result[0];
+    let count = row
+        .column(0)
+        .as_any()
+        .downcast_ref::<arrow::array::Int64Array>()
+        .unwrap()
+        .value(0);

Review Comment:
   Done.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to