WangGuangxin commented on code in PR #8931:
URL: https://github.com/apache/incubator-gluten/pull/8931#discussion_r2450194723
##########
backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarBuildSideRelation.scala:
##########
@@ -133,6 +148,85 @@ case class ColumnarBuildSideRelation(
override def asReadOnlyCopy(): ColumnarBuildSideRelation = this
+ private var hashTableData: Long = 0L
+
+ def buildHashTable(
+ broadCastContext: BroadcastHashJoinContext): (Long,
ColumnarBuildSideRelation) =
+ synchronized {
+ if (hashTableData == 0) {
+ val runtime = Runtimes.contextInstance(
+ BackendsApiManager.getBackendName,
+ "ColumnarBuildSideRelation#buildHashTable")
+ val jniWrapper = ColumnarBatchSerializerJniWrapper.create(runtime)
+ val serializeHandle: Long = {
+ val allocator = ArrowBufferAllocators.contextInstance()
+ val cSchema = ArrowSchema.allocateNew(allocator)
+ val arrowSchema = SparkArrowUtil.toArrowSchema(
+ SparkShimLoader.getSparkShims.structFromAttributes(output),
+ SQLConf.get.sessionLocalTimeZone)
+ ArrowAbiUtil.exportSchema(allocator, arrowSchema, cSchema)
+ val handle = jniWrapper
+ .init(cSchema.memoryAddress())
+ cSchema.close()
+ handle
+ }
+
+ val batchArray = new ArrayBuffer[Long]
+
+ var batchId = 0
+ while (batchId < batches.size) {
+ batchArray.append(jniWrapper.deserialize(serializeHandle,
batches(batchId)))
+ batchId += 1
+ }
+
+ logDebug(
+ s"BHJ value size: " +
+ s"${broadCastContext.buildHashTableId} = ${batches.length}")
+
+ val (keys, newOutput) = if (newBuildKeys.isEmpty) {
+ (
+ broadCastContext.buildSideJoinKeys.asJava,
+ broadCastContext.buildSideStructure.asJava
+ )
+ } else {
+ (
+ newBuildKeys.asJava,
+ output.asJava
+ )
+ }
+
+ val joinKey = keys.asScala
+ .map {
+ key =>
+ val attr = ConverterUtils.getAttrFromExpr(key)
+ ConverterUtils.genColumnNameWithExprId(attr)
+ }
+ .mkString(",")
+
+ // Build the hash table
+ hashTableData = HashJoinBuilder
+ .nativeBuild(
+ broadCastContext.buildHashTableId,
+ batchArray.toArray,
+ joinKey,
+ broadCastContext.substraitJoinType.ordinal(),
+ broadCastContext.hasMixedFiltCondition,
+ broadCastContext.isExistenceJoin,
+ SubstraitUtil.toNameStruct(newOutput).toByteArray,
+ broadCastContext.isNullAwareAntiJoin
+ )
+
+ jniWrapper.close(serializeHandle)
+ (hashTableData, this)
+ } else {
+ (HashJoinBuilder.cloneHashTable(hashTableData), null)
+ }
+ }
+
+ def reset(): Unit = synchronized {
+ hashTableData = 0
Review Comment:
When we invalid a `buildHashTableId`, why we only set `hashTableData` = 0,
but leaving the real data still in `gluten::hashTableObjStore`?
##########
backends-velox/src/main/scala/org/apache/gluten/execution/VeloxBroadcastBuildSideCache.scala:
##########
@@ -0,0 +1,112 @@
+/*
+ * 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.gluten.execution
+
+import org.apache.gluten.backendsapi.velox.VeloxBackendSettings
+import org.apache.gluten.vectorized.HashJoinBuilder
+
+import org.apache.spark.SparkEnv
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.execution.ColumnarBuildSideRelation
+import org.apache.spark.sql.execution.joins.BuildSideRelation
+import org.apache.spark.sql.execution.unsafe.UnsafeColumnarBuildSideRelation
+
+import com.github.benmanes.caffeine.cache.{Cache, Caffeine, RemovalCause,
RemovalListener}
+
+import java.util.concurrent.TimeUnit
+
+case class BroadcastHashTable(pointer: Long, relation: BuildSideRelation)
+
+/**
+ * `VeloxBroadcastBuildSideCache` is used for controlling to build bhj hash
table once.
+ *
+ * The complicated part is due to reuse exchange, where multiple BHJ IDs
correspond to a
+ * `BuildSideRelation`.
+ */
+object VeloxBroadcastBuildSideCache
+ extends Logging
+ with RemovalListener[String, BroadcastHashTable] {
+
+ private lazy val expiredTime = SparkEnv.get.conf.getLong(
+ VeloxBackendSettings.GLUTEN_VELOX_BROADCAST_CACHE_EXPIRED_TIME,
+ VeloxBackendSettings.GLUTEN_VELOX_BROADCAST_CACHE_EXPIRED_TIME_DEFAULT
+ )
+
+ // Use for controlling to build bhj hash table once.
+ // key: hashtable id, value is hashtable backend pointer(long to string).
+ private val buildSideRelationCache: Cache[String, BroadcastHashTable] =
+ Caffeine.newBuilder
+ .expireAfterAccess(expiredTime, TimeUnit.SECONDS)
+ .removalListener(this)
+ .build[String, BroadcastHashTable]()
+
+ def getOrBuildBroadcastHashTable(
+ broadcast: Broadcast[BuildSideRelation],
+ broadCastContext: BroadcastHashJoinContext): BroadcastHashTable =
synchronized {
Review Comment:
nit: broadCastContext => broadcastContext
--
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]