-
Notifications
You must be signed in to change notification settings - Fork 1.3k
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
chenkovsky
wants to merge
7
commits into
apache:main
Choose a base branch
from
chenkovsky:feature/metadata_columns
base: main
Could not load branches
Branch not found: {{ refName }}
Loading
Could not load tags
Nothing to show
Loading
Are you sure you want to change the base?
Some commits from the old base branch may be removed from the timeline,
and old review comments may become outdated.
Open
feat: metadata columns #14057
Changes from all commits
Commits
Show all changes
7 commits
Select commit
Hold shift + click to select a range
5bfc297
feat: metadata columns
chenkovsky 05a475b
format
chenkovsky 8e73cbe
format code
chenkovsky e9a0d6f
update metadata offset
chenkovsky a4dee3e
update
chenkovsky 1ab8c7d
add document, refine test
chenkovsky 5c4b5c4
add example
chenkovsky File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,330 @@ | ||
// 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 arrow_schema::SchemaBuilder; | ||
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::{ | ||
project_schema, DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, | ||
PlanProperties, SendableRecordBatchStream, | ||
}; | ||
|
||
use datafusion::prelude::*; | ||
|
||
use datafusion::catalog::Session; | ||
use datafusion_common::METADATA_OFFSET; | ||
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>>, | ||
schema: SchemaRef, | ||
) -> Result<Arc<dyn ExecutionPlan>> { | ||
Ok(Arc::new(CustomExec::new(projections, schema, 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>>, | ||
// filters and limit can be used here to inject some push-down operations if needed | ||
_filters: &[Expr], | ||
_limit: Option<usize>, | ||
) -> Result<Arc<dyn ExecutionPlan>> { | ||
let mut schema = self.schema(); | ||
let size = schema.fields.len(); | ||
if let Some(metadata) = self.metadata_columns() { | ||
let mut builder = SchemaBuilder::from(schema.as_ref()); | ||
for f in metadata.fields.iter() { | ||
builder.try_merge(f)?; | ||
} | ||
schema = Arc::new(builder.finish()); | ||
} | ||
|
||
let projection = match projection { | ||
Some(projection) => { | ||
let projection = projection | ||
.iter() | ||
.map(|idx| { | ||
if *idx >= METADATA_OFFSET { | ||
*idx - METADATA_OFFSET + size | ||
} else { | ||
*idx | ||
} | ||
}) | ||
.collect_vec(); | ||
Some(projection) | ||
} | ||
None => None, | ||
}; | ||
return self.create_physical_plan(projection.as_ref(), schema).await; | ||
} | ||
} | ||
|
||
#[derive(Debug, Clone)] | ||
struct CustomExec { | ||
db: CustomDataSource, | ||
projected_schema: SchemaRef, | ||
cache: PlanProperties, | ||
} | ||
|
||
impl CustomExec { | ||
fn new( | ||
projections: Option<&Vec<usize>>, | ||
schema: SchemaRef, | ||
db: CustomDataSource, | ||
) -> Self { | ||
let projected_schema = project_schema(&schema, projections).unwrap(); | ||
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, | ||
)?)) | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
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: