jackylee-ch commented on code in PR #8127: URL: https://github.com/apache/incubator-gluten/pull/8127#discussion_r1893760584
########## backends-velox/src/main/scala/org/apache/spark/sql/execution/unsafe/UnsafeBytesBufferArray.scala: ########## @@ -0,0 +1,165 @@ +/* + * 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.spark.internal.Logging +import org.apache.spark.memory.{MemoryConsumer, MemoryMode, TaskMemoryManager} +import org.apache.spark.unsafe.Platform +import org.apache.spark.unsafe.array.LongArray + +import java.security.MessageDigest + +/** + * Used to store broadcast variable off-heap memory for broadcast variable. The underlying data + * structure is a LongArray allocated in offheap memory. + * + * @param arraySize + * underlying array[array[byte]]'s length + * @param bytesBufferLengths + * underlying array[array[byte]] per bytesBuffer length + * @param totalBytes + * all bytesBuffer's length plus together + */ +// scalastyle:off no.finalize +case class UnsafeBytesBufferArray( + arraySize: Int, + bytesBufferLengths: Array[Int], + totalBytes: Long, + tmm: TaskMemoryManager) + extends MemoryConsumer(tmm, MemoryMode.OFF_HEAP) + with Logging { + + { + assert(arraySize == bytesBufferLengths.length) + assert(totalBytes >= 0) Review Comment: nit:add some message when assert failed, so we can find the reason from logs ########## backends-velox/src/main/scala/org/apache/spark/sql/execution/unsafe/UnsafeBytesBufferArray.scala: ########## @@ -0,0 +1,165 @@ +/* + * 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.spark.internal.Logging +import org.apache.spark.memory.{MemoryConsumer, MemoryMode, TaskMemoryManager} +import org.apache.spark.unsafe.Platform +import org.apache.spark.unsafe.array.LongArray + +import java.security.MessageDigest + +/** + * Used to store broadcast variable off-heap memory for broadcast variable. The underlying data + * structure is a LongArray allocated in offheap memory. + * + * @param arraySize + * underlying array[array[byte]]'s length + * @param bytesBufferLengths + * underlying array[array[byte]] per bytesBuffer length + * @param totalBytes + * all bytesBuffer's length plus together + */ +// scalastyle:off no.finalize +case class UnsafeBytesBufferArray( + arraySize: Int, + bytesBufferLengths: Array[Int], + totalBytes: Long, + tmm: TaskMemoryManager) + extends MemoryConsumer(tmm, MemoryMode.OFF_HEAP) + with Logging { + + { + assert(arraySize == bytesBufferLengths.length) + assert(totalBytes >= 0) + } + + /** + * A single array to store all bytesBufferArray's value, it's inited once when first time get + * accessed. + */ + private var longArray: LongArray = _ + + /** Index the start of each byteBuffer's offset to underlying LongArray's initial position. */ + private val bytesBufferOffset = if (bytesBufferLengths.isEmpty) { + new Array(0) + } else { + bytesBufferLengths.init.scanLeft(0)(_ + _) + } + + override def spill(l: Long, memoryConsumer: MemoryConsumer): Long = 0L + + /** + * Put bytesBuffer at specified array index. + * @param index + * @param bytesBuffer + */ + def putBytesBuffer(index: Int, bytesBuffer: Array[Byte]): Unit = this.synchronized { + assert(index < arraySize) + assert(bytesBuffer.length == bytesBufferLengths(index)) + // first to allocate underlying long array + if (null == longArray && index == 0) { + log.debug(s"allocate array $totalBytes, actual longArray size ${(totalBytes + 7) / 8}") + longArray = allocateArray((totalBytes + 7) / 8) + } + if (log.isDebugEnabled) { + log.debug(s"put bytesBuffer at index $index bytesBuffer's length is ${bytesBuffer.length}") + log.debug( + s"bytesBuffer at index $index " + + s"digest ${calculateMD5(bytesBuffer).mkString("Array(", ", ", ")")}") + } + Platform.copyMemory( + bytesBuffer, + Platform.BYTE_ARRAY_OFFSET, + longArray.getBaseObject, + longArray.getBaseOffset + bytesBufferOffset(index), + bytesBufferLengths(index)) + } + + /** + * Get bytesBuffer at specified index. + * @param index + * @return + */ + def getBytesBuffer(index: Int): Array[Byte] = { Review Comment: do we need a `ReadWriteLock` here? ########## backends-velox/src/main/scala/org/apache/spark/sql/execution/unsafe/UnsafeColumnarBuildSideRelation.scala: ########## @@ -0,0 +1,328 @@ +/* + * 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.internal.config.MEMORY_OFFHEAP_ENABLED +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") Review Comment: this seems like not a good log, can we delete it or refine it? -- 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]
