Skip to content
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

feat: metadata columns #14057

Open
wants to merge 19 commits into
base: main
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
1 change: 1 addition & 0 deletions datafusion-examples/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -65,6 +65,7 @@ datafusion = { workspace = true, default-features = true, features = ["avro"] }
datafusion-proto = { workspace = true }
env_logger = { workspace = true }
futures = { workspace = true }
itertools = { workspace = true }
log = { workspace = true }
mimalloc = { version = "0.1", default-features = false }
object_store = { workspace = true, features = ["aws", "http"] }
Expand Down
314 changes: 314 additions & 0 deletions datafusion-examples/examples/metadata_columns.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,314 @@
// 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.

use std::any::Any;
use std::fmt::{self, Debug, Formatter};
use std::sync::{Arc, Mutex};
use std::time::Duration;

use arrow::array::{ArrayRef, StringArray, UInt64Array};
use async_trait::async_trait;
use datafusion::arrow::array::{UInt64Builder, UInt8Builder};
use datafusion::arrow::datatypes::{DataType, Field, Schema, SchemaRef};
use datafusion::arrow::record_batch::RecordBatch;
use datafusion::datasource::{TableProvider, TableType};
use datafusion::error::Result;
use datafusion::execution::context::TaskContext;
use datafusion::physical_expr::EquivalenceProperties;
use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType};
use datafusion::physical_plan::memory::MemoryStream;
use datafusion::physical_plan::{
DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties,
SendableRecordBatchStream,
};

use datafusion::prelude::*;

use datafusion::catalog::Session;
use datafusion::common::FieldId;
use itertools::Itertools;
use tokio::time::timeout;

/// This example demonstrates executing a simple query against a custom datasource
#[tokio::main]
async fn main() -> Result<()> {
// create our custom datasource and adding some users
let db = CustomDataSource::default();
db.populate_users();

search_accounts(db.clone(), "select * from accounts", 3).await?;
search_accounts(
db.clone(),
"select _rowid, _file, * from accounts where _rowid > 1",
1,
)
.await?;
search_accounts(
db.clone(),
"select _rowid, _file, * from accounts where _file = 'file-0'",
1,
)
.await?;

Ok(())
}

async fn search_accounts(
db: CustomDataSource,
sql: &str,
expected_result_length: usize,
) -> Result<()> {
// create local execution context
let ctx = SessionContext::new();
ctx.register_table("accounts", Arc::new(db)).unwrap();
let options = SQLOptions::new().with_allow_ddl(false);

timeout(Duration::from_secs(10), async move {
let dataframe = ctx.sql_with_options(sql, options).await.unwrap();
let result = dataframe.collect().await.unwrap();
let record_batch = result.first().unwrap();

assert_eq!(expected_result_length, record_batch.column(1).len());
dbg!(record_batch.columns());
})
.await
.unwrap();

Ok(())
}

/// A User, with an id and a bank account
#[derive(Clone, Debug)]
struct User {
id: u8,
bank_account: u64,
}

/// A custom datasource, used to represent a datastore with a single index
#[derive(Clone)]
pub struct CustomDataSource {
inner: Arc<Mutex<CustomDataSourceInner>>,
metadata_columns: SchemaRef,
}

struct CustomDataSourceInner {
data: Vec<User>,
}

impl Debug for CustomDataSource {
fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result {
f.write_str("custom_db")
}
}

impl CustomDataSource {
pub(crate) async fn create_physical_plan(
&self,
projections: Option<&Vec<usize>>,
) -> Result<Arc<dyn ExecutionPlan>> {
Ok(Arc::new(CustomExec::new(projections, self.clone())))
}

pub(crate) fn populate_users(&self) {
self.add_user(User {
id: 1,
bank_account: 9_000,
});
self.add_user(User {
id: 2,
bank_account: 100,
});
self.add_user(User {
id: 3,
bank_account: 1_000,
});
}

fn add_user(&self, user: User) {
let mut inner = self.inner.lock().unwrap();
inner.data.push(user);
}
}

impl Default for CustomDataSource {
fn default() -> Self {
CustomDataSource {
inner: Arc::new(Mutex::new(CustomDataSourceInner {
data: Default::default(),
})),
metadata_columns: Arc::new(Schema::new(vec![
Field::new("_rowid", DataType::UInt64, false),
Field::new("_file", DataType::Utf8, false),
])),
}
}
}

#[async_trait]
impl TableProvider for CustomDataSource {
fn as_any(&self) -> &dyn Any {
self
}

fn schema(&self) -> SchemaRef {
SchemaRef::new(Schema::new(vec![
Field::new("id", DataType::UInt8, false),
Field::new("bank_account", DataType::UInt64, true),
]))
}

fn metadata_columns(&self) -> Option<SchemaRef> {
Some(self.metadata_columns.clone())
}

fn table_type(&self) -> TableType {
TableType::Base
}

async fn scan(
&self,
_state: &dyn Session,
projection: Option<&Vec<usize>>,
Copy link
Contributor

Choose a reason for hiding this comment

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

@adriangb If we want to avoid METAOFFSET, we might need breaking change for project_meta field, which I remember this method is widely use, so I'm not sure is there any way to minimize the breaking change and hide the METAOFFSET internally

Copy link
Contributor

@jayzhan211 jayzhan211 Jan 29, 2025

Choose a reason for hiding this comment

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

Field related API is also widely used, if we want to make METADATA_OFFSET private. We have two options: either introduce a breaking change or add field_with_meta, which could also create confusion for the user

    /// Returns an immutable reference of a specific `Field` instance selected using an
    /// offset within the internal `fields` vector
    pub fn field(&self, i: usize) -> &Field {
        if i >= METADATA_OFFSET {
            if let Some(metadata) = &self.metadata {
                return metadata.field(i - METADATA_OFFSET);
            }
        }
        self.inner.field(i)
    }

    /// Returns an immutable reference of a specific `Field` instance selected using an
    /// offset within the internal `fields` vector and its qualifier
    pub fn qualified_field(&self, i: usize) -> (Option<&TableReference>, &Field) {
        if i >= METADATA_OFFSET {
            if let Some(metadata) = &self.metadata {
                return metadata.qualified_field(i - METADATA_OFFSET);
            }
        }
        self.inner.qualified_field(i)
    }

// filters and limit can be used here to inject some push-down operations if needed
_filters: &[Expr],
_limit: Option<usize>,
) -> Result<Arc<dyn ExecutionPlan>> {
return self.create_physical_plan(projection).await;
}
}

#[derive(Debug, Clone)]
struct CustomExec {
db: CustomDataSource,
projected_schema: SchemaRef,
cache: PlanProperties,
}

impl CustomExec {
fn new(projections: Option<&Vec<usize>>, db: CustomDataSource) -> Self {
let schema = db.schema();
let metadata_schema = db.metadata_columns();
let projected_schema = match projections {
Some(projection) => {
let projection = projection
.iter()
.map(|idx| match FieldId::from(*idx) {
FieldId::Normal(i) => Arc::new(schema.field(i).clone()),
FieldId::Metadata(i) => {
Arc::new(metadata_schema.as_ref().unwrap().field(i).clone())
}
})
.collect_vec();
Arc::new(Schema::new(projection))
}
None => schema,
};
let cache = Self::compute_properties(projected_schema.clone());
Self {
db,
projected_schema,
cache,
}
}

/// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc.
fn compute_properties(schema: SchemaRef) -> PlanProperties {
let eq_properties = EquivalenceProperties::new(schema);
PlanProperties::new(
eq_properties,
Partitioning::UnknownPartitioning(1),
EmissionType::Incremental,
Boundedness::Bounded,
)
}
}

impl DisplayAs for CustomExec {
fn fmt_as(&self, _t: DisplayFormatType, f: &mut Formatter) -> fmt::Result {
write!(f, "CustomExec")
}
}

impl ExecutionPlan for CustomExec {
fn name(&self) -> &'static str {
"CustomExec"
}

fn as_any(&self) -> &dyn Any {
self
}

fn properties(&self) -> &PlanProperties {
&self.cache
}

fn children(&self) -> Vec<&Arc<dyn ExecutionPlan>> {
vec![]
}

fn with_new_children(
self: Arc<Self>,
_: Vec<Arc<dyn ExecutionPlan>>,
) -> Result<Arc<dyn ExecutionPlan>> {
Ok(self)
}

fn execute(
&self,
_partition: usize,
_context: Arc<TaskContext>,
) -> Result<SendableRecordBatchStream> {
let users: Vec<User> = {
let db = self.db.inner.lock().unwrap();
db.data.clone()
};

let mut id_array = UInt8Builder::with_capacity(users.len());
let mut account_array = UInt64Builder::with_capacity(users.len());
let len = users.len() as u64;

for user in users {
id_array.append_value(user.id);
account_array.append_value(user.bank_account);
}

let id_array = id_array.finish();
let account_array = account_array.finish();
let rowid_array = UInt64Array::from_iter_values(0_u64..len);
let file_array =
StringArray::from_iter_values((0_u64..len).map(|i| format!("file-{}", i)));

let arrays = self
.projected_schema
.fields
.iter()
.map(|f| match f.name().as_str() {
"_rowid" => Arc::new(rowid_array.clone()) as ArrayRef,
"id" => Arc::new(id_array.clone()) as ArrayRef,
"bank_account" => Arc::new(account_array.clone()) as ArrayRef,
"_file" => Arc::new(file_array.clone()) as ArrayRef,
_ => panic!("cannot reach here"),
})
.collect();

Ok(Box::pin(MemoryStream::try_new(
vec![RecordBatch::try_new(self.projected_schema.clone(), arrays)?],
self.schema(),
None,
)?))
}
}
45 changes: 45 additions & 0 deletions datafusion/catalog/src/table.rs
Original file line number Diff line number Diff line change
Expand Up @@ -55,6 +55,51 @@ pub trait TableProvider: Debug + Sync + Send {
/// Get a reference to the schema for this table
fn schema(&self) -> SchemaRef;

/// Return a reference to the schema for metadata columns.
///
/// Metadata columns are columns which meant to be semi-public stores of the internal details of the table.
/// For example, `ctid` in Postgres would be considered a metadata column
/// (Postgres calls these "system columns", see [the Postgres docs](https://www.postgresql.org/docs/current/ddl-system-columns.html) for more information and examples.
/// Spark has a `_metadata` column that it uses to include details about each file read in a query (see [Spark's docs](https://docs.databricks.com/en/ingestion/file-metadata-column.html)).
///
/// You can use this method to declare which columns in the table are "metadata" columns.
/// See `datafusion/core/tests/sql/metadata_columns.rs` for an example of this in action.
///
/// As an example of how this works in practice, if you have the following Postgres table:
///
/// ```sql
/// CREATE TABLE t (x int);
/// INSERT INTO t VALUES (1);
/// ```
///
/// And you do a `SELECT * FROM t`, you would get the following schema:
///
/// ```text
/// +---+
/// | x |
/// +---+
/// | 1 |
/// +---+
/// ```
///
/// But if you do `SELECT ctid, * FROM t`, you would get the following schema (ignore the meaning of the value of `ctid`, this is just an example):
///
/// ```text
/// +-----+---+
/// | ctid| x |
/// +-----+---+
/// | 0 | 1 |
/// +-----+---+
/// ```
///
/// Returns:
/// - `None` for tables that do not have metadata columns.
/// - `Some(SchemaRef)` for tables having metadata columns.
/// The returned schema should be be the schema of _only_ the metadata columns, not the full schema.
fn metadata_columns(&self) -> Option<SchemaRef> {
Copy link
Contributor

Choose a reason for hiding this comment

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

Can you please document this better -- specifically:

  1. A link to the prior art (spark metadata columns)
  2. A brief summary of what metadata columns are used for and an example (you can copy the content from the spark docs)

None
}

/// Get a reference to the constraints of the table.
/// Returns:
/// - `None` for tables that do not support constraints.
Expand Down
Loading