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

DNM: test dpp support #1396

Draft
wants to merge 7 commits into
base: main
Choose a base branch
from
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
19 changes: 16 additions & 3 deletions common/src/main/scala/org/apache/spark/sql/comet/util/Utils.scala
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@

package org.apache.spark.sql.comet.util

import java.io.{DataOutputStream, File}
import java.io.{DataInputStream, DataOutputStream, File}
import java.nio.ByteBuffer
import java.nio.channels.Channels

Expand All @@ -30,7 +30,7 @@ import org.apache.arrow.vector.{BigIntVector, BitVector, DateDayVector, DecimalV
import org.apache.arrow.vector.complex.MapVector
import org.apache.arrow.vector.complex.StructVector
import org.apache.arrow.vector.dictionary.DictionaryProvider
import org.apache.arrow.vector.ipc.ArrowStreamWriter
import org.apache.arrow.vector.ipc.{ArrowStreamReader, ArrowStreamWriter}
import org.apache.arrow.vector.types._
import org.apache.arrow.vector.types.pojo.{ArrowType, Field, FieldType, Schema}
import org.apache.spark.{SparkEnv, SparkException}
Expand All @@ -39,7 +39,8 @@ import org.apache.spark.sql.types._
import org.apache.spark.sql.vectorized.ColumnarBatch
import org.apache.spark.util.io.{ChunkedByteBuffer, ChunkedByteBufferOutputStream}

import org.apache.comet.vector.CometVector
import org.apache.comet.CometArrowAllocator
import org.apache.comet.vector.{CometVector, NativeUtil}

object Utils {
def getConfPath(confFileName: String): String = {
Expand Down Expand Up @@ -226,6 +227,18 @@ object Utils {
}
}

def deserializeBatches(
chunkedByteBuffers: Iterator[ChunkedByteBuffer]): Iterator[ColumnarBatch] = {
val codec = CompressionCodec.createCodec(SparkEnv.get.conf)
chunkedByteBuffers.map { chunkedByteBuffer =>
val cbbis = chunkedByteBuffer.toInputStream()
val in = new DataInputStream(codec.compressedInputStream(cbbis))
val reader = new ArrowStreamReader(in, CometArrowAllocator)
reader.loadNextBatch()
NativeUtil.rootAsBatch(reader.getVectorSchemaRoot, reader)
}
}

def getBatchFieldVectors(
batch: ColumnarBatch): (Seq[FieldVector], Option[DictionaryProvider]) = {
var provider: Option[DictionaryProvider] = None
Expand Down
7 changes: 7 additions & 0 deletions dev/ensure-jars-have-correct-contents.sh
Original file line number Diff line number Diff line change
Expand Up @@ -95,6 +95,13 @@ allowed_expr+="|^org/apache/spark/CometDriverPlugin.*$"
allowed_expr+="|^org/apache/spark/CometTaskMemoryManager.class$"
allowed_expr+="|^org/apache/spark/CometTaskMemoryManager.*$"

# TODO: add reason
allowed_expr+="|^org/apache/spark/sql/execution/$"
allowed_expr+="|^org/apache/spark/sql/execution/joins/$"
allowed_expr+="|^org/apache/spark/sql/execution/joins/CometHashedRelation.*$"
allowed_expr+="|^org/apache/spark/sql/execution/joins/CometLongHashedRelation.class$"
allowed_expr+="|^org/apache/spark/sql/execution/joins/CometUnsafeHashedRelation.class$"

allowed_expr+=")"
declare -i bad_artifacts=0
declare -a bad_contents
Expand Down
186 changes: 139 additions & 47 deletions spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala
Original file line number Diff line number Diff line change
Expand Up @@ -25,17 +25,18 @@ import org.apache.spark.SparkConf
import org.apache.spark.internal.Logging
import org.apache.spark.network.util.ByteUnit
import org.apache.spark.sql.{SparkSession, SparkSessionExtensions}
import org.apache.spark.sql.catalyst.expressions.{Attribute, Divide, DoubleLiteral, EqualNullSafe, EqualTo, Expression, FloatLiteral, GreaterThan, GreaterThanOrEqual, KnownFloatingPointNormalized, LessThan, LessThanOrEqual, NamedExpression, PlanExpression, Remainder}
import org.apache.spark.sql.catalyst.expressions.{Attribute, BindReferences, Divide, DoubleLiteral, EqualNullSafe, EqualTo, Expression, FloatLiteral, GreaterThan, GreaterThanOrEqual, KnownFloatingPointNormalized, LessThan, LessThanOrEqual, NamedExpression, PlanExpression, Remainder}
import org.apache.spark.sql.catalyst.expressions.aggregate.{Final, Partial}
import org.apache.spark.sql.catalyst.optimizer.NormalizeNaNAndZero
import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, NormalizeNaNAndZero}
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.catalyst.trees.TreeNode
import org.apache.spark.sql.catalyst.trees.TreePattern.PLAN_EXPRESSION
import org.apache.spark.sql.catalyst.util.MetadataColumnHelper
import org.apache.spark.sql.comet._
import org.apache.spark.sql.comet.execution.shuffle.{CometColumnarShuffle, CometNativeShuffle, CometShuffleExchangeExec, CometShuffleManager}
import org.apache.spark.sql.comet.util.Utils
import org.apache.spark.sql.execution._
import org.apache.spark.sql.execution.adaptive.{AQEShuffleReadExec, BroadcastQueryStageExec, ShuffleQueryStageExec}
import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, AdaptiveSparkPlanHelper, AQEShuffleReadExec, BroadcastQueryStageExec, ShuffleQueryStageExec}
import org.apache.spark.sql.execution.aggregate.{BaseAggregateExec, HashAggregateExec, ObjectHashAggregateExec}
import org.apache.spark.sql.execution.datasources._
import org.apache.spark.sql.execution.datasources.csv.CSVFileFormat
Expand All @@ -46,14 +47,14 @@ import org.apache.spark.sql.execution.datasources.v2.csv.CSVScan
import org.apache.spark.sql.execution.datasources.v2.json.JsonScan
import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan
import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ReusedExchangeExec, ShuffleExchangeExec}
import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, ShuffledHashJoinExec, SortMergeJoinExec}
import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, HashedRelationBroadcastMode, HashJoin, ShuffledHashJoinExec, SortMergeJoinExec}
import org.apache.spark.sql.execution.window.WindowExec
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types.{DoubleType, FloatType}

import org.apache.comet.CometConf._
import org.apache.comet.CometExplainInfo.getActualPlan
import org.apache.comet.CometSparkSessionExtensions.{createMessage, getCometBroadcastNotEnabledReason, getCometShuffleNotEnabledReason, isANSIEnabled, isCometBroadCastForceEnabled, isCometEnabled, isCometExecEnabled, isCometJVMShuffleMode, isCometNativeShuffleMode, isCometScan, isCometScanEnabled, isCometShuffleEnabled, isSpark34Plus, isSpark40Plus, shouldApplySparkToColumnar, withInfo, withInfos}
import org.apache.comet.CometSparkSessionExtensions.{createMessage, getCometShuffleNotEnabledReason, isANSIEnabled, isCometEnabled, isCometExecEnabled, isCometJVMShuffleMode, isCometNativeShuffleMode, isCometScan, isCometScanEnabled, isCometShuffleEnabled, isSpark34Plus, isSpark40Plus, shouldApplySparkToColumnar, withInfo, withInfos}
import org.apache.comet.parquet.{CometParquetScan, SupportsComet}
import org.apache.comet.rules.RewriteJoin
import org.apache.comet.serde.OperatorOuterClass.Operator
Expand All @@ -74,8 +75,12 @@ class CometSparkSessionExtensions
override def apply(extensions: SparkSessionExtensions): Unit = {
extensions.injectColumnar { session => CometScanColumnar(session) }
extensions.injectColumnar { session => CometExecColumnar(session) }
extensions.injectColumnar { session => CometSubqueryExecColumnar(session) }
extensions.injectQueryStagePrepRule { session => CometScanRule(session) }
extensions.injectQueryStagePrepRule { session => CometExecRule(session) }
extensions.injectQueryStagePrepRule { session =>
CometSubqueryExecRule(CometExecRule(session))
}
}

case class CometScanColumnar(session: SparkSession) extends ColumnarRule {
Expand Down Expand Up @@ -116,11 +121,11 @@ class CometSparkSessionExtensions
withInfo(scan, "Metadata column is not supported")
scan

case scanExec: FileSourceScanExec
if COMET_DPP_FALLBACK_ENABLED.get() &&
scanExec.partitionFilters.exists(isDynamicPruningFilter) =>
withInfo(scanExec, "DPP not supported")
scanExec
// case scanExec: FileSourceScanExec
// if COMET_DPP_FALLBACK_ENABLED.get() &&
// scanExec.partitionFilters.exists(isDynamicPruningFilter) =>
// withInfo(scanExec, "DPP not supported")
// scanExec

// data source V2
case scanExec: BatchScanExec
Expand Down Expand Up @@ -738,43 +743,16 @@ class CometSparkSessionExtensions
s
}

// `CometBroadcastExchangeExec`'s broadcast output is not compatible with Spark's broadcast
// exchange. It is only used for Comet native execution. We only transform Spark broadcast
// exchange to Comet broadcast exchange if its downstream is a Comet native plan or if the
// broadcast exchange is forced to be enabled by Comet config.
// Note that `CometBroadcastExchangeExec` is only supported for Spark 3.4+.
case plan if plan.children.exists(_.isInstanceOf[BroadcastExchangeExec]) =>
val newChildren = plan.children.map {
case b: BroadcastExchangeExec
if isCometNative(b.child) &&
CometConf.COMET_EXEC_BROADCAST_EXCHANGE_ENABLED.get(conf) &&
isSpark34Plus => // Spark 3.4+ only
QueryPlanSerde.operator2Proto(b) match {
case Some(nativeOp) =>
val cometOp = CometBroadcastExchangeExec(b, b.output, b.child)
CometSinkPlaceHolder(nativeOp, b, cometOp)
case None => b
}
case other => other
}
if (!newChildren.exists(_.isInstanceOf[BroadcastExchangeExec])) {
val newPlan = apply(plan.withNewChildren(newChildren))
if (isCometNative(newPlan) || isCometBroadCastForceEnabled(conf)) {
newPlan
} else {
if (isCometNative(newPlan)) {
val reason =
getCometBroadcastNotEnabledReason(conf).getOrElse("no reason available")
withInfo(plan, s"Broadcast is not enabled: $reason")
}
plan
}
} else {
withInfo(
plan,
s"${plan.nodeName} is not native because the following children are not native " +
s"${explainChildNotNative(plan)}")
plan
case b @ BroadcastExchangeExec(_: HashedRelationBroadcastMode, _)
if isCometNative(b.child) &&
CometConf.COMET_EXEC_BROADCAST_EXCHANGE_ENABLED.get(conf) &&
isSpark34Plus => // Spark 3.4+ only
QueryPlanSerde.operator2Proto(b) match {
case Some(nativeOp) =>
val cometOp = CometBroadcastExchangeExec(b, b.output, b.child)
CometSinkPlaceHolder(nativeOp, b, cometOp)
case None =>
b
}

// this case should be checked only after the previous case checking for a
Expand Down Expand Up @@ -1140,6 +1118,11 @@ class CometSparkSessionExtensions
override def apply(plan: SparkPlan): SparkPlan = {
val eliminatedPlan = plan transformUp {
case ColumnarToRowExec(sparkToColumnar: CometSparkToColumnarExec) => sparkToColumnar.child
case ColumnarToRowExec(child: CometBroadcastExchangeExec) =>
child
case ColumnarToRowExec(b @ BroadcastQueryStageExec(_, child, _))
if child.supportsColumnar =>
b
case c @ ColumnarToRowExec(child) if child.exists(_.isInstanceOf[CometPlan]) =>
val op = CometColumnarToRowExec(child)
if (c.logicalLink.isEmpty) {
Expand All @@ -1152,6 +1135,10 @@ class CometSparkSessionExtensions
case CometColumnarToRowExec(sparkToColumnar: CometSparkToColumnarExec) =>
sparkToColumnar.child
case CometSparkToColumnarExec(child: CometSparkToColumnarExec) => child
case CometColumnarToRowExec(child: CometBroadcastExchangeExec) =>
child
case b @ CometBroadcastExchangeExec(_, _, CometColumnarToRowExec(child)) =>
b.copy(child = child)
// Spark adds `RowToColumnar` under Comet columnar shuffle. But it's redundant as the
// shuffle takes row-based input.
case s @ CometShuffleExchangeExec(
Expand All @@ -1174,6 +1161,111 @@ class CometSparkSessionExtensions
}
}
}

case class CometSubqueryExecColumnar(session: SparkSession) extends ColumnarRule {
override def preColumnarTransitions: Rule[SparkPlan] = CometSubqueryExecRule(
CometExecRule(session))

override def postColumnarTransitions: Rule[SparkPlan] =
CometSubqueryExecRule(EliminateRedundantTransitions(session))
}

case class CometSubqueryExecRule(cometExecRule: Rule[SparkPlan])
extends Rule[SparkPlan]
with AdaptiveSparkPlanHelper {

override def apply(plan: SparkPlan): SparkPlan = {
if (!conf.dynamicPartitionPruningEnabled) {
return plan
}

val rootPlan = plan

def cleanSubqueryPlan(plan: SparkPlan): SparkPlan = {
plan transformUp {
case WholeStageCodegenExec(CometColumnarToRowExec(InputAdapter(child))) =>
CometColumnarToRowExec(child)
case b @ BroadcastExchangeExec(_, CometColumnarToRowExec(child)) =>
b.copy(child = child)
}
}

def replace_subquery(plan: SparkPlan): SparkPlan = {
plan.transformUpWithPruning(_.containsAnyPattern(PLAN_EXPRESSION)) { case p =>
p.transformExpressionsUpWithPruning(_.containsPattern(PLAN_EXPRESSION)) {
case sub: ExecSubqueryExpression =>
val newChild = sub.plan match {
case s @ SubqueryAdaptiveBroadcastExec(
name,
index,
_,
_,
buildKeys,
adaptivePlan: AdaptiveSparkPlanExec)
if adaptivePlan.executedPlan.isInstanceOf[CometPlan] =>
val packedKeys = BindReferences.bindReferences(
HashJoin.rewriteKeyExpr(buildKeys),
adaptivePlan.executedPlan.output)
val mode = HashedRelationBroadcastMode(packedKeys)
val exchange = BroadcastExchangeExec(mode, adaptivePlan.executedPlan)
exchange.setLogicalLink(adaptivePlan.executedPlan.logicalLink.get)
val cometExchange = cometExecRule.apply(cleanSubqueryPlan(exchange))
val canReuseExchange =
conf.exchangeReuseEnabled && buildKeys.nonEmpty && find(rootPlan) {
case CometBroadcastHashJoinExec(
_,
_,
_,
_,
_,
_,
_,
_,
BuildLeft,
left,
_,
_) =>
left.sameResult(cometExchange)
case CometBroadcastHashJoinExec(
_,
_,
_,
_,
_,
_,
_,
_,
BuildRight,
_,
right,
_) =>
right.sameResult(cometExchange)
case _ => false
}.isDefined

if (canReuseExchange) {
val newAdaptivePlan = adaptivePlan.copy(inputPlan = cometExchange)
val broadcastValues =
SubqueryBroadcastExec(name, index, buildKeys, newAdaptivePlan)
broadcastValues

} else {
s
}

case s: UnaryExecNode =>
val child = cleanSubqueryPlan(sub.plan.child)
val newChild = cometExecRule.apply(child)
s.withNewChildren(Seq(newChild))
case o => o
}
sub.withNewPlan(newChild.asInstanceOf[BaseSubqueryExec])
}
}
}
replace_subquery(plan)
}
}
}

object CometSparkSessionExtensions extends Logging {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -38,9 +38,10 @@ import org.apache.spark.sql.catalyst.expressions.Attribute
import org.apache.spark.sql.catalyst.plans.logical.Statistics
import org.apache.spark.sql.comet.execution.shuffle.ArrowReaderIterator
import org.apache.spark.sql.errors.QueryExecutionErrors
import org.apache.spark.sql.execution.{ColumnarToRowExec, SparkPlan, SQLExecution}
import org.apache.spark.sql.execution.{ColumnarToRowExec, LocalTableScanExec, RowToColumnarExec, SparkPlan, SQLExecution}
import org.apache.spark.sql.execution.adaptive.{AQEShuffleReadExec, ShuffleQueryStageExec}
import org.apache.spark.sql.execution.exchange.{BroadcastExchangeLike, ReusedExchangeExec}
import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, BroadcastExchangeLike, ReusedExchangeExec}
import org.apache.spark.sql.execution.joins.{CometHashedRelation, HashedRelationBroadcastMode}
import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics}
import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf}
import org.apache.spark.sql.vectorized.ColumnarBatch
Expand Down Expand Up @@ -143,6 +144,9 @@ case class CometBroadcastExchangeExec(
case ShuffleQueryStageExec(_, ReusedExchangeExec(_, plan), _)
if plan.isInstanceOf[CometPlan] =>
CometExec.getByteArrayRdd(plan.asInstanceOf[CometPlan]).collect()
// see AQEPropagateEmptyRelation
case RowToColumnarExec(LocalTableScanExec(_, rows)) if rows.isEmpty =>
Array.empty[(Long, ChunkedByteBuffer)]
case AQEShuffleReadExec(s: ShuffleQueryStageExec, _) =>
throw new CometRuntimeException(
"Child of CometBroadcastExchangeExec should be CometExec, " +
Expand Down Expand Up @@ -177,11 +181,18 @@ case class CometBroadcastExchangeExec(
dataSize)
}

val relation = CometHashedRelation(
batches,
originalPlan
.asInstanceOf[BroadcastExchangeExec]
.mode
.asInstanceOf[HashedRelationBroadcastMode])

val beforeBroadcast = System.nanoTime()
longMetric("buildTime") += NANOSECONDS.toMillis(beforeBroadcast - beforeBuild)

// (3.4 only) SPARK-39983 - Broadcast the relation without caching the unserialized object.
val broadcasted = doBroadcast(sparkContext, batches)
val broadcasted = doBroadcast(sparkContext, relation)
.asInstanceOf[broadcast.Broadcast[Any]]
longMetric("broadcastTime") += NANOSECONDS.toMillis(System.nanoTime() - beforeBroadcast)
val executionId = sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY)
Expand Down Expand Up @@ -233,7 +244,7 @@ case class CometBroadcastExchangeExec(

// This is basically for unit test only, called by `executeCollect` indirectly.
override protected def doExecuteColumnar(): RDD[ColumnarBatch] = {
val broadcasted = executeBroadcast[Array[ChunkedByteBuffer]]()
val broadcasted = executeBroadcast[CometHashedRelation]()

new CometBatchRDD(sparkContext, getNumPartitions(), broadcasted)
}
Expand Down Expand Up @@ -293,7 +304,7 @@ object CometBroadcastExchangeExec {
class CometBatchRDD(
sc: SparkContext,
numPartitions: Int,
value: broadcast.Broadcast[Array[ChunkedByteBuffer]])
value: broadcast.Broadcast[CometHashedRelation])
extends RDD[ColumnarBatch](sc, Nil) {

override def getPartitions: Array[Partition] = (0 until numPartitions).toArray.map { i =>
Expand All @@ -302,7 +313,7 @@ class CometBatchRDD(

override def compute(split: Partition, context: TaskContext): Iterator[ColumnarBatch] = {
val partition = split.asInstanceOf[CometBatchPartition]
partition.value.value.toIterator
partition.value.value.values.toIterator
.flatMap(decodeBatches(_, this.getClass.getSimpleName))
}

Expand All @@ -325,5 +336,5 @@ class CometBatchRDD(

class CometBatchPartition(
override val index: Int,
val value: broadcast.Broadcast[Array[ChunkedByteBuffer]])
val value: broadcast.Broadcast[CometHashedRelation])
extends Partition {}
Loading
Loading