You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@kyuubi.apache.org by "cxzl25 (via GitHub)" <gi...@apache.org> on 2023/04/07 08:33:29 UTC

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

cxzl25 commented on code in PR #4662:
URL: https://github.com/apache/kyuubi/pull/4662#discussion_r1160540646


##########
externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/sql/execution/arrow/KyuubiArrowConverters.scala:
##########
@@ -0,0 +1,309 @@
+/*
+ * 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.{ByteArrayInputStream, ByteArrayOutputStream}
+import java.nio.channels.Channels
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.arrow.vector._
+import org.apache.arrow.vector.ipc.{ArrowStreamWriter, ReadChannel, 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.SparkSession
+import org.apache.spark.sql.catalyst.{InternalRow, SQLConfHelper}
+import org.apache.spark.sql.catalyst.expressions.UnsafeRow
+import org.apache.spark.sql.execution.CollectLimitExec
+import org.apache.spark.sql.types._
+import org.apache.spark.sql.util.ArrowUtils
+import org.apache.spark.util.Utils
+
+object KyuubiArrowConverters extends SQLConfHelper with Logging {
+
+  type Batch = (Array[Byte], Long)
+
+  private val rootAllocator = ArrowUtils.rootAllocator.newChildAllocator(
+    s"to${this.getClass.getSimpleName}",
+    0,
+    Long.MaxValue)
+
+  /**
+   * this method is to slice the input Arrow record batch byte array `bytes`, starting from `start`
+   * and taking `length` number of elements.
+   */
+  def slice(
+      schema: StructType,
+      timeZoneId: String,
+      bytes: Array[Byte],
+      start: Int,
+      length: Int): Array[Byte] = {
+    val in = new ByteArrayInputStream(bytes)
+    val out = new ByteArrayOutputStream(bytes.length)
+
+    try {
+      val recordBatch = MessageSerializer.deserializeRecordBatch(
+        new ReadChannel(Channels.newChannel(in)),
+        rootAllocator)
+      val arrowSchema = ArrowUtils.toArrowSchema(schema, timeZoneId)
+
+      val root = VectorSchemaRoot.create(arrowSchema, rootAllocator)
+      val vectorLoader = new VectorLoader(root)
+      vectorLoader.load(recordBatch)
+      recordBatch.close()
+
+      val unloader = new VectorUnloader(root.slice(start, length))
+      val writeChannel = new WriteChannel(Channels.newChannel(out))
+      val batch = unloader.getRecordBatch()
+      MessageSerializer.serialize(writeChannel, batch)
+      batch.close()
+      out.toByteArray()
+    } finally {
+      in.close()
+      out.close()
+    }
+  }
+
+  /**
+   * Forked from `org.apache.spark.sql.execution.SparkPlan#executeTake()`, the algorithm can be
+   * summarized in the following steps:
+   * 1. If the limit specified in the CollectLimitExec object is 0, the function returns an empty
+   *    array of batches.
+   * 2. Otherwise, execute the child query plan of the CollectLimitExec object to obtain an RDD of
+   *    data to collect.
+   * 3. Use an iterative approach to collect data in batches until the specified limit is reached.
+   *    In each iteration, it selects a subset of the partitions of the RDD to scan and tries to
+   *    collect data from them.
+   * 4. For each partition subset, we use the runJob method of the Spark context to execute a
+   *    closure that scans the partition data and converts it to Arrow batches.
+   * 5. Check if the collected data reaches the specified limit. If not, it selects another subset
+   *    of partitions to scan and repeats the process until the limit is reached or all partitions
+   *    have been scanned.
+   * 6. Return an array of all the collected Arrow batches.
+   *
+   * Note that:
+   * 1. The returned Arrow batches row count >= limit, if the input df has more than the `limit`
+   *    row count
+   * 2. We don't implement the `takeFromEnd` logical
+   *
+   * @return
+   */
+  def takeAsArrowBatches(
+      collectLimitExec: CollectLimitExec,
+      maxRecordsPerBatch: Long,
+      maxEstimatedBatchSize: Long,
+      timeZoneId: String): Array[Batch] = {
+    val n = collectLimitExec.limit
+    val schema = collectLimitExec.schema
+    if (n == 0) {
+      return new Array[Batch](0)

Review Comment:
   nit: remove `return`



##########
externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/sql/execution/arrow/KyuubiArrowConverters.scala:
##########
@@ -0,0 +1,309 @@
+/*
+ * 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.{ByteArrayInputStream, ByteArrayOutputStream}
+import java.nio.channels.Channels
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.arrow.vector._
+import org.apache.arrow.vector.ipc.{ArrowStreamWriter, ReadChannel, 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.SparkSession
+import org.apache.spark.sql.catalyst.{InternalRow, SQLConfHelper}
+import org.apache.spark.sql.catalyst.expressions.UnsafeRow
+import org.apache.spark.sql.execution.CollectLimitExec
+import org.apache.spark.sql.types._
+import org.apache.spark.sql.util.ArrowUtils
+import org.apache.spark.util.Utils
+
+object KyuubiArrowConverters extends SQLConfHelper with Logging {
+
+  type Batch = (Array[Byte], Long)
+
+  private val rootAllocator = ArrowUtils.rootAllocator.newChildAllocator(
+    s"to${this.getClass.getSimpleName}",
+    0,
+    Long.MaxValue)
+
+  /**
+   * this method is to slice the input Arrow record batch byte array `bytes`, starting from `start`
+   * and taking `length` number of elements.
+   */
+  def slice(
+      schema: StructType,
+      timeZoneId: String,
+      bytes: Array[Byte],
+      start: Int,
+      length: Int): Array[Byte] = {
+    val in = new ByteArrayInputStream(bytes)
+    val out = new ByteArrayOutputStream(bytes.length)
+
+    try {
+      val recordBatch = MessageSerializer.deserializeRecordBatch(
+        new ReadChannel(Channels.newChannel(in)),
+        rootAllocator)
+      val arrowSchema = ArrowUtils.toArrowSchema(schema, timeZoneId)
+
+      val root = VectorSchemaRoot.create(arrowSchema, rootAllocator)
+      val vectorLoader = new VectorLoader(root)
+      vectorLoader.load(recordBatch)
+      recordBatch.close()
+
+      val unloader = new VectorUnloader(root.slice(start, length))
+      val writeChannel = new WriteChannel(Channels.newChannel(out))
+      val batch = unloader.getRecordBatch()
+      MessageSerializer.serialize(writeChannel, batch)
+      batch.close()
+      out.toByteArray()
+    } finally {
+      in.close()
+      out.close()
+    }
+  }
+
+  /**
+   * Forked from `org.apache.spark.sql.execution.SparkPlan#executeTake()`, the algorithm can be
+   * summarized in the following steps:
+   * 1. If the limit specified in the CollectLimitExec object is 0, the function returns an empty
+   *    array of batches.
+   * 2. Otherwise, execute the child query plan of the CollectLimitExec object to obtain an RDD of
+   *    data to collect.
+   * 3. Use an iterative approach to collect data in batches until the specified limit is reached.
+   *    In each iteration, it selects a subset of the partitions of the RDD to scan and tries to
+   *    collect data from them.
+   * 4. For each partition subset, we use the runJob method of the Spark context to execute a
+   *    closure that scans the partition data and converts it to Arrow batches.
+   * 5. Check if the collected data reaches the specified limit. If not, it selects another subset
+   *    of partitions to scan and repeats the process until the limit is reached or all partitions
+   *    have been scanned.
+   * 6. Return an array of all the collected Arrow batches.
+   *
+   * Note that:
+   * 1. The returned Arrow batches row count >= limit, if the input df has more than the `limit`
+   *    row count
+   * 2. We don't implement the `takeFromEnd` logical
+   *
+   * @return
+   */
+  def takeAsArrowBatches(
+      collectLimitExec: CollectLimitExec,
+      maxRecordsPerBatch: Long,
+      maxEstimatedBatchSize: Long,
+      timeZoneId: String): Array[Batch] = {
+    val n = collectLimitExec.limit
+    val schema = collectLimitExec.schema
+    if (n == 0) {
+      return new Array[Batch](0)
+    } else {
+      val limitScaleUpFactor = Math.max(conf.limitScaleUpFactor, 2)
+      // TODO: refactor and reuse the code from RDD's take()
+      val childRDD = collectLimitExec.child.execute()
+      val buf = new ArrayBuffer[Batch]
+      var bufferedRowSize = 0L
+      val totalParts = childRDD.partitions.length
+      var partsScanned = 0
+      while (bufferedRowSize < n && partsScanned < totalParts) {
+        // The number of partitions to try in this iteration. It is ok for this number to be
+        // greater than totalParts because we actually cap it at totalParts in runJob.
+        var numPartsToTry = limitInitialNumPartitions
+        if (partsScanned > 0) {
+          // If we didn't find any rows after the previous iteration, multiply by
+          // limitScaleUpFactor and retry. Otherwise, interpolate the number of partitions we need
+          // to try, but overestimate it by 50%. We also cap the estimation in the end.
+          if (buf.isEmpty) {
+            numPartsToTry = partsScanned * limitScaleUpFactor
+          } else {
+            val left = n - bufferedRowSize
+            // As left > 0, numPartsToTry is always >= 1
+            numPartsToTry = Math.ceil(1.5 * left * partsScanned / bufferedRowSize).toInt
+            numPartsToTry = Math.min(numPartsToTry, partsScanned * limitScaleUpFactor)
+          }
+        }
+
+        val partsToScan =
+          partsScanned.until(math.min(partsScanned + numPartsToTry, totalParts).toInt)

Review Comment:
   nit: remove `toInt`
   



-- 
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