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

ARROW-5227: [Rust] [DataFusion] Parallel Query Execution #4221

Closed
wants to merge 3 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
1 change: 1 addition & 0 deletions rust/datafusion/Cargo.toml
Expand Up @@ -49,6 +49,7 @@ sqlparser = "0.2.0"
clap = "2.33.0"
rustyline = "3.0.0"
prettytable-rs = "0.8.0"
crossbeam = "0.7.1"

[dev-dependencies]
criterion = "0.2.0"
Expand Down
48 changes: 48 additions & 0 deletions rust/datafusion/examples/parallel_query.rs
@@ -0,0 +1,48 @@
// 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::env;
use std::sync::Arc;

extern crate arrow;
extern crate datafusion;

use arrow::array::{BinaryArray, Float64Array};
use arrow::datatypes::{DataType, Field, Schema};

use datafusion::error::Result;
use datafusion::execution::context::ExecutionContext;

/// This example demonstrates executing a simple query against an Arrow data source and
/// fetching results
fn main() -> Result<()> {

let mut ctx = ExecutionContext::new();
ctx.register_parquet(
"tripdata",
"/home/andy/nyc-tripdata/parquet/year=2018/month=12/part-00000-0d20ebb7-f5fb-44e6-ae34-3a3e51060792-c000.snappy.parquet", // hack to pick up schema
)?;

let logical_plan = ctx.create_logical_plan("SELECT passenger_count, MIN(fare_amount), MAX(fare_amount) FROM tripdata GROUP BY passenger_count")?;
let optimized_plan = ctx.optimize(&logical_plan)?;
println!("{:?}", optimized_plan);

let physical_plan = ctx.create_physical_plan(&optimized_plan)?;
let result = ctx.execute_physical_plan(physical_plan)?;

Ok(())
}
239 changes: 237 additions & 2 deletions rust/datafusion/src/execution/context.rs
Expand Up @@ -20,23 +20,34 @@

use std::cell::RefCell;
use std::collections::HashMap;
use std::fs;
use std::rc::Rc;
use std::string::String;
use std::sync::Arc;
use std::thread;
use std::thread::JoinHandle;

use arrow::datatypes::*;
use arrow::record_batch::RecordBatch;

use sqlparser::sqlast::{SQLColumnDef, SQLType};

use crossbeam::channel::{unbounded, Receiver, Sender};

use crate::arrow::array::ArrayRef;
use crate::arrow::builder::BooleanBuilder;
use crate::datasource::csv::CsvFile;
use crate::datasource::parquet::ParquetTable;
use crate::datasource::TableProvider;
use crate::error::{ExecutionError, Result};
use crate::execution::aggregate::AggregateRelation;
use crate::execution::datasource::DataSourceRelation;
use crate::execution::expression::*;
use crate::execution::filter::FilterRelation;
use crate::execution::limit::LimitRelation;
use crate::execution::physical_plan::{BatchIterator, ExecutionPlan, Partition};
use crate::execution::projection::ProjectRelation;
use crate::execution::relation::{DataSourceRelation, Relation};
use crate::execution::relation::Relation;
use crate::execution::scalar_relation::ScalarRelation;
use crate::execution::table_impl::TableImpl;
use crate::logicalplan::*;
Expand All @@ -48,7 +59,158 @@ use crate::sql::parser::FileType;
use crate::sql::parser::{DFASTNode, DFParser};
use crate::sql::planner::{SchemaProvider, SqlToRel};
use crate::table::Table;
use sqlparser::sqlast::{SQLColumnDef, SQLType};

struct ParquetScanExec {
filename: String,
schema: Arc<Schema>,
}

impl ParquetScanExec {
/// Recursively build a list of parquet files in a directory
fn build_file_list(&self, dir: &str, filenames: &mut Vec<String>) -> Result<()> {
for entry in fs::read_dir(dir)? {
let entry = entry?;
let path = entry.path();
let path_name = path.as_os_str().to_str().unwrap();
if path.is_dir() {
self.build_file_list(path_name, filenames)?;
} else {
if path_name.ends_with(".parquet") {
filenames.push(path_name.to_string());
}
}
}
Ok(())
}
}

impl ExecutionPlan for ParquetScanExec {
fn schema(&self) -> Arc<Schema> {
self.schema.clone()
}

fn partitions(&self) -> Result<Vec<Arc<Partition>>> {
let mut filenames: Vec<String> = vec![];
self.build_file_list(&self.filename, &mut filenames)?;
let partitions = filenames
.iter()
.map(|filename| {
Arc::new(ParquetScanPartition::new(filename)) as Arc<Partition>
})
.collect();
Ok(partitions)
}
}

struct ParquetScanPartition {
request_tx: Sender<()>,
response_rx: Receiver<Result<Option<RecordBatch>>>,
}

impl ParquetScanPartition {
pub fn new(filename: &str) -> Self {
let (request_tx, request_rx): (Sender<()>, Receiver<()>) = unbounded();
let (response_tx, response_rx): (
Sender<Result<Option<RecordBatch>>>,
Receiver<Result<Option<RecordBatch>>>,
) = unbounded();

let filename = filename.to_string();
thread::spawn(move || {
//TODO reimplement to remove mutexes

println!("Opening {}", filename);

let table = ParquetTable::try_new(&filename).unwrap();
let partitions = table.scan(&None, 128 * 1024).unwrap();
let partition = partitions[0].clone();
while let Ok(_) = request_rx.recv() {
let mut partition = partition.lock().unwrap();
response_tx.send(partition.next()).unwrap();
}
});

Self {
request_tx,
response_rx,
}
}
}

impl Partition for ParquetScanPartition {
fn execute(&self) -> Result<Arc<BatchIterator>> {
Ok(Arc::new(TablePartitionIterator {
request_tx: self.request_tx.clone(),
response_rx: self.response_rx.clone(),
}))
}
}

pub struct TablePartitionIterator {
request_tx: Sender<()>,
response_rx: Receiver<Result<Option<RecordBatch>>>,
}

impl BatchIterator for TablePartitionIterator {
fn next(&self) -> Result<Option<RecordBatch>> {
self.request_tx.send(()).unwrap();
self.response_rx.recv().unwrap()
}
}

struct ProjectionExec {
input: Arc<ExecutionPlan>,
expr: Vec<CompiledExpr>,
}

impl ProjectionExec {
pub fn new(input: Arc<ExecutionPlan>, expr: Vec<CompiledExpr>) -> Self {
Self { input, expr }
}
}

impl ExecutionPlan for ProjectionExec {
fn schema(&self) -> Arc<Schema> {
unimplemented!()
}

fn partitions(&self) -> Result<Vec<Arc<Partition>>> {
//TODO: inject projection logic
Ok(self.input.partitions()?.iter().map(|p| p.clone()).collect())
}
}

struct FilterExec {
input: Arc<ExecutionPlan>,
expr: CompiledExpr,
}

impl FilterExec {
pub fn new(input: Arc<ExecutionPlan>, expr: CompiledExpr) -> Self {
Self { input, expr }
}
}

impl ExecutionPlan for FilterExec {
fn schema(&self) -> Arc<Schema> {
unimplemented!()
}

fn partitions(&self) -> Result<Vec<Arc<Partition>>> {
//TODO: inject filter logic
Ok(self.input.partitions()?.iter().map(|p| p.clone()).collect())
}
}

struct FilterPartition {
input: Arc<Partition>,
}

impl Partition for FilterPartition {
fn execute(&self) -> Result<Arc<BatchIterator>> {
unimplemented!()
}
}

/// Execution context for registering data sources and executing queries
pub struct ExecutionContext {
Expand Down Expand Up @@ -148,6 +310,79 @@ impl ExecutionContext {
}
}

pub fn create_physical_plan(
&mut self,
logical_plan: &Arc<LogicalPlan>,
) -> Result<Arc<ExecutionPlan>> {
match logical_plan.as_ref() {
LogicalPlan::TableScan { schema, .. } => {



let physical_plan = ParquetScanExec {
filename: "/home/andy/nyc-tripdata/parquet/year=2018"
.to_string(), /* TODO */
schema: schema.clone(),
};
Ok(Arc::new(physical_plan))
}
LogicalPlan::Projection { input, expr, .. } => {
let input = self.create_physical_plan(input)?;
let input_schema = input.as_ref().schema().clone();
let me = self;
let runtime_expr = expr
.iter()
.map(|e| compile_expr(&me, e, &input_schema))
.collect::<Result<Vec<_>>>()?;
Ok(Arc::new(ProjectionExec::new(input, runtime_expr)))
}
LogicalPlan::Selection { input, expr, .. } => {
let input = self.create_physical_plan(input)?;
let input_schema = input.as_ref().schema().clone();
let runtime_expr = compile_expr(&self, expr, &input_schema)?;
Ok(Arc::new(FilterExec::new(input, runtime_expr)))
}
_ => unimplemented!(),
}
}

pub fn execute_physical_plan(&mut self, plan: Arc<ExecutionPlan>) -> Result<()> {
// execute each partition on a thread
use std::sync::atomic::{AtomicUsize, Ordering};
let thread_id = AtomicUsize::new(1);
let threads: Vec<JoinHandle<Result<()>>> = plan
.partitions()?
.iter()
.map(|p| {
let thread_id = thread_id.fetch_add(1, Ordering::SeqCst);
let p = p.clone();
thread::spawn(move || {
let it = p.execute().unwrap();
while let Ok(Some(batch)) = it.next() {
println!(
"thread {} got batch with {} rows",
thread_id,
batch.num_rows()
);
}
Ok(())
})
})
.collect();

for thread in threads {
thread.join().unwrap().unwrap();
}

Ok(())
}

/// Register a Parquet file as a table so that it can be queried from SQL
pub fn register_parquet(&mut self, name: &str, filename: &str) -> Result<()> {
self.register_table(name, Rc::new(ParquetTable::try_new(filename)?));
Ok(())
}

/// Register a CSV file as a table so that it can be queried from SQL
pub fn register_csv(
&mut self,
Expand Down
50 changes: 50 additions & 0 deletions rust/datafusion/src/execution/datasource.rs
@@ -0,0 +1,50 @@
// 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.

//! Data source relation

use std::sync::{Arc, Mutex};

use arrow::datatypes::Schema;
use arrow::record_batch::RecordBatch;

use crate::datasource::RecordBatchIterator;
use crate::error::Result;
use crate::execution::relation::Relation;

/// Implementation of a relation that represents a DataFusion data source
pub struct DataSourceRelation {
schema: Arc<Schema>,
ds: Arc<Mutex<RecordBatchIterator>>,
}

impl DataSourceRelation {
pub fn new(ds: Arc<Mutex<RecordBatchIterator>>) -> Self {
let schema = ds.lock().unwrap().schema().clone();
Self { ds, schema }
}
}

impl Relation for DataSourceRelation {
fn next(&mut self) -> Result<Option<RecordBatch>> {
self.ds.lock().unwrap().next()
}

fn schema(&self) -> &Arc<Schema> {
&self.schema
}
}
2 changes: 2 additions & 0 deletions rust/datafusion/src/execution/mod.rs
Expand Up @@ -19,9 +19,11 @@

pub mod aggregate;
pub mod context;
pub mod datasource;
pub mod expression;
pub mod filter;
pub mod limit;
pub mod physical_plan;
pub mod projection;
pub mod relation;
pub mod scalar_relation;
Expand Down