zhztheplayer commented on code in PR #8127: URL: https://github.com/apache/incubator-gluten/pull/8127#discussion_r1891027017
########## backends-velox/src/main/scala/org/apache/spark/sql/execution/unsafe/UnsafeColumnarBuildSideRelation.scala: ########## @@ -0,0 +1,312 @@ +/* + * 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.unsafe + +import org.apache.gluten.backendsapi.BackendsApiManager +import org.apache.gluten.columnarbatch.ColumnarBatches +import org.apache.gluten.iterator.Iterators +import org.apache.gluten.memory.arrow.alloc.ArrowBufferAllocators +import org.apache.gluten.runtime.Runtimes +import org.apache.gluten.sql.shims.SparkShimLoader +import org.apache.gluten.utils.ArrowAbiUtil +import org.apache.gluten.vectorized.{ColumnarBatchSerializerJniWrapper, NativeColumnarToRowJniWrapper} + +import org.apache.spark.{SparkEnv, TaskContext} +import org.apache.spark.internal.Logging +import org.apache.spark.memory.{TaskMemoryManager, UnifiedMemoryManager} +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, UnsafeProjection, UnsafeRow} +import org.apache.spark.sql.catalyst.plans.physical.{BroadcastMode, IdentityBroadcastMode} +import org.apache.spark.sql.execution.joins.{BuildSideRelation, HashedRelationBroadcastMode} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.utils.SparkArrowUtil +import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.spark.task.TaskResources +import org.apache.spark.util.Utils + +import com.esotericsoftware.kryo.{Kryo, KryoSerializable} +import com.esotericsoftware.kryo.io.{Input, Output} +import org.apache.arrow.c.ArrowSchema + +import java.io.{Externalizable, ObjectInput, ObjectOutput} + +import scala.collection.JavaConverters.asScalaIteratorConverter + +/** + * UnsafeColumnarBuildSideRelation should backed by offheap to avoid on-heap oom. Almost the same as + * ColumnarBuildSideRelation, we should remove ColumnarBuildSideRelation when + * UnsafeColumnarBuildSideRelation get matured. + * + * @param output + * @param batches + */ +case class UnsafeColumnarBuildSideRelation( + private var output: Seq[Attribute], + private var batches: UnsafeBytesBufferArray, + var mode: BroadcastMode) + extends BuildSideRelation + with Externalizable + with Logging + with KryoSerializable { + + // Needed for serialization + def this() = { + this(null, null.asInstanceOf[UnsafeBytesBufferArray], null) + } + + def this(output: Seq[Attribute], bytesBufferArray: Array[Array[Byte]], mode: BroadcastMode) = { + // only used in driver side when broadcast the whole batches + this( + output, + UnsafeBytesBufferArray( + bytesBufferArray.length, + bytesBufferArray.map(_.length), + bytesBufferArray.map(_.length.toLong).sum, + TaskContext.get().taskMemoryManager + ), + mode + ) + val batchesSize = bytesBufferArray.length + for (i <- 0 until batchesSize) { + val length = bytesBufferArray(i).length + log.debug(s"this $i--- $length") + batches.putBytesBuffer(i, bytesBufferArray(i)) + } + } + + // should only be used on driver to serialize this relation + override def writeExternal(out: ObjectOutput): Unit = Utils.tryOrIOException { + out.writeObject(output) + out.writeObject(mode) + out.writeInt(batches.arraySize) + out.writeObject(batches.bytesBufferLengths) + out.writeLong(batches.totalBytes) + for (i <- 0 until batches.arraySize) { + val bytes = batches.getBytesBuffer(i) + out.write(bytes) + log.debug(s"writeExternal index $i with length ${bytes.length}") + } + } + + // should only be used on driver to serialize this relation + override def write(kryo: Kryo, out: Output): Unit = Utils.tryOrIOException { + kryo.writeObject(out, output.toList) + kryo.writeObject(out, mode) + out.writeInt(batches.arraySize) + kryo.writeObject(out, batches.bytesBufferLengths) + out.writeLong(batches.totalBytes) + for (i <- 0 until batches.arraySize) { + val bytes = batches.getBytesBuffer(i) + out.write(bytes) + log.debug(s"write index $i with length ${bytes.length}") + } + } + + // should only be used on executor to deserialize this relation + override def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException { + output = in.readObject().asInstanceOf[Seq[Attribute]] + mode = in.readObject().asInstanceOf[BroadcastMode] + val totalArraySize = in.readInt() + val bytesBufferLengths = in.readObject().asInstanceOf[Array[Int]] + val totalBytes = in.readLong() + + val taskMemoryManager = new TaskMemoryManager( + new UnifiedMemoryManager(SparkEnv.get.conf, Long.MaxValue, Long.MaxValue / 2, 1), + 0) + + batches = + UnsafeBytesBufferArray(totalArraySize, bytesBufferLengths, totalBytes, taskMemoryManager) + + for (i <- 0 until totalArraySize) { + val length = bytesBufferLengths(i) + log.debug(s"readExternal $i--- ${bytesBufferLengths(i)}") + val tmpBuffer = new Array[Byte](length) + in.read(tmpBuffer) + batches.putBytesBuffer(i, tmpBuffer) + } + } + + override def read(kryo: Kryo, in: Input): Unit = Utils.tryOrIOException { + output = kryo.readObject(in, classOf[List[_]]).asInstanceOf[Seq[Attribute]] + mode = kryo.readObject(in, classOf[BroadcastMode]) + val totalArraySize = in.readInt() + val bytesBufferLengths = kryo.readObject(in, classOf[Array[Int]]) + val totalBytes = in.readLong() + + val taskMemoryManager = new TaskMemoryManager( + new UnifiedMemoryManager(SparkEnv.get.conf, Long.MaxValue, Long.MaxValue / 2, 1), + 0) + + batches = + UnsafeBytesBufferArray(totalArraySize, bytesBufferLengths, totalBytes, taskMemoryManager) + + for (i <- 0 until totalArraySize) { + val length = bytesBufferLengths(i) + log.debug(s"readExternal $i--- $length") + val tmpBuffer = new Array[Byte](length) + in.read(tmpBuffer) + batches.putBytesBuffer(i, tmpBuffer) + } + } + + private def transformProjection: UnsafeProjection = { + mode match { + case HashedRelationBroadcastMode(k, _) => UnsafeProjection.create(k) + case IdentityBroadcastMode => UnsafeProjection.create(output, output) + } + } + + override def deserialized: Iterator[ColumnarBatch] = { + val runtime = + Runtimes.contextInstance(BackendsApiManager.getBackendName, "BuildSideRelation#transform") + val jniWrapper = ColumnarBatchSerializerJniWrapper.create(runtime) + val serializeHandle: Long = { Review Comment: nit: serializeHandle -> serializerHandle ########## gluten-arrow/src/main/java/org/apache/gluten/vectorized/ColumnarBatchSerializerJniWrapper.java: ########## @@ -42,5 +42,8 @@ public long rtHandle() { public native long deserialize(long serializerHandle, byte[] data); + // Return the native ColumnarBatch handle using memory address and length + public native long deserializeDirectAddress(long serializerHandle, long offset, int len); Review Comment: nit: Can we rename the method with `deserializeDirect`? ########## backends-velox/src/main/scala/org/apache/spark/sql/execution/unsafe/UnsafeColumnarBuildSideRelation.scala: ########## @@ -0,0 +1,312 @@ +/* + * 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.unsafe + +import org.apache.gluten.backendsapi.BackendsApiManager +import org.apache.gluten.columnarbatch.ColumnarBatches +import org.apache.gluten.iterator.Iterators +import org.apache.gluten.memory.arrow.alloc.ArrowBufferAllocators +import org.apache.gluten.runtime.Runtimes +import org.apache.gluten.sql.shims.SparkShimLoader +import org.apache.gluten.utils.ArrowAbiUtil +import org.apache.gluten.vectorized.{ColumnarBatchSerializerJniWrapper, NativeColumnarToRowJniWrapper} + +import org.apache.spark.{SparkEnv, TaskContext} +import org.apache.spark.internal.Logging +import org.apache.spark.memory.{TaskMemoryManager, UnifiedMemoryManager} +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, UnsafeProjection, UnsafeRow} +import org.apache.spark.sql.catalyst.plans.physical.{BroadcastMode, IdentityBroadcastMode} +import org.apache.spark.sql.execution.joins.{BuildSideRelation, HashedRelationBroadcastMode} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.utils.SparkArrowUtil +import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.spark.task.TaskResources +import org.apache.spark.util.Utils + +import com.esotericsoftware.kryo.{Kryo, KryoSerializable} +import com.esotericsoftware.kryo.io.{Input, Output} +import org.apache.arrow.c.ArrowSchema + +import java.io.{Externalizable, ObjectInput, ObjectOutput} + +import scala.collection.JavaConverters.asScalaIteratorConverter + +/** + * UnsafeColumnarBuildSideRelation should backed by offheap to avoid on-heap oom. Almost the same as + * ColumnarBuildSideRelation, we should remove ColumnarBuildSideRelation when + * UnsafeColumnarBuildSideRelation get matured. + * + * @param output + * @param batches + */ +case class UnsafeColumnarBuildSideRelation( + private var output: Seq[Attribute], + private var batches: UnsafeBytesBufferArray, + var mode: BroadcastMode) + extends BuildSideRelation + with Externalizable + with Logging + with KryoSerializable { + + // Needed for serialization + def this() = { + this(null, null.asInstanceOf[UnsafeBytesBufferArray], null) + } + + def this(output: Seq[Attribute], bytesBufferArray: Array[Array[Byte]], mode: BroadcastMode) = { + // only used in driver side when broadcast the whole batches + this( + output, + UnsafeBytesBufferArray( + bytesBufferArray.length, + bytesBufferArray.map(_.length), + bytesBufferArray.map(_.length.toLong).sum, + TaskContext.get().taskMemoryManager + ), + mode + ) + val batchesSize = bytesBufferArray.length + for (i <- 0 until batchesSize) { + val length = bytesBufferArray(i).length + log.debug(s"this $i--- $length") + batches.putBytesBuffer(i, bytesBufferArray(i)) + } + } + + // should only be used on driver to serialize this relation + override def writeExternal(out: ObjectOutput): Unit = Utils.tryOrIOException { + out.writeObject(output) + out.writeObject(mode) + out.writeInt(batches.arraySize) + out.writeObject(batches.bytesBufferLengths) + out.writeLong(batches.totalBytes) + for (i <- 0 until batches.arraySize) { + val bytes = batches.getBytesBuffer(i) + out.write(bytes) + log.debug(s"writeExternal index $i with length ${bytes.length}") + } + } + + // should only be used on driver to serialize this relation + override def write(kryo: Kryo, out: Output): Unit = Utils.tryOrIOException { + kryo.writeObject(out, output.toList) + kryo.writeObject(out, mode) + out.writeInt(batches.arraySize) + kryo.writeObject(out, batches.bytesBufferLengths) + out.writeLong(batches.totalBytes) + for (i <- 0 until batches.arraySize) { + val bytes = batches.getBytesBuffer(i) + out.write(bytes) + log.debug(s"write index $i with length ${bytes.length}") + } + } + + // should only be used on executor to deserialize this relation + override def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException { + output = in.readObject().asInstanceOf[Seq[Attribute]] + mode = in.readObject().asInstanceOf[BroadcastMode] + val totalArraySize = in.readInt() + val bytesBufferLengths = in.readObject().asInstanceOf[Array[Int]] + val totalBytes = in.readLong() + + val taskMemoryManager = new TaskMemoryManager( + new UnifiedMemoryManager(SparkEnv.get.conf, Long.MaxValue, Long.MaxValue / 2, 1), + 0) Review Comment: Would you like to explain a bit against this code? Thanks! ########## backends-velox/src/test/scala/org/apache/gluten/execution/VeloxTPCHSuite.scala: ########## @@ -305,6 +305,18 @@ class VeloxTPCHV1BhjSuite extends VeloxTPCHSuite { } } +/** BroadcastBuildSideRelation use offheap. */ +class VeloxTPCHV1BhjOffheapSuite extends VeloxTPCHSuite { + override def subType(): String = "v1-bhj-offheap" + + override protected def sparkConf: SparkConf = { + super.sparkConf + .set("spark.sql.sources.useV1SourceList", "parquet") + .set("spark.sql.autoBroadcastJoinThreshold", "30M") + .set("spark.gluten.velox.BroadcastBuildRelationUseOffheap.enabled", "true") + } +} Review Comment: Is the test still needed? Given off-heap relation is enabled by default. Perhaps add a suite for on-heap relation? ########## shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala: ########## @@ -2246,4 +2249,12 @@ object GlutenConfig { .doc("If enabled, gluten will convert the viewfs path to hdfs path in scala side") .booleanConf .createWithDefault(false) + + val VELOX_BROADCAST_BUILD_RELATION_USE_OFFHEAP = + buildConf("spark.gluten.velox.BroadcastBuildRelationUseOffheap.enabled") Review Comment: nit: Let's use camel case. Also could rename with `spark.gluten.velox.offHeapBroadcastBuildRelation`? Note `offHeap` is also camel case to match vanilla Spark's naming convention. -- 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
