You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by vi...@apache.org on 2024/02/22 19:26:17 UTC

(arrow-datafusion-comet) branch main updated: fix: Fix compilation error for CometBroadcastExchangeExec (#86)

This is an automated email from the ASF dual-hosted git repository.

viirya pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/arrow-datafusion-comet.git


The following commit(s) were added to refs/heads/main by this push:
     new 0cca52e  fix: Fix compilation error for CometBroadcastExchangeExec (#86)
0cca52e is described below

commit 0cca52e93fe27635676c72bfadcbb049302aaba4
Author: Liang-Chi Hsieh <vi...@gmail.com>
AuthorDate: Thu Feb 22 11:26:12 2024 -0800

    fix: Fix compilation error for CometBroadcastExchangeExec (#86)
---
 .../shims/ShimCometBroadcastExchangeExec.scala     | 51 ++++++++++++++++++++++
 .../sql/comet/CometBroadcastExchangeExec.scala     | 29 ++++++++----
 2 files changed, 72 insertions(+), 8 deletions(-)

diff --git a/spark/src/main/scala/org/apache/comet/shims/ShimCometBroadcastExchangeExec.scala b/spark/src/main/scala/org/apache/comet/shims/ShimCometBroadcastExchangeExec.scala
new file mode 100644
index 0000000..63ff2a2
--- /dev/null
+++ b/spark/src/main/scala/org/apache/comet/shims/ShimCometBroadcastExchangeExec.scala
@@ -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.shims
+
+import scala.reflect.ClassTag
+
+import org.apache.spark.SparkContext
+import org.apache.spark.broadcast.Broadcast
+
+trait ShimCometBroadcastExchangeExec {
+  // TODO: remove after dropping Spark 3.2 and 3.3 support
+  protected def doBroadcast[T: ClassTag](sparkContext: SparkContext, value: T): Broadcast[Any] = {
+    // Spark 3.4 has new API `broadcastInternal` to broadcast the relation without caching the
+    // unserialized object.
+    val classTag = implicitly[ClassTag[T]]
+    val broadcasted = sparkContext.getClass.getDeclaredMethods
+      .filter(_.getName == "broadcastInternal")
+      .map { a => a.setAccessible(true); a }
+      .map { method =>
+        method
+          .invoke(
+            sparkContext.asInstanceOf[Object],
+            value.asInstanceOf[Object],
+            true.asInstanceOf[Object],
+            classTag.asInstanceOf[Object])
+          .asInstanceOf[Broadcast[Any]]
+      }
+      .headOption
+    // Fallback to the old API if the new API is not available.
+    broadcasted
+      .getOrElse(sparkContext.broadcast(value.asInstanceOf[Object]))
+      .asInstanceOf[Broadcast[Any]]
+  }
+}
diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometBroadcastExchangeExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometBroadcastExchangeExec.scala
index 3b886a9..f115b2a 100644
--- a/spark/src/main/scala/org/apache/spark/sql/comet/CometBroadcastExchangeExec.scala
+++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometBroadcastExchangeExec.scala
@@ -27,11 +27,12 @@ import scala.concurrent.duration.NANOSECONDS
 import scala.util.control.NonFatal
 
 import org.apache.spark.{broadcast, Partition, SparkContext, TaskContext}
+import org.apache.spark.launcher.SparkLauncher
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.plans.logical.Statistics
 import org.apache.spark.sql.errors.QueryExecutionErrors
-import org.apache.spark.sql.execution.{ColumnarToRowExec, FileSourceScanExec, SparkPlan, SQLExecution}
+import org.apache.spark.sql.execution.{ColumnarToRowExec, SparkPlan, SQLExecution}
 import org.apache.spark.sql.execution.exchange.BroadcastExchangeLike
 import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics}
 import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf}
@@ -41,6 +42,8 @@ import org.apache.spark.util.io.ChunkedByteBuffer
 
 import com.google.common.base.Objects
 
+import org.apache.comet.shims.ShimCometBroadcastExchangeExec
+
 /**
  * A [[CometBroadcastExchangeExec]] collects, transforms and finally broadcasts the result of a
  * transformed SparkPlan. This is a copy of the [[BroadcastExchangeExec]] class with the necessary
@@ -51,9 +54,13 @@ import com.google.common.base.Objects
  * Note that this class cannot extend `CometExec` as usual similar to other Comet operators. As
  * the trait `BroadcastExchangeLike` in Spark extends abstract class `Exchange`, it limits the
  * flexibility to extend `CometExec` and `Exchange` at the same time.
+ *
+ * Note that this only supports Spark 3.4 and later, because the serialization class
+ * `ChunkedByteBuffer` is only serializable in Spark 3.4 and later.
  */
 case class CometBroadcastExchangeExec(originalPlan: SparkPlan, child: SparkPlan)
-    extends BroadcastExchangeLike {
+    extends BroadcastExchangeLike
+    with ShimCometBroadcastExchangeExec {
   import CometBroadcastExchangeExec._
 
   override val runId: UUID = UUID.randomUUID
@@ -129,9 +136,8 @@ case class CometBroadcastExchangeExec(originalPlan: SparkPlan, child: SparkPlan)
         val beforeBroadcast = System.nanoTime()
         longMetric("buildTime") += NANOSECONDS.toMillis(beforeBroadcast - beforeBuild)
 
-        // SPARK-39983 - Broadcast the relation without caching the unserialized object.
-        val broadcasted = sparkContext
-          .broadcastInternal(batches, serializedOnly = true)
+        // (3.4 only) SPARK-39983 - Broadcast the relation without caching the unserialized object.
+        val broadcasted = doBroadcast(sparkContext, batches)
           .asInstanceOf[broadcast.Broadcast[Any]]
         longMetric("broadcastTime") += NANOSECONDS.toMillis(System.nanoTime() - beforeBroadcast)
         val executionId = sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY)
@@ -143,9 +149,16 @@ case class CometBroadcastExchangeExec(originalPlan: SparkPlan, child: SparkPlan)
         // SparkFatalException, which is a subclass of Exception. ThreadUtils.awaitResult
         // will catch this exception and re-throw the wrapped fatal throwable.
         case oe: OutOfMemoryError =>
-          val tables = child.collect { case f: FileSourceScanExec => f.tableIdentifier }.flatten
-          val ex = new SparkFatalException(
-            QueryExecutionErrors.notEnoughMemoryToBuildAndBroadcastTableError(oe, tables))
+          // Spark 3.4 has two parameters for `notEnoughMemoryToBuildAndBroadcastTableError`, which
+          // is different to Spark 3.3. We simply create the error message here.
+          val error =
+            new OutOfMemoryError(
+              "Not enough memory to build and broadcast the table to all " +
+                "worker nodes. As a workaround, you can either disable broadcast by setting " +
+                s"${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key} to -1 or increase the spark " +
+                s"driver memory by setting ${SparkLauncher.DRIVER_MEMORY} to a higher value.")
+              .initCause(oe.getCause)
+          val ex = new SparkFatalException(error)
           promise.tryFailure(ex)
           throw ex
         case e if !NonFatal(e) =>