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

Support modulus op #510

Closed
wants to merge 26 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
26 commits
Select commit Hold shift + click to select a range
97fd9ff
Support modulus op
gangliao Jun 5, 2021
b84789a
Optimize cast function during planning stage (#513)
msathis Jun 5, 2021
ee2b9ef
Fix display of execution time (#514)
Dandandan Jun 6, 2021
767eeb0
closing up type checks (#506)
jimexist Jun 7, 2021
5773a03
refactor sort exec stream and combine batches (#515)
jimexist Jun 7, 2021
63accf8
closing up type checks (#518)
jimexist Jun 7, 2021
2f73e79
Refactor window aggregation, simplify batch processing logic (#516)
jimexist Jun 7, 2021
e39f311
110 support group by positions (#519)
jychen7 Jun 8, 2021
8495f95
Wrong aggregation arguments error. (#505)
jgoday Jun 8, 2021
42f908e
Define the unittests using pytest (#493)
kszucs Jun 9, 2021
d5bca0e
Add `partition by` constructs in window functions and modify logical …
jimexist Jun 9, 2021
8f84564
Return errors properly from RepartitionExec (#521)
alamb Jun 10, 2021
77775b7
add boundary check (#530)
jimexist Jun 10, 2021
5c88450
remove redundant into_iter() calls (#527)
jimexist Jun 10, 2021
3ef7f34
use nightly nightly-2021-05-10 (#536)
jimexist Jun 10, 2021
63e3045
Make BallistaContext::collect streaming (#535)
edrevo Jun 11, 2021
ad70a1e
reuse datafusion physical planner in ballista building from protobuf …
jimexist Jun 11, 2021
8f4078d
ShuffleReaderExec now supports multiple locations per partition (#541)
andygrove Jun 12, 2021
519698a
Refactor hash aggregates's planner building code (#539)
jimexist Jun 13, 2021
738f13b
turn on clippy rule for needless borrow (#545)
jimexist Jun 13, 2021
2568323
#420: Support for not_eq predicate in pruning predicates (#544)
jgoday Jun 13, 2021
d382854
Cleanup Repartition Exec code (#538)
alamb Jun 13, 2021
91af820
support table alias in join clause (#547)
Jun 14, 2021
fe810e9
reuse code for now function expr creation (#548)
Jun 14, 2021
9c23de0
Support modulus op
gangliao Jun 5, 2021
73902b6
Merge branch 'mod' of https://github.com/DSLAM-UMD/arrow-datafusion i…
gangliao Jun 14, 2021
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
2 changes: 1 addition & 1 deletion .env
Original file line number Diff line number Diff line change
Expand Up @@ -47,7 +47,7 @@ FEDORA=33
PYTHON=3.6
LLVM=11
CLANG_TOOLS=8
RUST=nightly-2021-03-24
RUST=nightly-2021-05-10
GO=1.15
NODE=14
MAVEN=3.5.4
Expand Down
2 changes: 1 addition & 1 deletion .github/workflows/python_build.yml
Original file line number Diff line number Diff line change
Expand Up @@ -39,7 +39,7 @@ jobs:

- uses: actions-rs/toolchain@v1
with:
toolchain: nightly-2021-01-06
toolchain: nightly-2021-05-10

- name: Install dependencies
run: |
Expand Down
6 changes: 3 additions & 3 deletions .github/workflows/python_test.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -25,8 +25,8 @@ jobs:
- uses: actions/checkout@v2
- name: Setup Rust toolchain
run: |
rustup toolchain install nightly-2021-01-06
rustup default nightly-2021-01-06
rustup toolchain install nightly-2021-05-10
rustup default nightly-2021-05-10
rustup component add rustfmt
- name: Cache Cargo
uses: actions/cache@v2
Expand All @@ -53,7 +53,7 @@ jobs:
pip install -r requirements.txt
maturin develop

python -m unittest discover tests
pytest -v .
env:
CARGO_HOME: "/home/runner/.cargo"
CARGO_TARGET_DIR: "/home/runner/target"
82 changes: 42 additions & 40 deletions ballista/rust/client/src/context.rs
Original file line number Diff line number Diff line change
Expand Up @@ -24,13 +24,14 @@ use std::{collections::HashMap, convert::TryInto};
use std::{fs, time::Duration};

use ballista_core::serde::protobuf::scheduler_grpc_client::SchedulerGrpcClient;
use ballista_core::serde::protobuf::PartitionLocation;
use ballista_core::serde::protobuf::{
execute_query_params::Query, job_status, ExecuteQueryParams, GetJobStatusParams,
GetJobStatusResult,
};
use ballista_core::utils::WrappedStream;
use ballista_core::{
client::BallistaClient, datasource::DfTableAdapter, memory_stream::MemoryStream,
utils::create_datafusion_context,
client::BallistaClient, datasource::DfTableAdapter, utils::create_datafusion_context,
};

use datafusion::arrow::datatypes::Schema;
Expand All @@ -39,6 +40,8 @@ use datafusion::error::{DataFusionError, Result};
use datafusion::logical_plan::LogicalPlan;
use datafusion::physical_plan::csv::CsvReadOptions;
use datafusion::{dataframe::DataFrame, physical_plan::RecordBatchStream};
use futures::future;
use futures::StreamExt;
use log::{error, info};

#[allow(dead_code)]
Expand Down Expand Up @@ -155,6 +158,29 @@ impl BallistaContext {
ctx.sql(sql)
}

async fn fetch_partition(
location: PartitionLocation,
) -> Result<Pin<Box<dyn RecordBatchStream + Send + Sync>>> {
let metadata = location.executor_meta.ok_or_else(|| {
DataFusionError::Internal("Received empty executor metadata".to_owned())
})?;
let partition_id = location.partition_id.ok_or_else(|| {
DataFusionError::Internal("Received empty partition id".to_owned())
})?;
let mut ballista_client =
BallistaClient::try_new(metadata.host.as_str(), metadata.port as u16)
.await
.map_err(|e| DataFusionError::Execution(format!("{:?}", e)))?;
Ok(ballista_client
.fetch_partition(
&partition_id.job_id,
partition_id.stage_id as usize,
partition_id.partition_id as usize,
)
.await
.map_err(|e| DataFusionError::Execution(format!("{:?}", e)))?)
}

pub async fn collect(
&self,
plan: &LogicalPlan,
Expand Down Expand Up @@ -222,45 +248,21 @@ impl BallistaContext {
break Err(DataFusionError::Execution(msg));
}
job_status::Status::Completed(completed) => {
// TODO: use streaming. Probably need to change the signature of fetch_partition to achieve that
let mut result = vec![];
for location in completed.partition_location {
let metadata = location.executor_meta.ok_or_else(|| {
DataFusionError::Internal(
"Received empty executor metadata".to_owned(),
)
})?;
let partition_id = location.partition_id.ok_or_else(|| {
DataFusionError::Internal(
"Received empty partition id".to_owned(),
)
})?;
let mut ballista_client = BallistaClient::try_new(
metadata.host.as_str(),
metadata.port as u16,
)
.await
.map_err(|e| DataFusionError::Execution(format!("{:?}", e)))?;
let stream = ballista_client
.fetch_partition(
&partition_id.job_id,
partition_id.stage_id as usize,
partition_id.partition_id as usize,
)
.await
.map_err(|e| {
DataFusionError::Execution(format!("{:?}", e))
})?;
result.append(
&mut datafusion::physical_plan::common::collect(stream)
.await?,
);
}
break Ok(Box::pin(MemoryStream::try_new(
result,
let result = future::join_all(
completed
.partition_location
.into_iter()
.map(BallistaContext::fetch_partition),
)
.await
.into_iter()
.collect::<Result<Vec<_>>>()?;

let result = WrappedStream::new(
Box::pin(futures::stream::iter(result).flatten()),
Arc::new(schema),
None,
)?));
);
break Ok(Box::pin(result));
}
};
}
Expand Down
2 changes: 1 addition & 1 deletion ballista/rust/core/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,7 @@ futures = "0.3"
log = "0.4"
prost = "0.7"
serde = {version = "1", features = ["derive"]}
sqlparser = "0.8"
sqlparser = "0.9.0"
tokio = "1.0"
tonic = "0.4"
uuid = { version = "0.8", features = ["v4"] }
Expand Down
15 changes: 10 additions & 5 deletions ballista/rust/core/proto/ballista.proto
Original file line number Diff line number Diff line change
Expand Up @@ -174,12 +174,12 @@ message WindowExprNode {
// udaf = 3
}
LogicalExprNode expr = 4;
// repeated LogicalExprNode partition_by = 5;
repeated LogicalExprNode partition_by = 5;
repeated LogicalExprNode order_by = 6;
// repeated LogicalExprNode filter = 7;
// oneof window_frame {
// WindowFrame frame = 8;
// }
oneof window_frame {
WindowFrame frame = 8;
}
}

message BetweenNode {
Expand Down Expand Up @@ -489,10 +489,15 @@ message HashAggregateExecNode {
}

message ShuffleReaderExecNode {
repeated PartitionLocation partition_location = 1;
repeated ShuffleReaderPartition partition = 1;
Schema schema = 2;
}

message ShuffleReaderPartition {
// each partition of a shuffle read can read data from multiple locations
repeated PartitionLocation location = 1;
}

message GlobalLimitExecNode {
PhysicalPlanNode input = 1;
uint32 limit = 2;
Expand Down
2 changes: 1 addition & 1 deletion ballista/rust/core/src/execution_plans/query_stage.rs
Original file line number Diff line number Diff line change
Expand Up @@ -139,7 +139,7 @@ impl ExecutionPlan for QueryStageExec {
info!("Writing results to {}", path);

// stream results to disk
let stats = utils::write_stream_to_disk(&mut stream, &path)
let stats = utils::write_stream_to_disk(&mut stream, path)
.await
.map_err(|e| DataFusionError::Execution(format!("{:?}", e)))?;

Expand Down
94 changes: 57 additions & 37 deletions ballista/rust/core/src/execution_plans/shuffle_reader.rs
Original file line number Diff line number Diff line change
Expand Up @@ -15,42 +15,43 @@
// specific language governing permissions and limitations
// under the License.

use std::fmt::Formatter;
use std::sync::Arc;
use std::{any::Any, pin::Pin};

use crate::client::BallistaClient;
use crate::memory_stream::MemoryStream;
use crate::serde::scheduler::PartitionLocation;

use crate::utils::WrappedStream;
use async_trait::async_trait;
use datafusion::arrow::datatypes::SchemaRef;
use datafusion::arrow::error::Result as ArrowResult;
use datafusion::arrow::record_batch::RecordBatch;
use datafusion::physical_plan::{DisplayFormatType, ExecutionPlan, Partitioning};
use datafusion::{
error::{DataFusionError, Result},
physical_plan::RecordBatchStream,
};
use futures::{future, Stream, StreamExt};
use log::info;
use std::fmt::Formatter;

/// ShuffleReaderExec reads partitions that have already been materialized by an executor.
/// ShuffleReaderExec reads partitions that have already been materialized by a query stage
/// being executed by an executor
#[derive(Debug, Clone)]
pub struct ShuffleReaderExec {
// The query stage that is responsible for producing the shuffle partitions that
// this operator will read
pub(crate) partition_location: Vec<PartitionLocation>,
/// Each partition of a shuffle can read data from multiple locations
pub(crate) partition: Vec<Vec<PartitionLocation>>,
pub(crate) schema: SchemaRef,
}

impl ShuffleReaderExec {
/// Create a new ShuffleReaderExec
pub fn try_new(
partition_meta: Vec<PartitionLocation>,
partition: Vec<Vec<PartitionLocation>>,
schema: SchemaRef,
) -> Result<Self> {
Ok(Self {
partition_location: partition_meta,
schema,
})
Ok(Self { partition, schema })
}
}

Expand All @@ -65,7 +66,7 @@ impl ExecutionPlan for ShuffleReaderExec {
}

fn output_partitioning(&self) -> Partitioning {
Partitioning::UnknownPartitioning(self.partition_location.len())
Partitioning::UnknownPartitioning(self.partition.len())
}

fn children(&self) -> Vec<Arc<dyn ExecutionPlan>> {
Expand All @@ -86,23 +87,18 @@ impl ExecutionPlan for ShuffleReaderExec {
partition: usize,
) -> Result<Pin<Box<dyn RecordBatchStream + Send + Sync>>> {
info!("ShuffleReaderExec::execute({})", partition);
let partition_location = &self.partition_location[partition];

let mut client = BallistaClient::try_new(
&partition_location.executor_meta.host,
partition_location.executor_meta.port,
)
.await
.map_err(|e| DataFusionError::Execution(format!("Ballista Error: {:?}", e)))?;

client
.fetch_partition(
&partition_location.partition_id.job_id,
partition_location.partition_id.stage_id,
partition,
)
let partition_locations = &self.partition[partition];
let result = future::join_all(partition_locations.iter().map(fetch_partition))
.await
.map_err(|e| DataFusionError::Execution(format!("Ballista Error: {:?}", e)))
.into_iter()
.collect::<Result<Vec<_>>>()?;

let result = WrappedStream::new(
Box::pin(futures::stream::iter(result).flatten()),
Arc::new(self.schema.as_ref().clone()),
);
Ok(Box::pin(result))
}

fn fmt_as(
Expand All @@ -113,22 +109,46 @@ impl ExecutionPlan for ShuffleReaderExec {
match t {
DisplayFormatType::Default => {
let loc_str = self
.partition_location
.partition
.iter()
.map(|l| {
format!(
"[executor={} part={}:{}:{} stats={:?}]",
l.executor_meta.id,
l.partition_id.job_id,
l.partition_id.stage_id,
l.partition_id.partition_id,
l.partition_stats
)
.map(|x| {
x.iter()
.map(|l| {
format!(
"[executor={} part={}:{}:{} stats={:?}]",
l.executor_meta.id,
l.partition_id.job_id,
l.partition_id.stage_id,
l.partition_id.partition_id,
l.partition_stats
)
})
.collect::<Vec<String>>()
.join(",")
})
.collect::<Vec<String>>()
.join(",");
.join("\n");
write!(f, "ShuffleReaderExec: partition_locations={}", loc_str)
}
}
}
}

async fn fetch_partition(
location: &PartitionLocation,
) -> Result<Pin<Box<dyn RecordBatchStream + Send + Sync>>> {
let metadata = &location.executor_meta;
let partition_id = &location.partition_id;
let mut ballista_client =
BallistaClient::try_new(metadata.host.as_str(), metadata.port as u16)
.await
.map_err(|e| DataFusionError::Execution(format!("{:?}", e)))?;
Ok(ballista_client
.fetch_partition(
&partition_id.job_id,
partition_id.stage_id as usize,
partition_id.partition_id as usize,
)
.await
.map_err(|e| DataFusionError::Execution(format!("{:?}", e)))?)
}
Loading