diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala index ffd9068b166a8..c85b48143756a 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala @@ -413,5 +413,4 @@ object CHBackendSettings extends BackendSettingsApi with Logging { .getConf(SQLConf.ADAPTIVE_AUTO_BROADCASTJOIN_THRESHOLD) .getOrElse(conf.autoBroadcastJoinThreshold) } - } diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala index cb261e6e416c2..826712dcfea7f 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala @@ -540,22 +540,20 @@ class CHSparkPlanExecApi extends SparkPlanExecApi { CHExecUtil.buildSideRDD(dataSize, newChild).collect val batches = countsAndBytes.map(_._2) - val totalBatchesBytes = batches.map(_.length).sum - // totalBatchesBytes could be larger than the shuffle written bytes, so we double the threshold - // here. + val totalBatchesSize = batches.map(_.length).sum + val rawSize = dataSize.value if ( - totalBatchesBytes < 0 || - totalBatchesBytes.toLong > CHBackendSettings.getBroadcastThreshold * 2 + rawSize >= BroadcastExchangeExec.MAX_BROADCAST_TABLE_BYTES || + rawSize > CHBackendSettings.getBroadcastThreshold ) { throw new GlutenException( - s"Cannot broadcast the table ($totalBatchesBytes) that is larger than threshold:" + - s" ${CHBackendSettings.getBroadcastThreshold}. Ensure the shuffle written" + - s"bytes is collected properly.") + s"Cannot broadcast the table that is larger than 8GB or broadcast threshold:" + + s" ${rawSize >> 30} GB") } - val rawSize = dataSize.value - if (rawSize >= BroadcastExchangeExec.MAX_BROADCAST_TABLE_BYTES) { + if ((rawSize == 0 && totalBatchesSize != 0) || totalBatchesSize < 0) { throw new GlutenException( - s"Cannot broadcast the table that is larger than 8GB: ${rawSize >> 30} GB") + s"Invalid rawSize($rawSize) or totalBatchesSize ($totalBatchesSize). Ensure the shuffle" + + s" written bytes is correct.") } val rowCount = countsAndBytes.map(_._1).sum numOutputRows += rowCount diff --git a/cpp-ch/local-engine/local_engine_jni.cpp b/cpp-ch/local-engine/local_engine_jni.cpp index ce536799d94a3..9727fca1937d8 100644 --- a/cpp-ch/local-engine/local_engine_jni.cpp +++ b/cpp-ch/local-engine/local_engine_jni.cpp @@ -680,7 +680,7 @@ JNIEXPORT jobject Java_org_apache_gluten_vectorized_CHShuffleSplitterJniWrapper_ // AQE has dependency on total_bytes_written, if the data is wrong, it will generate inappropriate plan // add a log here for remining this. if (!result.total_bytes_written) - LOG_WARNING(getLogger("_CHShuffleSplitterJniWrapper"), "total_bytes_written is 0, something may be wrong"); + LOG_WARNING(getLogger("CHShuffleSplitterJniWrapper"), "total_bytes_written is 0, something may be wrong"); jobject split_result = env->NewObject( split_result_class,