Skip to content

Commit

Permalink
Merge remote-tracking branch 'upstream/main' into hash_join
Browse files Browse the repository at this point in the history
  • Loading branch information
viirya committed Mar 18, 2024
2 parents b573f53 + 8aab44c commit a236051
Show file tree
Hide file tree
Showing 47 changed files with 4,046 additions and 3,300 deletions.
19 changes: 18 additions & 1 deletion README.md
Original file line number Diff line number Diff line change
Expand Up @@ -113,4 +113,21 @@ INFO src/lib.rs: Comet native library initialized
+- CometScan parquet [a#14] Batched: true, DataFilters: [isnotnull(a#14), (a#14 > 5)],
Format: CometParquet, Location: InMemoryFileIndex(1 paths)[file:/tmp/test], PartitionFilters: [],
PushedFilters: [IsNotNull(a), GreaterThan(a,5)], ReadSchema: struct<a:int>
```
```

### Enable Comet shuffle

Comet shuffle feature is disabled by default. To enable it, please add related configs:

```
--conf spark.shuffle.manager=org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager
--conf spark.comet.exec.shuffle.enabled=true
```

Above configs enable Comet native shuffle which only supports hash partiting and single partition.
Comet native shuffle doesn't support complext types yet.

To enable columnar shuffle which supports all partitioning and basic complex types, one more config is required:
```
--conf spark.comet.columnar.shuffle.enabled=true
```
Original file line number Diff line number Diff line change
Expand Up @@ -105,15 +105,13 @@ public UTF8String getUTF8String(int rowId) {
int length = Platform.getInt(null, offsetBufferAddress + (rowId + 1L) * 4L) - offset;
return UTF8String.fromAddress(null, valueBufferAddress + offset, length);
} else {
// Iceberg maps UUID to StringType.
// The data type here must be UUID because the only FLBA -> String mapping we have is UUID.
BaseFixedWidthVector fixedWidthVector = (BaseFixedWidthVector) valueVector;
int length = fixedWidthVector.getTypeWidth();
int offset = rowId * length;
byte[] result = new byte[length];
Platform.copyMemory(
null, valueBufferAddress + offset, result, Platform.BYTE_ARRAY_OFFSET, length);
return UTF8String.fromString(convertToUuid(result).toString());
return UTF8String.fromBytes(result);
}
}

Expand Down
11 changes: 11 additions & 0 deletions core/src/common/bit.rs
Original file line number Diff line number Diff line change
Expand Up @@ -131,6 +131,17 @@ pub fn read_num_bytes_u32(size: usize, src: &[u8]) -> u32 {
trailing_bits(v as u64, size * 8) as u32
}

/// Similar to the `read_num_bytes` but read nums from bytes in big-endian order
/// This is used to read bytes from Java's OutputStream which writes bytes in big-endian
macro_rules! read_num_be_bytes {
($ty:ty, $size:expr, $src:expr) => {{
debug_assert!($size <= $src.len());
let mut buffer = <$ty as $crate::common::bit::FromBytes>::Buffer::default();
buffer.as_mut()[..$size].copy_from_slice(&$src[..$size]);
<$ty>::from_be_bytes(buffer)
}};
}

/// Converts value `val` of type `T` to a byte vector, by reading `num_bytes` from `val`.
/// NOTE: if `val` is less than the size of `T` then it can be truncated.
#[inline]
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,152 @@
// 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 crate::{
execution::datafusion::util::spark_bloom_filter::SparkBloomFilter, parquet::data_type::AsBytes,
};
use arrow::record_batch::RecordBatch;
use arrow_array::cast::as_primitive_array;
use arrow_schema::{DataType, Schema};
use datafusion::physical_plan::ColumnarValue;
use datafusion_common::{internal_err, DataFusionError, Result, ScalarValue};
use datafusion_physical_expr::{aggregate::utils::down_cast_any_ref, PhysicalExpr};
use std::{
any::Any,
fmt::Display,
hash::{Hash, Hasher},
sync::Arc,
};

/// A physical expression that checks if a value might be in a bloom filter. It corresponds to the
/// Spark's `BloomFilterMightContain` expression.
#[derive(Debug, Hash)]
pub struct BloomFilterMightContain {
pub bloom_filter_expr: Arc<dyn PhysicalExpr>,
pub value_expr: Arc<dyn PhysicalExpr>,
bloom_filter: Option<SparkBloomFilter>,
}

impl Display for BloomFilterMightContain {
fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result {
write!(
f,
"BloomFilterMightContain [bloom_filter_expr: {}, value_expr: {}]",
self.bloom_filter_expr, self.value_expr
)
}
}

impl PartialEq<dyn Any> for BloomFilterMightContain {
fn eq(&self, _other: &dyn Any) -> bool {
down_cast_any_ref(_other)
.downcast_ref::<Self>()
.map(|other| {
self.bloom_filter_expr.eq(&other.bloom_filter_expr)
&& self.value_expr.eq(&other.value_expr)
})
.unwrap_or(false)
}
}

fn evaluate_bloom_filter(
bloom_filter_expr: &Arc<dyn PhysicalExpr>,
) -> Result<Option<SparkBloomFilter>> {
// bloom_filter_expr must be a literal/scalar subquery expression, so we can evaluate it
// with an empty batch with empty schema
let batch = RecordBatch::new_empty(Arc::new(Schema::empty()));
let bloom_filter_bytes = bloom_filter_expr.evaluate(&batch)?;
match bloom_filter_bytes {
ColumnarValue::Scalar(ScalarValue::Binary(v)) => {
Ok(v.map(|v| SparkBloomFilter::new(v.as_bytes())))
}
_ => internal_err!("Bloom filter expression should be evaluated as a scalar binary value"),
}
}

impl BloomFilterMightContain {
pub fn try_new(
bloom_filter_expr: Arc<dyn PhysicalExpr>,
value_expr: Arc<dyn PhysicalExpr>,
) -> Result<Self> {
// early evaluate the bloom_filter_expr to get the actual bloom filter
let bloom_filter = evaluate_bloom_filter(&bloom_filter_expr)?;
Ok(Self {
bloom_filter_expr,
value_expr,
bloom_filter,
})
}
}

impl PhysicalExpr for BloomFilterMightContain {
fn as_any(&self) -> &dyn Any {
self
}

fn data_type(&self, _input_schema: &Schema) -> Result<DataType> {
Ok(DataType::Boolean)
}

fn nullable(&self, _input_schema: &Schema) -> Result<bool> {
Ok(true)
}

fn evaluate(&self, batch: &RecordBatch) -> Result<ColumnarValue> {
self.bloom_filter
.as_ref()
.map(|spark_filter| {
let values = self.value_expr.evaluate(batch)?;
match values {
ColumnarValue::Array(array) => {
let boolean_array =
spark_filter.might_contain_longs(as_primitive_array(&array));
Ok(ColumnarValue::Array(Arc::new(boolean_array)))
}
ColumnarValue::Scalar(ScalarValue::Int64(v)) => {
let result = v.map(|v| spark_filter.might_contain_long(v));
Ok(ColumnarValue::Scalar(ScalarValue::Boolean(result)))
}
_ => internal_err!("value expression should be int64 type"),
}
})
.unwrap_or_else(|| {
// when the bloom filter is null, we should return null for all the input
Ok(ColumnarValue::Scalar(ScalarValue::Boolean(None)))
})
}

fn children(&self) -> Vec<Arc<dyn PhysicalExpr>> {
vec![self.bloom_filter_expr.clone(), self.value_expr.clone()]
}

fn with_new_children(
self: Arc<Self>,
children: Vec<Arc<dyn PhysicalExpr>>,
) -> Result<Arc<dyn PhysicalExpr>> {
Ok(Arc::new(BloomFilterMightContain::try_new(
children[0].clone(),
children[1].clone(),
)?))
}

fn dyn_hash(&self, state: &mut dyn Hasher) {
let mut s = state;
self.bloom_filter_expr.hash(&mut s);
self.value_expr.hash(&mut s);
self.hash(&mut s);
}
}
1 change: 1 addition & 0 deletions core/src/execution/datafusion/expressions/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ pub mod scalar_funcs;
pub use normalize_nan::NormalizeNaNAndZero;
pub mod avg;
pub mod avg_decimal;
pub mod bloom_filter_might_contain;
pub mod strings;
pub mod subquery;
pub mod sum_decimal;
Expand Down
1 change: 1 addition & 0 deletions core/src/execution/datafusion/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -22,3 +22,4 @@ mod operators;
pub mod planner;
pub(crate) mod shuffle_writer;
mod spark_hash;
mod util;
43 changes: 34 additions & 9 deletions core/src/execution/datafusion/planner.rs
Original file line number Diff line number Diff line change
Expand Up @@ -27,24 +27,23 @@ use datafusion::{
physical_expr::{
execution_props::ExecutionProps,
expressions::{
in_list, BinaryExpr, CaseExpr, CastExpr, Column, Count, FirstValue, InListExpr,
IsNotNullExpr, IsNullExpr, LastValue, Literal as DataFusionLiteral, Max, Min,
NegativeExpr, NotExpr, Sum, UnKnownColumn,
in_list, BinaryExpr, BitAnd, BitOr, BitXor, CaseExpr, CastExpr, Column, Count,
FirstValue, InListExpr, IsNotNullExpr, IsNullExpr, LastValue,
Literal as DataFusionLiteral, Max, Min, NegativeExpr, NotExpr, Sum, UnKnownColumn,
},
functions::create_physical_expr,
AggregateExpr, PhysicalExpr, PhysicalSortExpr, ScalarFunctionExpr,
},
physical_plan::{
aggregates::{AggregateMode as DFAggregateMode, PhysicalGroupBy},
filter::FilterExec,
joins::{utils::JoinFilter, HashJoinExec, PartitionMode},
joins::{utils::JoinFilter, HashJoinExec, PartitionMode, SortMergeJoinExec},
limit::LocalLimitExec,
projection::ProjectionExec,
sorts::sort::SortExec,
ExecutionPlan, Partitioning,
},
};
use datafusion::physical_plan::joins::SortMergeJoinExec;
use datafusion_common::{
tree_node::{TreeNode, TreeNodeRewriter, VisitRecursion},
JoinType as DFJoinType, ScalarValue,
Expand All @@ -61,6 +60,7 @@ use crate::{
avg::Avg,
avg_decimal::AvgDecimal,
bitwise_not::BitwiseNotExpr,
bloom_filter_might_contain::BloomFilterMightContain,
cast::Cast,
checkoverflow::CheckOverflow,
if_expr::IfExpr,
Expand Down Expand Up @@ -539,6 +539,15 @@ impl PhysicalPlanner {
let data_type = to_arrow_datatype(expr.datatype.as_ref().unwrap());
Ok(Arc::new(Subquery::new(self.exec_context_id, id, data_type)))
}
ExprStruct::BloomFilterMightContain(expr) => {
let bloom_filter_expr =
self.create_expr(expr.bloom_filter.as_ref().unwrap(), input_schema.clone())?;
let value_expr = self.create_expr(expr.value.as_ref().unwrap(), input_schema)?;
Ok(Arc::new(BloomFilterMightContain::try_new(
bloom_filter_expr,
value_expr,
)?))
}
expr => Err(ExecutionError::GeneralError(format!(
"Not implemented: {:?}",
expr
Expand Down Expand Up @@ -924,7 +933,8 @@ impl PhysicalPlanner {
left
};

let right = if crate::execution::datafusion::planner::can_reuse_input_batch(&right) {
let right = if crate::execution::datafusion::planner::can_reuse_input_batch(&right)
{
Arc::new(CopyExec::new(right))
} else {
right
Expand Down Expand Up @@ -1044,13 +1054,13 @@ impl PhysicalPlanner {
// DataFusion `HashJoinExec` operator keeps the input batch internally. We need
// to copy the input batch to avoid the data corruption from reusing the input
// batch.
let left = if op_reuse_array(&left) {
let left = if can_reuse_input_batch(&left) {
Arc::new(CopyExec::new(left))
} else {
left
};

let right = if op_reuse_array(&right) {
let right = if can_reuse_input_batch(&right) {
Arc::new(CopyExec::new(right))
} else {
right
Expand Down Expand Up @@ -1150,6 +1160,21 @@ impl PhysicalPlanner {
vec![],
)))
}
AggExprStruct::BitAndAgg(expr) => {
let child = self.create_expr(expr.child.as_ref().unwrap(), schema)?;
let datatype = to_arrow_datatype(expr.datatype.as_ref().unwrap());
Ok(Arc::new(BitAnd::new(child, "bit_and", datatype)))
}
AggExprStruct::BitOrAgg(expr) => {
let child = self.create_expr(expr.child.as_ref().unwrap(), schema)?;
let datatype = to_arrow_datatype(expr.datatype.as_ref().unwrap());
Ok(Arc::new(BitOr::new(child, "bit_or", datatype)))
}
AggExprStruct::BitXorAgg(expr) => {
let child = self.create_expr(expr.child.as_ref().unwrap(), schema)?;
let datatype = to_arrow_datatype(expr.datatype.as_ref().unwrap());
Ok(Arc::new(BitXor::new(child, "bit_xor", datatype)))
}
}
}

Expand Down Expand Up @@ -1239,7 +1264,7 @@ impl From<ExpressionError> for DataFusionError {
/// Returns true if given operator can return input array as output array without
/// modification. This is used to determine if we need to copy the input batch to avoid
/// data corruption from reusing the input batch.
fn op_reuse_array(op: &Arc<dyn ExecutionPlan>) -> bool {
fn can_reuse_input_batch(op: &Arc<dyn ExecutionPlan>) -> bool {
op.as_any().downcast_ref::<ScanExec>().is_some()
|| op.as_any().downcast_ref::<LocalLimitExec>().is_some()
|| op.as_any().downcast_ref::<ProjectionExec>().is_some()
Expand Down
2 changes: 1 addition & 1 deletion core/src/execution/datafusion/spark_hash.rs
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,7 @@ use datafusion::{
};

#[inline]
fn spark_compatible_murmur3_hash<T: AsRef<[u8]>>(data: T, seed: u32) -> u32 {
pub(crate) fn spark_compatible_murmur3_hash<T: AsRef<[u8]>>(data: T, seed: u32) -> u32 {
#[inline]
fn mix_k1(mut k1: i32) -> i32 {
k1 = k1.mul_wrapping(0xcc9e2d51u32 as i32);
Expand Down
19 changes: 19 additions & 0 deletions core/src/execution/datafusion/util/mod.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,19 @@
// 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.

pub mod spark_bit_array;
pub mod spark_bloom_filter;
Loading

0 comments on commit a236051

Please sign in to comment.