You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@kyuubi.apache.org by "ulysses-you (via GitHub)" <gi...@apache.org> on 2023/04/04 09:39:22 UTC

[GitHub] [kyuubi] ulysses-you commented on a diff in pull request #4662: [ARROW] Arrow serialization should not introduce extra shuffle for outermost limit

ulysses-you commented on code in PR #4662:
URL: https://github.com/apache/kyuubi/pull/4662#discussion_r1156978732


##########
externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConvertersHelper.scala:
##########
@@ -0,0 +1,148 @@
+/*
+ * 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.spark.sql.execution.arrow
+
+import java.io.ByteArrayOutputStream
+import java.nio.channels.Channels
+
+import org.apache.arrow.vector._
+import org.apache.arrow.vector.ipc.{ArrowStreamWriter, WriteChannel}
+import org.apache.arrow.vector.ipc.message.{IpcOption, MessageSerializer}
+import org.apache.spark.TaskContext
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.UnsafeRow
+import org.apache.spark.sql.types._
+import org.apache.spark.sql.util.ArrowUtils
+import org.apache.spark.util.Utils
+
+object ArrowConvertersHelper extends Logging {
+
+  /**
+   * Different from [[org.apache.spark.sql.execution.arrow.ArrowConvertersHelper.toBatchIterator]],
+   * each output arrow batch contains this batch row count.
+   */
+  def toBatchIterator(
+      rowIter: Iterator[InternalRow],
+      schema: StructType,
+      maxRecordsPerBatch: Long,
+      maxEstimatedBatchSize: Long,
+      limit: Long,
+      timeZoneId: String): ArrowBatchIterator = {
+    new ArrowBatchIterator(
+      rowIter,
+      schema,
+      maxRecordsPerBatch,
+      maxEstimatedBatchSize,
+      limit,
+      timeZoneId,
+      TaskContext.get)
+  }
+
+  private[sql] class ArrowBatchIterator(
+      rowIter: Iterator[InternalRow],
+      schema: StructType,
+      maxRecordsPerBatch: Long,
+      maxEstimatedBatchSize: Long,
+      limit: Long,
+      timeZoneId: String,
+      context: TaskContext)
+    extends Iterator[Array[Byte]] {
+
+    protected val arrowSchema = ArrowUtils.toArrowSchema(schema, timeZoneId)
+    private val allocator =
+      ArrowUtils.rootAllocator.newChildAllocator(
+        s"to${this.getClass.getSimpleName}",
+        0,
+        Long.MaxValue)
+
+    private val root = VectorSchemaRoot.create(arrowSchema, allocator)
+    protected val unloader = new VectorUnloader(root)
+    protected val arrowWriter = ArrowWriter.create(root)
+
+    Option(context).foreach {
+      _.addTaskCompletionListener[Unit] { _ =>
+        root.close()
+        allocator.close()
+      }
+    }
+
+    override def hasNext: Boolean = (rowIter.hasNext && rowCount < limit) || {
+      root.close()
+      allocator.close()
+      false
+    }
+
+    var rowCountInLastBatch: Long = 0
+    var rowCount: Long = 0
+
+    override def next(): Array[Byte] = {
+      val out = new ByteArrayOutputStream()
+      val writeChannel = new WriteChannel(Channels.newChannel(out))
+
+      rowCountInLastBatch = 0
+      var estimatedBatchSize = 0

Review Comment:
   should it be long ?



##########
externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConvertersHelper.scala:
##########
@@ -0,0 +1,148 @@
+/*
+ * 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.spark.sql.execution.arrow
+
+import java.io.ByteArrayOutputStream
+import java.nio.channels.Channels
+
+import org.apache.arrow.vector._
+import org.apache.arrow.vector.ipc.{ArrowStreamWriter, WriteChannel}
+import org.apache.arrow.vector.ipc.message.{IpcOption, MessageSerializer}
+import org.apache.spark.TaskContext
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.UnsafeRow
+import org.apache.spark.sql.types._
+import org.apache.spark.sql.util.ArrowUtils
+import org.apache.spark.util.Utils
+
+object ArrowConvertersHelper extends Logging {
+
+  /**
+   * Different from [[org.apache.spark.sql.execution.arrow.ArrowConvertersHelper.toBatchIterator]],
+   * each output arrow batch contains this batch row count.
+   */
+  def toBatchIterator(
+      rowIter: Iterator[InternalRow],
+      schema: StructType,
+      maxRecordsPerBatch: Long,
+      maxEstimatedBatchSize: Long,
+      limit: Long,
+      timeZoneId: String): ArrowBatchIterator = {
+    new ArrowBatchIterator(
+      rowIter,
+      schema,
+      maxRecordsPerBatch,
+      maxEstimatedBatchSize,
+      limit,
+      timeZoneId,
+      TaskContext.get)
+  }
+
+  private[sql] class ArrowBatchIterator(
+      rowIter: Iterator[InternalRow],
+      schema: StructType,
+      maxRecordsPerBatch: Long,
+      maxEstimatedBatchSize: Long,
+      limit: Long,
+      timeZoneId: String,
+      context: TaskContext)
+    extends Iterator[Array[Byte]] {
+
+    protected val arrowSchema = ArrowUtils.toArrowSchema(schema, timeZoneId)
+    private val allocator =
+      ArrowUtils.rootAllocator.newChildAllocator(
+        s"to${this.getClass.getSimpleName}",
+        0,
+        Long.MaxValue)
+
+    private val root = VectorSchemaRoot.create(arrowSchema, allocator)
+    protected val unloader = new VectorUnloader(root)
+    protected val arrowWriter = ArrowWriter.create(root)
+
+    Option(context).foreach {
+      _.addTaskCompletionListener[Unit] { _ =>
+        root.close()
+        allocator.close()
+      }
+    }
+
+    override def hasNext: Boolean = (rowIter.hasNext && rowCount < limit) || {
+      root.close()
+      allocator.close()
+      false
+    }
+
+    var rowCountInLastBatch: Long = 0
+    var rowCount: Long = 0
+
+    override def next(): Array[Byte] = {
+      val out = new ByteArrayOutputStream()
+      val writeChannel = new WriteChannel(Channels.newChannel(out))
+
+      rowCountInLastBatch = 0
+      var estimatedBatchSize = 0
+      Utils.tryWithSafeFinally {
+
+        // Always write the first row.
+        while (rowIter.hasNext && (
+            // For maxBatchSize and maxRecordsPerBatch, respect whatever smaller.
+            // If the size in bytes is positive (set properly), always write the first row.
+            rowCountInLastBatch == 0 && maxEstimatedBatchSize > 0 ||
+              // If the size in bytes of rows are 0 or negative, unlimit it.
+              estimatedBatchSize <= 0 ||
+              estimatedBatchSize < maxEstimatedBatchSize ||
+              // If the size of rows are 0 or negative, unlimit it.
+              maxRecordsPerBatch <= 0 ||
+              rowCountInLastBatch < maxRecordsPerBatch ||
+              rowCount < limit)) {
+          val row = rowIter.next()
+          arrowWriter.write(row)
+          estimatedBatchSize += (row match {
+            case ur: UnsafeRow => ur.getSizeInBytes
+            // Trying to estimate the size of the current row, assuming 16 bytes per value.
+            case ir: InternalRow => ir.numFields * 16

Review Comment:
   in general, we can infer row size by `schema.defaultSize`



##########
externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/sql/kyuubi/SparkDatasetHelper.scala:
##########
@@ -17,18 +17,95 @@
 
 package org.apache.spark.sql.kyuubi
 
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.TaskContext
+import org.apache.spark.network.util.{ByteUnit, JavaUtils}
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.{DataFrame, Dataset, Row}
+import org.apache.spark.sql.execution.{CollectLimitExec, SparkPlan, SQLExecution}
+import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec
+import org.apache.spark.sql.execution.arrow.{ArrowCollectUtils, ArrowConverters, KyuubiArrowUtils}
 import org.apache.spark.sql.functions._
 import org.apache.spark.sql.types._
 
+import org.apache.kyuubi.engine.spark.KyuubiSparkUtil
 import org.apache.kyuubi.engine.spark.schema.RowSet
 
 object SparkDatasetHelper {
+
   def toArrowBatchRdd[T](ds: Dataset[T]): RDD[Array[Byte]] = {
     ds.toArrowBatchRdd
   }
 
+  /**
+   * Forked from [[Dataset.toArrowBatchRdd(plan: SparkPlan)]].
+   * Convert to an RDD of serialized ArrowRecordBatches.
+   */
+  def toArrowBatchRdd(plan: SparkPlan): RDD[Array[Byte]] = {
+    val schemaCaptured = plan.schema
+    val maxRecordsPerBatch = plan.session.sessionState.conf.arrowMaxRecordsPerBatch
+    val timeZoneId = plan.session.sessionState.conf.sessionLocalTimeZone
+    plan.execute().mapPartitionsInternal { iter =>
+      val context = TaskContext.get()
+      ArrowConverters.toBatchIterator(
+        iter,
+        schemaCaptured,
+        maxRecordsPerBatch,
+        timeZoneId,
+        context)
+    }
+  }
+
+  def doCollectLimit(collectLimit: CollectLimitExec): Array[Array[Byte]] = {
+    val timeZoneId = collectLimit.session.sessionState.conf.sessionLocalTimeZone
+    val maxRecordsPerBatch = collectLimit.session.sessionState.conf.arrowMaxRecordsPerBatch
+
+    val batches = ArrowCollectUtils.takeAsArrowBatches(
+      collectLimit,
+      maxRecordsPerBatch,
+      maxBatchSize,
+      timeZoneId)
+
+    // note that the number of rows in the returned arrow batches may be >= `limit`, preform
+    // the slicing operation of result
+    val result = ArrayBuffer[Array[Byte]]()
+    var i = 0
+    var rest = collectLimit.limit
+    while (i < batches.length && rest > 0) {
+      val (batch, size) = batches(i)
+      if (size <= rest) {
+        result += batch
+        // returned ArrowRecordBatch has less than `limit` row count, safety to do conversion
+        rest -= size.toInt
+      } else { // size > rest
+        result += KyuubiArrowUtils.slice(collectLimit.schema, timeZoneId, batch, 0, rest)
+        rest = 0
+      }
+      i += 1
+    }
+    result.toArray
+  }
+
+  def executeCollect(df: DataFrame): Array[Array[Byte]] = withNewExecutionId(df) {
+    executeArrowBatchCollect(df.queryExecution.executedPlan)
+  }
+
+  def toArrowBatchLocalIterator(df: DataFrame): Iterator[Array[Byte]] = {
+    withNewExecutionId(df) {
+      toArrowBatchRdd(df).toLocalIterator
+    }
+  }
+
+  def executeArrowBatchCollect: SparkPlan => Array[Array[Byte]] = {
+    case adaptiveSparkPlan: AdaptiveSparkPlanExec =>
+      executeArrowBatchCollect(adaptiveSparkPlan.finalPhysicalPlan)

Review Comment:
   sall we reflect some related private method to workaround ? it's unacceptable if we break the compatibility.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org
For additional commands, e-mail: notifications-help@kyuubi.apache.org