Skip to content
Merged
Show file tree
Hide file tree
Changes from 16 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
The table of contents is too big for display.
Diff view
Diff view
  •  
  •  
  •  
5 changes: 5 additions & 0 deletions docs/source/user-guide/latest/configs.md
Original file line number Diff line number Diff line change
Expand Up @@ -214,6 +214,7 @@ These settings can be used to determine which parts of the plan are accelerated
| `spark.comet.expression.BitwiseNot.enabled` | Enable Comet acceleration for `BitwiseNot` | true |
| `spark.comet.expression.BitwiseOr.enabled` | Enable Comet acceleration for `BitwiseOr` | true |
| `spark.comet.expression.BitwiseXor.enabled` | Enable Comet acceleration for `BitwiseXor` | true |
| `spark.comet.expression.BloomFilterMightContain.enabled` | Enable Comet acceleration for `BloomFilterMightContain` | true |
| `spark.comet.expression.CaseWhen.enabled` | Enable Comet acceleration for `CaseWhen` | true |
| `spark.comet.expression.Cast.enabled` | Enable Comet acceleration for `Cast` | true |
| `spark.comet.expression.Ceil.enabled` | Enable Comet acceleration for `Ceil` | true |
Expand Down Expand Up @@ -258,6 +259,7 @@ These settings can be used to determine which parts of the plan are accelerated
| `spark.comet.expression.IsNaN.enabled` | Enable Comet acceleration for `IsNaN` | true |
| `spark.comet.expression.IsNotNull.enabled` | Enable Comet acceleration for `IsNotNull` | true |
| `spark.comet.expression.IsNull.enabled` | Enable Comet acceleration for `IsNull` | true |
| `spark.comet.expression.KnownFloatingPointNormalized.enabled` | Enable Comet acceleration for `KnownFloatingPointNormalized` | true |
| `spark.comet.expression.Length.enabled` | Enable Comet acceleration for `Length` | true |
| `spark.comet.expression.LessThan.enabled` | Enable Comet acceleration for `LessThan` | true |
| `spark.comet.expression.LessThanOrEqual.enabled` | Enable Comet acceleration for `LessThanOrEqual` | true |
Expand All @@ -267,6 +269,7 @@ These settings can be used to determine which parts of the plan are accelerated
| `spark.comet.expression.Log10.enabled` | Enable Comet acceleration for `Log10` | true |
| `spark.comet.expression.Log2.enabled` | Enable Comet acceleration for `Log2` | true |
| `spark.comet.expression.Lower.enabled` | Enable Comet acceleration for `Lower` | true |
| `spark.comet.expression.MakeDecimal.enabled` | Enable Comet acceleration for `MakeDecimal` | true |
| `spark.comet.expression.MapEntries.enabled` | Enable Comet acceleration for `MapEntries` | true |
| `spark.comet.expression.MapFromArrays.enabled` | Enable Comet acceleration for `MapFromArrays` | true |
| `spark.comet.expression.MapKeys.enabled` | Enable Comet acceleration for `MapKeys` | true |
Expand All @@ -289,6 +292,7 @@ These settings can be used to determine which parts of the plan are accelerated
| `spark.comet.expression.Remainder.enabled` | Enable Comet acceleration for `Remainder` | true |
| `spark.comet.expression.Reverse.enabled` | Enable Comet acceleration for `Reverse` | true |
| `spark.comet.expression.Round.enabled` | Enable Comet acceleration for `Round` | true |
| `spark.comet.expression.ScalarSubquery.enabled` | Enable Comet acceleration for `ScalarSubquery` | true |
| `spark.comet.expression.Second.enabled` | Enable Comet acceleration for `Second` | true |
| `spark.comet.expression.Sha1.enabled` | Enable Comet acceleration for `Sha1` | true |
| `spark.comet.expression.Sha2.enabled` | Enable Comet acceleration for `Sha2` | true |
Expand Down Expand Up @@ -320,6 +324,7 @@ These settings can be used to determine which parts of the plan are accelerated
| `spark.comet.expression.TruncTimestamp.enabled` | Enable Comet acceleration for `TruncTimestamp` | true |
| `spark.comet.expression.UnaryMinus.enabled` | Enable Comet acceleration for `UnaryMinus` | true |
| `spark.comet.expression.Unhex.enabled` | Enable Comet acceleration for `Unhex` | true |
| `spark.comet.expression.UnscaledValue.enabled` | Enable Comet acceleration for `UnscaledValue` | true |
| `spark.comet.expression.Upper.enabled` | Enable Comet acceleration for `Upper` | true |
| `spark.comet.expression.WeekDay.enabled` | Enable Comet acceleration for `WeekDay` | true |
| `spark.comet.expression.WeekOfYear.enabled` | Enable Comet acceleration for `WeekOfYear` | true |
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,51 @@
/*
* 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.
*/

package org.apache.comet.serde

import org.apache.spark.sql.catalyst.expressions.Attribute
import org.apache.spark.sql.execution.ScalarSubquery

import org.apache.comet.CometSparkSessionExtensions.withInfo
import org.apache.comet.serde.QueryPlanSerde.{serializeDataType, supportedDataType}

object CometScalarSubquery extends CometExpressionSerde[ScalarSubquery] {
override def convert(
expr: ScalarSubquery,
inputs: Seq[Attribute],
binding: Boolean): Option[ExprOuterClass.Expr] = {
if (supportedDataType(expr.dataType)) {
val dataType = serializeDataType(expr.dataType)
if (dataType.isEmpty) {
withInfo(expr, s"Scalar subquery returns unsupported datatype ${expr.dataType}")
return None
}

val builder = ExprOuterClass.Subquery
.newBuilder()
.setId(expr.exprId.id)
.setDatatype(dataType.get)
Some(ExprOuterClass.Expr.newBuilder().setSubquery(builder).build())
} else {
withInfo(expr, s"Unsupported data type: ${expr.dataType}")
None
}

}
}
89 changes: 9 additions & 80 deletions spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala
Original file line number Diff line number Diff line change
Expand Up @@ -25,10 +25,8 @@ import org.apache.spark.internal.Logging
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.expressions.aggregate._
import org.apache.spark.sql.catalyst.expressions.objects.StaticInvoke
import org.apache.spark.sql.catalyst.optimizer.NormalizeNaNAndZero
import org.apache.spark.sql.comet._
import org.apache.spark.sql.execution
import org.apache.spark.sql.execution._
import org.apache.spark.sql.comet.DecimalPrecision
import org.apache.spark.sql.execution.{ScalarSubquery, SparkPlan}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types._

Expand Down Expand Up @@ -153,7 +151,7 @@ object QueryPlanSerde extends Logging with CometExprShim {
classOf[Lower] -> CometLower,
classOf[OctetLength] -> CometScalarFunction("octet_length"),
classOf[RegExpReplace] -> CometRegExpReplace,
classOf[Reverse] -> CometScalarFunction("reverse"),
classOf[Reverse] -> CometReverse,
classOf[RLike] -> CometRLike,
classOf[StartsWith] -> CometScalarFunction("starts_with"),
classOf[StringInstr] -> CometScalarFunction("instr"),
Expand Down Expand Up @@ -203,21 +201,20 @@ object QueryPlanSerde extends Logging with CometExprShim {

private val miscExpressions: Map[Class[_ <: Expression], CometExpressionSerde[_]] = Map(
// TODO PromotePrecision
// TODO KnownFloatingPointNormalized
// TODO ScalarSubquery
// TODO UnscaledValue
// TODO MakeDecimal
// TODO BloomFilterMightContain
// TODO RegExpReplace
classOf[Alias] -> CometAlias,
classOf[AttributeReference] -> CometAttributeReference,
classOf[BloomFilterMightContain] -> CometBloomFilterMightContain,
classOf[CheckOverflow] -> CometCheckOverflow,
classOf[Coalesce] -> CometCoalesce,
classOf[KnownFloatingPointNormalized] -> CometKnownFloatingPointNormalized,
classOf[Literal] -> CometLiteral,
classOf[MakeDecimal] -> CometMakeDecimal,
classOf[MonotonicallyIncreasingID] -> CometMonotonicallyIncreasingId,
classOf[ScalarSubquery] -> CometScalarSubquery,
classOf[SparkPartitionID] -> CometSparkPartitionId,
classOf[SortOrder] -> CometSortOrder,
classOf[StaticInvoke] -> CometStaticInvoke)
classOf[StaticInvoke] -> CometStaticInvoke,
classOf[UnscaledValue] -> CometUnscaledValue)

/**
* Mapping of Spark expression class to Comet expression handler.
Expand Down Expand Up @@ -541,74 +538,6 @@ object QueryPlanSerde extends Logging with CometExprShim {
// `PromotePrecision` is just a wrapper, don't need to serialize it.
exprToProtoInternal(child, inputs, binding)

case KnownFloatingPointNormalized(NormalizeNaNAndZero(expr)) =>
val dataType = serializeDataType(expr.dataType)
if (dataType.isEmpty) {
withInfo(expr, s"Unsupported datatype ${expr.dataType}")
return None
}
val ex = exprToProtoInternal(expr, inputs, binding)
ex.map { child =>
val builder = ExprOuterClass.NormalizeNaNAndZero
.newBuilder()
.setChild(child)
.setDatatype(dataType.get)
ExprOuterClass.Expr.newBuilder().setNormalizeNanAndZero(builder).build()
}

case s @ execution.ScalarSubquery(_, _) =>
if (supportedDataType(s.dataType)) {
val dataType = serializeDataType(s.dataType)
if (dataType.isEmpty) {
withInfo(s, s"Scalar subquery returns unsupported datatype ${s.dataType}")
return None
}

val builder = ExprOuterClass.Subquery
.newBuilder()
.setId(s.exprId.id)
.setDatatype(dataType.get)
Some(ExprOuterClass.Expr.newBuilder().setSubquery(builder).build())
} else {
withInfo(s, s"Unsupported data type: ${s.dataType}")
None
}

case UnscaledValue(child) =>
val childExpr = exprToProtoInternal(child, inputs, binding)
val optExpr =
scalarFunctionExprToProtoWithReturnType("unscaled_value", LongType, false, childExpr)
optExprWithInfo(optExpr, expr, child)

case MakeDecimal(child, precision, scale, true) =>
val childExpr = exprToProtoInternal(child, inputs, binding)
val optExpr = scalarFunctionExprToProtoWithReturnType(
"make_decimal",
DecimalType(precision, scale),
false,
childExpr)
optExprWithInfo(optExpr, expr, child)

case b @ BloomFilterMightContain(_, _) =>
val bloomFilter = b.left
val value = b.right
val bloomFilterExpr = exprToProtoInternal(bloomFilter, inputs, binding)
val valueExpr = exprToProtoInternal(value, inputs, binding)
if (bloomFilterExpr.isDefined && valueExpr.isDefined) {
val builder = ExprOuterClass.BloomFilterMightContain.newBuilder()
builder.setBloomFilter(bloomFilterExpr.get)
builder.setValue(valueExpr.get)
Some(
ExprOuterClass.Expr
.newBuilder()
.setBloomFilterMightContain(builder)
.build())
} else {
withInfo(expr, bloomFilter, value)
None
}
case r @ Reverse(child) if child.dataType.isInstanceOf[ArrayType] =>
convert(r, CometArrayReverse)
case expr =>
QueryPlanSerde.exprSerdeMap.get(expr.getClass) match {
case Some(handler) =>
Expand Down
115 changes: 115 additions & 0 deletions spark/src/main/scala/org/apache/comet/serde/misc.scala
Original file line number Diff line number Diff line change
@@ -0,0 +1,115 @@
/*
* 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.
*/

package org.apache.comet.serde

import org.apache.spark.sql.catalyst.expressions.{Attribute, BloomFilterMightContain, KnownFloatingPointNormalized, MakeDecimal, UnscaledValue}
import org.apache.spark.sql.catalyst.optimizer.NormalizeNaNAndZero
import org.apache.spark.sql.types.{DecimalType, LongType}

import org.apache.comet.CometSparkSessionExtensions.withInfo
import org.apache.comet.serde.QueryPlanSerde.{exprToProtoInternal, optExprWithInfo, scalarFunctionExprToProtoWithReturnType, serializeDataType}

object CometUnscaledValue extends CometExpressionSerde[UnscaledValue] {
override def convert(
expr: UnscaledValue,
inputs: Seq[Attribute],
binding: Boolean): Option[ExprOuterClass.Expr] = {
val childExpr = exprToProtoInternal(expr.child, inputs, binding)
val optExpr =
scalarFunctionExprToProtoWithReturnType("unscaled_value", LongType, false, childExpr)
optExprWithInfo(optExpr, expr, expr.child)

}
}
object CometKnownFloatingPointNormalized
extends CometExpressionSerde[KnownFloatingPointNormalized] {

override def getSupportLevel(expr: KnownFloatingPointNormalized): SupportLevel = {
expr.child match {
case _: NormalizeNaNAndZero => Compatible()
case _ => Unsupported()
}
}

override def convert(
expr: KnownFloatingPointNormalized,
inputs: Seq[Attribute],
binding: Boolean): Option[ExprOuterClass.Expr] = {

val wrapped = expr.child.asInstanceOf[NormalizeNaNAndZero].child

val dataType = serializeDataType(wrapped.dataType)
if (dataType.isEmpty) {
withInfo(wrapped, s"Unsupported datatype ${wrapped.dataType}")
return None
}
val ex = exprToProtoInternal(wrapped, inputs, binding)
ex.map { child =>
val builder = ExprOuterClass.NormalizeNaNAndZero
.newBuilder()
.setChild(child)
.setDatatype(dataType.get)
ExprOuterClass.Expr.newBuilder().setNormalizeNanAndZero(builder).build()
}
}
}

object CometMakeDecimal extends CometExpressionSerde[MakeDecimal] {
override def convert(
expr: MakeDecimal,
inputs: Seq[Attribute],
binding: Boolean): Option[ExprOuterClass.Expr] = {
val childExpr = exprToProtoInternal(expr.child, inputs, binding)
val optExpr = scalarFunctionExprToProtoWithReturnType(
"make_decimal",
DecimalType(expr.precision, expr.scale),
failOnError = !expr.nullOnOverflow,
childExpr)
optExprWithInfo(optExpr, expr, expr.child)

}
}

object CometBloomFilterMightContain extends CometExpressionSerde[BloomFilterMightContain] {

override def convert(
expr: BloomFilterMightContain,
inputs: Seq[Attribute],
binding: Boolean): Option[ExprOuterClass.Expr] = {

val bloomFilter = expr.left
val value = expr.right
val bloomFilterExpr = exprToProtoInternal(bloomFilter, inputs, binding)
val valueExpr = exprToProtoInternal(value, inputs, binding)
if (bloomFilterExpr.isDefined && valueExpr.isDefined) {
val builder = ExprOuterClass.BloomFilterMightContain.newBuilder()
builder.setBloomFilter(bloomFilterExpr.get)
builder.setValue(valueExpr.get)
Some(
ExprOuterClass.Expr
.newBuilder()
.setBloomFilterMightContain(builder)
.build())
} else {
withInfo(expr, bloomFilter, value)
None
}
}
}
23 changes: 21 additions & 2 deletions spark/src/main/scala/org/apache/comet/serde/strings.scala
Original file line number Diff line number Diff line change
Expand Up @@ -21,15 +21,34 @@ package org.apache.comet.serde

import java.util.Locale

import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, Concat, Expression, InitCap, Length, Like, Literal, Lower, RegExpReplace, RLike, StringLPad, StringRepeat, StringRPad, Substring, Upper}
import org.apache.spark.sql.types.{BinaryType, DataTypes, LongType, StringType}
import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, Concat, Expression, InitCap, Length, Like, Literal, Lower, RegExpReplace, Reverse, RLike, StringLPad, StringRepeat, StringRPad, Substring, Upper}
import org.apache.spark.sql.types.{ArrayType, BinaryType, DataTypes, LongType, StringType}

import org.apache.comet.CometConf
import org.apache.comet.CometSparkSessionExtensions.withInfo
import org.apache.comet.expressions.{CometCast, CometEvalMode, RegExp}
import org.apache.comet.serde.ExprOuterClass.Expr
import org.apache.comet.serde.QueryPlanSerde.{createBinaryExpr, exprToProtoInternal, optExprWithInfo, scalarFunctionExprToProto}

object CometReverse extends CometScalarFunction[Reverse]("reverse") {
Copy link
Member

Choose a reason for hiding this comment

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

Copy link
Member Author

Choose a reason for hiding this comment

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

Thanks. Moved to match Spark organization.


override def getSupportLevel(expr: Reverse): SupportLevel = {
if (expr.child.dataType.isInstanceOf[ArrayType]) {
CometArrayReverse.getSupportLevel(expr)
} else {
Compatible()
}
}

override def convert(expr: Reverse, inputs: Seq[Attribute], binding: Boolean): Option[Expr] = {
if (expr.child.dataType.isInstanceOf[ArrayType]) {
CometArrayReverse.convert(expr, inputs, binding)
} else {
super.convert(expr, inputs, binding)
}
}
}

object CometStringRepeat extends CometExpressionSerde[StringRepeat] {

override def convert(
Expand Down
Loading
Loading