zhztheplayer commented on code in PR #8931:
URL: https://github.com/apache/incubator-gluten/pull/8931#discussion_r2904871526
##########
cpp/velox/jni/VeloxJniWrapper.cc:
##########
@@ -914,18 +922,181 @@ JNIEXPORT jobject JNICALL
Java_org_apache_gluten_execution_IcebergWriteJniWrappe
auto writer = ObjectStore::retrieve<IcebergWriter>(writerHandle);
auto writeStats = writer->writeStats();
jobject writeMetrics = env->NewObject(
- batchWriteMetricsClass,
- batchWriteMetricsConstructor,
- writeStats.numWrittenBytes,
- writeStats.numWrittenFiles,
- writeStats.writeIOTimeNs,
- writeStats.writeWallNs);
+ batchWriteMetricsClass,
+ batchWriteMetricsConstructor,
+ writeStats.numWrittenBytes,
+ writeStats.numWrittenFiles,
+ writeStats.writeIOTimeNs,
+ writeStats.writeWallNs);
return writeMetrics;
JNI_METHOD_END(nullptr)
}
#endif
+JNIEXPORT jlong JNICALL
Java_org_apache_gluten_vectorized_HashJoinBuilder_nativeBuild( // NOLINT
+ JNIEnv* env,
+ jclass,
+ jstring tableId,
+ jlongArray batchHandles,
+ jstring joinKey,
+ jint joinType,
+ jboolean hasMixedJoinCondition,
+ jboolean isExistenceJoin,
+ jbyteArray namedStruct,
+ jboolean isNullAwareAntiJoin,
+ jlong bloomFilterPushdownSize,
+ jint broadcastHashTableBuildThreads) {
+ JNI_METHOD_START
+ const auto hashTableId = jStringToCString(env, tableId);
+ const auto hashJoinKey = jStringToCString(env, joinKey);
+ const auto inputType = gluten::getByteArrayElementsSafe(env, namedStruct);
+ std::string structString{
+ reinterpret_cast<const char*>(inputType.elems()),
static_cast<std::string::size_type>(inputType.length())};
+
+ substrait::NamedStruct substraitStruct;
+ substraitStruct.ParseFromString(structString);
+
+ std::vector<facebook::velox::TypePtr> veloxTypeList;
+ veloxTypeList = SubstraitParser::parseNamedStruct(substraitStruct);
+
+ const auto& substraitNames = substraitStruct.names();
+
+ std::vector<std::string> names;
+ names.reserve(substraitNames.size());
+ for (const auto& name : substraitNames) {
+ names.emplace_back(name);
+ }
+
+ std::vector<std::shared_ptr<ColumnarBatch>> cb;
+ int handleCount = env->GetArrayLength(batchHandles);
+ auto safeArray = getLongArrayElementsSafe(env, batchHandles);
+ for (int i = 0; i < handleCount; ++i) {
+ int64_t handle = safeArray.elems()[i];
+ cb.push_back(ObjectStore::retrieve<ColumnarBatch>(handle));
+ }
+
+ size_t maxThreads = broadcastHashTableBuildThreads > 0
+ ? std::min((size_t)broadcastHashTableBuildThreads, (size_t)32)
+ : std::min((size_t)std::thread::hardware_concurrency(), (size_t)32);
+
+ // Heuristic: Each thread should process at least a certain number of
batches to justify parallelism overhead.
+ // 32 batches is roughly 128k rows, which is a reasonable granularity for a
single thread.
+ constexpr size_t kMinBatchesPerThread = 32;
+ size_t numThreads = std::min(maxThreads, (handleCount + kMinBatchesPerThread
- 1) / kMinBatchesPerThread);
+ numThreads = std::max((size_t)1, numThreads);
+
+ if (numThreads <= 1) {
+ auto builder = nativeHashTableBuild(
+ hashJoinKey,
+ names,
+ veloxTypeList,
+ joinType,
+ hasMixedJoinCondition,
+ isExistenceJoin,
+ isNullAwareAntiJoin,
+ bloomFilterPushdownSize,
+ cb,
+ defaultLeafVeloxMemoryPool());
+
+ auto mainTable = builder->uniqueTable();
+ mainTable->prepareJoinTable(
+ {},
+ facebook::velox::exec::BaseHashTable::kNoSpillInputStartPartitionBit,
+ 1'000'000,
+ builder->dropDuplicates(),
+ nullptr);
+ builder->setHashTable(std::move(mainTable));
+
+ return gluten::hashTableObjStore->save(builder);
+ }
+
+ std::vector<std::thread> threads;
+
+ std::vector<std::shared_ptr<gluten::HashTableBuilder>>
hashTableBuilders(numThreads);
+ std::vector<std::unique_ptr<facebook::velox::exec::BaseHashTable>>
otherTables(numThreads);
+
+ for (size_t t = 0; t < numThreads; ++t) {
+ size_t start = (handleCount * t) / numThreads;
+ size_t end = (handleCount * (t + 1)) / numThreads;
+
+ threads.emplace_back([&, t, start, end]() {
+ std::vector<std::shared_ptr<gluten::ColumnarBatch>> threadBatches;
+ for (size_t i = start; i < end; ++i) {
+ threadBatches.push_back(cb[i]);
+ }
+
+ auto builder = nativeHashTableBuild(
+ hashJoinKey,
+ names,
+ veloxTypeList,
+ joinType,
+ hasMixedJoinCondition,
+ isExistenceJoin,
+ isNullAwareAntiJoin,
+ bloomFilterPushdownSize,
+ threadBatches,
+ defaultLeafVeloxMemoryPool());
Review Comment:
`defaultLeafVeloxMemoryPool` should already be counted into off-heap memory
pool, but I suggest to verify it if you'd like to.
##########
backends-velox/src/main/scala/org/apache/spark/rpc/GlutenRpcMessages.scala:
##########
@@ -0,0 +1,53 @@
+/*
+ * 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.rpc
+
+import java.util
+
+trait GlutenRpcMessage extends Serializable
+
+object GlutenRpcMessages {
+ case class GlutenRegisterExecutor(
+ executorId: String,
+ executorRef: RpcEndpointRef
+ ) extends GlutenRpcMessage
+
+ case class GlutenOnExecutionStart(executionId: String) extends
GlutenRpcMessage
+
+ case class GlutenOnExecutionEnd(executionId: String) extends GlutenRpcMessage
+
+ case class GlutenExecutorRemoved(executorId: String) extends GlutenRpcMessage
+
+ case class GlutenCleanExecutionResource(executionId: String,
broadcastHashIds: util.Set[String])
+ extends GlutenRpcMessage
+
+ // for mergetree cache
+ case class GlutenMergeTreeCacheLoad(
+ mergeTreeTable: String,
+ columns: util.Set[String],
+ onlyMetaCache: Boolean)
+ extends GlutenRpcMessage
+
+ case class GlutenCacheLoadStatus(jobId: String)
+
+ case class CacheJobInfo(status: Boolean, jobId: String, reason: String = "")
+ extends GlutenRpcMessage
+
+ case class GlutenFilesCacheLoad(files: Array[Byte]) extends GlutenRpcMessage
+
+ case class GlutenFilesCacheLoadStatus(jobId: String)
Review Comment:
nit: Are these used?
##########
backends-velox/src/main/scala/org/apache/spark/sql/execution/unsafe/UnsafeColumnarBuildSideRelation.scala:
##########
@@ -96,37 +109,128 @@ class UnsafeColumnarBuildSideRelation(
case _ => None
}
+ def isOffload: Boolean = offload
+
/** needed for serialization. */
def this() = {
- this(null, null, null)
+ this(null, null, null, Seq.empty, false)
}
private[unsafe] def getBatches(): Seq[UnsafeByteArray] = {
batches
}
+ private var hashTableData: Long = 0L
+
+ def buildHashTable(broadcastContext: BroadcastHashJoinContext): (Long,
BuildSideRelation) =
+ synchronized {
+ if (hashTableData == 0) {
+ val runtime = Runtimes.contextInstance(
+ BackendsApiManager.getBackendName,
+ "UnsafeColumnarBuildSideRelation#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) {
+ val (offset, length) = (batches(batchId).address(),
batches(batchId).size())
+ batchArray.append(jniWrapper.deserializeDirect(serializeHandle,
offset, length.toInt))
+ batchId += 1
+ }
+
+ logDebug(
+ s"BHJ value size: " +
+ s"${broadcastContext.buildHashTableId} = ${batches.size}")
+
+ 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,
+ broadcastContext.bloomFilterPushdownSize,
+ broadcastContext.broadcastHashTableBuildThreads
+ )
+
+ jniWrapper.close(serializeHandle)
+ (hashTableData, this)
+ } else {
+ (HashJoinBuilder.cloneHashTable(hashTableData), null)
Review Comment:
When is this code called?
--
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]