From 793abb9689f94e994af8efe83fcd06965ab6bb14 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Fri, 13 Jun 2025 14:26:32 +0800 Subject: [PATCH 01/25] Add an example of embedding indexes inside a parquet file --- Cargo.lock | 1 + datafusion-examples/Cargo.toml | 2 + .../examples/embedding_parquet_indexes.rs | 155 ++++++++++++++++++ 3 files changed, 158 insertions(+) create mode 100644 datafusion-examples/examples/embedding_parquet_indexes.rs diff --git a/Cargo.lock b/Cargo.lock index 7a847276055a..799e4d734690 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2159,6 +2159,7 @@ dependencies = [ "arrow-flight", "arrow-schema", "async-trait", + "base64 0.22.1", "bytes", "dashmap", "datafusion", diff --git a/datafusion-examples/Cargo.toml b/datafusion-examples/Cargo.toml index b31708a5c1cc..6f2e1639fbe8 100644 --- a/datafusion-examples/Cargo.toml +++ b/datafusion-examples/Cargo.toml @@ -81,3 +81,5 @@ uuid = "1.17" [target.'cfg(not(target_os = "windows"))'.dev-dependencies] nix = { version = "0.30.1", features = ["fs"] } +[dependencies] +base64 = "0.22.1" \ No newline at end of file diff --git a/datafusion-examples/examples/embedding_parquet_indexes.rs b/datafusion-examples/examples/embedding_parquet_indexes.rs new file mode 100644 index 000000000000..a23715d0e345 --- /dev/null +++ b/datafusion-examples/examples/embedding_parquet_indexes.rs @@ -0,0 +1,155 @@ + +//! Example: embedding a "distinct values" index in a Parquet file's metadata +//! +//! 1. Read an existing Parquet file +//! 2. Compute distinct values for a target column using DataFusion +//! 3. Serialize the distinct list to bytes and write a new Parquet file +//! with these bytes appended as a custom metadata entry +//! 4. Read the new file, extract and deserialize the index from footer +//! 5. Use the index to answer membership queries without scanning data pages + +use arrow::array::{ArrayRef, StringArray, StringBuilder, StringViewArray}; +use arrow::record_batch::RecordBatch; +use arrow::util::pretty::pretty_format_batches; +use datafusion::prelude::*; +use datafusion::common::{DataFusionError, Result}; +use datafusion::datasource::listing::PartitionedFile; +use datafusion::datasource::memory::MemTable; +use datafusion::execution::object_store::ObjectStoreUrl; +use datafusion::parquet::arrow::ArrowWriter; +use datafusion::parquet::file::properties::WriterProperties; +use datafusion::parquet::file::metadata::KeyValue; +use datafusion::parquet::file::reader::{FileReader, SerializedFileReader}; +use datafusion::parquet::file::writer; +use std::fs::{File, read_dir}; +use std::io::{Read, Write}; +use std::path::Path; +use std::sync::Arc; +use arrow_schema::{DataType, Field, Schema}; +use tempfile::TempDir; +use datafusion::logical_expr::UserDefinedLogicalNode; +use datafusion::parquet::arrow::arrow_reader::ParquetRecordBatchReaderBuilder; + +#[tokio::main] +async fn main() -> Result<()> { + + let tmpdir = TempDir::new()?; + + let input_path = tmpdir.path().join("input.parquet"); + let indexed_path = tmpdir.path().join("output_with_index.parquet"); + let column = "category"; + + + let schema = Arc::new(Schema::new(vec![ + Field::new("category", DataType::Utf8, false), + ])); + + let mut builder = StringBuilder::new(); + builder.append_value("foo"); + builder.append_value("bar"); + builder.append_value("baz"); + builder.append_value("foo"); + builder.append_value("qux"); + builder.append_value("baz"); + builder.append_value("bar"); + builder.append_value("quux"); + builder.append_value("corge"); + builder.append_value("grault"); + + let array = Arc::new(builder.finish()) as ArrayRef; + + let batch = RecordBatch::try_new(schema.clone(), vec![array])?; + + let input_file = File::create(&input_path)?; + let mut writer = ArrowWriter::try_new(input_file, schema, None)?; + writer.write(&batch)?; + writer.finish()?; + + + // 1. Compute distinct values for `column` + let ctx = SessionContext::new(); + ctx.register_parquet("t", input_path.to_str().unwrap().to_string(), ParquetReadOptions::default()).await?; + + let df = ctx.sql(&format!("SELECT DISTINCT {col} FROM t", col=column)).await?; + let batches = df.collect().await?; + + println!("batches: {:#?}", batches); + + // Flatten distinct strings + let mut distinct: Vec = Vec::new(); + for batch in &batches { + // ut8 default read to ut8view + let col_array = batch.column(0) + .as_any().downcast_ref::().unwrap(); + for i in col_array.iter() { + distinct.push(i.unwrap().to_string()); + } + } + + // 2. Serialize distinct list (simple newline-delimited) + let serialized = distinct.join("\n"); + let bytes = serialized.as_bytes(); + + + let mut props = WriterProperties::builder(); + // store index length and bytes in metadata + props = props.set_key_value_metadata(Some(vec![ + KeyValue::new("distinct_index_size".to_string(), bytes.len().to_string()), + KeyValue::new("distinct_index_data".to_string(), base64::encode(bytes)), + ])); + let props = props.build(); + + // use ArrowWriter to copy data pages + let file = File::open(&input_path).map_err(|e| { + DataFusionError::from(e).context(format!("Error opening file {input_path:?}")) + })?; + + let write_file = File::create(&indexed_path).map_err(|e| { + DataFusionError::from(e).context(format!("Error creating file {indexed_path:?}")) + })?; + + + let reader = ParquetRecordBatchReaderBuilder::try_new(file)?; + + let schema = reader.schema().clone(); + let mut writer = ArrowWriter::try_new( + write_file, + schema, + Some(props), + )?; + + let file = File::open(&input_path).map_err(|e| { + DataFusionError::from(e).context(format!("Error opening file {input_path:?}")) + })?; + + // stream record batches using a fresh reader + let mut batch_reader = ParquetRecordBatchReaderBuilder::try_new(file)?.build()?; + while let Some(Ok(batch)) = batch_reader.next() { + println!("batch = {:#?}", batch); + writer.write(&batch)?; + } + writer.finish()?; + + + // 4. Open new file and extract index bytes + let file = File::open(&indexed_path).map_err(|e| { + DataFusionError::from(e).context(format!("Error opening file {input_path:?}")) + })?; + + let reader2 = ParquetRecordBatchReaderBuilder::try_new(file)?; + let meta = reader2.metadata().file_metadata(); + let kv = meta.key_value_metadata().unwrap(); + let map = kv.iter().map(|kv| (kv.key.as_str(), kv.value.as_deref().unwrap_or(""))).collect::>(); + let size: usize = map.get("distinct_index_size").unwrap().parse().unwrap(); + let data_b64 = map.get("distinct_index_data").unwrap(); + let data = base64::decode(data_b64).unwrap(); + + // 5. Demonstrate using the index: membership check + let loaded = String::from_utf8(data).unwrap(); + let loaded_set: std::collections::HashSet<&str> = loaded.lines().collect(); + println!("Index contains {} distinct values", loaded_set.len()); + println!("Contains 'foo'? {}", loaded_set.contains("foo")); + + Ok(()) +} + From 1f480ee23d63c1c259137757a8a2bf9fb8dccf17 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Fri, 13 Jun 2025 14:45:13 +0800 Subject: [PATCH 02/25] Add page image --- .../examples/embedding_parquet_indexes.rs | 49 +++++++++++++++++++ 1 file changed, 49 insertions(+) diff --git a/datafusion-examples/examples/embedding_parquet_indexes.rs b/datafusion-examples/examples/embedding_parquet_indexes.rs index a23715d0e345..eafd618bc577 100644 --- a/datafusion-examples/examples/embedding_parquet_indexes.rs +++ b/datafusion-examples/examples/embedding_parquet_indexes.rs @@ -1,3 +1,19 @@ +// 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. //! Example: embedding a "distinct values" index in a Parquet file's metadata //! @@ -30,6 +46,39 @@ use tempfile::TempDir; use datafusion::logical_expr::UserDefinedLogicalNode; use datafusion::parquet::arrow::arrow_reader::ParquetRecordBatchReaderBuilder; + +/// Example creating parquet file that +/// contains specialized indexes that +/// are ignored by other readers +/// +/// ```text +/// ┌──────────────────────┐ +/// │┌───────────────────┐ │ +/// ││ DataPage │ │ Standard Parquet +/// │└───────────────────┘ │ Data / pages +/// │┌───────────────────┐ │ +/// ││ DataPage │ │ +/// │└───────────────────┘ │ +/// │ ... │ +/// │ │ +/// │┌───────────────────┐ │ +/// ││ DataPage │ │ +/// │└───────────────────┘ │ +/// │┏━━━━━━━━━━━━━━━━━━━┓ │ +/// │┃ ┃ │ key/value metadata +/// │┃ Special Index ┃◀┼──── that points at the +/// │┃ ┃ │ │ special index +/// │┗━━━━━━━━━━━━━━━━━━━┛ │ +/// │╔═══════════════════╗ │ │ +/// │║ ║ │ +/// │║ Parquet Footer ║ │ │ Footer includes +/// │║ ║ ┼────── thrift-encoded +/// │║ ║ │ ParquetMetadata +/// │╚═══════════════════╝ │ +/// └──────────────────────┘ +/// +/// Parquet File +/// ``` #[tokio::main] async fn main() -> Result<()> { From 2a0ecacbf0f0a5ee7c8fdb8fb153a3c49ec6d1db Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Fri, 13 Jun 2025 16:00:26 +0800 Subject: [PATCH 03/25] Add prune file example --- .../examples/embedding_parquet_indexes.rs | 245 ++++++++++-------- 1 file changed, 134 insertions(+), 111 deletions(-) diff --git a/datafusion-examples/examples/embedding_parquet_indexes.rs b/datafusion-examples/examples/embedding_parquet_indexes.rs index eafd618bc577..e9f3236e0811 100644 --- a/datafusion-examples/examples/embedding_parquet_indexes.rs +++ b/datafusion-examples/examples/embedding_parquet_indexes.rs @@ -28,7 +28,7 @@ use arrow::array::{ArrayRef, StringArray, StringBuilder, StringViewArray}; use arrow::record_batch::RecordBatch; use arrow::util::pretty::pretty_format_batches; use datafusion::prelude::*; -use datafusion::common::{DataFusionError, Result}; +use datafusion::common::{DataFusionError, HashMap, HashSet, Result}; use datafusion::datasource::listing::PartitionedFile; use datafusion::datasource::memory::MemTable; use datafusion::execution::object_store::ObjectStoreUrl; @@ -37,15 +37,21 @@ use datafusion::parquet::file::properties::WriterProperties; use datafusion::parquet::file::metadata::KeyValue; use datafusion::parquet::file::reader::{FileReader, SerializedFileReader}; use datafusion::parquet::file::writer; -use std::fs::{File, read_dir}; +use std::fs::{File, read_dir, create_dir_all}; use std::io::{Read, Write}; -use std::path::Path; +use std::path::{Path, PathBuf}; use std::sync::Arc; -use arrow_schema::{DataType, Field, Schema}; +use arrow_schema::{DataType, Field, Schema, SchemaRef}; +use async_trait::async_trait; use tempfile::TempDir; -use datafusion::logical_expr::UserDefinedLogicalNode; +use datafusion::catalog::{Session, TableProvider}; +use datafusion::datasource::physical_plan::{FileScanConfigBuilder, ParquetSource}; +use datafusion::datasource::memory::DataSourceExec; +use datafusion::datasource::TableType; +use datafusion::logical_expr::{BinaryExpr, Operator, TableProviderFilterPushDown, UserDefinedLogicalNode}; use datafusion::parquet::arrow::arrow_reader::ParquetRecordBatchReaderBuilder; - +use datafusion::physical_plan::ExecutionPlan; +use datafusion::scalar::ScalarValue; /// Example creating parquet file that /// contains specialized indexes that @@ -79,125 +85,142 @@ use datafusion::parquet::arrow::arrow_reader::ParquetRecordBatchReaderBuilder; /// /// Parquet File /// ``` -#[tokio::main] -async fn main() -> Result<()> { - - let tmpdir = TempDir::new()?; - - let input_path = tmpdir.path().join("input.parquet"); - let indexed_path = tmpdir.path().join("output_with_index.parquet"); - let column = "category"; - let schema = Arc::new(Schema::new(vec![ - Field::new("category", DataType::Utf8, false), - ])); - let mut builder = StringBuilder::new(); - builder.append_value("foo"); - builder.append_value("bar"); - builder.append_value("baz"); - builder.append_value("foo"); - builder.append_value("qux"); - builder.append_value("baz"); - builder.append_value("bar"); - builder.append_value("quux"); - builder.append_value("corge"); - builder.append_value("grault"); - - let array = Arc::new(builder.finish()) as ArrayRef; +/// TableProvider that prunes Parquet files by their embedded distinct‑values index +#[derive(Debug)] +struct DistinctIndexTable { + schema: SchemaRef, + index: HashMap>, + dir: PathBuf, +} - let batch = RecordBatch::try_new(schema.clone(), vec![array])?; +impl DistinctIndexTable { + /// Scan a directory, read each file's footer metadata into a map + fn try_new(dir: impl Into, schema: SchemaRef) -> Result { + let dir = dir.into(); + let mut index = HashMap::new(); + for entry in read_dir(&dir)? { + let p = entry?.path(); + if p.extension().and_then(|s| s.to_str()) != Some("parquet") { + continue; + } + let name = p.file_name().unwrap().to_string_lossy().into_owned(); + let reader = SerializedFileReader::new(File::open(&p)?)?; + if let Some(kv) = reader.metadata().file_metadata().key_value_metadata() { + if let Some(e) = kv.iter().find(|kv| kv.key == "distinct_index_data") { + let raw = base64::decode(e.value.as_deref().unwrap()).unwrap(); + let s = String::from_utf8(raw).unwrap(); + let set = s.lines().map(|l| l.to_string()).collect(); + index.insert(name, set); + } + } + } + Ok(Self { schema, index, dir }) + } +} - let input_file = File::create(&input_path)?; - let mut writer = ArrowWriter::try_new(input_file, schema, None)?; +// Write a Parquet file and embed its distinct "category" values in footer metadata +fn write_file_with_index(path: &Path, values: &[&str]) -> Result<()> { + let field = Field::new("category", DataType::Utf8, false); + let schema = Arc::new(Schema::new(vec![field.clone()])); + let arr: ArrayRef = Arc::new(StringArray::from(values.to_vec())); + let batch = RecordBatch::try_new(schema.clone(), vec![arr])?; + + // Compute distinct values, serialize & Base64‑encode + let distinct: HashSet<_> = values.iter().copied().collect(); + let serialized = distinct.iter().cloned().collect::>().join("\n"); + let b64 = base64::encode(serialized.as_bytes()); + + let props = WriterProperties::builder() + .set_key_value_metadata(Some(vec![ + KeyValue::new("distinct_index_data".into(), b64), + ])) + .build(); + + let file = File::create(path)?; + let mut writer = ArrowWriter::try_new(file, schema, Some(props))?; writer.write(&batch)?; writer.finish()?; + Ok(()) +} - - // 1. Compute distinct values for `column` - let ctx = SessionContext::new(); - ctx.register_parquet("t", input_path.to_str().unwrap().to_string(), ParquetReadOptions::default()).await?; - - let df = ctx.sql(&format!("SELECT DISTINCT {col} FROM t", col=column)).await?; - let batches = df.collect().await?; - - println!("batches: {:#?}", batches); - - // Flatten distinct strings - let mut distinct: Vec = Vec::new(); - for batch in &batches { - // ut8 default read to ut8view - let col_array = batch.column(0) - .as_any().downcast_ref::().unwrap(); - for i in col_array.iter() { - distinct.push(i.unwrap().to_string()); +#[async_trait] +impl TableProvider for DistinctIndexTable { + fn as_any(&self) -> &dyn std::any::Any { self } + fn schema(&self) -> SchemaRef { self.schema.clone() } + fn table_type(&self) -> TableType { TableType::Base } + + /// Prune files before reading: only keep files whose distinct set contains the filter value + async fn scan( + &self, + _ctx: &dyn Session, + _proj: Option<&Vec>, + filters: &[Expr], + _limit: Option, + ) -> Result> { + // Look for a single `category = 'X'` filter + let mut target: Option = None; + + if filters.len() == 1 { + if let Expr::BinaryExpr(expr) = &filters[0] { + if expr.op == Operator::Eq { + if let (Expr::Column(c), Expr::Literal(ScalarValue::Utf8(Some(v)), _)) = (&*expr.left, &*expr.right) { + if c.name == "category" { + target = Some(v.clone()); + } + } + } + } } + // Determine which files to scan + let keep: Vec = self.index.iter() + .filter(|(_f,set)| target.as_ref().map_or(true, |v| set.contains(v))) + .map(|(f,_)| f.clone()) + .collect(); + + println!("Pruned files: {:?}", keep.clone()); + + // Build ParquetSource for kept files + let url = ObjectStoreUrl::parse("file://")?; + let source = Arc::new(ParquetSource::default()); + let mut builder = FileScanConfigBuilder::new(url, self.schema.clone(), source); + for file in keep { + let path = self.dir.join(&file); + let len = std::fs::metadata(&path)?.len(); + builder = builder.with_file(PartitionedFile::new(path.to_string_lossy().into_owned(), len)); + } + Ok(DataSourceExec::from_data_source(builder.build())) } - // 2. Serialize distinct list (simple newline-delimited) - let serialized = distinct.join("\n"); - let bytes = serialized.as_bytes(); - - - let mut props = WriterProperties::builder(); - // store index length and bytes in metadata - props = props.set_key_value_metadata(Some(vec![ - KeyValue::new("distinct_index_size".to_string(), bytes.len().to_string()), - KeyValue::new("distinct_index_data".to_string(), base64::encode(bytes)), - ])); - let props = props.build(); - - // use ArrowWriter to copy data pages - let file = File::open(&input_path).map_err(|e| { - DataFusionError::from(e).context(format!("Error opening file {input_path:?}")) - })?; - - let write_file = File::create(&indexed_path).map_err(|e| { - DataFusionError::from(e).context(format!("Error creating file {indexed_path:?}")) - })?; - - - let reader = ParquetRecordBatchReaderBuilder::try_new(file)?; - - let schema = reader.schema().clone(); - let mut writer = ArrowWriter::try_new( - write_file, - schema, - Some(props), - )?; - - let file = File::open(&input_path).map_err(|e| { - DataFusionError::from(e).context(format!("Error opening file {input_path:?}")) - })?; - - // stream record batches using a fresh reader - let mut batch_reader = ParquetRecordBatchReaderBuilder::try_new(file)?.build()?; - while let Some(Ok(batch)) = batch_reader.next() { - println!("batch = {:#?}", batch); - writer.write(&batch)?; + fn supports_filters_pushdown(&self, fs: &[&Expr]) -> Result> { + // Mark as inexact since pruning is file‑granular + Ok(vec![TableProviderFilterPushDown::Inexact; fs.len()]) } - writer.finish()?; +} +#[tokio::main] +async fn main() -> Result<()> { + + // 1. Create temp dir and write 3 Parquet files with different category sets + let tmp = TempDir::new()?; + let dir = tmp.path(); + create_dir_all(&dir)?; + write_file_with_index(&dir.join("a.parquet"), &["foo","bar","foo"])?; + write_file_with_index(&dir.join("b.parquet"), &["baz","qux"])?; + write_file_with_index(&dir.join("c.parquet"), &["foo","quux"])?; + + // 2. Register our custom TableProvider + let field = Field::new("category", DataType::Utf8, false); + let schema_ref = Arc::new(Schema::new(vec![field])); + let provider = Arc::new(DistinctIndexTable::try_new(dir, schema_ref.clone())?); + let ctx = SessionContext::new(); + ctx.register_table("t", provider)?; - // 4. Open new file and extract index bytes - let file = File::open(&indexed_path).map_err(|e| { - DataFusionError::from(e).context(format!("Error opening file {input_path:?}")) - })?; - - let reader2 = ParquetRecordBatchReaderBuilder::try_new(file)?; - let meta = reader2.metadata().file_metadata(); - let kv = meta.key_value_metadata().unwrap(); - let map = kv.iter().map(|kv| (kv.key.as_str(), kv.value.as_deref().unwrap_or(""))).collect::>(); - let size: usize = map.get("distinct_index_size").unwrap().parse().unwrap(); - let data_b64 = map.get("distinct_index_data").unwrap(); - let data = base64::decode(data_b64).unwrap(); - - // 5. Demonstrate using the index: membership check - let loaded = String::from_utf8(data).unwrap(); - let loaded_set: std::collections::HashSet<&str> = loaded.lines().collect(); - println!("Index contains {} distinct values", loaded_set.len()); - println!("Contains 'foo'? {}", loaded_set.contains("foo")); + // 3. Run a query: only files containing 'foo' get scanned + let df = ctx.sql("SELECT * FROM t WHERE category = 'foo'").await?; + df.show().await?; Ok(()) } From 4e61d0ea51a7679826b9842d97ca26adf1005f53 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Fri, 13 Jun 2025 16:19:43 +0800 Subject: [PATCH 04/25] Fix clippy --- .../examples/embedding_parquet_indexes.rs | 100 ++++++++++-------- 1 file changed, 57 insertions(+), 43 deletions(-) diff --git a/datafusion-examples/examples/embedding_parquet_indexes.rs b/datafusion-examples/examples/embedding_parquet_indexes.rs index e9f3236e0811..5abaceca727a 100644 --- a/datafusion-examples/examples/embedding_parquet_indexes.rs +++ b/datafusion-examples/examples/embedding_parquet_indexes.rs @@ -24,34 +24,31 @@ //! 4. Read the new file, extract and deserialize the index from footer //! 5. Use the index to answer membership queries without scanning data pages -use arrow::array::{ArrayRef, StringArray, StringBuilder, StringViewArray}; +use arrow::array::{ArrayRef, StringArray}; use arrow::record_batch::RecordBatch; -use arrow::util::pretty::pretty_format_batches; -use datafusion::prelude::*; -use datafusion::common::{DataFusionError, HashMap, HashSet, Result}; +use arrow_schema::{DataType, Field, Schema, SchemaRef}; +use async_trait::async_trait; +use base64::engine::general_purpose; +use base64::Engine; +use datafusion::catalog::{Session, TableProvider}; +use datafusion::common::{HashMap, HashSet, Result}; use datafusion::datasource::listing::PartitionedFile; -use datafusion::datasource::memory::MemTable; +use datafusion::datasource::memory::DataSourceExec; +use datafusion::datasource::physical_plan::{FileScanConfigBuilder, ParquetSource}; +use datafusion::datasource::TableType; use datafusion::execution::object_store::ObjectStoreUrl; +use datafusion::logical_expr::{Operator, TableProviderFilterPushDown}; use datafusion::parquet::arrow::ArrowWriter; -use datafusion::parquet::file::properties::WriterProperties; use datafusion::parquet::file::metadata::KeyValue; +use datafusion::parquet::file::properties::WriterProperties; use datafusion::parquet::file::reader::{FileReader, SerializedFileReader}; -use datafusion::parquet::file::writer; -use std::fs::{File, read_dir, create_dir_all}; -use std::io::{Read, Write}; +use datafusion::physical_plan::ExecutionPlan; +use datafusion::prelude::*; +use datafusion::scalar::ScalarValue; +use std::fs::{create_dir_all, read_dir, File}; use std::path::{Path, PathBuf}; use std::sync::Arc; -use arrow_schema::{DataType, Field, Schema, SchemaRef}; -use async_trait::async_trait; use tempfile::TempDir; -use datafusion::catalog::{Session, TableProvider}; -use datafusion::datasource::physical_plan::{FileScanConfigBuilder, ParquetSource}; -use datafusion::datasource::memory::DataSourceExec; -use datafusion::datasource::TableType; -use datafusion::logical_expr::{BinaryExpr, Operator, TableProviderFilterPushDown, UserDefinedLogicalNode}; -use datafusion::parquet::arrow::arrow_reader::ParquetRecordBatchReaderBuilder; -use datafusion::physical_plan::ExecutionPlan; -use datafusion::scalar::ScalarValue; /// Example creating parquet file that /// contains specialized indexes that @@ -85,10 +82,7 @@ use datafusion::scalar::ScalarValue; /// /// Parquet File /// ``` - - - -/// TableProvider that prunes Parquet files by their embedded distinct‑values index +/// DistinctIndexTable is a custom TableProvider that reads Parquet files #[derive(Debug)] struct DistinctIndexTable { schema: SchemaRef, @@ -110,7 +104,9 @@ impl DistinctIndexTable { let reader = SerializedFileReader::new(File::open(&p)?)?; if let Some(kv) = reader.metadata().file_metadata().key_value_metadata() { if let Some(e) = kv.iter().find(|kv| kv.key == "distinct_index_data") { - let raw = base64::decode(e.value.as_deref().unwrap()).unwrap(); + let raw = general_purpose::STANDARD_NO_PAD + .decode(e.value.as_deref().unwrap()) + .unwrap(); let s = String::from_utf8(raw).unwrap(); let set = s.lines().map(|l| l.to_string()).collect(); index.insert(name, set); @@ -131,12 +127,13 @@ fn write_file_with_index(path: &Path, values: &[&str]) -> Result<()> { // Compute distinct values, serialize & Base64‑encode let distinct: HashSet<_> = values.iter().copied().collect(); let serialized = distinct.iter().cloned().collect::>().join("\n"); - let b64 = base64::encode(serialized.as_bytes()); + let b64 = general_purpose::STANDARD_NO_PAD.encode(serialized.as_bytes()); let props = WriterProperties::builder() - .set_key_value_metadata(Some(vec![ - KeyValue::new("distinct_index_data".into(), b64), - ])) + .set_key_value_metadata(Some(vec![KeyValue::new( + "distinct_index_data".into(), + b64, + )])) .build(); let file = File::create(path)?; @@ -146,11 +143,18 @@ fn write_file_with_index(path: &Path, values: &[&str]) -> Result<()> { Ok(()) } +/// Implement TableProvider for DistinctIndexTable, using the distinct index to prune files #[async_trait] impl TableProvider for DistinctIndexTable { - fn as_any(&self) -> &dyn std::any::Any { self } - fn schema(&self) -> SchemaRef { self.schema.clone() } - fn table_type(&self) -> TableType { TableType::Base } + fn as_any(&self) -> &dyn std::any::Any { + self + } + fn schema(&self) -> SchemaRef { + self.schema.clone() + } + fn table_type(&self) -> TableType { + TableType::Base + } /// Prune files before reading: only keep files whose distinct set contains the filter value async fn scan( @@ -166,7 +170,11 @@ impl TableProvider for DistinctIndexTable { if filters.len() == 1 { if let Expr::BinaryExpr(expr) = &filters[0] { if expr.op == Operator::Eq { - if let (Expr::Column(c), Expr::Literal(ScalarValue::Utf8(Some(v)), _)) = (&*expr.left, &*expr.right) { + if let ( + Expr::Column(c), + Expr::Literal(ScalarValue::Utf8(Some(v)), _), + ) = (&*expr.left, &*expr.right) + { if c.name == "category" { target = Some(v.clone()); } @@ -175,9 +183,11 @@ impl TableProvider for DistinctIndexTable { } } // Determine which files to scan - let keep: Vec = self.index.iter() - .filter(|(_f,set)| target.as_ref().map_or(true, |v| set.contains(v))) - .map(|(f,_)| f.clone()) + let keep: Vec = self + .index + .iter() + .filter(|(_f, set)| target.as_ref().is_none_or(|v| set.contains(v))) + .map(|(f, _)| f.clone()) .collect(); println!("Pruned files: {:?}", keep.clone()); @@ -189,12 +199,18 @@ impl TableProvider for DistinctIndexTable { for file in keep { let path = self.dir.join(&file); let len = std::fs::metadata(&path)?.len(); - builder = builder.with_file(PartitionedFile::new(path.to_string_lossy().into_owned(), len)); + builder = builder.with_file(PartitionedFile::new( + path.to_string_lossy().into_owned(), + len, + )); } Ok(DataSourceExec::from_data_source(builder.build())) } - fn supports_filters_pushdown(&self, fs: &[&Expr]) -> Result> { + fn supports_filters_pushdown( + &self, + fs: &[&Expr], + ) -> Result> { // Mark as inexact since pruning is file‑granular Ok(vec![TableProviderFilterPushDown::Inexact; fs.len()]) } @@ -202,14 +218,13 @@ impl TableProvider for DistinctIndexTable { #[tokio::main] async fn main() -> Result<()> { - // 1. Create temp dir and write 3 Parquet files with different category sets let tmp = TempDir::new()?; let dir = tmp.path(); - create_dir_all(&dir)?; - write_file_with_index(&dir.join("a.parquet"), &["foo","bar","foo"])?; - write_file_with_index(&dir.join("b.parquet"), &["baz","qux"])?; - write_file_with_index(&dir.join("c.parquet"), &["foo","quux"])?; + create_dir_all(dir)?; + write_file_with_index(&dir.join("a.parquet"), &["foo", "bar", "foo"])?; + write_file_with_index(&dir.join("b.parquet"), &["baz", "qux"])?; + write_file_with_index(&dir.join("c.parquet"), &["foo", "quux"])?; // 2. Register our custom TableProvider let field = Field::new("category", DataType::Utf8, false); @@ -224,4 +239,3 @@ async fn main() -> Result<()> { Ok(()) } - From a8da658c6c651f66de5438eebaf4ebcaf9908ced Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Fri, 13 Jun 2025 16:47:22 +0800 Subject: [PATCH 05/25] polish code --- .../examples/embedding_parquet_indexes.rs | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) diff --git a/datafusion-examples/examples/embedding_parquet_indexes.rs b/datafusion-examples/examples/embedding_parquet_indexes.rs index 5abaceca727a..df51283c89a8 100644 --- a/datafusion-examples/examples/embedding_parquet_indexes.rs +++ b/datafusion-examples/examples/embedding_parquet_indexes.rs @@ -17,12 +17,12 @@ //! Example: embedding a "distinct values" index in a Parquet file's metadata //! -//! 1. Read an existing Parquet file +//! 1. Read existing Parquet files //! 2. Compute distinct values for a target column using DataFusion -//! 3. Serialize the distinct list to bytes and write a new Parquet file -//! with these bytes appended as a custom metadata entry -//! 4. Read the new file, extract and deserialize the index from footer -//! 5. Use the index to answer membership queries without scanning data pages +//! 3. Serialize the distinct index to bytes and write to the new Parquet file +//! with these encoded bytes appended as a custom metadata entry +//! 4. Read each new parquet file, extract and deserialize the index from footer +//! 5. Use the distinct index to prune files when querying use arrow::array::{ArrayRef, StringArray}; use arrow::record_batch::RecordBatch; @@ -109,6 +109,7 @@ impl DistinctIndexTable { .unwrap(); let s = String::from_utf8(raw).unwrap(); let set = s.lines().map(|l| l.to_string()).collect(); + println!("Inserting File: {name}, Distinct Values: {set:?}"); index.insert(name, set); } } @@ -176,6 +177,7 @@ impl TableProvider for DistinctIndexTable { ) = (&*expr.left, &*expr.right) { if c.name == "category" { + println!("Filtering for category: {v}"); target = Some(v.clone()); } } @@ -200,7 +202,7 @@ impl TableProvider for DistinctIndexTable { let path = self.dir.join(&file); let len = std::fs::metadata(&path)?.len(); builder = builder.with_file(PartitionedFile::new( - path.to_string_lossy().into_owned(), + path.to_str().unwrap().to_string(), len, )); } @@ -224,7 +226,7 @@ async fn main() -> Result<()> { create_dir_all(dir)?; write_file_with_index(&dir.join("a.parquet"), &["foo", "bar", "foo"])?; write_file_with_index(&dir.join("b.parquet"), &["baz", "qux"])?; - write_file_with_index(&dir.join("c.parquet"), &["foo", "quux"])?; + write_file_with_index(&dir.join("c.parquet"), &["foo", "quux", "quux"])?; // 2. Register our custom TableProvider let field = Field::new("category", DataType::Utf8, false); From c1ab4b9f17894e26a1ba4d5ef7dd88b0b05ece1c Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Fri, 13 Jun 2025 16:52:56 +0800 Subject: [PATCH 06/25] Fmt --- datafusion-examples/Cargo.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion-examples/Cargo.toml b/datafusion-examples/Cargo.toml index 6f2e1639fbe8..4bb061353d1e 100644 --- a/datafusion-examples/Cargo.toml +++ b/datafusion-examples/Cargo.toml @@ -82,4 +82,4 @@ uuid = "1.17" [target.'cfg(not(target_os = "windows"))'.dev-dependencies] nix = { version = "0.30.1", features = ["fs"] } [dependencies] -base64 = "0.22.1" \ No newline at end of file +base64 = "0.22.1" From baf03118155c238cb418bce196772dad9054b61a Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Fri, 13 Jun 2025 23:21:35 +0800 Subject: [PATCH 07/25] address comments --- .../examples/embedding_parquet_indexes.rs | 168 +++++++++++++++--- 1 file changed, 143 insertions(+), 25 deletions(-) diff --git a/datafusion-examples/examples/embedding_parquet_indexes.rs b/datafusion-examples/examples/embedding_parquet_indexes.rs index df51283c89a8..22c3d98fec1c 100644 --- a/datafusion-examples/examples/embedding_parquet_indexes.rs +++ b/datafusion-examples/examples/embedding_parquet_indexes.rs @@ -38,7 +38,7 @@ use datafusion::datasource::physical_plan::{FileScanConfigBuilder, ParquetSource use datafusion::datasource::TableType; use datafusion::execution::object_store::ObjectStoreUrl; use datafusion::logical_expr::{Operator, TableProviderFilterPushDown}; -use datafusion::parquet::arrow::ArrowWriter; +use datafusion::parquet::arrow::{ArrowSchemaConverter, ArrowWriter}; use datafusion::parquet::file::metadata::KeyValue; use datafusion::parquet::file::properties::WriterProperties; use datafusion::parquet::file::reader::{FileReader, SerializedFileReader}; @@ -46,9 +46,16 @@ use datafusion::physical_plan::ExecutionPlan; use datafusion::prelude::*; use datafusion::scalar::ScalarValue; use std::fs::{create_dir_all, read_dir, File}; +use std::io::{Read, Seek, SeekFrom, Write}; use std::path::{Path, PathBuf}; use std::sync::Arc; +use futures::AsyncWriteExt; use tempfile::TempDir; +use datafusion::parquet::column::writer::ColumnWriter; +use datafusion::parquet::data_type::{ByteArray, ByteArrayType}; +use datafusion::parquet::errors::ParquetError; +use datafusion::parquet::file::writer::{SerializedColumnWriter, SerializedFileWriter}; +use datafusion_proto::protobuf::FromProtoError::DataFusionError; /// Example creating parquet file that /// contains specialized indexes that @@ -95,29 +102,42 @@ impl DistinctIndexTable { fn try_new(dir: impl Into, schema: SchemaRef) -> Result { let dir = dir.into(); let mut index = HashMap::new(); + for entry in read_dir(&dir)? { - let p = entry?.path(); - if p.extension().and_then(|s| s.to_str()) != Some("parquet") { + let path = entry?.path(); + if path.extension().and_then(|s| s.to_str()) != Some("parquet") { continue; } - let name = p.file_name().unwrap().to_string_lossy().into_owned(); - let reader = SerializedFileReader::new(File::open(&p)?)?; - if let Some(kv) = reader.metadata().file_metadata().key_value_metadata() { - if let Some(e) = kv.iter().find(|kv| kv.key == "distinct_index_data") { - let raw = general_purpose::STANDARD_NO_PAD - .decode(e.value.as_deref().unwrap()) - .unwrap(); - let s = String::from_utf8(raw).unwrap(); - let set = s.lines().map(|l| l.to_string()).collect(); - println!("Inserting File: {name}, Distinct Values: {set:?}"); - index.insert(name, set); - } - } + let file_name = path.file_name().unwrap().to_string_lossy().to_string(); + + // 直接用工具函数读取该文件的 distinct index + let distinct_set = read_distinct_index(&path)?; + index.insert(file_name, distinct_set); } + Ok(Self { schema, index, dir }) } } +pub struct IndexedParquetWriter { + writer: SerializedFileWriter, +} + +impl IndexedParquetWriter { + /// 构造:传入已经创建好的文件、schema 和普通的 WriterProperties(不要在 metadata 里放 Base64) + pub fn try_new( + sink: W, + schema: Arc, + props: WriterProperties, + ) -> Result { + let schema_desc = ArrowSchemaConverter::new().convert(schema.as_ref())?; + let props_ptr = Arc::new(props); + let writer = SerializedFileWriter::new(sink, schema_desc.root_schema_ptr(), props_ptr)?; + Ok(Self { writer }) + } +} + + // Write a Parquet file and embed its distinct "category" values in footer metadata fn write_file_with_index(path: &Path, values: &[&str]) -> Result<()> { let field = Field::new("category", DataType::Utf8, false); @@ -127,23 +147,121 @@ fn write_file_with_index(path: &Path, values: &[&str]) -> Result<()> { // Compute distinct values, serialize & Base64‑encode let distinct: HashSet<_> = values.iter().copied().collect(); - let serialized = distinct.iter().cloned().collect::>().join("\n"); - let b64 = general_purpose::STANDARD_NO_PAD.encode(serialized.as_bytes()); + let serialized = distinct.into_iter().collect::>().join("\n"); + let index_bytes = serialized.into_bytes(); let props = WriterProperties::builder() - .set_key_value_metadata(Some(vec![KeyValue::new( - "distinct_index_data".into(), - b64, - )])) .build(); let file = File::create(path)?; - let mut writer = ArrowWriter::try_new(file, schema, Some(props))?; - writer.write(&batch)?; - writer.finish()?; + + let mut writer = IndexedParquetWriter::try_new(file, schema.clone(), props)?; + { + // 1) next_row_group + let mut rg_writer = writer.writer.next_row_group()?; + + // 2) 拿到 SerializedColumnWriter + let mut ser_col_writer: SerializedColumnWriter<'_> = + rg_writer + .next_column()? + .ok_or_else(|| ParquetError::General("No column writer".into()))?; + + // 3) 通过 typed 拿到具体的 ByteArrayColumnWriter 引用 + let col_writer = ser_col_writer.typed::(); + + // 4) 写入数据 + let values_bytes: Vec = batch + .column(0) + .as_any() + .downcast_ref::() + .unwrap() + .iter() + .map(|opt| ByteArray::from(opt.unwrap())) + .collect(); + col_writer.write_batch(&values_bytes, None, None)?; + + // 5) 关闭这个 column writer(.close(self) 会消费 ser_col_writer) + ser_col_writer.close()?; + + // 6) 关闭 row‑group + rg_writer.close()?; + } + + let offset = writer.writer + .inner() + .seek(SeekFrom::Current(0))?; + + println!("Writing distinct index at offset: {} path: {}", offset, path.display()); + + writer.writer.inner().write_all(&index_bytes)?; + + let final_props = WriterProperties::builder() + .set_key_value_metadata(Some(vec![ + KeyValue::new("distinct_index_offset".into(), offset.to_string()), + KeyValue::new("distinct_index_length".into(), index_bytes.len().to_string()), + ])) + .build(); + + let mut footer_writer = + SerializedFileWriter::new( + writer.writer.inner(), + ArrowSchemaConverter::new().convert(schema.as_ref())?.root_schema_ptr(), + Arc::new(final_props), + )?; + + footer_writer.close()?; + + println!("Finished writing file"); + Ok(()) } + +fn read_distinct_index(path: &Path) -> Result> { + // 1. Open reader for metadata + let reader = SerializedFileReader::new( + File::open(path) + .map_err(|e| ParquetError::General(e.to_string()))? + )?; + let meta = reader.metadata().file_metadata(); + let (off_kv, len_kv) = meta + .key_value_metadata() + .and_then(|vec| { + let off = vec.iter().find(|kv| kv.key == "distinct_index_offset")?; + let len = vec.iter().find(|kv| kv.key == "distinct_index_length")?; + Some((off, len)) + }) + .ok_or_else(|| ParquetError::General("missing index offset/length metadata".into()))?; + + // 2. Parse offset and length, converting any ParseIntError to String + let offset: u64 = off_kv + .value + .as_ref() + .ok_or_else(|| ParquetError::General("empty offset".into()))? + .parse::() + .map_err(|e| ParquetError::General(e.to_string()))?; + let length: usize = len_kv + .value + .as_ref() + .ok_or_else(|| ParquetError::General("empty length".into()))? + .parse::() + .map_err(|e| ParquetError::General(e.to_string()))?; + + // 3. Seek & read exactly `length` bytes at `offset` + let mut file = File::open(path) + .map_err(|e| ParquetError::General(e.to_string()))?; + file.seek(SeekFrom::Start(offset)) + .map_err(|e| ParquetError::General(e.to_string()))?; + let mut buf = vec![0u8; length]; + file.read_exact(&mut buf) + .map_err(|e| ParquetError::General(e.to_string()))?; + + // 4. Decode UTF-8 & split into lines + let s = String::from_utf8(buf) + .map_err(|e| ParquetError::General(e.to_string()))?; + Ok(s.lines().map(|l| l.to_string()).collect()) +} + /// Implement TableProvider for DistinctIndexTable, using the distinct index to prune files #[async_trait] impl TableProvider for DistinctIndexTable { From 18e7028a26ab404ea74b4ad1f68f60c942597805 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Fri, 13 Jun 2025 23:50:30 +0800 Subject: [PATCH 08/25] Add debug --- .../examples/embedding_parquet_indexes.rs | 108 +++++++++++------- 1 file changed, 64 insertions(+), 44 deletions(-) diff --git a/datafusion-examples/examples/embedding_parquet_indexes.rs b/datafusion-examples/examples/embedding_parquet_indexes.rs index 22c3d98fec1c..9412156e2751 100644 --- a/datafusion-examples/examples/embedding_parquet_indexes.rs +++ b/datafusion-examples/examples/embedding_parquet_indexes.rs @@ -112,6 +112,8 @@ impl DistinctIndexTable { // 直接用工具函数读取该文件的 distinct index let distinct_set = read_distinct_index(&path)?; + + println!("Read distinct index for {}: {:?}", file_name, distinct_set); index.insert(file_name, distinct_set); } @@ -156,36 +158,39 @@ fn write_file_with_index(path: &Path, values: &[&str]) -> Result<()> { let file = File::create(path)?; let mut writer = IndexedParquetWriter::try_new(file, schema.clone(), props)?; - { - // 1) next_row_group - let mut rg_writer = writer.writer.next_row_group()?; - - // 2) 拿到 SerializedColumnWriter - let mut ser_col_writer: SerializedColumnWriter<'_> = - rg_writer - .next_column()? - .ok_or_else(|| ParquetError::General("No column writer".into()))?; - - // 3) 通过 typed 拿到具体的 ByteArrayColumnWriter 引用 - let col_writer = ser_col_writer.typed::(); - - // 4) 写入数据 - let values_bytes: Vec = batch - .column(0) - .as_any() - .downcast_ref::() - .unwrap() - .iter() - .map(|opt| ByteArray::from(opt.unwrap())) - .collect(); - col_writer.write_batch(&values_bytes, None, None)?; - - // 5) 关闭这个 column writer(.close(self) 会消费 ser_col_writer) - ser_col_writer.close()?; - - // 6) 关闭 row‑group - rg_writer.close()?; - } + // { + // // 1) next_row_group + // let mut rg_writer = writer.writer.next_row_group()?; + // + // // 2) 拿到 SerializedColumnWriter + // let mut ser_col_writer: SerializedColumnWriter<'_> = + // rg_writer + // .next_column()? + // .ok_or_else(|| ParquetError::General("No column writer".into()))?; + // + // // 3) 通过 typed 拿到具体的 ByteArrayColumnWriter 引用 + // let col_writer = ser_col_writer.typed::(); + // + // // 4) 写入数据 + // let values_bytes: Vec = batch + // .column(0) + // .as_any() + // .downcast_ref::() + // .unwrap() + // .iter() + // .map(|opt| ByteArray::from(opt.unwrap())) + // .collect(); + // + // println!("Writing values: {:?}", values_bytes); + // + // col_writer.write_batch(&values_bytes, None, None)?; + // + // // 5) 关闭这个 column writer(.close(self) 会消费 ser_col_writer) + // ser_col_writer.close()?; + // + // // 6) 关闭 row‑group + // rg_writer.close()?; + // } let offset = writer.writer .inner() @@ -195,21 +200,36 @@ fn write_file_with_index(path: &Path, values: &[&str]) -> Result<()> { writer.writer.inner().write_all(&index_bytes)?; - let final_props = WriterProperties::builder() - .set_key_value_metadata(Some(vec![ - KeyValue::new("distinct_index_offset".into(), offset.to_string()), - KeyValue::new("distinct_index_length".into(), index_bytes.len().to_string()), - ])) - .build(); - - let mut footer_writer = - SerializedFileWriter::new( - writer.writer.inner(), - ArrowSchemaConverter::new().convert(schema.as_ref())?.root_schema_ptr(), - Arc::new(final_props), - )?; - footer_writer.close()?; + writer.writer.append_key_value_metadata( + KeyValue::new( + "distinct_index_offset".into(), + offset.to_string(), + ), + ); + + writer.writer.append_key_value_metadata( + KeyValue::new( + "distinct_index_length".into(), + index_bytes.len().to_string(), + ), + ); + + // let final_props = WriterProperties::builder() + // .set_key_value_metadata(Some(vec![ + // KeyValue::new("distinct_index_offset".into(), offset.to_string()), + // KeyValue::new("distinct_index_length".into(), index_bytes.len().to_string()), + // ])) + // .build(); + // + // let mut footer_writer = + // SerializedFileWriter::new( + // writer.writer.inner(), + // ArrowSchemaConverter::new().convert(schema.as_ref())?.root_schema_ptr(), + // Arc::new(final_props), + // )?; + + writer.writer.close()?; println!("Finished writing file"); From 66dc5e4c1cde6a666efec31d993da69cd46f2c10 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Sat, 14 Jun 2025 22:27:35 +0800 Subject: [PATCH 09/25] Add new example, but it will fail with page index --- .../examples/embedding_parquet_indexes.rs | 207 +++++++++--------- 1 file changed, 98 insertions(+), 109 deletions(-) diff --git a/datafusion-examples/examples/embedding_parquet_indexes.rs b/datafusion-examples/examples/embedding_parquet_indexes.rs index 9412156e2751..6f498332b8ea 100644 --- a/datafusion-examples/examples/embedding_parquet_indexes.rs +++ b/datafusion-examples/examples/embedding_parquet_indexes.rs @@ -140,146 +140,135 @@ impl IndexedParquetWriter { } -// Write a Parquet file and embed its distinct "category" values in footer metadata +const PARQUET_MAGIC: &[u8] = b"PAR1"; +const INDEX_MAGIC: &[u8] = b"IDX1"; // 自定义索引魔术字 + +// 修改 write_file_with_index 函数 fn write_file_with_index(path: &Path, values: &[&str]) -> Result<()> { let field = Field::new("category", DataType::Utf8, false); let schema = Arc::new(Schema::new(vec![field.clone()])); let arr: ArrayRef = Arc::new(StringArray::from(values.to_vec())); let batch = RecordBatch::try_new(schema.clone(), vec![arr])?; - // Compute distinct values, serialize & Base64‑encode + // 计算不同值索引 let distinct: HashSet<_> = values.iter().copied().collect(); let serialized = distinct.into_iter().collect::>().join("\n"); let index_bytes = serialized.into_bytes(); - let props = WriterProperties::builder() - .build(); - + let props = WriterProperties::builder().build(); let file = File::create(path)?; let mut writer = IndexedParquetWriter::try_new(file, schema.clone(), props)?; - // { - // // 1) next_row_group - // let mut rg_writer = writer.writer.next_row_group()?; - // - // // 2) 拿到 SerializedColumnWriter - // let mut ser_col_writer: SerializedColumnWriter<'_> = - // rg_writer - // .next_column()? - // .ok_or_else(|| ParquetError::General("No column writer".into()))?; - // - // // 3) 通过 typed 拿到具体的 ByteArrayColumnWriter 引用 - // let col_writer = ser_col_writer.typed::(); - // - // // 4) 写入数据 - // let values_bytes: Vec = batch - // .column(0) - // .as_any() - // .downcast_ref::() - // .unwrap() - // .iter() - // .map(|opt| ByteArray::from(opt.unwrap())) - // .collect(); - // - // println!("Writing values: {:?}", values_bytes); - // - // col_writer.write_batch(&values_bytes, None, None)?; - // - // // 5) 关闭这个 column writer(.close(self) 会消费 ser_col_writer) - // ser_col_writer.close()?; - // - // // 6) 关闭 row‑group - // rg_writer.close()?; - // } - - let offset = writer.writer - .inner() - .seek(SeekFrom::Current(0))?; - - println!("Writing distinct index at offset: {} path: {}", offset, path.display()); - writer.writer.inner().write_all(&index_bytes)?; + // 写入数据 + { + let mut rg_writer = writer.writer.next_row_group()?; + let mut ser_col_writer = rg_writer + .next_column()? + .ok_or_else(|| ParquetError::General("No column writer".into()))?; + + let col_writer = ser_col_writer.typed::(); + let values_bytes: Vec = batch + .column(0) + .as_any() + .downcast_ref::() + .unwrap() + .iter() + .map(|opt| ByteArray::from(opt.unwrap())) + .collect(); + println!("Writing values: {:?}", values_bytes); + col_writer.write_batch(&values_bytes, None, None)?; + ser_col_writer.close()?; + rg_writer.close()?; + } - writer.writer.append_key_value_metadata( - KeyValue::new( - "distinct_index_offset".into(), - offset.to_string(), - ), - ); - - writer.writer.append_key_value_metadata( - KeyValue::new( - "distinct_index_length".into(), - index_bytes.len().to_string(), - ), - ); - - // let final_props = WriterProperties::builder() - // .set_key_value_metadata(Some(vec![ - // KeyValue::new("distinct_index_offset".into(), offset.to_string()), - // KeyValue::new("distinct_index_length".into(), index_bytes.len().to_string()), - // ])) - // .build(); - // - // let mut footer_writer = - // SerializedFileWriter::new( - // writer.writer.inner(), - // ArrowSchemaConverter::new().convert(schema.as_ref())?.root_schema_ptr(), - // Arc::new(final_props), - // )?; + // 获取当前写入位置(索引开始位置) + let offset = writer.writer.inner().seek(SeekFrom::Current(0))?; - writer.writer.close()?; + // 写入索引魔术字和长度 + let index_len = index_bytes.len() as u64; + writer.writer.inner().write_all(b"IDX1")?; // 4字节魔术字 + writer.writer.inner().write_all(&index_len.to_le_bytes())?; // 8字节长度 - println!("Finished writing file"); + // 写入索引数据 + writer.writer.inner().write_all(&index_bytes)?; + + // 记录索引元数据 + writer.writer.append_key_value_metadata(KeyValue::new( + "distinct_index_offset".to_string(), + offset.to_string(), + )); + writer.writer.append_key_value_metadata(KeyValue::new( + "distinct_index_length".to_string(), + index_bytes.len().to_string(), + )); + + // 关闭写入器(这会写入页脚) + writer.writer.close()?; + println!("Finished writing file to {}", path.display()); Ok(()) } -fn read_distinct_index(path: &Path) -> Result> { - // 1. Open reader for metadata - let reader = SerializedFileReader::new( - File::open(path) - .map_err(|e| ParquetError::General(e.to_string()))? - )?; +// 修改后的读取函数 +fn read_distinct_index(path: &Path) -> Result, ParquetError> { + let mut file = File::open(path)?; + + + let file_size = file.metadata()?.len(); + println!("Reading index from {} (size: {})", path.display(), file_size); + + // 1. 读取元数据获取索引位置 + let reader = SerializedFileReader::new(file.try_clone()?)?; let meta = reader.metadata().file_metadata(); - let (off_kv, len_kv) = meta - .key_value_metadata() - .and_then(|vec| { - let off = vec.iter().find(|kv| kv.key == "distinct_index_offset")?; - let len = vec.iter().find(|kv| kv.key == "distinct_index_length")?; - Some((off, len)) - }) - .ok_or_else(|| ParquetError::General("missing index offset/length metadata".into()))?; - - // 2. Parse offset and length, converting any ParseIntError to String - let offset: u64 = off_kv - .value - .as_ref() - .ok_or_else(|| ParquetError::General("empty offset".into()))? + + + let offset = meta.key_value_metadata() + .and_then(|kvs| kvs.iter().find(|kv| kv.key == "distinct_index_offset")) + .and_then(|kv| kv.value.as_ref()) + .ok_or_else(|| ParquetError::General("Missing index offset".into()))? .parse::() .map_err(|e| ParquetError::General(e.to_string()))?; - let length: usize = len_kv - .value - .as_ref() - .ok_or_else(|| ParquetError::General("empty length".into()))? + + let length = meta.key_value_metadata() + .and_then(|kvs| kvs.iter().find(|kv| kv.key == "distinct_index_length")) + .and_then(|kv| kv.value.as_ref()) + .ok_or_else(|| ParquetError::General("Missing index length".into()))? .parse::() .map_err(|e| ParquetError::General(e.to_string()))?; - // 3. Seek & read exactly `length` bytes at `offset` - let mut file = File::open(path) - .map_err(|e| ParquetError::General(e.to_string()))?; - file.seek(SeekFrom::Start(offset)) - .map_err(|e| ParquetError::General(e.to_string()))?; - let mut buf = vec![0u8; length]; - file.read_exact(&mut buf) - .map_err(|e| ParquetError::General(e.to_string()))?; + println!("Reading index at offset: {}, length: {}", offset, length); + + // 2. 定位并读取索引 + file.seek(SeekFrom::Start(offset))?; - // 4. Decode UTF-8 & split into lines - let s = String::from_utf8(buf) + // 验证魔术字 + let mut magic_buf = [0u8; 4]; + file.read_exact(&mut magic_buf)?; + if &magic_buf != INDEX_MAGIC { + return Err(ParquetError::General("Invalid index magic".into())); + } + + // 读取索引长度 + let mut len_buf = [0u8; 8]; + file.read_exact(&mut len_buf)?; + let stored_len = u64::from_le_bytes(len_buf) as usize; + + if stored_len != length { + return Err(ParquetError::General("Index length mismatch".into())); + } + + // 读取索引数据 + let mut index_buf = vec![0u8; length]; + file.read_exact(&mut index_buf)?; + + // 3. 解码索引 + let s = String::from_utf8(index_buf) .map_err(|e| ParquetError::General(e.to_string()))?; - Ok(s.lines().map(|l| l.to_string()).collect()) + + Ok(s.lines().map(|s| s.to_string()).collect()) } /// Implement TableProvider for DistinctIndexTable, using the distinct index to prune files From eb9b62e70a8c66e39d20abbd3f39587d85a72dac Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Sat, 14 Jun 2025 22:49:48 +0800 Subject: [PATCH 10/25] add debug --- datafusion-examples/examples/embedding_parquet_indexes.rs | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/datafusion-examples/examples/embedding_parquet_indexes.rs b/datafusion-examples/examples/embedding_parquet_indexes.rs index 6f498332b8ea..f8e07d819953 100644 --- a/datafusion-examples/examples/embedding_parquet_indexes.rs +++ b/datafusion-examples/examples/embedding_parquet_indexes.rs @@ -360,8 +360,13 @@ async fn main() -> Result<()> { let schema_ref = Arc::new(Schema::new(vec![field])); let provider = Arc::new(DistinctIndexTable::try_new(dir, schema_ref.clone())?); let ctx = SessionContext::new(); + ctx.register_table("t", provider)?; + // 3. Run a query: only files containing 'foo' get scanned + let df = ctx.sql("SELECT * FROM t").await?; + df.show().await?; + // 3. Run a query: only files containing 'foo' get scanned let df = ctx.sql("SELECT * FROM t WHERE category = 'foo'").await?; df.show().await?; From 310576eba016d74a6426122e134b6d339ea71d87 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Sat, 14 Jun 2025 23:34:23 +0800 Subject: [PATCH 11/25] add debug --- Cargo.lock | 32 ------------------- Cargo.toml | 14 ++++---- .../examples/embedding_parquet_indexes.rs | 21 ++++++++++++ 3 files changed, 28 insertions(+), 39 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 799e4d734690..349b4e0ae136 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -247,8 +247,6 @@ checksum = "7c02d123df017efcdfbd739ef81735b36c5ba83ec3c59c80a9d7ecc718f92e50" [[package]] name = "arrow" version = "55.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b1bb018b6960c87fd9d025009820406f74e83281185a8bdcb44880d2aa5c9a87" dependencies = [ "arrow-arith", "arrow-array", @@ -271,8 +269,6 @@ dependencies = [ [[package]] name = "arrow-arith" version = "55.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "44de76b51473aa888ecd6ad93ceb262fb8d40d1f1154a4df2f069b3590aa7575" dependencies = [ "arrow-array", "arrow-buffer", @@ -285,8 +281,6 @@ dependencies = [ [[package]] name = "arrow-array" version = "55.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "29ed77e22744475a9a53d00026cf8e166fe73cf42d89c4c4ae63607ee1cfcc3f" dependencies = [ "ahash 0.8.12", "arrow-buffer", @@ -302,8 +296,6 @@ dependencies = [ [[package]] name = "arrow-buffer" version = "55.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b0391c96eb58bf7389171d1e103112d3fc3e5625ca6b372d606f2688f1ea4cce" dependencies = [ "bytes", "half", @@ -313,8 +305,6 @@ dependencies = [ [[package]] name = "arrow-cast" version = "55.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f39e1d774ece9292697fcbe06b5584401b26bd34be1bec25c33edae65c2420ff" dependencies = [ "arrow-array", "arrow-buffer", @@ -334,8 +324,6 @@ dependencies = [ [[package]] name = "arrow-csv" version = "55.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9055c972a07bf12c2a827debfd34f88d3b93da1941d36e1d9fee85eebe38a12a" dependencies = [ "arrow-array", "arrow-cast", @@ -350,8 +338,6 @@ dependencies = [ [[package]] name = "arrow-data" version = "55.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cf75ac27a08c7f48b88e5c923f267e980f27070147ab74615ad85b5c5f90473d" dependencies = [ "arrow-buffer", "arrow-schema", @@ -362,8 +348,6 @@ dependencies = [ [[package]] name = "arrow-flight" version = "55.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "91efc67a4f5a438833dd76ef674745c80f6f6b9a428a3b440cbfbf74e32867e6" dependencies = [ "arrow-arith", "arrow-array", @@ -389,8 +373,6 @@ dependencies = [ [[package]] name = "arrow-ipc" version = "55.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a222f0d93772bd058d1268f4c28ea421a603d66f7979479048c429292fac7b2e" dependencies = [ "arrow-array", "arrow-buffer", @@ -403,8 +385,6 @@ dependencies = [ [[package]] name = "arrow-json" version = "55.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9085342bbca0f75e8cb70513c0807cc7351f1fbf5cb98192a67d5e3044acb033" dependencies = [ "arrow-array", "arrow-buffer", @@ -425,8 +405,6 @@ dependencies = [ [[package]] name = "arrow-ord" version = "55.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ab2f1065a5cad7b9efa9e22ce5747ce826aa3855766755d4904535123ef431e7" dependencies = [ "arrow-array", "arrow-buffer", @@ -438,8 +416,6 @@ dependencies = [ [[package]] name = "arrow-row" version = "55.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3703a0e3e92d23c3f756df73d2dc9476873f873a76ae63ef9d3de17fda83b2d8" dependencies = [ "arrow-array", "arrow-buffer", @@ -451,8 +427,6 @@ dependencies = [ [[package]] name = "arrow-schema" version = "55.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "73a47aa0c771b5381de2b7f16998d351a6f4eb839f1e13d48353e17e873d969b" dependencies = [ "bitflags 2.9.1", "serde", @@ -462,8 +436,6 @@ dependencies = [ [[package]] name = "arrow-select" version = "55.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "24b7b85575702b23b85272b01bc1c25a01c9b9852305e5d0078c79ba25d995d4" dependencies = [ "ahash 0.8.12", "arrow-array", @@ -476,8 +448,6 @@ dependencies = [ [[package]] name = "arrow-string" version = "55.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9260fddf1cdf2799ace2b4c2fc0356a9789fa7551e0953e35435536fecefebbd" dependencies = [ "arrow-array", "arrow-buffer", @@ -4421,8 +4391,6 @@ dependencies = [ [[package]] name = "parquet" version = "55.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "be7b2d778f6b841d37083ebdf32e33a524acde1266b5884a8ca29bf00dfa1231" dependencies = [ "ahash 0.8.12", "arrow-array", diff --git a/Cargo.toml b/Cargo.toml index 93d64cb6b6ef..30333b971c7c 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -88,19 +88,19 @@ ahash = { version = "0.8", default-features = false, features = [ "runtime-rng", ] } apache-avro = { version = "0.17", default-features = false } -arrow = { version = "55.1.0", features = [ +arrow = { path = "/Users/zhuqi/arrow-rs/arrow", features = [ "prettyprint", "chrono-tz", ] } -arrow-buffer = { version = "55.0.0", default-features = false } -arrow-flight = { version = "55.1.0", features = [ +arrow-buffer = { path = "/Users/zhuqi/arrow-rs/arrow-buffer", default-features = false } +arrow-flight = { path = "/Users/zhuqi/arrow-rs/arrow-flight", features = [ "flight-sql-experimental", ] } -arrow-ipc = { version = "55.0.0", default-features = false, features = [ +arrow-ipc = { path = "/Users/zhuqi/arrow-rs/arrow-ipc", default-features = false, features = [ "lz4", ] } -arrow-ord = { version = "55.0.0", default-features = false } -arrow-schema = { version = "55.0.0", default-features = false } +arrow-ord = { path = "/Users/zhuqi/arrow-rs/arrow-ord", default-features = false } +arrow-schema = { path = "/Users/zhuqi/arrow-rs/arrow-schema", default-features = false } async-trait = "0.1.88" bigdecimal = "0.4.8" bytes = "1.10" @@ -152,7 +152,7 @@ itertools = "0.14" log = "^0.4" object_store = { version = "0.12.0", default-features = false } parking_lot = "0.12" -parquet = { version = "55.1.0", default-features = false, features = [ +parquet = { path = "/Users/zhuqi/arrow-rs/parquet", default-features = false, features = [ "arrow", "async", "object_store", diff --git a/datafusion-examples/examples/embedding_parquet_indexes.rs b/datafusion-examples/examples/embedding_parquet_indexes.rs index f8e07d819953..70bea93157f4 100644 --- a/datafusion-examples/examples/embedding_parquet_indexes.rs +++ b/datafusion-examples/examples/embedding_parquet_indexes.rs @@ -373,3 +373,24 @@ async fn main() -> Result<()> { Ok(()) } + + +// ┌──────────────────────┐ +// │┌───────────────────┐│ +// ││ DataPage ││ ← 实际数据存储 +// │└───────────────────┘│ +// │┌───────────────────┐│ +// ││ DataPage ││ +// │└───────────────────┘│ +// │ ... │ +// │┌───────────────────┐│ +// ││ ColumnIndex ││ ← 列索引 +// │└───────────────────┘│ +// │┌───────────────────┐│ +// ││ OffsetIndex ││ ← 偏移索引 +// │└───────────────────┘│ +// │╔═══════════════════╗│ +// │║ Parquet Footer ║│ ← 页脚元数据 +// │╚═══════════════════╝│ +// │ PAR1 │ ← 文件结束标记 +// └──────────────────────┘ \ No newline at end of file From fbeecfe093bd9ca8ed9873bc0a117203ed39e471 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Sun, 15 Jun 2025 22:25:08 +0800 Subject: [PATCH 12/25] polish --- .../examples/embedding_parquet_indexes.rs | 80 +++++-------------- 1 file changed, 22 insertions(+), 58 deletions(-) diff --git a/datafusion-examples/examples/embedding_parquet_indexes.rs b/datafusion-examples/examples/embedding_parquet_indexes.rs index 70bea93157f4..9462ed09223c 100644 --- a/datafusion-examples/examples/embedding_parquet_indexes.rs +++ b/datafusion-examples/examples/embedding_parquet_indexes.rs @@ -28,8 +28,6 @@ use arrow::array::{ArrayRef, StringArray}; use arrow::record_batch::RecordBatch; use arrow_schema::{DataType, Field, Schema, SchemaRef}; use async_trait::async_trait; -use base64::engine::general_purpose; -use base64::Engine; use datafusion::catalog::{Session, TableProvider}; use datafusion::common::{HashMap, HashSet, Result}; use datafusion::datasource::listing::PartitionedFile; @@ -38,25 +36,26 @@ use datafusion::datasource::physical_plan::{FileScanConfigBuilder, ParquetSource use datafusion::datasource::TableType; use datafusion::execution::object_store::ObjectStoreUrl; use datafusion::logical_expr::{Operator, TableProviderFilterPushDown}; -use datafusion::parquet::arrow::{ArrowSchemaConverter, ArrowWriter}; +use datafusion::parquet::arrow::ArrowSchemaConverter; +use datafusion::parquet::data_type::{ByteArray, ByteArrayType}; +use datafusion::parquet::errors::ParquetError; use datafusion::parquet::file::metadata::KeyValue; use datafusion::parquet::file::properties::WriterProperties; use datafusion::parquet::file::reader::{FileReader, SerializedFileReader}; +use datafusion::parquet::file::writer::SerializedFileWriter; use datafusion::physical_plan::ExecutionPlan; use datafusion::prelude::*; use datafusion::scalar::ScalarValue; +use futures::AsyncWriteExt; use std::fs::{create_dir_all, read_dir, File}; use std::io::{Read, Seek, SeekFrom, Write}; use std::path::{Path, PathBuf}; use std::sync::Arc; -use futures::AsyncWriteExt; use tempfile::TempDir; -use datafusion::parquet::column::writer::ColumnWriter; -use datafusion::parquet::data_type::{ByteArray, ByteArrayType}; -use datafusion::parquet::errors::ParquetError; -use datafusion::parquet::file::writer::{SerializedColumnWriter, SerializedFileWriter}; -use datafusion_proto::protobuf::FromProtoError::DataFusionError; +/// We should disable page index support in the Parquet reader +/// when we ennable this feature, since we are using a custom index. +/// /// Example creating parquet file that /// contains specialized indexes that /// are ignored by other readers @@ -126,7 +125,6 @@ pub struct IndexedParquetWriter { } impl IndexedParquetWriter { - /// 构造:传入已经创建好的文件、schema 和普通的 WriterProperties(不要在 metadata 里放 Base64) pub fn try_new( sink: W, schema: Arc, @@ -134,23 +132,20 @@ impl IndexedParquetWriter { ) -> Result { let schema_desc = ArrowSchemaConverter::new().convert(schema.as_ref())?; let props_ptr = Arc::new(props); - let writer = SerializedFileWriter::new(sink, schema_desc.root_schema_ptr(), props_ptr)?; + let writer = + SerializedFileWriter::new(sink, schema_desc.root_schema_ptr(), props_ptr)?; Ok(Self { writer }) } } +const INDEX_MAGIC: &[u8] = b"IDX1"; -const PARQUET_MAGIC: &[u8] = b"PAR1"; -const INDEX_MAGIC: &[u8] = b"IDX1"; // 自定义索引魔术字 - -// 修改 write_file_with_index 函数 fn write_file_with_index(path: &Path, values: &[&str]) -> Result<()> { let field = Field::new("category", DataType::Utf8, false); let schema = Arc::new(Schema::new(vec![field.clone()])); let arr: ArrayRef = Arc::new(StringArray::from(values.to_vec())); let batch = RecordBatch::try_new(schema.clone(), vec![arr])?; - // 计算不同值索引 let distinct: HashSet<_> = values.iter().copied().collect(); let serialized = distinct.into_iter().collect::>().join("\n"); let index_bytes = serialized.into_bytes(); @@ -160,7 +155,6 @@ fn write_file_with_index(path: &Path, values: &[&str]) -> Result<()> { let mut writer = IndexedParquetWriter::try_new(file, schema.clone(), props)?; - // 写入数据 { let mut rg_writer = writer.writer.next_row_group()?; let mut ser_col_writer = rg_writer @@ -183,18 +177,14 @@ fn write_file_with_index(path: &Path, values: &[&str]) -> Result<()> { rg_writer.close()?; } - // 获取当前写入位置(索引开始位置) let offset = writer.writer.inner().seek(SeekFrom::Current(0))?; - // 写入索引魔术字和长度 let index_len = index_bytes.len() as u64; writer.writer.inner().write_all(b"IDX1")?; // 4字节魔术字 writer.writer.inner().write_all(&index_len.to_le_bytes())?; // 8字节长度 - // 写入索引数据 writer.writer.inner().write_all(&index_bytes)?; - // 记录索引元数据 writer.writer.append_key_value_metadata(KeyValue::new( "distinct_index_offset".to_string(), offset.to_string(), @@ -204,35 +194,35 @@ fn write_file_with_index(path: &Path, values: &[&str]) -> Result<()> { index_bytes.len().to_string(), )); - // 关闭写入器(这会写入页脚) writer.writer.close()?; println!("Finished writing file to {}", path.display()); Ok(()) } - -// 修改后的读取函数 fn read_distinct_index(path: &Path) -> Result, ParquetError> { let mut file = File::open(path)?; - let file_size = file.metadata()?.len(); - println!("Reading index from {} (size: {})", path.display(), file_size); + println!( + "Reading index from {} (size: {})", + path.display(), + file_size + ); - // 1. 读取元数据获取索引位置 let reader = SerializedFileReader::new(file.try_clone()?)?; let meta = reader.metadata().file_metadata(); - - let offset = meta.key_value_metadata() + let offset = meta + .key_value_metadata() .and_then(|kvs| kvs.iter().find(|kv| kv.key == "distinct_index_offset")) .and_then(|kv| kv.value.as_ref()) .ok_or_else(|| ParquetError::General("Missing index offset".into()))? .parse::() .map_err(|e| ParquetError::General(e.to_string()))?; - let length = meta.key_value_metadata() + let length = meta + .key_value_metadata() .and_then(|kvs| kvs.iter().find(|kv| kv.key == "distinct_index_length")) .and_then(|kv| kv.value.as_ref()) .ok_or_else(|| ParquetError::General("Missing index length".into()))? @@ -241,17 +231,14 @@ fn read_distinct_index(path: &Path) -> Result, ParquetError> { println!("Reading index at offset: {}, length: {}", offset, length); - // 2. 定位并读取索引 file.seek(SeekFrom::Start(offset))?; - // 验证魔术字 let mut magic_buf = [0u8; 4]; file.read_exact(&mut magic_buf)?; if &magic_buf != INDEX_MAGIC { return Err(ParquetError::General("Invalid index magic".into())); } - // 读取索引长度 let mut len_buf = [0u8; 8]; file.read_exact(&mut len_buf)?; let stored_len = u64::from_le_bytes(len_buf) as usize; @@ -260,13 +247,11 @@ fn read_distinct_index(path: &Path) -> Result, ParquetError> { return Err(ParquetError::General("Index length mismatch".into())); } - // 读取索引数据 let mut index_buf = vec![0u8; length]; file.read_exact(&mut index_buf)?; - // 3. 解码索引 - let s = String::from_utf8(index_buf) - .map_err(|e| ParquetError::General(e.to_string()))?; + let s = + String::from_utf8(index_buf).map_err(|e| ParquetError::General(e.to_string()))?; Ok(s.lines().map(|s| s.to_string()).collect()) } @@ -373,24 +358,3 @@ async fn main() -> Result<()> { Ok(()) } - - -// ┌──────────────────────┐ -// │┌───────────────────┐│ -// ││ DataPage ││ ← 实际数据存储 -// │└───────────────────┘│ -// │┌───────────────────┐│ -// ││ DataPage ││ -// │└───────────────────┘│ -// │ ... │ -// │┌───────────────────┐│ -// ││ ColumnIndex ││ ← 列索引 -// │└───────────────────┘│ -// │┌───────────────────┐│ -// ││ OffsetIndex ││ ← 偏移索引 -// │└───────────────────┘│ -// │╔═══════════════════╗│ -// │║ Parquet Footer ║│ ← 页脚元数据 -// │╚═══════════════════╝│ -// │ PAR1 │ ← 文件结束标记 -// └──────────────────────┘ \ No newline at end of file From 88fc6a6da8f73aedd8459bf859280340c5e87184 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Sun, 15 Jun 2025 22:27:47 +0800 Subject: [PATCH 13/25] debug --- datafusion-examples/examples/embedding_parquet_indexes.rs | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/datafusion-examples/examples/embedding_parquet_indexes.rs b/datafusion-examples/examples/embedding_parquet_indexes.rs index 9462ed09223c..66dd711b9b8c 100644 --- a/datafusion-examples/examples/embedding_parquet_indexes.rs +++ b/datafusion-examples/examples/embedding_parquet_indexes.rs @@ -109,7 +109,6 @@ impl DistinctIndexTable { } let file_name = path.file_name().unwrap().to_string_lossy().to_string(); - // 直接用工具函数读取该文件的 distinct index let distinct_set = read_distinct_index(&path)?; println!("Read distinct index for {}: {:?}", file_name, distinct_set); @@ -180,8 +179,8 @@ fn write_file_with_index(path: &Path, values: &[&str]) -> Result<()> { let offset = writer.writer.inner().seek(SeekFrom::Current(0))?; let index_len = index_bytes.len() as u64; - writer.writer.inner().write_all(b"IDX1")?; // 4字节魔术字 - writer.writer.inner().write_all(&index_len.to_le_bytes())?; // 8字节长度 + writer.writer.inner().write_all(b"IDX1")?; + writer.writer.inner().write_all(&index_len.to_le_bytes())?; writer.writer.inner().write_all(&index_bytes)?; From 32abcb96ba57c1cfdf14f69f2cf802965381b3cf Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Wed, 18 Jun 2025 11:41:23 +0800 Subject: [PATCH 14/25] Using low level API to support --- .../examples/embedding_parquet_indexes.rs | 34 +++++++++++-------- 1 file changed, 19 insertions(+), 15 deletions(-) diff --git a/datafusion-examples/examples/embedding_parquet_indexes.rs b/datafusion-examples/examples/embedding_parquet_indexes.rs index 66dd711b9b8c..09a9cf8004d5 100644 --- a/datafusion-examples/examples/embedding_parquet_indexes.rs +++ b/datafusion-examples/examples/embedding_parquet_indexes.rs @@ -46,7 +46,6 @@ use datafusion::parquet::file::writer::SerializedFileWriter; use datafusion::physical_plan::ExecutionPlan; use datafusion::prelude::*; use datafusion::scalar::ScalarValue; -use futures::AsyncWriteExt; use std::fs::{create_dir_all, read_dir, File}; use std::io::{Read, Seek, SeekFrom, Write}; use std::path::{Path, PathBuf}; @@ -54,9 +53,9 @@ use std::sync::Arc; use tempfile::TempDir; /// We should disable page index support in the Parquet reader -/// when we ennable this feature, since we are using a custom index. +/// when we enable this feature, since we are using a custom index. /// -/// Example creating parquet file that +/// Example creating the parquet file that /// contains specialized indexes that /// are ignored by other readers /// @@ -111,7 +110,7 @@ impl DistinctIndexTable { let distinct_set = read_distinct_index(&path)?; - println!("Read distinct index for {}: {:?}", file_name, distinct_set); + println!("Read distinct index for {file_name}: {file_name:?}"); index.insert(file_name, distinct_set); } @@ -137,6 +136,7 @@ impl IndexedParquetWriter { } } +/// Magic bytes to identify our custom index format const INDEX_MAGIC: &[u8] = b"IDX1"; fn write_file_with_index(path: &Path, values: &[&str]) -> Result<()> { @@ -154,6 +154,7 @@ fn write_file_with_index(path: &Path, values: &[&str]) -> Result<()> { let mut writer = IndexedParquetWriter::try_new(file, schema.clone(), props)?; + // Write data to the Parquet file, we only write one column since our schema has one field { let mut rg_writer = writer.writer.next_row_group()?; let mut ser_col_writer = rg_writer @@ -170,20 +171,23 @@ fn write_file_with_index(path: &Path, values: &[&str]) -> Result<()> { .map(|opt| ByteArray::from(opt.unwrap())) .collect(); - println!("Writing values: {:?}", values_bytes); + println!("Writing values: {values_bytes:?}"); col_writer.write_batch(&values_bytes, None, None)?; ser_col_writer.close()?; rg_writer.close()?; } - let offset = writer.writer.inner().seek(SeekFrom::Current(0))?; - + let offset = writer.writer.inner().stream_position()?; let index_len = index_bytes.len() as u64; + + // Write the index magic and length to the file writer.writer.inner().write_all(b"IDX1")?; writer.writer.inner().write_all(&index_len.to_le_bytes())?; + // Write the index bytes writer.writer.inner().write_all(&index_bytes)?; - + + // Append metadata about the index to the Parquet file footer writer.writer.append_key_value_metadata(KeyValue::new( "distinct_index_offset".to_string(), offset.to_string(), @@ -228,13 +232,13 @@ fn read_distinct_index(path: &Path) -> Result, ParquetError> { .parse::() .map_err(|e| ParquetError::General(e.to_string()))?; - println!("Reading index at offset: {}, length: {}", offset, length); + println!("Reading index at offset: {offset}, length: {length}"); file.seek(SeekFrom::Start(offset))?; let mut magic_buf = [0u8; 4]; file.read_exact(&mut magic_buf)?; - if &magic_buf != INDEX_MAGIC { + if magic_buf != INDEX_MAGIC { return Err(ParquetError::General("Invalid index magic".into())); } @@ -307,7 +311,10 @@ impl TableProvider for DistinctIndexTable { // Build ParquetSource for kept files let url = ObjectStoreUrl::parse("file://")?; - let source = Arc::new(ParquetSource::default()); + + // Note: we disable page index support here since we are using a custom index, it has conflicts when testing. + // TODO: Remove this when we have a better solution for custom indexes with page index support. + let source = Arc::new(ParquetSource::default().with_enable_page_index(false)); let mut builder = FileScanConfigBuilder::new(url, self.schema.clone(), source); for file in keep { let path = self.dir.join(&file); @@ -343,14 +350,11 @@ async fn main() -> Result<()> { let field = Field::new("category", DataType::Utf8, false); let schema_ref = Arc::new(Schema::new(vec![field])); let provider = Arc::new(DistinctIndexTable::try_new(dir, schema_ref.clone())?); + let ctx = SessionContext::new(); ctx.register_table("t", provider)?; - // 3. Run a query: only files containing 'foo' get scanned - let df = ctx.sql("SELECT * FROM t").await?; - df.show().await?; - // 3. Run a query: only files containing 'foo' get scanned let df = ctx.sql("SELECT * FROM t WHERE category = 'foo'").await?; df.show().await?; From 284510ceb6442f3b9f14c009c6956803f5ab65a6 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Wed, 18 Jun 2025 11:44:03 +0800 Subject: [PATCH 15/25] polish --- Cargo.lock | 186 ++++++++++-------- Cargo.toml | 14 +- .../examples/embedding_parquet_indexes.rs | 6 +- 3 files changed, 119 insertions(+), 87 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 349b4e0ae136..c6af946e6e05 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -247,6 +247,8 @@ checksum = "7c02d123df017efcdfbd739ef81735b36c5ba83ec3c59c80a9d7ecc718f92e50" [[package]] name = "arrow" version = "55.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b1bb018b6960c87fd9d025009820406f74e83281185a8bdcb44880d2aa5c9a87" dependencies = [ "arrow-arith", "arrow-array", @@ -269,6 +271,8 @@ dependencies = [ [[package]] name = "arrow-arith" version = "55.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "44de76b51473aa888ecd6ad93ceb262fb8d40d1f1154a4df2f069b3590aa7575" dependencies = [ "arrow-array", "arrow-buffer", @@ -281,6 +285,8 @@ dependencies = [ [[package]] name = "arrow-array" version = "55.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "29ed77e22744475a9a53d00026cf8e166fe73cf42d89c4c4ae63607ee1cfcc3f" dependencies = [ "ahash 0.8.12", "arrow-buffer", @@ -296,6 +302,8 @@ dependencies = [ [[package]] name = "arrow-buffer" version = "55.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b0391c96eb58bf7389171d1e103112d3fc3e5625ca6b372d606f2688f1ea4cce" dependencies = [ "bytes", "half", @@ -305,6 +313,8 @@ dependencies = [ [[package]] name = "arrow-cast" version = "55.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f39e1d774ece9292697fcbe06b5584401b26bd34be1bec25c33edae65c2420ff" dependencies = [ "arrow-array", "arrow-buffer", @@ -324,6 +334,8 @@ dependencies = [ [[package]] name = "arrow-csv" version = "55.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9055c972a07bf12c2a827debfd34f88d3b93da1941d36e1d9fee85eebe38a12a" dependencies = [ "arrow-array", "arrow-cast", @@ -338,6 +350,8 @@ dependencies = [ [[package]] name = "arrow-data" version = "55.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cf75ac27a08c7f48b88e5c923f267e980f27070147ab74615ad85b5c5f90473d" dependencies = [ "arrow-buffer", "arrow-schema", @@ -348,6 +362,8 @@ dependencies = [ [[package]] name = "arrow-flight" version = "55.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "91efc67a4f5a438833dd76ef674745c80f6f6b9a428a3b440cbfbf74e32867e6" dependencies = [ "arrow-arith", "arrow-array", @@ -373,6 +389,8 @@ dependencies = [ [[package]] name = "arrow-ipc" version = "55.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a222f0d93772bd058d1268f4c28ea421a603d66f7979479048c429292fac7b2e" dependencies = [ "arrow-array", "arrow-buffer", @@ -385,6 +403,8 @@ dependencies = [ [[package]] name = "arrow-json" version = "55.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9085342bbca0f75e8cb70513c0807cc7351f1fbf5cb98192a67d5e3044acb033" dependencies = [ "arrow-array", "arrow-buffer", @@ -405,6 +425,8 @@ dependencies = [ [[package]] name = "arrow-ord" version = "55.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ab2f1065a5cad7b9efa9e22ce5747ce826aa3855766755d4904535123ef431e7" dependencies = [ "arrow-array", "arrow-buffer", @@ -416,6 +438,8 @@ dependencies = [ [[package]] name = "arrow-row" version = "55.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3703a0e3e92d23c3f756df73d2dc9476873f873a76ae63ef9d3de17fda83b2d8" dependencies = [ "arrow-array", "arrow-buffer", @@ -427,6 +451,8 @@ dependencies = [ [[package]] name = "arrow-schema" version = "55.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "73a47aa0c771b5381de2b7f16998d351a6f4eb839f1e13d48353e17e873d969b" dependencies = [ "bitflags 2.9.1", "serde", @@ -436,6 +462,8 @@ dependencies = [ [[package]] name = "arrow-select" version = "55.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "24b7b85575702b23b85272b01bc1c25a01c9b9852305e5d0078c79ba25d995d4" dependencies = [ "ahash 0.8.12", "arrow-array", @@ -448,6 +476,8 @@ dependencies = [ [[package]] name = "arrow-string" version = "55.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9260fddf1cdf2799ace2b4c2fc0356a9789fa7551e0953e35435536fecefebbd" dependencies = [ "arrow-array", "arrow-buffer", @@ -522,7 +552,7 @@ checksum = "3b43422f69d8ff38f95f1b2bb76517c91589a924d1559a0e935d7c8ce0274c11" dependencies = [ "proc-macro2", "quote", - "syn 2.0.102", + "syn 2.0.103", ] [[package]] @@ -544,7 +574,7 @@ checksum = "c7c24de15d275a1ecfd47a380fb4d5ec9bfe0933f309ed5e705b775596a3574d" dependencies = [ "proc-macro2", "quote", - "syn 2.0.102", + "syn 2.0.103", ] [[package]] @@ -555,7 +585,7 @@ checksum = "e539d3fca749fcee5236ab05e93a52867dd549cc157c8cb7f99595f3cedffdb5" dependencies = [ "proc-macro2", "quote", - "syn 2.0.102", + "syn 2.0.103", ] [[package]] @@ -581,9 +611,9 @@ checksum = "ace50bade8e6234aa140d9a2f552bbee1db4d353f69b8217bc503490fc1a9f26" [[package]] name = "aws-config" -version = "1.6.3" +version = "1.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "02a18fd934af6ae7ca52410d4548b98eb895aab0f1ea417d168d85db1434a141" +checksum = "455e9fb7743c6f6267eb2830ccc08686fbb3d13c9a689369562fd4d4ef9ea462" dependencies = [ "aws-credential-types", "aws-runtime", @@ -646,9 +676,9 @@ dependencies = [ [[package]] name = "aws-runtime" -version = "1.5.7" +version = "1.5.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6c4063282c69991e57faab9e5cb21ae557e59f5b0fb285c196335243df8dc25c" +checksum = "4f6c68419d8ba16d9a7463671593c54f81ba58cab466e9b759418da606dcc2e2" dependencies = [ "aws-credential-types", "aws-sigv4", @@ -670,9 +700,9 @@ dependencies = [ [[package]] name = "aws-sdk-sso" -version = "1.70.0" +version = "1.73.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "83447efb7179d8e2ad2afb15ceb9c113debbc2ecdf109150e338e2e28b86190b" +checksum = "b2ac1674cba7872061a29baaf02209fefe499ff034dfd91bd4cc59e4d7741489" dependencies = [ "aws-credential-types", "aws-runtime", @@ -692,9 +722,9 @@ dependencies = [ [[package]] name = "aws-sdk-ssooidc" -version = "1.71.0" +version = "1.74.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c5f9bfbbda5e2b9fe330de098f14558ee8b38346408efe9f2e9cee82dc1636a4" +checksum = "3a6a22f077f5fd3e3c0270d4e1a110346cddf6769e9433eb9e6daceb4ca3b149" dependencies = [ "aws-credential-types", "aws-runtime", @@ -714,9 +744,9 @@ dependencies = [ [[package]] name = "aws-sdk-sts" -version = "1.71.0" +version = "1.74.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e17b984a66491ec08b4f4097af8911251db79296b3e4a763060b45805746264f" +checksum = "19d440e1d368759bd10df0dbdddbfff6473d7cd73e9d9ef2363dc9995ac2d711" dependencies = [ "aws-credential-types", "aws-runtime", @@ -737,9 +767,9 @@ dependencies = [ [[package]] name = "aws-sigv4" -version = "1.3.2" +version = "1.3.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3734aecf9ff79aa401a6ca099d076535ab465ff76b46440cf567c8e70b65dc13" +checksum = "ddfb9021f581b71870a17eac25b52335b82211cdc092e02b6876b2bcefa61666" dependencies = [ "aws-credential-types", "aws-smithy-http", @@ -813,9 +843,9 @@ dependencies = [ [[package]] name = "aws-smithy-json" -version = "0.61.3" +version = "0.61.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "92144e45819cae7dc62af23eac5a038a58aa544432d2102609654376a900bd07" +checksum = "a16e040799d29c17412943bdbf488fd75db04112d0c0d4b9290bacf5ae0014b9" dependencies = [ "aws-smithy-types", ] @@ -865,9 +895,9 @@ dependencies = [ [[package]] name = "aws-smithy-runtime-api" -version = "1.8.0" +version = "1.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a1e5d9e3a80a18afa109391fb5ad09c3daf887b516c6fd805a157c6ea7994a57" +checksum = "bd8531b6d8882fd8f48f82a9754e682e29dd44cff27154af51fa3eb730f59efb" dependencies = [ "aws-smithy-async", "aws-smithy-types", @@ -882,9 +912,9 @@ dependencies = [ [[package]] name = "aws-smithy-types" -version = "1.3.1" +version = "1.3.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "40076bd09fadbc12d5e026ae080d0930defa606856186e31d83ccc6a255eeaf3" +checksum = "d498595448e43de7f4296b7b7a18a8a02c61ec9349128c80a368f7c3b4ab11a8" dependencies = [ "base64-simd", "bytes", @@ -905,9 +935,9 @@ dependencies = [ [[package]] name = "aws-smithy-xml" -version = "0.60.9" +version = "0.60.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ab0b0166827aa700d3dc519f72f8b3a91c35d0b8d042dc5d643a91e6f80648fc" +checksum = "3db87b96cb1b16c024980f133968d52882ca0daaee3a086c6decc500f6c99728" dependencies = [ "xmlparser", ] @@ -1043,7 +1073,7 @@ dependencies = [ "regex", "rustc-hash 1.1.0", "shlex", - "syn 2.0.102", + "syn 2.0.103", "which", ] @@ -1172,7 +1202,7 @@ dependencies = [ "proc-macro-crate", "proc-macro2", "quote", - "syn 2.0.102", + "syn 2.0.103", ] [[package]] @@ -1440,7 +1470,7 @@ dependencies = [ "heck 0.5.0", "proc-macro2", "quote", - "syn 2.0.102", + "syn 2.0.103", ] [[package]] @@ -1742,7 +1772,7 @@ dependencies = [ "proc-macro2", "quote", "strsim", - "syn 2.0.102", + "syn 2.0.103", ] [[package]] @@ -1753,7 +1783,7 @@ checksum = "fc34b93ccb385b40dc71c6fceac4b2ad23662c7eeb248cf10d529b7e055b6ead" dependencies = [ "darling_core", "quote", - "syn 2.0.102", + "syn 2.0.103", ] [[package]] @@ -2353,7 +2383,7 @@ version = "48.0.0" dependencies = [ "datafusion-expr", "quote", - "syn 2.0.102", + "syn 2.0.103", ] [[package]] @@ -2700,7 +2730,7 @@ checksum = "97369cbbc041bc366949bc74d34658d6cda5621039731c6310521892a3a20ae0" dependencies = [ "proc-macro2", "quote", - "syn 2.0.102", + "syn 2.0.103", ] [[package]] @@ -2756,7 +2786,7 @@ dependencies = [ "enum-ordinalize", "proc-macro2", "quote", - "syn 2.0.102", + "syn 2.0.103", ] [[package]] @@ -2794,7 +2824,7 @@ checksum = "0d28318a75d4aead5c4db25382e8ef717932d0346600cacae6357eb5941bc5ff" dependencies = [ "proc-macro2", "quote", - "syn 2.0.102", + "syn 2.0.103", ] [[package]] @@ -3058,7 +3088,7 @@ checksum = "162ee34ebcb7c64a8abebc059ce0fee27c2262618d7b60ed8faf72fef13c3650" dependencies = [ "proc-macro2", "quote", - "syn 2.0.102", + "syn 2.0.103", ] [[package]] @@ -3761,7 +3791,7 @@ checksum = "6c6e1db7ed32c6c71b759497fae34bf7933636f75a251b9e736555da426f6442" dependencies = [ "proc-macro2", "quote", - "syn 2.0.102", + "syn 2.0.103", ] [[package]] @@ -4391,6 +4421,8 @@ dependencies = [ [[package]] name = "parquet" version = "55.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "be7b2d778f6b841d37083ebdf32e33a524acde1266b5884a8ca29bf00dfa1231" dependencies = [ "ahash 0.8.12", "arrow-array", @@ -4444,7 +4476,7 @@ dependencies = [ "regex", "regex-syntax", "structmeta", - "syn 2.0.102", + "syn 2.0.103", ] [[package]] @@ -4582,7 +4614,7 @@ checksum = "6e918e4ff8c4549eb882f14b3a4bc8c8bc93de829416eacf579f1207a8fbf861" dependencies = [ "proc-macro2", "quote", - "syn 2.0.102", + "syn 2.0.103", ] [[package]] @@ -4655,7 +4687,7 @@ dependencies = [ "heck 0.5.0", "proc-macro2", "quote", - "syn 2.0.102", + "syn 2.0.103", ] [[package]] @@ -4750,7 +4782,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "664ec5419c51e34154eec046ebcba56312d5a2fc3b09a06da188e1ad21afadf6" dependencies = [ "proc-macro2", - "syn 2.0.102", + "syn 2.0.103", ] [[package]] @@ -4821,7 +4853,7 @@ dependencies = [ "prost", "prost-types", "regex", - "syn 2.0.102", + "syn 2.0.103", "tempfile", ] @@ -4835,7 +4867,7 @@ dependencies = [ "itertools 0.14.0", "proc-macro2", "quote", - "syn 2.0.102", + "syn 2.0.103", ] [[package]] @@ -4932,7 +4964,7 @@ dependencies = [ "proc-macro2", "pyo3-macros-backend", "quote", - "syn 2.0.102", + "syn 2.0.103", ] [[package]] @@ -4945,7 +4977,7 @@ dependencies = [ "proc-macro2", "pyo3-build-config", "quote", - "syn 2.0.102", + "syn 2.0.103", ] [[package]] @@ -5156,7 +5188,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "76009fbe0614077fc1a2ce255e3a1881a2e3a3527097d5dc6d8212c585e7e38b" dependencies = [ "quote", - "syn 2.0.102", + "syn 2.0.103", ] [[package]] @@ -5378,7 +5410,7 @@ dependencies = [ "regex", "relative-path", "rustc_version", - "syn 2.0.102", + "syn 2.0.103", "unicode-ident", ] @@ -5390,14 +5422,14 @@ checksum = "b3a8fb4672e840a587a66fc577a5491375df51ddb88f2a2c2a792598c326fe14" dependencies = [ "quote", "rand 0.8.5", - "syn 2.0.102", + "syn 2.0.103", ] [[package]] name = "rust_decimal" -version = "1.37.1" +version = "1.37.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "faa7de2ba56ac291bd90c6b9bece784a52ae1411f9506544b3eae36dd2356d50" +checksum = "b203a6425500a03e0919c42d3c47caca51e79f1132046626d2c8871c5092035d" dependencies = [ "arrayvec", "borsh", @@ -5594,7 +5626,7 @@ dependencies = [ "proc-macro2", "quote", "serde_derive_internals", - "syn 2.0.102", + "syn 2.0.103", ] [[package]] @@ -5673,7 +5705,7 @@ checksum = "5b0276cf7f2c73365f7157c8123c21cd9a50fbbd844757af28ca1f5925fc2a00" dependencies = [ "proc-macro2", "quote", - "syn 2.0.102", + "syn 2.0.103", ] [[package]] @@ -5684,7 +5716,7 @@ checksum = "18d26a20a969b9e3fdf2fc2d9f21eda6c40e2de84c9408bb5d3b05d499aae711" dependencies = [ "proc-macro2", "quote", - "syn 2.0.102", + "syn 2.0.103", ] [[package]] @@ -5707,7 +5739,7 @@ checksum = "175ee3e80ae9982737ca543e96133087cbd9a485eecc3bc4de9c1a37b47ea59c" dependencies = [ "proc-macro2", "quote", - "syn 2.0.102", + "syn 2.0.103", ] [[package]] @@ -5719,7 +5751,7 @@ dependencies = [ "proc-macro2", "quote", "serde", - "syn 2.0.102", + "syn 2.0.103", ] [[package]] @@ -5761,7 +5793,7 @@ dependencies = [ "darling", "proc-macro2", "quote", - "syn 2.0.102", + "syn 2.0.103", ] [[package]] @@ -5923,7 +5955,7 @@ checksum = "da5fc6819faabb412da764b99d3b713bb55083c11e7e0c00144d386cd6a1939c" dependencies = [ "proc-macro2", "quote", - "syn 2.0.102", + "syn 2.0.103", ] [[package]] @@ -5977,7 +6009,7 @@ dependencies = [ "proc-macro2", "quote", "structmeta-derive", - "syn 2.0.102", + "syn 2.0.103", ] [[package]] @@ -5988,7 +6020,7 @@ checksum = "152a0b65a590ff6c3da95cabe2353ee04e6167c896b28e3b14478c2636c922fc" dependencies = [ "proc-macro2", "quote", - "syn 2.0.102", + "syn 2.0.103", ] [[package]] @@ -6031,7 +6063,7 @@ dependencies = [ "proc-macro2", "quote", "rustversion", - "syn 2.0.102", + "syn 2.0.103", ] [[package]] @@ -6065,7 +6097,7 @@ dependencies = [ "serde", "serde_json", "serde_yaml", - "syn 2.0.102", + "syn 2.0.103", "typify", "walkdir", ] @@ -6089,9 +6121,9 @@ dependencies = [ [[package]] name = "syn" -version = "2.0.102" +version = "2.0.103" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f6397daf94fa90f058bd0fd88429dd9e5738999cca8d701813c80723add80462" +checksum = "e4307e30089d6fd6aff212f2da3a1f9e32f3223b1f010fb09b7c95f90f3ca1e8" dependencies = [ "proc-macro2", "quote", @@ -6115,7 +6147,7 @@ checksum = "728a70f3dbaf5bab7f0c4b1ac8d7ae5ea60a4b5549c8a5914361c99147a709d2" dependencies = [ "proc-macro2", "quote", - "syn 2.0.102", + "syn 2.0.103", ] [[package]] @@ -6247,7 +6279,7 @@ checksum = "4fee6c4efc90059e10f81e6d42c60a18f76588c3d74cb83a0b242a2b6c7504c1" dependencies = [ "proc-macro2", "quote", - "syn 2.0.102", + "syn 2.0.103", ] [[package]] @@ -6258,7 +6290,7 @@ checksum = "7f7cf42b4507d8ea322120659672cf1b9dbb93f8f2d4ecfd6e51350ff5b17a1d" dependencies = [ "proc-macro2", "quote", - "syn 2.0.102", + "syn 2.0.103", ] [[package]] @@ -6383,7 +6415,7 @@ checksum = "6e06d43f1345a3bcd39f6a56dbb7dcab2ba47e68e8ac134855e7e2bdbaf8cab8" dependencies = [ "proc-macro2", "quote", - "syn 2.0.102", + "syn 2.0.103", ] [[package]] @@ -6574,7 +6606,7 @@ checksum = "395ae124c09f9e6918a2310af6038fba074bcf474ac352496d5910dd59a2226d" dependencies = [ "proc-macro2", "quote", - "syn 2.0.102", + "syn 2.0.103", ] [[package]] @@ -6672,7 +6704,7 @@ checksum = "f9534daa9fd3ed0bd911d462a37f172228077e7abf18c18a5f67199d959205f8" dependencies = [ "proc-macro2", "quote", - "syn 2.0.102", + "syn 2.0.103", ] [[package]] @@ -6706,7 +6738,7 @@ dependencies = [ "semver", "serde", "serde_json", - "syn 2.0.102", + "syn 2.0.103", "thiserror 2.0.12", "unicode-ident", ] @@ -6724,7 +6756,7 @@ dependencies = [ "serde", "serde_json", "serde_tokenstream", - "syn 2.0.102", + "syn 2.0.103", "typify-impl", ] @@ -6922,7 +6954,7 @@ dependencies = [ "log", "proc-macro2", "quote", - "syn 2.0.102", + "syn 2.0.103", "wasm-bindgen-shared", ] @@ -6957,7 +6989,7 @@ checksum = "8ae87ea40c9f689fc23f209965b6fb8a99ad69aeeb0231408be24920604395de" dependencies = [ "proc-macro2", "quote", - "syn 2.0.102", + "syn 2.0.103", "wasm-bindgen-backend", "wasm-bindgen-shared", ] @@ -6992,7 +7024,7 @@ checksum = "17d5042cc5fa009658f9a7333ef24291b1291a25b6382dd68862a7f3b969f69b" dependencies = [ "proc-macro2", "quote", - "syn 2.0.102", + "syn 2.0.103", ] [[package]] @@ -7136,7 +7168,7 @@ checksum = "a47fddd13af08290e67f4acabf4b459f647552718f683a7b415d290ac744a836" dependencies = [ "proc-macro2", "quote", - "syn 2.0.102", + "syn 2.0.103", ] [[package]] @@ -7147,7 +7179,7 @@ checksum = "bd9211b69f8dcdfa817bfd14bf1c97c9188afa36f4750130fcdf3f400eca9fa8" dependencies = [ "proc-macro2", "quote", - "syn 2.0.102", + "syn 2.0.103", ] [[package]] @@ -7437,7 +7469,7 @@ checksum = "38da3c9736e16c5d3c8c597a9aaa5d1fa565d0532ae05e27c24aa62fb32c0ab6" dependencies = [ "proc-macro2", "quote", - "syn 2.0.102", + "syn 2.0.103", "synstructure", ] @@ -7458,7 +7490,7 @@ checksum = "28a6e20d751156648aa063f3800b706ee209a32c0b4d9f24be3d980b01be55ef" dependencies = [ "proc-macro2", "quote", - "syn 2.0.102", + "syn 2.0.103", ] [[package]] @@ -7478,7 +7510,7 @@ checksum = "d71e5d6e06ab090c67b5e44993ec16b72dcbaabc526db883a360057678b48502" dependencies = [ "proc-macro2", "quote", - "syn 2.0.102", + "syn 2.0.103", "synstructure", ] @@ -7518,7 +7550,7 @@ checksum = "5b96237efa0c878c64bd89c436f661be4e46b2f3eff1ebb976f7ef2321d2f58f" dependencies = [ "proc-macro2", "quote", - "syn 2.0.102", + "syn 2.0.103", ] [[package]] diff --git a/Cargo.toml b/Cargo.toml index 30333b971c7c..93d64cb6b6ef 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -88,19 +88,19 @@ ahash = { version = "0.8", default-features = false, features = [ "runtime-rng", ] } apache-avro = { version = "0.17", default-features = false } -arrow = { path = "/Users/zhuqi/arrow-rs/arrow", features = [ +arrow = { version = "55.1.0", features = [ "prettyprint", "chrono-tz", ] } -arrow-buffer = { path = "/Users/zhuqi/arrow-rs/arrow-buffer", default-features = false } -arrow-flight = { path = "/Users/zhuqi/arrow-rs/arrow-flight", features = [ +arrow-buffer = { version = "55.0.0", default-features = false } +arrow-flight = { version = "55.1.0", features = [ "flight-sql-experimental", ] } -arrow-ipc = { path = "/Users/zhuqi/arrow-rs/arrow-ipc", default-features = false, features = [ +arrow-ipc = { version = "55.0.0", default-features = false, features = [ "lz4", ] } -arrow-ord = { path = "/Users/zhuqi/arrow-rs/arrow-ord", default-features = false } -arrow-schema = { path = "/Users/zhuqi/arrow-rs/arrow-schema", default-features = false } +arrow-ord = { version = "55.0.0", default-features = false } +arrow-schema = { version = "55.0.0", default-features = false } async-trait = "0.1.88" bigdecimal = "0.4.8" bytes = "1.10" @@ -152,7 +152,7 @@ itertools = "0.14" log = "^0.4" object_store = { version = "0.12.0", default-features = false } parking_lot = "0.12" -parquet = { path = "/Users/zhuqi/arrow-rs/parquet", default-features = false, features = [ +parquet = { version = "55.1.0", default-features = false, features = [ "arrow", "async", "object_store", diff --git a/datafusion-examples/examples/embedding_parquet_indexes.rs b/datafusion-examples/examples/embedding_parquet_indexes.rs index 09a9cf8004d5..d90b819d6b9e 100644 --- a/datafusion-examples/examples/embedding_parquet_indexes.rs +++ b/datafusion-examples/examples/embedding_parquet_indexes.rs @@ -179,14 +179,14 @@ fn write_file_with_index(path: &Path, values: &[&str]) -> Result<()> { let offset = writer.writer.inner().stream_position()?; let index_len = index_bytes.len() as u64; - + // Write the index magic and length to the file writer.writer.inner().write_all(b"IDX1")?; writer.writer.inner().write_all(&index_len.to_le_bytes())?; // Write the index bytes writer.writer.inner().write_all(&index_bytes)?; - + // Append metadata about the index to the Parquet file footer writer.writer.append_key_value_metadata(KeyValue::new( "distinct_index_offset".to_string(), @@ -311,7 +311,7 @@ impl TableProvider for DistinctIndexTable { // Build ParquetSource for kept files let url = ObjectStoreUrl::parse("file://")?; - + // Note: we disable page index support here since we are using a custom index, it has conflicts when testing. // TODO: Remove this when we have a better solution for custom indexes with page index support. let source = Arc::new(ParquetSource::default().with_enable_page_index(false)); From 0410bd8a3239cc4fd1bc974b1cadde7135e9132f Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Wed, 18 Jun 2025 11:44:54 +0800 Subject: [PATCH 16/25] fix --- Cargo.lock | 14 +++++++------- Cargo.toml | 2 +- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index c6af946e6e05..bc7cd8201e06 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2159,7 +2159,6 @@ dependencies = [ "arrow-flight", "arrow-schema", "async-trait", - "base64 0.22.1", "bytes", "dashmap", "datafusion", @@ -2466,6 +2465,7 @@ dependencies = [ "itertools 0.14.0", "log", "recursive", + "tokio", ] [[package]] @@ -3892,9 +3892,9 @@ dependencies = [ [[package]] name = "libc" -version = "0.2.172" +version = "0.2.173" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d750af042f7ef4f724306de029d18836c26c1765a54a6a3f094cbd23a7267ffa" +checksum = "d8cfeafaffdbc32176b64fb251369d52ea9f0a8fbc6f8759edffef7b525d64bb" [[package]] name = "libflate" @@ -3948,9 +3948,9 @@ checksum = "f9fbbcab51052fe104eb5e5d351cf728d30a5be1fe14d9be8a3b097481fb97de" [[package]] name = "libmimalloc-sys" -version = "0.1.42" +version = "0.1.43" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ec9d6fac27761dabcd4ee73571cdb06b7022dc99089acbe5435691edffaac0f4" +checksum = "bf88cd67e9de251c1781dbe2f641a1a3ad66eaae831b8a2c38fbdc5ddae16d4d" dependencies = [ "cc", "libc", @@ -4081,9 +4081,9 @@ dependencies = [ [[package]] name = "mimalloc" -version = "0.1.46" +version = "0.1.47" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "995942f432bbb4822a7e9c3faa87a695185b0d09273ba85f097b54f4e458f2af" +checksum = "b1791cbe101e95af5764f06f20f6760521f7158f69dbf9d6baf941ee1bf6bc40" dependencies = [ "libmimalloc-sys", ] diff --git a/Cargo.toml b/Cargo.toml index 93d64cb6b6ef..f2cd6f72c7e6 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -167,7 +167,7 @@ recursive = "0.1.1" regex = "1.8" rstest = "0.25.0" serde_json = "1" -sqlparser = { version = "0.55.0", features = ["visitor"] } +sqlparser = { version = "0.55.0", default-features = false, features = ["std", "visitor"] } tempfile = "3" tokio = { version = "1.45", features = ["macros", "rt", "sync"] } url = "2.5.4" From 12ce9c242403b102be8b8a764c56aef5be6b582c Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Wed, 18 Jun 2025 11:45:35 +0800 Subject: [PATCH 17/25] merge --- Cargo.lock | 1 + 1 file changed, 1 insertion(+) diff --git a/Cargo.lock b/Cargo.lock index bc7cd8201e06..691829cc0eb6 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2159,6 +2159,7 @@ dependencies = [ "arrow-flight", "arrow-schema", "async-trait", + "base64 0.22.1", "bytes", "dashmap", "datafusion", From 0c093acf0188f7d82e9296feced7a51d91735050 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Wed, 18 Jun 2025 11:50:23 +0800 Subject: [PATCH 18/25] fix --- Cargo.lock | 1 - datafusion-examples/Cargo.toml | 2 -- 2 files changed, 3 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 691829cc0eb6..bc7cd8201e06 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2159,7 +2159,6 @@ dependencies = [ "arrow-flight", "arrow-schema", "async-trait", - "base64 0.22.1", "bytes", "dashmap", "datafusion", diff --git a/datafusion-examples/Cargo.toml b/datafusion-examples/Cargo.toml index 4bb061353d1e..b31708a5c1cc 100644 --- a/datafusion-examples/Cargo.toml +++ b/datafusion-examples/Cargo.toml @@ -81,5 +81,3 @@ uuid = "1.17" [target.'cfg(not(target_os = "windows"))'.dev-dependencies] nix = { version = "0.30.1", features = ["fs"] } -[dependencies] -base64 = "0.22.1" From 9c758149d2b2b1dfc0eb33733f187d9516f43e66 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Thu, 19 Jun 2025 17:27:11 +0800 Subject: [PATCH 19/25] complte solution --- Cargo.lock | 49 ++++++------------- Cargo.toml | 14 +++--- .../examples/embedding_parquet_indexes.rs | 12 ++--- 3 files changed, 28 insertions(+), 47 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index bc7cd8201e06..9c29cb12eb09 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -247,8 +247,7 @@ checksum = "7c02d123df017efcdfbd739ef81735b36c5ba83ec3c59c80a9d7ecc718f92e50" [[package]] name = "arrow" version = "55.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b1bb018b6960c87fd9d025009820406f74e83281185a8bdcb44880d2aa5c9a87" +source = "git+https://github.com/zhuqi-lucas/arrow-rs.git?branch=add_pub_write_all_api#a291e4b593c43b0050e003aecb18bd4c54f7d30b" dependencies = [ "arrow-arith", "arrow-array", @@ -271,8 +270,7 @@ dependencies = [ [[package]] name = "arrow-arith" version = "55.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "44de76b51473aa888ecd6ad93ceb262fb8d40d1f1154a4df2f069b3590aa7575" +source = "git+https://github.com/zhuqi-lucas/arrow-rs.git?branch=add_pub_write_all_api#a291e4b593c43b0050e003aecb18bd4c54f7d30b" dependencies = [ "arrow-array", "arrow-buffer", @@ -285,8 +283,7 @@ dependencies = [ [[package]] name = "arrow-array" version = "55.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "29ed77e22744475a9a53d00026cf8e166fe73cf42d89c4c4ae63607ee1cfcc3f" +source = "git+https://github.com/zhuqi-lucas/arrow-rs.git?branch=add_pub_write_all_api#a291e4b593c43b0050e003aecb18bd4c54f7d30b" dependencies = [ "ahash 0.8.12", "arrow-buffer", @@ -302,8 +299,7 @@ dependencies = [ [[package]] name = "arrow-buffer" version = "55.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b0391c96eb58bf7389171d1e103112d3fc3e5625ca6b372d606f2688f1ea4cce" +source = "git+https://github.com/zhuqi-lucas/arrow-rs.git?branch=add_pub_write_all_api#a291e4b593c43b0050e003aecb18bd4c54f7d30b" dependencies = [ "bytes", "half", @@ -313,8 +309,7 @@ dependencies = [ [[package]] name = "arrow-cast" version = "55.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f39e1d774ece9292697fcbe06b5584401b26bd34be1bec25c33edae65c2420ff" +source = "git+https://github.com/zhuqi-lucas/arrow-rs.git?branch=add_pub_write_all_api#a291e4b593c43b0050e003aecb18bd4c54f7d30b" dependencies = [ "arrow-array", "arrow-buffer", @@ -334,8 +329,7 @@ dependencies = [ [[package]] name = "arrow-csv" version = "55.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9055c972a07bf12c2a827debfd34f88d3b93da1941d36e1d9fee85eebe38a12a" +source = "git+https://github.com/zhuqi-lucas/arrow-rs.git?branch=add_pub_write_all_api#a291e4b593c43b0050e003aecb18bd4c54f7d30b" dependencies = [ "arrow-array", "arrow-cast", @@ -343,15 +337,13 @@ dependencies = [ "chrono", "csv", "csv-core", - "lazy_static", "regex", ] [[package]] name = "arrow-data" version = "55.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cf75ac27a08c7f48b88e5c923f267e980f27070147ab74615ad85b5c5f90473d" +source = "git+https://github.com/zhuqi-lucas/arrow-rs.git?branch=add_pub_write_all_api#a291e4b593c43b0050e003aecb18bd4c54f7d30b" dependencies = [ "arrow-buffer", "arrow-schema", @@ -362,8 +354,7 @@ dependencies = [ [[package]] name = "arrow-flight" version = "55.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "91efc67a4f5a438833dd76ef674745c80f6f6b9a428a3b440cbfbf74e32867e6" +source = "git+https://github.com/zhuqi-lucas/arrow-rs.git?branch=add_pub_write_all_api#a291e4b593c43b0050e003aecb18bd4c54f7d30b" dependencies = [ "arrow-arith", "arrow-array", @@ -389,8 +380,7 @@ dependencies = [ [[package]] name = "arrow-ipc" version = "55.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a222f0d93772bd058d1268f4c28ea421a603d66f7979479048c429292fac7b2e" +source = "git+https://github.com/zhuqi-lucas/arrow-rs.git?branch=add_pub_write_all_api#a291e4b593c43b0050e003aecb18bd4c54f7d30b" dependencies = [ "arrow-array", "arrow-buffer", @@ -403,8 +393,7 @@ dependencies = [ [[package]] name = "arrow-json" version = "55.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9085342bbca0f75e8cb70513c0807cc7351f1fbf5cb98192a67d5e3044acb033" +source = "git+https://github.com/zhuqi-lucas/arrow-rs.git?branch=add_pub_write_all_api#a291e4b593c43b0050e003aecb18bd4c54f7d30b" dependencies = [ "arrow-array", "arrow-buffer", @@ -425,8 +414,7 @@ dependencies = [ [[package]] name = "arrow-ord" version = "55.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ab2f1065a5cad7b9efa9e22ce5747ce826aa3855766755d4904535123ef431e7" +source = "git+https://github.com/zhuqi-lucas/arrow-rs.git?branch=add_pub_write_all_api#a291e4b593c43b0050e003aecb18bd4c54f7d30b" dependencies = [ "arrow-array", "arrow-buffer", @@ -438,8 +426,7 @@ dependencies = [ [[package]] name = "arrow-row" version = "55.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3703a0e3e92d23c3f756df73d2dc9476873f873a76ae63ef9d3de17fda83b2d8" +source = "git+https://github.com/zhuqi-lucas/arrow-rs.git?branch=add_pub_write_all_api#a291e4b593c43b0050e003aecb18bd4c54f7d30b" dependencies = [ "arrow-array", "arrow-buffer", @@ -451,8 +438,7 @@ dependencies = [ [[package]] name = "arrow-schema" version = "55.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "73a47aa0c771b5381de2b7f16998d351a6f4eb839f1e13d48353e17e873d969b" +source = "git+https://github.com/zhuqi-lucas/arrow-rs.git?branch=add_pub_write_all_api#a291e4b593c43b0050e003aecb18bd4c54f7d30b" dependencies = [ "bitflags 2.9.1", "serde", @@ -462,8 +448,7 @@ dependencies = [ [[package]] name = "arrow-select" version = "55.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "24b7b85575702b23b85272b01bc1c25a01c9b9852305e5d0078c79ba25d995d4" +source = "git+https://github.com/zhuqi-lucas/arrow-rs.git?branch=add_pub_write_all_api#a291e4b593c43b0050e003aecb18bd4c54f7d30b" dependencies = [ "ahash 0.8.12", "arrow-array", @@ -476,8 +461,7 @@ dependencies = [ [[package]] name = "arrow-string" version = "55.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9260fddf1cdf2799ace2b4c2fc0356a9789fa7551e0953e35435536fecefebbd" +source = "git+https://github.com/zhuqi-lucas/arrow-rs.git?branch=add_pub_write_all_api#a291e4b593c43b0050e003aecb18bd4c54f7d30b" dependencies = [ "arrow-array", "arrow-buffer", @@ -4421,8 +4405,7 @@ dependencies = [ [[package]] name = "parquet" version = "55.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "be7b2d778f6b841d37083ebdf32e33a524acde1266b5884a8ca29bf00dfa1231" +source = "git+https://github.com/zhuqi-lucas/arrow-rs.git?branch=add_pub_write_all_api#a291e4b593c43b0050e003aecb18bd4c54f7d30b" dependencies = [ "ahash 0.8.12", "arrow-array", diff --git a/Cargo.toml b/Cargo.toml index f2cd6f72c7e6..f70c390d98ce 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -88,19 +88,19 @@ ahash = { version = "0.8", default-features = false, features = [ "runtime-rng", ] } apache-avro = { version = "0.17", default-features = false } -arrow = { version = "55.1.0", features = [ +arrow = { git = "https://github.com/zhuqi-lucas/arrow-rs.git", branch = "add_pub_write_all_api", features = [ "prettyprint", "chrono-tz", ] } -arrow-buffer = { version = "55.0.0", default-features = false } -arrow-flight = { version = "55.1.0", features = [ +arrow-buffer = { git = "https://github.com/zhuqi-lucas/arrow-rs.git", branch = "add_pub_write_all_api", default-features = false } +arrow-flight = { git = "https://github.com/zhuqi-lucas/arrow-rs.git", branch = "add_pub_write_all_api", features = [ "flight-sql-experimental", ] } -arrow-ipc = { version = "55.0.0", default-features = false, features = [ +arrow-ipc = { git = "https://github.com/zhuqi-lucas/arrow-rs.git", branch = "add_pub_write_all_api", default-features = false, features = [ "lz4", ] } -arrow-ord = { version = "55.0.0", default-features = false } -arrow-schema = { version = "55.0.0", default-features = false } +arrow-ord = { git = "https://github.com/zhuqi-lucas/arrow-rs.git", branch = "add_pub_write_all_api", default-features = false } +arrow-schema = { git = "https://github.com/zhuqi-lucas/arrow-rs.git", branch = "add_pub_write_all_api", default-features = false } async-trait = "0.1.88" bigdecimal = "0.4.8" bytes = "1.10" @@ -152,7 +152,7 @@ itertools = "0.14" log = "^0.4" object_store = { version = "0.12.0", default-features = false } parking_lot = "0.12" -parquet = { version = "55.1.0", default-features = false, features = [ +parquet = { git = "https://github.com/zhuqi-lucas/arrow-rs.git", branch = "add_pub_write_all_api", default-features = false, features = [ "arrow", "async", "object_store", diff --git a/datafusion-examples/examples/embedding_parquet_indexes.rs b/datafusion-examples/examples/embedding_parquet_indexes.rs index d90b819d6b9e..fbcf9836b875 100644 --- a/datafusion-examples/examples/embedding_parquet_indexes.rs +++ b/datafusion-examples/examples/embedding_parquet_indexes.rs @@ -180,12 +180,13 @@ fn write_file_with_index(path: &Path, values: &[&str]) -> Result<()> { let offset = writer.writer.inner().stream_position()?; let index_len = index_bytes.len() as u64; + println!("Writing custom index at offset: {offset}, length: {index_len}"); // Write the index magic and length to the file - writer.writer.inner().write_all(b"IDX1")?; - writer.writer.inner().write_all(&index_len.to_le_bytes())?; + writer.writer.write_all(b"IDX1")?; + writer.writer.write_all(&index_len.to_le_bytes())?; // Write the index bytes - writer.writer.inner().write_all(&index_bytes)?; + writer.writer.write_all(&index_bytes)?; // Append metadata about the index to the Parquet file footer writer.writer.append_key_value_metadata(KeyValue::new( @@ -311,10 +312,7 @@ impl TableProvider for DistinctIndexTable { // Build ParquetSource for kept files let url = ObjectStoreUrl::parse("file://")?; - - // Note: we disable page index support here since we are using a custom index, it has conflicts when testing. - // TODO: Remove this when we have a better solution for custom indexes with page index support. - let source = Arc::new(ParquetSource::default().with_enable_page_index(false)); + let source = Arc::new(ParquetSource::default().with_enable_page_index(true)); let mut builder = FileScanConfigBuilder::new(url, self.schema.clone(), source); for file in keep { let path = self.dir.join(&file); From 06d6f081a923f94f648f47dc76b205587bc62d39 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Thu, 19 Jun 2025 17:42:59 +0800 Subject: [PATCH 20/25] polish comments --- .../examples/embedding_parquet_indexes.rs | 31 +++++++++++++------ 1 file changed, 22 insertions(+), 9 deletions(-) diff --git a/datafusion-examples/examples/embedding_parquet_indexes.rs b/datafusion-examples/examples/embedding_parquet_indexes.rs index fbcf9836b875..97458bdabee3 100644 --- a/datafusion-examples/examples/embedding_parquet_indexes.rs +++ b/datafusion-examples/examples/embedding_parquet_indexes.rs @@ -15,14 +15,29 @@ // specific language governing permissions and limitations // under the License. -//! Example: embedding a "distinct values" index in a Parquet file's metadata +//! Example: embedding and using a custom “distinct values” index in Parquet files //! -//! 1. Read existing Parquet files -//! 2. Compute distinct values for a target column using DataFusion -//! 3. Serialize the distinct index to bytes and write to the new Parquet file -//! with these encoded bytes appended as a custom metadata entry -//! 4. Read each new parquet file, extract and deserialize the index from footer -//! 5. Use the distinct index to prune files when querying +//! This example shows how to build and leverage a file‑level distinct‑values index +//! for pruning in DataFusion’s Parquet scans. +//! +//! Steps: +//! 1. Compute the distinct values for a target column and serialize them into bytes. +//! 2. Write each Parquet file with: +//! - regular data pages for your column +//! - the magic marker `IDX1` and a little‑endian length, to identify our custom index format +//! - the serialized distinct‑values bytes +//! - footer key/value metadata entries (`distinct_index_offset` and `distinct_index_length`) +//! 3. Read back each file’s footer metadata to locate and deserialize the index. +//! 4. Build a `DistinctIndexTable` (a custom `TableProvider`) that scans footers +//! into a map of filename → `HashSet` of distinct values. +//! 5. In `scan()`, prune out any Parquet files whose distinct set doesn’t match the +//! `category = 'X'` filter, then only read data from the remaining files. +//! +//! This technique embeds a lightweight, application‑specific index directly in Parquet +//! metadata to achieve efficient file‑level pruning without modifying the Parquet format. +//! +//! And it's very efficient, since we don't add any additional info to the metadata, we write the custom index +//! after the data pages, and we only read it when needed. use arrow::array::{ArrayRef, StringArray}; use arrow::record_batch::RecordBatch; @@ -52,8 +67,6 @@ use std::path::{Path, PathBuf}; use std::sync::Arc; use tempfile::TempDir; -/// We should disable page index support in the Parquet reader -/// when we enable this feature, since we are using a custom index. /// /// Example creating the parquet file that /// contains specialized indexes that From 6bd7d3e806ff38c008489e78aa33f1adfb1a7816 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Thu, 19 Jun 2025 17:47:07 +0800 Subject: [PATCH 21/25] adjust image --- .../examples/embedding_parquet_indexes.rs | 24 ++++++++++++------- 1 file changed, 15 insertions(+), 9 deletions(-) diff --git a/datafusion-examples/examples/embedding_parquet_indexes.rs b/datafusion-examples/examples/embedding_parquet_indexes.rs index 97458bdabee3..845ddf019b74 100644 --- a/datafusion-examples/examples/embedding_parquet_indexes.rs +++ b/datafusion-examples/examples/embedding_parquet_indexes.rs @@ -68,15 +68,17 @@ use std::sync::Arc; use tempfile::TempDir; /// -/// Example creating the parquet file that -/// contains specialized indexes that -/// are ignored by other readers +/// Example creating the Parquet file that +/// contains specialized indexes and a page‑index offset +/// +/// Note: the page index offset will after the custom index, which +/// is originally after the data pages. /// /// ```text /// ┌──────────────────────┐ /// │┌───────────────────┐ │ /// ││ DataPage │ │ Standard Parquet -/// │└───────────────────┘ │ Data / pages +/// │└───────────────────┘ │ Data pages /// │┌───────────────────┐ │ /// ││ DataPage │ │ /// │└───────────────────┘ │ @@ -87,20 +89,24 @@ use tempfile::TempDir; /// │└───────────────────┘ │ /// │┏━━━━━━━━━━━━━━━━━━━┓ │ /// │┃ ┃ │ key/value metadata -/// │┃ Special Index ┃◀┼──── that points at the -/// │┃ ┃ │ │ special index +/// │┃ Special Index ┃◀┼──── that points to the +/// │┃ ┃ │ │ custom index blob /// │┗━━━━━━━━━━━━━━━━━━━┛ │ +/// │┏───────────────────┓ │ +/// │┃ Page Index Offset ┃◀┼──── little‑endian u64 +/// │┗───────────────────┛ │ │ sitting after the custom index /// │╔═══════════════════╗ │ │ /// │║ ║ │ -/// │║ Parquet Footer ║ │ │ Footer includes -/// │║ ║ ┼────── thrift-encoded -/// │║ ║ │ ParquetMetadata +/// │║ Parquet Footer ║ │ │ thrift‑encoded +/// │║ ║ ┼────── ParquetMetadata +/// │║ ║ │ /// │╚═══════════════════╝ │ /// └──────────────────────┘ /// /// Parquet File /// ``` /// DistinctIndexTable is a custom TableProvider that reads Parquet files + #[derive(Debug)] struct DistinctIndexTable { schema: SchemaRef, From 23d71259e22b885a0fa5fe9b66712fe2209bc8f4 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Fri, 20 Jun 2025 23:37:36 +0800 Subject: [PATCH 22/25] add comments part 1 --- Cargo.lock | 32 ++++++------ .../examples/embedding_parquet_indexes.rs | 50 ++++--------------- 2 files changed, 27 insertions(+), 55 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index a7ed890d4e95..b57c175bdbe7 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -247,7 +247,7 @@ checksum = "7c02d123df017efcdfbd739ef81735b36c5ba83ec3c59c80a9d7ecc718f92e50" [[package]] name = "arrow" version = "55.1.0" -source = "git+https://github.com/zhuqi-lucas/arrow-rs.git?branch=add_pub_write_all_api#a291e4b593c43b0050e003aecb18bd4c54f7d30b" +source = "git+https://github.com/zhuqi-lucas/arrow-rs.git?branch=add_pub_write_all_api#5c52e40f177702e5dfcc67bd700dbd2234d51222" dependencies = [ "arrow-arith", "arrow-array", @@ -270,7 +270,7 @@ dependencies = [ [[package]] name = "arrow-arith" version = "55.1.0" -source = "git+https://github.com/zhuqi-lucas/arrow-rs.git?branch=add_pub_write_all_api#a291e4b593c43b0050e003aecb18bd4c54f7d30b" +source = "git+https://github.com/zhuqi-lucas/arrow-rs.git?branch=add_pub_write_all_api#5c52e40f177702e5dfcc67bd700dbd2234d51222" dependencies = [ "arrow-array", "arrow-buffer", @@ -283,7 +283,7 @@ dependencies = [ [[package]] name = "arrow-array" version = "55.1.0" -source = "git+https://github.com/zhuqi-lucas/arrow-rs.git?branch=add_pub_write_all_api#a291e4b593c43b0050e003aecb18bd4c54f7d30b" +source = "git+https://github.com/zhuqi-lucas/arrow-rs.git?branch=add_pub_write_all_api#5c52e40f177702e5dfcc67bd700dbd2234d51222" dependencies = [ "ahash 0.8.12", "arrow-buffer", @@ -299,7 +299,7 @@ dependencies = [ [[package]] name = "arrow-buffer" version = "55.1.0" -source = "git+https://github.com/zhuqi-lucas/arrow-rs.git?branch=add_pub_write_all_api#a291e4b593c43b0050e003aecb18bd4c54f7d30b" +source = "git+https://github.com/zhuqi-lucas/arrow-rs.git?branch=add_pub_write_all_api#5c52e40f177702e5dfcc67bd700dbd2234d51222" dependencies = [ "bytes", "half", @@ -309,7 +309,7 @@ dependencies = [ [[package]] name = "arrow-cast" version = "55.1.0" -source = "git+https://github.com/zhuqi-lucas/arrow-rs.git?branch=add_pub_write_all_api#a291e4b593c43b0050e003aecb18bd4c54f7d30b" +source = "git+https://github.com/zhuqi-lucas/arrow-rs.git?branch=add_pub_write_all_api#5c52e40f177702e5dfcc67bd700dbd2234d51222" dependencies = [ "arrow-array", "arrow-buffer", @@ -329,7 +329,7 @@ dependencies = [ [[package]] name = "arrow-csv" version = "55.1.0" -source = "git+https://github.com/zhuqi-lucas/arrow-rs.git?branch=add_pub_write_all_api#a291e4b593c43b0050e003aecb18bd4c54f7d30b" +source = "git+https://github.com/zhuqi-lucas/arrow-rs.git?branch=add_pub_write_all_api#5c52e40f177702e5dfcc67bd700dbd2234d51222" dependencies = [ "arrow-array", "arrow-cast", @@ -343,7 +343,7 @@ dependencies = [ [[package]] name = "arrow-data" version = "55.1.0" -source = "git+https://github.com/zhuqi-lucas/arrow-rs.git?branch=add_pub_write_all_api#a291e4b593c43b0050e003aecb18bd4c54f7d30b" +source = "git+https://github.com/zhuqi-lucas/arrow-rs.git?branch=add_pub_write_all_api#5c52e40f177702e5dfcc67bd700dbd2234d51222" dependencies = [ "arrow-buffer", "arrow-schema", @@ -354,7 +354,7 @@ dependencies = [ [[package]] name = "arrow-flight" version = "55.1.0" -source = "git+https://github.com/zhuqi-lucas/arrow-rs.git?branch=add_pub_write_all_api#a291e4b593c43b0050e003aecb18bd4c54f7d30b" +source = "git+https://github.com/zhuqi-lucas/arrow-rs.git?branch=add_pub_write_all_api#5c52e40f177702e5dfcc67bd700dbd2234d51222" dependencies = [ "arrow-arith", "arrow-array", @@ -380,7 +380,7 @@ dependencies = [ [[package]] name = "arrow-ipc" version = "55.1.0" -source = "git+https://github.com/zhuqi-lucas/arrow-rs.git?branch=add_pub_write_all_api#a291e4b593c43b0050e003aecb18bd4c54f7d30b" +source = "git+https://github.com/zhuqi-lucas/arrow-rs.git?branch=add_pub_write_all_api#5c52e40f177702e5dfcc67bd700dbd2234d51222" dependencies = [ "arrow-array", "arrow-buffer", @@ -393,7 +393,7 @@ dependencies = [ [[package]] name = "arrow-json" version = "55.1.0" -source = "git+https://github.com/zhuqi-lucas/arrow-rs.git?branch=add_pub_write_all_api#a291e4b593c43b0050e003aecb18bd4c54f7d30b" +source = "git+https://github.com/zhuqi-lucas/arrow-rs.git?branch=add_pub_write_all_api#5c52e40f177702e5dfcc67bd700dbd2234d51222" dependencies = [ "arrow-array", "arrow-buffer", @@ -414,7 +414,7 @@ dependencies = [ [[package]] name = "arrow-ord" version = "55.1.0" -source = "git+https://github.com/zhuqi-lucas/arrow-rs.git?branch=add_pub_write_all_api#a291e4b593c43b0050e003aecb18bd4c54f7d30b" +source = "git+https://github.com/zhuqi-lucas/arrow-rs.git?branch=add_pub_write_all_api#5c52e40f177702e5dfcc67bd700dbd2234d51222" dependencies = [ "arrow-array", "arrow-buffer", @@ -426,7 +426,7 @@ dependencies = [ [[package]] name = "arrow-row" version = "55.1.0" -source = "git+https://github.com/zhuqi-lucas/arrow-rs.git?branch=add_pub_write_all_api#a291e4b593c43b0050e003aecb18bd4c54f7d30b" +source = "git+https://github.com/zhuqi-lucas/arrow-rs.git?branch=add_pub_write_all_api#5c52e40f177702e5dfcc67bd700dbd2234d51222" dependencies = [ "arrow-array", "arrow-buffer", @@ -438,7 +438,7 @@ dependencies = [ [[package]] name = "arrow-schema" version = "55.1.0" -source = "git+https://github.com/zhuqi-lucas/arrow-rs.git?branch=add_pub_write_all_api#a291e4b593c43b0050e003aecb18bd4c54f7d30b" +source = "git+https://github.com/zhuqi-lucas/arrow-rs.git?branch=add_pub_write_all_api#5c52e40f177702e5dfcc67bd700dbd2234d51222" dependencies = [ "bitflags 2.9.1", "serde", @@ -448,7 +448,7 @@ dependencies = [ [[package]] name = "arrow-select" version = "55.1.0" -source = "git+https://github.com/zhuqi-lucas/arrow-rs.git?branch=add_pub_write_all_api#a291e4b593c43b0050e003aecb18bd4c54f7d30b" +source = "git+https://github.com/zhuqi-lucas/arrow-rs.git?branch=add_pub_write_all_api#5c52e40f177702e5dfcc67bd700dbd2234d51222" dependencies = [ "ahash 0.8.12", "arrow-array", @@ -461,7 +461,7 @@ dependencies = [ [[package]] name = "arrow-string" version = "55.1.0" -source = "git+https://github.com/zhuqi-lucas/arrow-rs.git?branch=add_pub_write_all_api#a291e4b593c43b0050e003aecb18bd4c54f7d30b" +source = "git+https://github.com/zhuqi-lucas/arrow-rs.git?branch=add_pub_write_all_api#5c52e40f177702e5dfcc67bd700dbd2234d51222" dependencies = [ "arrow-array", "arrow-buffer", @@ -4420,7 +4420,7 @@ dependencies = [ [[package]] name = "parquet" version = "55.1.0" -source = "git+https://github.com/zhuqi-lucas/arrow-rs.git?branch=add_pub_write_all_api#a291e4b593c43b0050e003aecb18bd4c54f7d30b" +source = "git+https://github.com/zhuqi-lucas/arrow-rs.git?branch=add_pub_write_all_api#5c52e40f177702e5dfcc67bd700dbd2234d51222" dependencies = [ "ahash 0.8.12", "arrow-array", diff --git a/datafusion-examples/examples/embedding_parquet_indexes.rs b/datafusion-examples/examples/embedding_parquet_indexes.rs index 845ddf019b74..b41e7dc92c82 100644 --- a/datafusion-examples/examples/embedding_parquet_indexes.rs +++ b/datafusion-examples/examples/embedding_parquet_indexes.rs @@ -51,13 +51,10 @@ use datafusion::datasource::physical_plan::{FileScanConfigBuilder, ParquetSource use datafusion::datasource::TableType; use datafusion::execution::object_store::ObjectStoreUrl; use datafusion::logical_expr::{Operator, TableProviderFilterPushDown}; -use datafusion::parquet::arrow::ArrowSchemaConverter; -use datafusion::parquet::data_type::{ByteArray, ByteArrayType}; +use datafusion::parquet::arrow::ArrowWriter; use datafusion::parquet::errors::ParquetError; use datafusion::parquet::file::metadata::KeyValue; -use datafusion::parquet::file::properties::WriterProperties; use datafusion::parquet::file::reader::{FileReader, SerializedFileReader}; -use datafusion::parquet::file::writer::SerializedFileWriter; use datafusion::physical_plan::ExecutionPlan; use datafusion::prelude::*; use datafusion::scalar::ScalarValue; @@ -138,19 +135,12 @@ impl DistinctIndexTable { } pub struct IndexedParquetWriter { - writer: SerializedFileWriter, + writer: ArrowWriter, } impl IndexedParquetWriter { - pub fn try_new( - sink: W, - schema: Arc, - props: WriterProperties, - ) -> Result { - let schema_desc = ArrowSchemaConverter::new().convert(schema.as_ref())?; - let props_ptr = Arc::new(props); - let writer = - SerializedFileWriter::new(sink, schema_desc.root_schema_ptr(), props_ptr)?; + pub fn try_new(sink: W, schema: Arc) -> Result { + let writer = ArrowWriter::try_new(sink, schema, None)?; Ok(Self { writer }) } } @@ -168,35 +158,17 @@ fn write_file_with_index(path: &Path, values: &[&str]) -> Result<()> { let serialized = distinct.into_iter().collect::>().join("\n"); let index_bytes = serialized.into_bytes(); - let props = WriterProperties::builder().build(); let file = File::create(path)?; - let mut writer = IndexedParquetWriter::try_new(file, schema.clone(), props)?; - - // Write data to the Parquet file, we only write one column since our schema has one field - { - let mut rg_writer = writer.writer.next_row_group()?; - let mut ser_col_writer = rg_writer - .next_column()? - .ok_or_else(|| ParquetError::General("No column writer".into()))?; - - let col_writer = ser_col_writer.typed::(); - let values_bytes: Vec = batch - .column(0) - .as_any() - .downcast_ref::() - .unwrap() - .iter() - .map(|opt| ByteArray::from(opt.unwrap())) - .collect(); + let mut writer = IndexedParquetWriter::try_new(file, schema.clone())?; - println!("Writing values: {values_bytes:?}"); - col_writer.write_batch(&values_bytes, None, None)?; - ser_col_writer.close()?; - rg_writer.close()?; - } + // Write the data pages + writer.writer.write(&batch)?; + // Close row group + writer.writer.flush()?; - let offset = writer.writer.inner().stream_position()?; + // Set the offset for the index + let offset = writer.writer.bytes_written(); let index_len = index_bytes.len() as u64; println!("Writing custom index at offset: {offset}, length: {index_len}"); From 13b74aca148b1f1c1fd12a7344f5cb7a1f8f3546 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Sat, 21 Jun 2025 11:11:33 +0800 Subject: [PATCH 23/25] pin to new arrow-rs --- Cargo.toml | 38 ++++++++++++++++++++++++++++++-------- 1 file changed, 30 insertions(+), 8 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index f70c390d98ce..d4c0963e146a 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -88,19 +88,20 @@ ahash = { version = "0.8", default-features = false, features = [ "runtime-rng", ] } apache-avro = { version = "0.17", default-features = false } -arrow = { git = "https://github.com/zhuqi-lucas/arrow-rs.git", branch = "add_pub_write_all_api", features = [ +arrow = { version = "55.2.0", features = [ "prettyprint", "chrono-tz", ] } -arrow-buffer = { git = "https://github.com/zhuqi-lucas/arrow-rs.git", branch = "add_pub_write_all_api", default-features = false } -arrow-flight = { git = "https://github.com/zhuqi-lucas/arrow-rs.git", branch = "add_pub_write_all_api", features = [ +arrow-buffer = { version = "55.2.0", default-features = false } +arrow-flight = { version = "55.2.0", features = [ "flight-sql-experimental", ] } -arrow-ipc = { git = "https://github.com/zhuqi-lucas/arrow-rs.git", branch = "add_pub_write_all_api", default-features = false, features = [ +arrow-ipc = { version = "55.2.0", default-features = false, features = [ "lz4", + "zstd", ] } -arrow-ord = { git = "https://github.com/zhuqi-lucas/arrow-rs.git", branch = "add_pub_write_all_api", default-features = false } -arrow-schema = { git = "https://github.com/zhuqi-lucas/arrow-rs.git", branch = "add_pub_write_all_api", default-features = false } +arrow-ord = { version = "55.2.0", default-features = false } +arrow-schema = { version = "55.2.0", default-features = false } async-trait = "0.1.88" bigdecimal = "0.4.8" bytes = "1.10" @@ -152,7 +153,7 @@ itertools = "0.14" log = "^0.4" object_store = { version = "0.12.0", default-features = false } parking_lot = "0.12" -parquet = { git = "https://github.com/zhuqi-lucas/arrow-rs.git", branch = "add_pub_write_all_api", default-features = false, features = [ +parquet = { version = "55.2.0", default-features = false, features = [ "arrow", "async", "object_store", @@ -217,5 +218,26 @@ unnecessary_lazy_evaluations = "warn" uninlined_format_args = "warn" [workspace.lints.rust] -unexpected_cfgs = { level = "warn", check-cfg = ["cfg(tarpaulin)", "cfg(tarpaulin_include)"] } +unexpected_cfgs = { level = "warn", check-cfg = [ + 'cfg(datafusion_coop, values("tokio", "tokio_fallback", "per_stream"))', + "cfg(tarpaulin)", + "cfg(tarpaulin_include)", +] } unused_qualifications = "deny" + + +## Temporary arrow-rs patch until 52.2.0 is released +# https://github.com/apache/arrow-rs/issues/7394 +[patch.crates-io] +arrow = { git = "https://github.com/apache/arrow-rs.git", rev = "2628c8cf025b28b74cab48b6ea0e3594fa9b37d2" } +arrow-array = { git = "https://github.com/apache/arrow-rs.git", rev = "2628c8cf025b28b74cab48b6ea0e3594fa9b37d2" } +arrow-buffer = { git = "https://github.com/apache/arrow-rs.git", rev = "2628c8cf025b28b74cab48b6ea0e3594fa9b37d2" } +arrow-cast = { git = "https://github.com/apache/arrow-rs.git", rev = "2628c8cf025b28b74cab48b6ea0e3594fa9b37d2" } +arrow-data = { git = "https://github.com/apache/arrow-rs.git", rev = "2628c8cf025b28b74cab48b6ea0e3594fa9b37d2" } +arrow-ipc = { git = "https://github.com/apache/arrow-rs.git", rev = "2628c8cf025b28b74cab48b6ea0e3594fa9b37d2" } +arrow-schema = { git = "https://github.com/apache/arrow-rs.git", rev = "2628c8cf025b28b74cab48b6ea0e3594fa9b37d2" } +arrow-select = { git = "https://github.com/apache/arrow-rs.git", rev = "2628c8cf025b28b74cab48b6ea0e3594fa9b37d2" } +arrow-string = { git = "https://github.com/apache/arrow-rs.git", rev = "2628c8cf025b28b74cab48b6ea0e3594fa9b37d2" } +arrow-ord = { git = "https://github.com/apache/arrow-rs.git", rev = "2628c8cf025b28b74cab48b6ea0e3594fa9b37d2" } +arrow-flight = { git = "https://github.com/apache/arrow-rs.git", rev = "2628c8cf025b28b74cab48b6ea0e3594fa9b37d2" } +parquet = { git = "https://github.com/apache/arrow-rs.git", rev = "2628c8cf025b28b74cab48b6ea0e3594fa9b37d2" } From 1b0501c0ba5b9ae6ad9f752441ca3a870e09ee16 Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Sat, 21 Jun 2025 11:11:42 +0800 Subject: [PATCH 24/25] pin to new arrow-rs --- Cargo.toml | 38 ++++++++------------------------------ 1 file changed, 8 insertions(+), 30 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index d4c0963e146a..f70c390d98ce 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -88,20 +88,19 @@ ahash = { version = "0.8", default-features = false, features = [ "runtime-rng", ] } apache-avro = { version = "0.17", default-features = false } -arrow = { version = "55.2.0", features = [ +arrow = { git = "https://github.com/zhuqi-lucas/arrow-rs.git", branch = "add_pub_write_all_api", features = [ "prettyprint", "chrono-tz", ] } -arrow-buffer = { version = "55.2.0", default-features = false } -arrow-flight = { version = "55.2.0", features = [ +arrow-buffer = { git = "https://github.com/zhuqi-lucas/arrow-rs.git", branch = "add_pub_write_all_api", default-features = false } +arrow-flight = { git = "https://github.com/zhuqi-lucas/arrow-rs.git", branch = "add_pub_write_all_api", features = [ "flight-sql-experimental", ] } -arrow-ipc = { version = "55.2.0", default-features = false, features = [ +arrow-ipc = { git = "https://github.com/zhuqi-lucas/arrow-rs.git", branch = "add_pub_write_all_api", default-features = false, features = [ "lz4", - "zstd", ] } -arrow-ord = { version = "55.2.0", default-features = false } -arrow-schema = { version = "55.2.0", default-features = false } +arrow-ord = { git = "https://github.com/zhuqi-lucas/arrow-rs.git", branch = "add_pub_write_all_api", default-features = false } +arrow-schema = { git = "https://github.com/zhuqi-lucas/arrow-rs.git", branch = "add_pub_write_all_api", default-features = false } async-trait = "0.1.88" bigdecimal = "0.4.8" bytes = "1.10" @@ -153,7 +152,7 @@ itertools = "0.14" log = "^0.4" object_store = { version = "0.12.0", default-features = false } parking_lot = "0.12" -parquet = { version = "55.2.0", default-features = false, features = [ +parquet = { git = "https://github.com/zhuqi-lucas/arrow-rs.git", branch = "add_pub_write_all_api", default-features = false, features = [ "arrow", "async", "object_store", @@ -218,26 +217,5 @@ unnecessary_lazy_evaluations = "warn" uninlined_format_args = "warn" [workspace.lints.rust] -unexpected_cfgs = { level = "warn", check-cfg = [ - 'cfg(datafusion_coop, values("tokio", "tokio_fallback", "per_stream"))', - "cfg(tarpaulin)", - "cfg(tarpaulin_include)", -] } +unexpected_cfgs = { level = "warn", check-cfg = ["cfg(tarpaulin)", "cfg(tarpaulin_include)"] } unused_qualifications = "deny" - - -## Temporary arrow-rs patch until 52.2.0 is released -# https://github.com/apache/arrow-rs/issues/7394 -[patch.crates-io] -arrow = { git = "https://github.com/apache/arrow-rs.git", rev = "2628c8cf025b28b74cab48b6ea0e3594fa9b37d2" } -arrow-array = { git = "https://github.com/apache/arrow-rs.git", rev = "2628c8cf025b28b74cab48b6ea0e3594fa9b37d2" } -arrow-buffer = { git = "https://github.com/apache/arrow-rs.git", rev = "2628c8cf025b28b74cab48b6ea0e3594fa9b37d2" } -arrow-cast = { git = "https://github.com/apache/arrow-rs.git", rev = "2628c8cf025b28b74cab48b6ea0e3594fa9b37d2" } -arrow-data = { git = "https://github.com/apache/arrow-rs.git", rev = "2628c8cf025b28b74cab48b6ea0e3594fa9b37d2" } -arrow-ipc = { git = "https://github.com/apache/arrow-rs.git", rev = "2628c8cf025b28b74cab48b6ea0e3594fa9b37d2" } -arrow-schema = { git = "https://github.com/apache/arrow-rs.git", rev = "2628c8cf025b28b74cab48b6ea0e3594fa9b37d2" } -arrow-select = { git = "https://github.com/apache/arrow-rs.git", rev = "2628c8cf025b28b74cab48b6ea0e3594fa9b37d2" } -arrow-string = { git = "https://github.com/apache/arrow-rs.git", rev = "2628c8cf025b28b74cab48b6ea0e3594fa9b37d2" } -arrow-ord = { git = "https://github.com/apache/arrow-rs.git", rev = "2628c8cf025b28b74cab48b6ea0e3594fa9b37d2" } -arrow-flight = { git = "https://github.com/apache/arrow-rs.git", rev = "2628c8cf025b28b74cab48b6ea0e3594fa9b37d2" } -parquet = { git = "https://github.com/apache/arrow-rs.git", rev = "2628c8cf025b28b74cab48b6ea0e3594fa9b37d2" } From c344843a6eb843cf647e6474502a4996db21d4ed Mon Sep 17 00:00:00 2001 From: zhuqi-lucas <821684824@qq.com> Date: Sat, 21 Jun 2025 18:18:27 +0800 Subject: [PATCH 25/25] add comments part 2 --- .../examples/embedding_parquet_indexes.rs | 101 ++++++++++++------ 1 file changed, 70 insertions(+), 31 deletions(-) diff --git a/datafusion-examples/examples/embedding_parquet_indexes.rs b/datafusion-examples/examples/embedding_parquet_indexes.rs index b41e7dc92c82..d6515fec19f5 100644 --- a/datafusion-examples/examples/embedding_parquet_indexes.rs +++ b/datafusion-examples/examples/embedding_parquet_indexes.rs @@ -104,10 +104,71 @@ use tempfile::TempDir; /// ``` /// DistinctIndexTable is a custom TableProvider that reads Parquet files +#[derive(Debug, Clone)] +struct DistinctIndex { + inner: HashSet, +} + +impl DistinctIndex { + // Init from iterator of distinct values + pub fn new>(iter: I) -> Self { + Self { + inner: iter.into_iter().collect(), + } + } + + // serialize the distinct index to a writer + fn serialize( + &self, + arrow_writer: &mut ArrowWriter, + ) -> Result<()> { + let distinct: HashSet<_> = self.inner.iter().collect(); + let serialized = distinct + .into_iter() + .map(|s| s.as_str()) + .collect::>() + .join("\n"); + let index_bytes = serialized.into_bytes(); + + // Set the offset for the index + let offset = arrow_writer.bytes_written(); + let index_len = index_bytes.len() as u64; + + println!("Writing custom index at offset: {offset}, length: {index_len}"); + // Write the index magic and length to the file + arrow_writer.write_all(b"IDX1")?; + arrow_writer.write_all(&index_len.to_le_bytes())?; + + // Write the index bytes + arrow_writer.write_all(&index_bytes)?; + + // Append metadata about the index to the Parquet file footer + arrow_writer.append_key_value_metadata(KeyValue::new( + "distinct_index_offset".to_string(), + offset.to_string(), + )); + arrow_writer.append_key_value_metadata(KeyValue::new( + "distinct_index_length".to_string(), + index_bytes.len().to_string(), + )); + Ok(()) + } + + // create a new distinct index from the specified bytes + fn new_from_bytes(serialized: &[u8]) -> Result { + let s = String::from_utf8(serialized.to_vec()) + .map_err(|e| ParquetError::General(e.to_string()))?; + + Ok(Self { + inner: s.lines().map(|s| s.to_string()).collect(), + }) + } +} + #[derive(Debug)] struct DistinctIndexTable { schema: SchemaRef, - index: HashMap>, + index: HashMap, dir: PathBuf, } @@ -154,10 +215,6 @@ fn write_file_with_index(path: &Path, values: &[&str]) -> Result<()> { let arr: ArrayRef = Arc::new(StringArray::from(values.to_vec())); let batch = RecordBatch::try_new(schema.clone(), vec![arr])?; - let distinct: HashSet<_> = values.iter().copied().collect(); - let serialized = distinct.into_iter().collect::>().join("\n"); - let index_bytes = serialized.into_bytes(); - let file = File::create(path)?; let mut writer = IndexedParquetWriter::try_new(file, schema.clone())?; @@ -167,27 +224,10 @@ fn write_file_with_index(path: &Path, values: &[&str]) -> Result<()> { // Close row group writer.writer.flush()?; - // Set the offset for the index - let offset = writer.writer.bytes_written(); - let index_len = index_bytes.len() as u64; + let distinct_index: DistinctIndex = + DistinctIndex::new(values.iter().map(|s| s.to_string())); - println!("Writing custom index at offset: {offset}, length: {index_len}"); - // Write the index magic and length to the file - writer.writer.write_all(b"IDX1")?; - writer.writer.write_all(&index_len.to_le_bytes())?; - - // Write the index bytes - writer.writer.write_all(&index_bytes)?; - - // Append metadata about the index to the Parquet file footer - writer.writer.append_key_value_metadata(KeyValue::new( - "distinct_index_offset".to_string(), - offset.to_string(), - )); - writer.writer.append_key_value_metadata(KeyValue::new( - "distinct_index_length".to_string(), - index_bytes.len().to_string(), - )); + distinct_index.serialize(&mut writer.writer)?; writer.writer.close()?; @@ -195,7 +235,7 @@ fn write_file_with_index(path: &Path, values: &[&str]) -> Result<()> { Ok(()) } -fn read_distinct_index(path: &Path) -> Result, ParquetError> { +fn read_distinct_index(path: &Path) -> Result { let mut file = File::open(path)?; let file_size = file.metadata()?.len(); @@ -245,10 +285,9 @@ fn read_distinct_index(path: &Path) -> Result, ParquetError> { let mut index_buf = vec![0u8; length]; file.read_exact(&mut index_buf)?; - let s = - String::from_utf8(index_buf).map_err(|e| ParquetError::General(e.to_string()))?; - - Ok(s.lines().map(|s| s.to_string()).collect()) + let index = DistinctIndex::new_from_bytes(&index_buf) + .map_err(|e| ParquetError::General(e.to_string()))?; + Ok(index) } /// Implement TableProvider for DistinctIndexTable, using the distinct index to prune files @@ -295,7 +334,7 @@ impl TableProvider for DistinctIndexTable { let keep: Vec = self .index .iter() - .filter(|(_f, set)| target.as_ref().is_none_or(|v| set.contains(v))) + .filter(|(_f, set)| target.as_ref().is_none_or(|v| set.inner.contains(v))) .map(|(f, _)| f.clone()) .collect();