Skip to content
Draft
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
82 changes: 50 additions & 32 deletions native/core/src/execution/planner.rs
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ use crate::{
use arrow::compute::CastOptions;
use arrow::datatypes::{DataType, Field, Schema, TimeUnit, DECIMAL128_MAX_PRECISION};
use datafusion::functions_aggregate::bit_and_or_xor::{bit_and_udaf, bit_or_udaf, bit_xor_udaf};
use datafusion::functions_aggregate::count::count_udaf;
use datafusion::functions_aggregate::min_max::max_udaf;
use datafusion::functions_aggregate::min_max::min_udaf;
use datafusion::functions_aggregate::sum::sum_udaf;
Expand Down Expand Up @@ -1899,41 +1900,58 @@ impl PhysicalPlanner {
match spark_expr.expr_struct.as_ref().unwrap() {
AggExprStruct::Count(expr) => {
assert!(!expr.children.is_empty());
// Using `count_udaf` from Comet is exceptionally slow for some reason, so
// as a workaround we translate it to `SUM(IF(expr IS NOT NULL, 1, 0))`
// https://github.com/apache/datafusion-comet/issues/744

let children = expr
.children
.iter()
.map(|child| self.create_expr(child, Arc::clone(&schema)))
.collect::<Result<Vec<_>, _>>()?;
if spark_expr.distinct {
let children = expr
.children
.iter()
.map(|child| self.create_expr(child, Arc::clone(&schema)))
.collect::<Result<Vec<_>, _>>()?;

AggregateExprBuilder::new(count_udaf(), children)
.schema(schema)
.alias("count")
.with_ignore_nulls(false)
.with_distinct(true)
.build()
.map_err(|e| ExecutionError::DataFusionError(e.to_string()))
} else {
// Using `count_udaf` from Comet is exceptionally slow for some reason, so
// as a workaround we translate it to `SUM(IF(expr IS NOT NULL, 1, 0))`
// https://github.com/apache/datafusion-comet/issues/744

// create `IS NOT NULL expr` and join them with `AND` if there are multiple
let not_null_expr: Arc<dyn PhysicalExpr> = children.iter().skip(1).fold(
Arc::new(IsNotNullExpr::new(Arc::clone(&children[0]))) as Arc<dyn PhysicalExpr>,
|acc, child| {
Arc::new(BinaryExpr::new(
acc,
DataFusionOperator::And,
Arc::new(IsNotNullExpr::new(Arc::clone(child))),
))
},
);
let children = expr
.children
.iter()
.map(|child| self.create_expr(child, Arc::clone(&schema)))
.collect::<Result<Vec<_>, _>>()?;

// create `IS NOT NULL expr` and join them with `AND` if there are multiple
let not_null_expr: Arc<dyn PhysicalExpr> = children.iter().skip(1).fold(
Arc::new(IsNotNullExpr::new(Arc::clone(&children[0])))
as Arc<dyn PhysicalExpr>,
|acc, child| {
Arc::new(BinaryExpr::new(
acc,
DataFusionOperator::And,
Arc::new(IsNotNullExpr::new(Arc::clone(child))),
))
},
);

let child = Arc::new(IfExpr::new(
not_null_expr,
Arc::new(Literal::new(ScalarValue::Int64(Some(1)))),
Arc::new(Literal::new(ScalarValue::Int64(Some(0)))),
));
let child = Arc::new(IfExpr::new(
not_null_expr,
Arc::new(Literal::new(ScalarValue::Int64(Some(1)))),
Arc::new(Literal::new(ScalarValue::Int64(Some(0)))),
));

AggregateExprBuilder::new(sum_udaf(), vec![child])
.schema(schema)
.alias("count")
.with_ignore_nulls(false)
.with_distinct(false)
.build()
.map_err(|e| ExecutionError::DataFusionError(e.to_string()))
AggregateExprBuilder::new(sum_udaf(), vec![child])
.schema(schema)
.alias("count")
.with_ignore_nulls(false)
.with_distinct(false)
.build()
.map_err(|e| ExecutionError::DataFusionError(e.to_string()))
}
}
AggExprStruct::Min(expr) => {
let child = self.create_expr(expr.child.as_ref().unwrap(), Arc::clone(&schema))?;
Expand Down
1 change: 1 addition & 0 deletions native/proto/src/proto/expr.proto
Original file line number Diff line number Diff line change
Expand Up @@ -90,6 +90,7 @@ message Expr {
}

message AggExpr {
bool distinct = 1;
oneof expr_struct {
Count count = 2;
Sum sum = 3;
Expand Down
18 changes: 9 additions & 9 deletions spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala
Original file line number Diff line number Diff line change
Expand Up @@ -508,19 +508,17 @@ object QueryPlanSerde extends Logging with CometExprShim {
binding: Boolean,
conf: SQLConf): Option[AggExpr] = {

if (aggExpr.isDistinct) {
// https://github.com/apache/datafusion-comet/issues/1260
withInfo(aggExpr, "distinct aggregates are not supported")
return None
}

val fn = aggExpr.aggregateFunction
val cometExpr = aggrSerdeMap.get(fn.getClass)
cometExpr match {
case Some(handler) =>
handler
.asInstanceOf[CometAggregateExpressionSerde[AggregateFunction]]
.convert(aggExpr, fn, inputs, binding, conf)
val aggSerde = handler.asInstanceOf[CometAggregateExpressionSerde[AggregateFunction]]
if (aggExpr.isDistinct && !aggSerde.supportsDistinct) {
// https://github.com/apache/datafusion-comet/issues/1260
withInfo(aggExpr, s"distinct aggregate not supported: $fn")
return None
}
aggSerde.convert(aggExpr, fn, inputs, binding, conf)
case _ =>
withInfo(
aggExpr,
Expand Down Expand Up @@ -2181,6 +2179,8 @@ trait CometExpressionSerde[T <: Expression] {
*/
trait CometAggregateExpressionSerde[T <: AggregateFunction] {

def supportsDistinct: Boolean = false

/**
* Convert a Spark expression into a protocol buffer representation that can be passed into
* native code.
Expand Down
8 changes: 8 additions & 0 deletions spark/src/main/scala/org/apache/comet/serde/aggregates.scala
Original file line number Diff line number Diff line change
Expand Up @@ -103,19 +103,27 @@ object CometMax extends CometAggregateExpressionSerde[Max] {
}

object CometCount extends CometAggregateExpressionSerde[Count] {

override def supportsDistinct: Boolean = true

override def convert(
aggExpr: AggregateExpression,
expr: Count,
inputs: Seq[Attribute],
binding: Boolean,
conf: SQLConf): Option[ExprOuterClass.AggExpr] = {
if (expr.children.length > 1) {
withInfo(aggExpr, "COUNT only supports a single argument")
return None
}
val exprChildren = expr.children.map(exprToProto(_, inputs, binding))
if (exprChildren.forall(_.isDefined)) {
val builder = ExprOuterClass.Count.newBuilder()
builder.addAllChildren(exprChildren.map(_.get).asJava)
Some(
ExprOuterClass.AggExpr
.newBuilder()
.setDistinct(aggExpr.isDistinct)
.setCount(builder)
.build())
} else {
Expand Down
Loading