Skip to content

Add an example of embedding indexes inside a parquet file #16395

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Merged
merged 38 commits into from
Jul 3, 2025

Conversation

zhuqi-lucas
Copy link
Contributor

@zhuqi-lucas zhuqi-lucas commented Jun 13, 2025

Which issue does this PR close?

Rationale for this change

//! Example: embedding and using a custom “distinct values” index in Parquet files
//!
//! 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<String>` 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.

What changes are included in this PR?

Are these changes tested?

Are there any user-facing changes?

// Compute distinct values, serialize & Base64‑encode
let distinct: HashSet<_> = values.iter().copied().collect();
let serialized = distinct.iter().cloned().collect::<Vec<_>>().join("\n");
let b64 = general_purpose::STANDARD_NO_PAD.encode(serialized.as_bytes());
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think this writes the index into the footer itself (as an opaque string)

This has at least 2 downsides

  1. The footer metadata will be much larger / longer to parse
  2. A binary index must be converted to/from strings (as you are doing here with b64)

Is it possible to write the binary data directly into the parquet file?

Specifically, so then the metadata looks something like

// Find out where the current write position is
let offset_to_index_in_file = file.current_position()
file.write_all(distinct_index)?;

// now, finalize the file with the parquet metadata:
let props = WriterProperties::builder()
        .set_key_value_metadata(Some(vec![KeyValue::new(
            "distinct_index_data".into(),
            offset_to_index_in_file.to_string(),
        )]))
        .build();

I am not sure how easy this would be to do with the current API

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Very good point @alamb! Thank you.

I will try to find a better solution, i agree the following downsides.

This has at least 2 downsides

  1. The footer metadata will be much larger / longer to parse
  2. A binary index must be converted to/from strings (as you are doing here with b64)

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I tried today, but found it's hard for current API to support this, will try it again.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Try to using low level API, but it only works when we disable page index, if we setting page index, it will follow up the real row group data, and it conflicts with our embedding indexes.

Copy link
Contributor Author

@zhuqi-lucas zhuqi-lucas Jun 15, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The code is here:

// 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
//!
//! 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

use arrow::array::{ArrayRef, StringArray};
use arrow::record_batch::RecordBatch;
use arrow_schema::{DataType, Field, Schema, SchemaRef};
use async_trait::async_trait;
use datafusion::catalog::{Session, TableProvider};
use datafusion::common::{HashMap, HashSet, Result};
use datafusion::datasource::listing::PartitionedFile;
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::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 tempfile::TempDir;

/// 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
///
/// ```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
/// ```
/// DistinctIndexTable is a custom TableProvider that reads Parquet files
#[derive(Debug)]
struct DistinctIndexTable {
    schema: SchemaRef,
    index: HashMap<String, HashSet<String>>,
    dir: PathBuf,
}

impl DistinctIndexTable {
    /// Scan a directory, read each file's footer metadata into a map
    fn try_new(dir: impl Into<PathBuf>, schema: SchemaRef) -> Result<Self> {
        let dir = dir.into();
        let mut index = HashMap::new();

        for entry in read_dir(&dir)? {
            let path = entry?.path();
            if path.extension().and_then(|s| s.to_str()) != Some("parquet") {
                continue;
            }
            let file_name = path.file_name().unwrap().to_string_lossy().to_string();

            let distinct_set = read_distinct_index(&path)?;

            println!("Read distinct index for {}: {:?}", file_name, distinct_set);
            index.insert(file_name, distinct_set);
        }

        Ok(Self { schema, index, dir })
    }
}

pub struct IndexedParquetWriter<W: Write + Seek> {
    writer: SerializedFileWriter<W>,
}

impl<W: Write + Seek + Send> IndexedParquetWriter<W> {
    pub fn try_new(
        sink: W,
        schema: Arc<Schema>,
        props: WriterProperties,
    ) -> Result<Self> {
        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 })
    }
}

const INDEX_MAGIC: &[u8] = b"IDX1";

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::<Vec<_>>().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)?;

    {
        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::<ByteArrayType>();
        let values_bytes: Vec<ByteArray> = batch
            .column(0)
            .as_any()
            .downcast_ref::<StringArray>()
            .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()?;
    }

    let offset = writer.writer.inner().seek(SeekFrom::Current(0))?;

    let index_len = index_bytes.len() as u64;
    writer.writer.inner().write_all(b"IDX1")?;
    writer.writer.inner().write_all(&index_len.to_le_bytes())?;

    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<HashSet<String>, ParquetError> {
    let mut file = File::open(path)?;

    let file_size = file.metadata()?.len();
    println!(
        "Reading index from {} (size: {})",
        path.display(),
        file_size
    );

    let reader = SerializedFileReader::new(file.try_clone()?)?;
    let meta = reader.metadata().file_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::<u64>()
        .map_err(|e| ParquetError::General(e.to_string()))?;

    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::<usize>()
        .map_err(|e| ParquetError::General(e.to_string()))?;

    println!("Reading index at offset: {}, length: {}", offset, length);

    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;

    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)?;

    let s =
        String::from_utf8(index_buf).map_err(|e| ParquetError::General(e.to_string()))?;

    Ok(s.lines().map(|s| s.to_string()).collect())
}

/// 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
    }

    /// Prune files before reading: only keep files whose distinct set contains the filter value
    async fn scan(
        &self,
        _ctx: &dyn Session,
        _proj: Option<&Vec<usize>>,
        filters: &[Expr],
        _limit: Option<usize>,
    ) -> Result<Arc<dyn ExecutionPlan>> {
        // Look for a single `category = 'X'` filter
        let mut target: Option<String> = 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" {
                            println!("Filtering for category: {v}");
                            target = Some(v.clone());
                        }
                    }
                }
            }
        }
        // Determine which files to scan
        let keep: Vec<String> = 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());

        // 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_str().unwrap().to_string(),
                len,
            ));
        }
        Ok(DataSourceExec::from_data_source(builder.build()))
    }

    fn supports_filters_pushdown(
        &self,
        fs: &[&Expr],
    ) -> Result<Vec<TableProviderFilterPushDown>> {
        // Mark as inexact since pruning is file‑granular
        Ok(vec![TableProviderFilterPushDown::Inexact; fs.len()])
    }
}

#[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", "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)?;

    // 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?;

    Ok(())
}

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This looks super cool @zhuqi-lucas

Try to using low level API, but it only works when we disable page index, if we setting page index, it will follow up the real row group data, and it conflicts with our embedding indexes.

I don't fully understand this concern -- I would probably have to play around with it some more

Are you willing to update this PR with this new example? I have some ideas on the various APIs we could use (like we could potentially encapsulate the index writing some more)

We could also then file a ticket upstream i arrow-rs with a description of what wasn't working with page indexes

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thank you @alamb , updated the code without page index using low level API, i will continue debugging the case that our self defined index with page index.

Copy link
Contributor

@alamb alamb left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thank you @zhuqi-lucas -- this is really neat. I left some thoughts -- let me know what you think


// 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));
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thank you @alamb Currently, i disable the page index for reading, so this example will succeed, but if we enable page index, it will fail due to:

  1. We are writing the self defined index just after the data.
  2. But it seems, the page index offset info will write to the same place.
  3. I can't find a solution until now, need some help.

Thanks!

@zhuqi-lucas
Copy link
Contributor Author

Thank you @alamb,
I am excited to update today that i resolve the page index conflicts by adding new API in arrow-rs which can write bytes to the buf, and it can make the buf-wirtten metrics consistent, and the buf-wirtten will be used by page index also, so it's safe now, and i enable the page index now for the example, the testing result is good!

I am currently using this arrow-rs branch before the code merge:
apache/arrow-rs#7714

The example print logs, it's good, thanks!

Writing values: [ByteArray { data: "foo" }, ByteArray { data: "bar" }, ByteArray { data: "foo" }]
Writing custom index at offset: 68, length: 7
Finished writing file to /var/folders/q7/zjtv8rvx2hz0_t_rjjq8p9k00000gp/T/.tmp9zCIJt/a.parquet
Writing values: [ByteArray { data: "baz" }, ByteArray { data: "qux" }]
Writing custom index at offset: 68, length: 7
Finished writing file to /var/folders/q7/zjtv8rvx2hz0_t_rjjq8p9k00000gp/T/.tmp9zCIJt/b.parquet
Writing values: [ByteArray { data: "foo" }, ByteArray { data: "quux" }, ByteArray { data: "quux" }]
Writing custom index at offset: 70, length: 8
Finished writing file to /var/folders/q7/zjtv8rvx2hz0_t_rjjq8p9k00000gp/T/.tmp9zCIJt/c.parquet
Reading index from /var/folders/q7/zjtv8rvx2hz0_t_rjjq8p9k00000gp/T/.tmp9zCIJt/a.parquet (size: 363)
Reading index at offset: 68, length: 7
Read distinct index for a.parquet: "a.parquet"
Reading index from /var/folders/q7/zjtv8rvx2hz0_t_rjjq8p9k00000gp/T/.tmp9zCIJt/b.parquet (size: 363)
Reading index at offset: 68, length: 7
Read distinct index for b.parquet: "b.parquet"
Reading index from /var/folders/q7/zjtv8rvx2hz0_t_rjjq8p9k00000gp/T/.tmp9zCIJt/c.parquet (size: 368)
Reading index at offset: 70, length: 8
Read distinct index for c.parquet: "c.parquet"
Filtering for category: foo
Pruned files: ["c.parquet", "a.parquet"]
+----------+
| category |
+----------+
| foo      |
| foo      |
| foo      |
+----------+

@alamb
Copy link
Contributor

alamb commented Jun 21, 2025

FYI @XiangpengHao and @@JigaoLuo -- here is another example of the somewhat crazy things you can do with parquet

@alamb
Copy link
Contributor

alamb commented Jun 21, 2025

How does it ensure that this extra index can be safely ignored by other readers? If another parquet reader implementation decides to do a sequential whole file scan, will it read into the extra custom data?

I agree with what @zhuqi-lucas says too

The way I think about this is that the parquet file's footer contains pointers (offsets) to the actual data in the file. There is no requirement that the footer points to all bytes within the file

There are other interesting things that can be done with this setup too (for example, concatenating parquet files together without having to re-encode the data (you can just copy the bytes around and rewrite the footer)

@zhuqi-lucas
Copy link
Contributor Author

This is amazing @alamb ! Thanks!

There are other interesting things that can be done with this setup too (for example, concatenating parquet files together without having to re-encode the data (you can just copy the bytes around and rewrite the footer)

@alamb
Copy link
Contributor

alamb commented Jun 26, 2025

I have a PR up to upgrade to the necessary version of arrow here

Hopefully once that merges we can polish this PR up (and make a big deal about it)

@zhuqi-lucas
Copy link
Contributor Author

Thank you @alamb , i will polish the code after we changing to arrow/parquet 55.2.0!

I have a PR up to upgrade to the necessary version of arrow here

Hopefully once that merges we can polish this PR up (and make a big deal about it)

@zhuqi-lucas
Copy link
Contributor Author

Updated the code with the merged PR: #16575
And also added more comments.

Copy link
Contributor

@alamb alamb left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thank you @zhuqi-lucas -- I started going through this PR again in detail

I renamed the example to align with the other parquet examples, and I added it to the list of examples.

I also took a pass through the comments.

I have run out of time today, but I'll finish it up first thing tomorrow and hopefully merge

Thank you again so much!

// 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)?;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

BTW I also double checked that these files can be read by duckdb, and the had no problems:

D select * from read_parquet('/tmp/parquet_index_data/*');
┌──────────┐
│ category │
│ varchar  │
├──────────┤
│ foo      │
│ bar      │
│ foo      │
│ baz      │
│ qux      │
│ foo      │
│ quux     │
│ quux     │
└──────────┘

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thank you @alamb for the crossing checking! This is very valuable!

@zhuqi-lucas
Copy link
Contributor Author

Thank you @zhuqi-lucas -- I started going through this PR again in detail

I renamed the example to align with the other parquet examples, and I added it to the list of examples.

I also took a pass through the comments.

I have run out of time today, but I'll finish it up first thing tomorrow and hopefully merge

Thank you again so much!

Thank you very much @alamb ! This looks pretty good.

@alamb
Copy link
Contributor

alamb commented Jul 2, 2025

Hi @zhuqi-lucas -- spent a while this morning going over this PR carefully -- it is great!

I hope you don't mind but I made some substantial edits to try and make it read a bit better:

  1. Revamped the documentation and overview
  2. Updated the ASCII art
  3. moved reading the index into DistinctIndex
  4. Added a bunch more comments
  5. Simplified the code to only write the offset index (the length is stored inline)

In my mind none of this was required but since I plan to make a Huge Deal (TM) about this example publically I figured spending some extra time polishing it would be worthwhle

@alamb
Copy link
Contributor

alamb commented Jul 2, 2025

I think it is now ready to merge, but it would probably be good for someone else to go over it one last time to make sure it is clear

@zhuqi-lucas
Copy link
Contributor Author

zhuqi-lucas commented Jul 2, 2025

Hi @zhuqi-lucas -- spent a while this morning going over this PR carefully -- it is great!

I hope you don't mind but I made some substantial edits to try and make it read a bit better:

  1. Revamped the documentation and overview
  2. Updated the ASCII art
  3. moved reading the index into DistinctIndex
  4. Added a bunch more comments
  5. Simplified the code to only write the offset index (the length is stored inline)

In my mind none of this was required but since I plan to make a Huge Deal (TM) about this example publically I figured spending some extra time polishing it would be worthwhle

Thank you @alamb looks great to me!

Simplified the code to only write the offset index (the length is stored inline)

Perfect for this change!

@jcsherin
Copy link
Contributor

jcsherin commented Jul 2, 2025

@alamb The overview documentation is very clear and love the ASCII art.

@alamb
Copy link
Contributor

alamb commented Jul 3, 2025

This is so great -- now we just need to write up a blog post 🎣

Thanks again @zhuqi-lucas -- this is going to be great

@alamb alamb merged commit 5a48857 into apache:main Jul 3, 2025
27 checks passed
@zhuqi-lucas
Copy link
Contributor Author

Thank you @alamb , i will submit a draft blog soon in:

#16372

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

Add an example of embedding indexes *inside* a parquet file
4 participants