Skip to content
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.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
1 change: 0 additions & 1 deletion rust/arrow/src/array/builder.rs
Original file line number Diff line number Diff line change
Expand Up @@ -1458,5 +1458,4 @@ mod tests {
let mut builder = StructBuilder::new(fields, field_builders);
assert!(builder.field_builder::<BinaryBuilder>(0).is_none());
}

}
1 change: 0 additions & 1 deletion rust/arrow/src/bitmap.rs
Original file line number Diff line number Diff line change
Expand Up @@ -122,5 +122,4 @@ mod tests {
assert_eq!(true, bitmap.is_set(6));
assert_eq!(false, bitmap.is_set(7));
}

}
1 change: 0 additions & 1 deletion rust/datafusion/src/execution/aggregate.rs
Original file line number Diff line number Diff line change
Expand Up @@ -1471,5 +1471,4 @@ mod tests {
ds,
)))))
}

}
1 change: 0 additions & 1 deletion rust/datafusion/src/execution/context.rs
Original file line number Diff line number Diff line change
Expand Up @@ -682,5 +682,4 @@ mod tests {

Ok(ctx)
}

}
18 changes: 4 additions & 14 deletions rust/datafusion/src/execution/limit.rs
Original file line number Diff line number Diff line change
Expand Up @@ -21,12 +21,11 @@ use std::cell::RefCell;
use std::rc::Rc;
use std::sync::Arc;

use arrow::array::*;
use arrow::compute::array_ops::limit;
use arrow::datatypes::Schema;
use arrow::record_batch::RecordBatch;

use crate::error::{ExecutionError, Result};
use crate::error::Result;
use crate::execution::physical_plan::limit::truncate_batch;
use crate::execution::relation::Relation;

/// Implementation of a LIMIT relation
Expand Down Expand Up @@ -67,18 +66,9 @@ impl Relation for LimitRelation {
return Ok(None);
}

if batch.num_rows() >= capacity {
let limited_columns: Result<Vec<ArrayRef>> = (0..batch.num_columns())
.map(|i| match limit(batch.column(i), capacity) {
Ok(result) => Ok(result),
Err(error) => Err(ExecutionError::from(error)),
})
.collect();

let limited_batch: RecordBatch =
RecordBatch::try_new(self.schema.clone(), limited_columns?)?;
if batch.num_rows() > capacity {
let limited_batch = truncate_batch(&batch, capacity)?;
self.num_consumed_rows += capacity;

Ok(Some(limited_batch))
} else {
self.num_consumed_rows += batch.num_rows();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -720,5 +720,4 @@ mod tests {

Ok(())
}

}
208 changes: 208 additions & 0 deletions rust/datafusion/src/execution/physical_plan/limit.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,208 @@
// 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.

//! Defines the LIMIT plan

use crate::error::{ExecutionError, Result};
use crate::execution::physical_plan::common::RecordBatchIterator;
use crate::execution::physical_plan::ExecutionPlan;
use crate::execution::physical_plan::{BatchIterator, Partition};
use arrow::array::ArrayRef;
use arrow::compute::array_ops::limit;
use arrow::datatypes::Schema;
use arrow::record_batch::RecordBatch;
use std::sync::{Arc, Mutex};
use std::thread;
use std::thread::JoinHandle;

/// Limit execution plan
pub struct LimitExec {
/// Input schema
schema: Arc<Schema>,
/// Input partitions
partitions: Vec<Arc<dyn Partition>>,
/// Maximum number of rows to return
limit: usize,
}

impl LimitExec {
/// Create a new MergeExec
pub fn new(
schema: Arc<Schema>,
partitions: Vec<Arc<dyn Partition>>,
limit: usize,
) -> Self {
LimitExec {
schema,
partitions,
limit,
}
}
}

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

fn partitions(&self) -> Result<Vec<Arc<dyn Partition>>> {
Ok(vec![Arc::new(LimitPartition {
schema: self.schema.clone(),
partitions: self.partitions.clone(),
limit: self.limit,
})])
}
}

struct LimitPartition {
/// Input schema
schema: Arc<Schema>,
/// Input partitions
partitions: Vec<Arc<dyn Partition>>,
/// Maximum number of rows to return
limit: usize,
}

impl Partition for LimitPartition {
fn execute(&self) -> Result<Arc<Mutex<dyn BatchIterator>>> {
// collect up to "limit" rows on each partition
let threads: Vec<JoinHandle<Result<Vec<RecordBatch>>>> = self
.partitions
.iter()
Copy link
Contributor

Choose a reason for hiding this comment

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

Would be an overkill to use rayon's par_iter()?

Copy link
Member Author

Choose a reason for hiding this comment

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

For this round my plan was to keep things simple and do a second round of optimizations that I've started tracking in https://jira.apache.org/jira/browse/ARROW-6689 and in particular see the subtask https://jira.apache.org/jira/browse/ARROW-6691 related to threading.

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 that's a "no" for rayon :D

.map(|p| {
let p = p.clone();
let limit = self.limit;
thread::spawn(move || {
let it = p.execute()?;
collect_with_limit(it, limit)
})
})
.collect();

// combine the results from each thread, up to the limit
let mut combined_results: Vec<Arc<RecordBatch>> = vec![];
let mut count = 0;
for thread in threads {
let join = thread.join().expect("Failed to join thread");
let result = join?;
for batch in result {
let capacity = self.limit - count;
if batch.num_rows() <= capacity {
count += batch.num_rows();
combined_results.push(Arc::new(batch.clone()))
} else {
let batch = truncate_batch(&batch, capacity)?;
count += batch.num_rows();
combined_results.push(Arc::new(batch.clone()))
}
if count == self.limit {
break;
}
}
}

Ok(Arc::new(Mutex::new(RecordBatchIterator::new(
self.schema.clone(),
combined_results,
))))
}
}

/// Truncate a RecordBatch to maximum of n rows
pub fn truncate_batch(batch: &RecordBatch, n: usize) -> Result<RecordBatch> {
let limited_columns: Result<Vec<ArrayRef>> = (0..batch.num_columns())
.map(|i| match limit(batch.column(i), n) {
Ok(result) => Ok(result),
Err(error) => Err(ExecutionError::from(error)),
})
.collect();

Ok(RecordBatch::try_new(
batch.schema().clone(),
limited_columns?,
)?)
}

/// Create a vector of record batches from an iterator
fn collect_with_limit(
it: Arc<Mutex<dyn BatchIterator>>,
limit: usize,
) -> Result<Vec<RecordBatch>> {
let mut count = 0;
let mut it = it.lock().unwrap();
let mut results: Vec<RecordBatch> = vec![];
loop {
match it.next() {
Ok(Some(batch)) => {
let capacity = limit - count;
if batch.num_rows() <= capacity {
count += batch.num_rows();
results.push(batch);
} else {
let batch = truncate_batch(&batch, capacity)?;
count += batch.num_rows();
results.push(batch);
}
if count == limit {
return Ok(results);
}
}
Ok(None) => {
// end of result set
return Ok(results);
}
Err(e) => return Err(e),
}
}
}

#[cfg(test)]
mod tests {

use super::*;
use crate::execution::physical_plan::common;
use crate::execution::physical_plan::csv::CsvExec;
use crate::test;

#[test]
fn limit() -> Result<()> {
let schema = test::aggr_test_schema();

let num_partitions = 4;
let path =
test::create_partitioned_csv("aggregate_test_100.csv", num_partitions)?;

let csv = CsvExec::try_new(&path, schema.clone(), true, None, 1024)?;

// input should have 4 partitions
let input = csv.partitions()?;
assert_eq!(input.len(), num_partitions);

let limit = LimitExec::new(schema.clone(), input, 7);
let partitions = limit.partitions()?;

// the result should contain 4 batches (one per input partition)
let iter = partitions[0].execute()?;
let batches = common::collect(iter)?;

// there should be a total of 100 rows
let row_count: usize = batches.iter().map(|batch| batch.num_rows()).sum();
assert_eq!(row_count, 7);

Ok(())
}
}
1 change: 0 additions & 1 deletion rust/datafusion/src/execution/physical_plan/merge.rs
Original file line number Diff line number Diff line change
Expand Up @@ -134,5 +134,4 @@ mod tests {

Ok(())
}

}
1 change: 1 addition & 0 deletions rust/datafusion/src/execution/physical_plan/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -86,6 +86,7 @@ pub mod csv;
pub mod datasource;
pub mod expressions;
pub mod hash_aggregate;
pub mod limit;
pub mod merge;
pub mod projection;
pub mod selection;
1 change: 0 additions & 1 deletion rust/datafusion/src/execution/physical_plan/projection.rs
Original file line number Diff line number Diff line change
Expand Up @@ -171,5 +171,4 @@ mod tests {

Ok(())
}

}
1 change: 0 additions & 1 deletion rust/datafusion/src/execution/physical_plan/selection.rs
Original file line number Diff line number Diff line change
Expand Up @@ -180,5 +180,4 @@ mod tests {

Ok(())
}

}
1 change: 0 additions & 1 deletion rust/datafusion/src/execution/projection.rs
Original file line number Diff line number Diff line change
Expand Up @@ -141,5 +141,4 @@ mod tests {

assert_eq!("c1", batch.schema().field(0).name());
}

}
1 change: 0 additions & 1 deletion rust/datafusion/src/execution/table_impl.rs
Original file line number Diff line number Diff line change
Expand Up @@ -315,5 +315,4 @@ mod tests {
true,
);
}

}
1 change: 0 additions & 1 deletion rust/datafusion/src/logicalplan.rs
Original file line number Diff line number Diff line change
Expand Up @@ -641,5 +641,4 @@ mod tests {
println!("plan: {:?}", plan1);
});
}

}
1 change: 0 additions & 1 deletion rust/datafusion/src/sql/planner.rs
Original file line number Diff line number Diff line change
Expand Up @@ -569,5 +569,4 @@ mod tests {
}
}
}

}
1 change: 0 additions & 1 deletion rust/parquet/src/compression.rs
Original file line number Diff line number Diff line change
Expand Up @@ -338,5 +338,4 @@ mod tests {
fn test_codec_zstd() {
test_codec(CodecType::ZSTD);
}

}
2 changes: 1 addition & 1 deletion rust/rust-toolchain
Original file line number Diff line number Diff line change
@@ -1 +1 @@
nightly-2019-07-30
nightly-2019-09-25