From de725bd9ad1328992e2c57eb6521f4307ee37c09 Mon Sep 17 00:00:00 2001 From: Arnav Balyan <60175178+ArnavBalyan@users.noreply.github.com> Date: Thu, 9 Jan 2025 13:14:52 +0530 Subject: [PATCH 01/28] [GLUTEN-8455][VL] Fallback Scan for Encrypted Parquet Files (#8456) --- .../backendsapi/velox/VeloxBackend.scala | 31 +++- .../gluten/utils/ParquetMetadataUtils.scala | 136 ++++++++++++++++++ .../apache/gluten/config/GlutenConfig.scala | 10 ++ 3 files changed, 174 insertions(+), 3 deletions(-) create mode 100644 backends-velox/src/main/scala/org/apache/gluten/utils/ParquetMetadataUtils.scala diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxBackend.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxBackend.scala index 6c51ad484c7a..519e98c5d459 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxBackend.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxBackend.scala @@ -191,10 +191,35 @@ object VeloxBackendSettings extends BackendSettingsApi { } } - validateScheme().orElse(validateFormat()) match { - case Some(reason) => ValidationResult.failed(reason) - case _ => ValidationResult.succeeded + def validateEncryption(): Option[String] = { + + val encryptionValidationEnabled = GlutenConfig.get.parquetEncryptionValidationEnabled + if (!encryptionValidationEnabled) { + return None + } + + val encryptionResult = + ParquetMetadataUtils.validateEncryption(format, rootPaths, serializableHadoopConf) + if (encryptionResult.ok()) { + None + } else { + Some(s"Detected encrypted parquet files: ${encryptionResult.reason()}") + } } + + val validationChecks = Seq( + validateScheme(), + validateFormat(), + validateEncryption() + ) + + for (check <- validationChecks) { + if (check.isDefined) { + return ValidationResult.failed(check.get) + } + } + + ValidationResult.succeeded } def distinctRootPaths(paths: Seq[String]): Seq[String] = { diff --git a/backends-velox/src/main/scala/org/apache/gluten/utils/ParquetMetadataUtils.scala b/backends-velox/src/main/scala/org/apache/gluten/utils/ParquetMetadataUtils.scala new file mode 100644 index 000000000000..6a96789d512e --- /dev/null +++ b/backends-velox/src/main/scala/org/apache/gluten/utils/ParquetMetadataUtils.scala @@ -0,0 +1,136 @@ +/* + * 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.utils + +import org.apache.gluten.extension.ValidationResult +import org.apache.gluten.substrait.rel.LocalFilesNode.ReadFileFormat +import org.apache.gluten.substrait.rel.LocalFilesNode.ReadFileFormat.ParquetReadFormat + +import org.apache.spark.util.SerializableConfiguration + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileSystem, LocatedFileStatus, Path, RemoteIterator} +import org.apache.parquet.crypto.ParquetCryptoRuntimeException +import org.apache.parquet.hadoop.ParquetFileReader + +object ParquetMetadataUtils { + + /** + * Validates whether Parquet encryption is enabled for the given paths. + * + * - If the file format is not Parquet, skip this check and return success. + * - If there is at least one Parquet file with encryption enabled, fail the validation. + * + * @param format + * File format, e.g., `ParquetReadFormat` + * @param rootPaths + * List of file paths to scan + * @param serializableHadoopConf + * Optional Hadoop configuration + * @return + * [[ValidationResult]] validation success or failure + */ + def validateEncryption( + format: ReadFileFormat, + rootPaths: Seq[String], + serializableHadoopConf: Option[SerializableConfiguration] + ): ValidationResult = { + if (format != ParquetReadFormat || rootPaths.isEmpty) { + return ValidationResult.succeeded + } + + val conf = serializableHadoopConf.map(_.value).getOrElse(new Configuration()) + + rootPaths.foreach { + rootPath => + val fs = new Path(rootPath).getFileSystem(conf) + try { + val encryptionDetected = + checkForEncryptionWithLimit(fs, new Path(rootPath), conf, fileLimit = 10) + if (encryptionDetected) { + return ValidationResult.failed("Encrypted Parquet file detected.") + } + } catch { + case e: Exception => + } + } + ValidationResult.succeeded + } + + /** + * Check any Parquet file under the given path is encrypted using a recursive iterator. Only the + * first `fileLimit` files are processed for efficiency. + * + * @param fs + * FileSystem to use + * @param path + * Root path to check + * @param conf + * Hadoop configuration + * @param fileLimit + * Maximum number of files to inspect + * @return + * True if an encrypted file is detected, false otherwise + */ + private def checkForEncryptionWithLimit( + fs: FileSystem, + path: Path, + conf: Configuration, + fileLimit: Int + ): Boolean = { + + val filesIterator: RemoteIterator[LocatedFileStatus] = fs.listFiles(path, true) + var checkedFileCount = 0 + while (filesIterator.hasNext && checkedFileCount < fileLimit) { + val fileStatus = filesIterator.next() + checkedFileCount += 1 + try { + ParquetFileReader.readFooter(conf, fileStatus.getPath).toString + } catch { + case e: Exception if hasCause(e, classOf[ParquetCryptoRuntimeException]) => + return true + case e: Exception => + } + } + false + } + + /** + * Utility to check the exception for the specified type. Parquet 1.12 does not provide direct + * utility to check for encryption. Newer versions provide utility to check encryption from read + * footer which can be used in the future once Spark brings it in. + * + * @param throwable + * Exception to check + * @param causeType + * Class of the cause to look for + * @tparam T + * Type of the cause + * @return + * True if the cause is found; false otherwise + */ + private def hasCause[T <: Throwable](throwable: Throwable, causeType: Class[T]): Boolean = { + var current = throwable + while (current != null) { + if (causeType.isInstance(current)) { + return true + } + current = current.getCause + } + false + } +} diff --git a/shims/common/src/main/scala/org/apache/gluten/config/GlutenConfig.scala b/shims/common/src/main/scala/org/apache/gluten/config/GlutenConfig.scala index cee4ea7d1645..d4083d5896eb 100644 --- a/shims/common/src/main/scala/org/apache/gluten/config/GlutenConfig.scala +++ b/shims/common/src/main/scala/org/apache/gluten/config/GlutenConfig.scala @@ -497,6 +497,9 @@ class GlutenConfig(conf: SQLConf) extends Logging { def enableBroadcastBuildRelationInOffheap: Boolean = getConf(VELOX_BROADCAST_BUILD_RELATION_USE_OFFHEAP) + + def parquetEncryptionValidationEnabled: Boolean = getConf(ENCRYPTED_PARQUET_FALLBACK_ENABLED) + } object GlutenConfig { @@ -2278,4 +2281,11 @@ object GlutenConfig { "Otherwise, broadcast build relation will use onheap memory.") .booleanConf .createWithDefault(false) + + val ENCRYPTED_PARQUET_FALLBACK_ENABLED = + buildConf("spark.gluten.sql.fallbackEncryptedParquet") + .internal() + .doc("If enabled, gluten will not offload scan when encrypted parquet files are detected") + .booleanConf + .createWithDefault(false) } From 6b90393f62ac05dae98e20991ae831f7fa9e9e0b Mon Sep 17 00:00:00 2001 From: Gluten Performance Bot <137994563+GlutenPerfBot@users.noreply.github.com> Date: Thu, 9 Jan 2025 15:59:31 +0800 Subject: [PATCH 02/28] [GLUTEN-6887][VL] Daily Update Velox Version (2025_01_09) (#8472) --- ep/build-velox/src/get_velox.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh index 82e7fb0c1cb7..676c062c1bf4 100755 --- a/ep/build-velox/src/get_velox.sh +++ b/ep/build-velox/src/get_velox.sh @@ -17,7 +17,7 @@ set -exu VELOX_REPO=https://github.com/oap-project/velox.git -VELOX_BRANCH=2025_01_08 +VELOX_BRANCH=2025_01_09 VELOX_HOME="" OS=`uname -s` From d101cb80e4ab973f3bee2ba3c4983ffe53759678 Mon Sep 17 00:00:00 2001 From: jackylee Date: Thu, 9 Jan 2025 16:23:50 +0800 Subject: [PATCH 03/28] [CORE] Refactor columnar noop write rule (#8422) --- .../backendsapi/clickhouse/CHRuleApi.scala | 2 + .../backendsapi/velox/VeloxRuleApi.scala | 3 + .../GlutenWriterColumnarRules.scala | 59 +++++------- .../noop/GlutenNoopWriterRule.scala | 42 +++++++++ .../clickhouse/ClickHouseTestSettings.scala | 3 + .../utils/velox/VeloxTestSettings.scala | 3 - .../ClickHouseAdaptiveQueryExecSuite.scala | 89 ++++++++++++++++++- .../velox/VeloxAdaptiveQueryExecSuite.scala | 89 ++++++++++++++++++- .../GlutenWriterColumnarRulesSuite.scala | 54 ----------- .../clickhouse/ClickHouseTestSettings.scala | 3 + .../utils/velox/VeloxTestSettings.scala | 3 - .../ClickHouseAdaptiveQueryExecSuite.scala | 89 ++++++++++++++++++- .../velox/VeloxAdaptiveQueryExecSuite.scala | 89 ++++++++++++++++++- .../GlutenWriterColumnarRulesSuite.scala | 54 ----------- .../clickhouse/ClickHouseTestSettings.scala | 2 + .../utils/velox/VeloxTestSettings.scala | 3 +- .../ClickHouseAdaptiveQueryExecSuite.scala | 38 +++++++- .../velox/VeloxAdaptiveQueryExecSuite.scala | 38 +++++++- .../clickhouse/ClickHouseTestSettings.scala | 2 + .../utils/velox/VeloxTestSettings.scala | 1 - .../ClickHouseAdaptiveQueryExecSuite.scala | 38 +++++++- .../velox/VeloxAdaptiveQueryExecSuite.scala | 38 +++++++- .../GlutenNoopWriterRuleSuite.scala | 70 +++++++++++++++ 23 files changed, 652 insertions(+), 160 deletions(-) create mode 100644 gluten-substrait/src/main/scala/org/apache/spark/sql/execution/datasources/noop/GlutenNoopWriterRule.scala delete mode 100644 gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenWriterColumnarRulesSuite.scala delete mode 100644 gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenWriterColumnarRulesSuite.scala create mode 100644 gluten-ut/test/src/test/scala/org/apache/spark/sql/datasources/GlutenNoopWriterRuleSuite.scala diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala index 139b8b3131ff..21ae342a2263 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala @@ -38,6 +38,7 @@ import org.apache.spark.sql.delta.DeltaLogFileIndex import org.apache.spark.sql.delta.rules.CHOptimizeMetadataOnlyDeltaQuery import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper +import org.apache.spark.sql.execution.datasources.noop.GlutenNoopWriterRule import org.apache.spark.sql.execution.datasources.v2.V2CommandExec import org.apache.spark.util.SparkPlanRules @@ -132,6 +133,7 @@ object CHRuleApi { c => intercept( SparkPlanRules.extendedColumnarRule(c.glutenConf.extendedColumnarPostRules)(c.session))) + injector.injectPost(c => GlutenNoopWriterRule.apply(c.session)) // Gluten columnar: Final rules. injector.injectFinal(c => RemoveGlutenTableCacheColumnarToRow(c.session)) diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxRuleApi.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxRuleApi.scala index 4f19375e85ce..f3c75cd98318 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxRuleApi.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxRuleApi.scala @@ -36,6 +36,7 @@ import org.apache.gluten.sql.shims.SparkShimLoader import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, ObjectHashAggregateExec, SortAggregateExec} import org.apache.spark.sql.execution.datasources.WriteFilesExec +import org.apache.spark.sql.execution.datasources.noop.GlutenNoopWriterRule import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanExecBase import org.apache.spark.sql.execution.exchange.Exchange import org.apache.spark.sql.execution.joins.BaseJoinExec @@ -110,6 +111,7 @@ object VeloxRuleApi { .getExtendedColumnarPostRules() .foreach(each => injector.injectPost(c => each(c.session))) injector.injectPost(c => ColumnarCollapseTransformStages(c.glutenConf)) + injector.injectPost(c => GlutenNoopWriterRule(c.session)) // Gluten columnar: Final rules. injector.injectFinal(c => RemoveGlutenTableCacheColumnarToRow(c.session)) @@ -188,6 +190,7 @@ object VeloxRuleApi { .getExtendedColumnarPostRules() .foreach(each => injector.injectPostTransform(c => each(c.session))) injector.injectPostTransform(c => ColumnarCollapseTransformStages(c.glutenConf)) + injector.injectPostTransform(c => GlutenNoopWriterRule(c.session)) injector.injectPostTransform(c => RemoveGlutenTableCacheColumnarToRow(c.session)) injector.injectPostTransform(c => GlutenFallbackReporter(c.glutenConf, c.session)) injector.injectPostTransform(_ => RemoveFallbackTagRule()) diff --git a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/datasources/GlutenWriterColumnarRules.scala b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/datasources/GlutenWriterColumnarRules.scala index 126417bf18a5..54b5a3463991 100644 --- a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/datasources/GlutenWriterColumnarRules.scala +++ b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/datasources/GlutenWriterColumnarRules.scala @@ -30,7 +30,6 @@ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec import org.apache.spark.sql.execution.command.{CreateDataSourceTableAsSelectCommand, DataWritingCommand, DataWritingCommandExec} -import org.apache.spark.sql.execution.datasources.v2.{AppendDataExec, OverwriteByExpressionExec} import org.apache.spark.sql.hive.execution.{CreateHiveTableAsSelectCommand, InsertIntoHiveDirCommand, InsertIntoHiveTable} import org.apache.spark.sql.sources.DataSourceRegister import org.apache.spark.sql.vectorized.ColumnarBatch @@ -133,19 +132,33 @@ object GlutenWriterColumnarRules { } } - case class NativeWritePostRule(session: SparkSession) extends Rule[SparkPlan] { + private[datasources] def injectFakeRowAdaptor(command: SparkPlan, child: SparkPlan): SparkPlan = { + child match { + // if the child is columnar, we can just wrap & transfer the columnar data + case c2r: ColumnarToRowExecBase => + command.withNewChildren(Array(FakeRowAdaptor(c2r.child))) + // If the child is aqe, we make aqe "support columnar", + // then aqe itself will guarantee to generate columnar outputs. + // So FakeRowAdaptor will always consumes columnar data, + // thus avoiding the case of c2r->aqe->r2c->writer + case aqe: AdaptiveSparkPlanExec => + command.withNewChildren( + Array( + FakeRowAdaptor( + AdaptiveSparkPlanExec( + aqe.inputPlan, + aqe.context, + aqe.preprocessingRules, + aqe.isSubquery, + supportsColumnar = true + )))) + case other => command.withNewChildren(Array(FakeRowAdaptor(other))) + } + } - private val NOOP_WRITE = "org.apache.spark.sql.execution.datasources.noop.NoopWrite$" + case class NativeWritePostRule(session: SparkSession) extends Rule[SparkPlan] { override def apply(p: SparkPlan): SparkPlan = p match { - case rc @ AppendDataExec(_, _, write) - if write.getClass.getName == NOOP_WRITE && - BackendsApiManager.getSettings.enableNativeWriteFiles() => - injectFakeRowAdaptor(rc, rc.child) - case rc @ OverwriteByExpressionExec(_, _, write) - if write.getClass.getName == NOOP_WRITE && - BackendsApiManager.getSettings.enableNativeWriteFiles() => - injectFakeRowAdaptor(rc, rc.child) case rc @ DataWritingCommandExec(cmd, child) => // The same thread can set these properties in the last query submission. val fields = child.output.toStructType.fields @@ -165,30 +178,6 @@ object GlutenWriterColumnarRules { case plan: SparkPlan => plan.withNewChildren(plan.children.map(apply)) } - - private def injectFakeRowAdaptor(command: SparkPlan, child: SparkPlan): SparkPlan = { - child match { - // if the child is columnar, we can just wrap&transfer the columnar data - case c2r: ColumnarToRowExecBase => - command.withNewChildren(Array(FakeRowAdaptor(c2r.child))) - // If the child is aqe, we make aqe "support columnar", - // then aqe itself will guarantee to generate columnar outputs. - // So FakeRowAdaptor will always consumes columnar data, - // thus avoiding the case of c2r->aqe->r2c->writer - case aqe: AdaptiveSparkPlanExec => - command.withNewChildren( - Array( - FakeRowAdaptor( - AdaptiveSparkPlanExec( - aqe.inputPlan, - aqe.context, - aqe.preprocessingRules, - aqe.isSubquery, - supportsColumnar = true - )))) - case other => command.withNewChildren(Array(FakeRowAdaptor(other))) - } - } } def injectSparkLocalProperty(spark: SparkSession, format: Option[String]): Unit = { diff --git a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/datasources/noop/GlutenNoopWriterRule.scala b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/datasources/noop/GlutenNoopWriterRule.scala new file mode 100644 index 000000000000..bedf006510d3 --- /dev/null +++ b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/datasources/noop/GlutenNoopWriterRule.scala @@ -0,0 +1,42 @@ +/* + * 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.datasources.noop + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.datasources.GlutenWriterColumnarRules.injectFakeRowAdaptor +import org.apache.spark.sql.execution.datasources.v2.{AppendDataExec, OverwriteByExpressionExec} + +/** + * A rule that injects a FakeRowAdaptor for NoopWrite. + * + * The current V2 Command does not support columnar. Therefore, when its child node is a + * ColumnarNode, Vanilla Spark inserts a ColumnarToRow conversion between V2 Command and its child. + * This rule replaces the inserted ColumnarToRow with a FakeRowAdaptor, effectively bypassing the + * ColumnarToRow operation for NoopWrite. Since NoopWrite does not actually perform any data + * operations, it can accept input data in either row-based or columnar format. + */ +case class GlutenNoopWriterRule(session: SparkSession) extends Rule[SparkPlan] { + override def apply(p: SparkPlan): SparkPlan = p match { + case rc @ AppendDataExec(_, _, NoopWrite) => + injectFakeRowAdaptor(rc, rc.child) + case rc @ OverwriteByExpressionExec(_, _, NoopWrite) => + injectFakeRowAdaptor(rc, rc.child) + case _ => p + } +} diff --git a/gluten-ut/spark32/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala b/gluten-ut/spark32/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala index b985347f2041..8c62e3b0fd9b 100644 --- a/gluten-ut/spark32/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala +++ b/gluten-ut/spark32/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala @@ -1123,6 +1123,9 @@ class ClickHouseTestSettings extends BackendTestSettings { .exclude("Change merge join to broadcast join without local shuffle read") .exclude( "Avoid changing merge join to broadcast join if too many empty partitions on build plan") + .exclude("SPARK-32932: Do not use local shuffle read at final stage on write command") + .exclude( + "SPARK-30953: InsertAdaptiveSparkPlan should apply AQE on child plan of v2 write commands") .exclude("SPARK-29544: adaptive skew join with different join types") .exclude("SPARK-34682: AQEShuffleReadExec operating on canonicalized plan") .exclude("SPARK-32717: AQEOptimizer should respect excludedRules configuration") diff --git a/gluten-ut/spark32/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala b/gluten-ut/spark32/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala index f5071d2f3fc4..62ab86836301 100644 --- a/gluten-ut/spark32/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala +++ b/gluten-ut/spark32/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala @@ -166,14 +166,12 @@ class VeloxTestSettings extends BackendTestSettings { "SPARK-30403", "SPARK-30719", "SPARK-31384", - "SPARK-30953", "SPARK-31658", "SPARK-32717", "SPARK-32649", "SPARK-34533", "SPARK-34781", "SPARK-35585", - "SPARK-32932", "SPARK-33494", // "SPARK-33933", "SPARK-31220", @@ -1053,7 +1051,6 @@ class VeloxTestSettings extends BackendTestSettings { enableSuite[GlutenSupportsCatalogOptionsSuite] enableSuite[GlutenTableCapabilityCheckSuite] enableSuite[GlutenWriteDistributionAndOrderingSuite] - enableSuite[GlutenWriterColumnarRulesSuite] enableSuite[GlutenBucketedReadWithoutHiveSupportSuite] // Exclude the following suite for plan changed from SMJ to SHJ. .exclude("avoid shuffle when join 2 bucketed tables") diff --git a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/adaptive/clickhouse/ClickHouseAdaptiveQueryExecSuite.scala b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/adaptive/clickhouse/ClickHouseAdaptiveQueryExecSuite.scala index 6d3c3e865d58..928dc38985ce 100644 --- a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/adaptive/clickhouse/ClickHouseAdaptiveQueryExecSuite.scala +++ b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/adaptive/clickhouse/ClickHouseAdaptiveQueryExecSuite.scala @@ -16,6 +16,7 @@ */ package org.apache.spark.sql.execution.adaptive.clickhouse +import org.apache.gluten.config.GlutenConfig import org.apache.gluten.execution.{BroadcastHashJoinExecTransformerBase, ShuffledHashJoinExecTransformerBase, SortExecTransformer, SortMergeJoinExecTransformer} import org.apache.spark.SparkConf @@ -25,14 +26,20 @@ import org.apache.spark.sql.GlutenTestConstants.GLUTEN_TEST import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight} import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.adaptive._ +import org.apache.spark.sql.execution.command.DataWritingCommandExec +import org.apache.spark.sql.execution.datasources.FakeRowAdaptor +import org.apache.spark.sql.execution.datasources.noop.NoopDataSource +import org.apache.spark.sql.execution.datasources.v2.V2TableWriteExec import org.apache.spark.sql.execution.exchange._ import org.apache.spark.sql.execution.joins.{BaseJoinExec, BroadcastHashJoinExec, ShuffledHashJoinExec, SortMergeJoinExec} import org.apache.spark.sql.execution.metric.SQLShuffleReadMetricsReporter import org.apache.spark.sql.execution.ui.SparkListenerSQLAdaptiveExecutionUpdate import org.apache.spark.sql.functions.when import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.SQLConf.PartitionOverwriteMode import org.apache.spark.sql.test.SQLTestData.TestData import org.apache.spark.sql.types.{IntegerType, StructType} +import org.apache.spark.sql.util.QueryExecutionListener import org.apache.log4j.Level @@ -42,7 +49,7 @@ class ClickHouseAdaptiveQueryExecSuite extends AdaptiveQueryExecSuite with Glute override def sparkConf: SparkConf = { super.sparkConf - .set("spark.gluten.sql.columnar.forceShuffledHashJoin", "false") + .set(GlutenConfig.COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED.key, "false") .set(SQLConf.SHUFFLE_PARTITIONS.key, "5") } @@ -1196,6 +1203,86 @@ class ClickHouseAdaptiveQueryExecSuite extends AdaptiveQueryExecSuite with Glute } } + testGluten("SPARK-32932: Do not use local shuffle read at final stage on write command") { + withSQLConf( + SQLConf.PARTITION_OVERWRITE_MODE.key -> PartitionOverwriteMode.DYNAMIC.toString, + SQLConf.SHUFFLE_PARTITIONS.key -> "5", + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true" + ) { + val data = + for ( + i <- 1L to 10L; + j <- 1L to 3L + ) yield (i, j) + + val df = data.toDF("i", "j").repartition($"j") + var noLocalread: Boolean = false + val listener = new QueryExecutionListener { + override def onSuccess(funcName: String, qe: QueryExecution, durationNs: Long): Unit = { + qe.executedPlan match { + case plan @ (_: DataWritingCommandExec | _: V2TableWriteExec) => + noLocalread = collect(plan) { + case exec: AQEShuffleReadExec if exec.isLocalRead => exec + }.isEmpty + case _ => // ignore other events + } + } + override def onFailure( + funcName: String, + qe: QueryExecution, + exception: Exception): Unit = {} + } + spark.listenerManager.register(listener) + + withTable("t") { + df.write.partitionBy("j").saveAsTable("t") + sparkContext.listenerBus.waitUntilEmpty() + assert(noLocalread) + noLocalread = false + } + + // Test DataSource v2 + val format = classOf[NoopDataSource].getName + df.write.format(format).mode("overwrite").save() + sparkContext.listenerBus.waitUntilEmpty() + assert(noLocalread) + noLocalread = false + + spark.listenerManager.unregister(listener) + } + } + + testGluten( + "SPARK-30953: InsertAdaptiveSparkPlan should apply AQE on child plan of v2 write commands") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.ADAPTIVE_EXECUTION_FORCE_APPLY.key -> "true") { + var plan: SparkPlan = null + val listener = new QueryExecutionListener { + override def onSuccess(funcName: String, qe: QueryExecution, durationNs: Long): Unit = { + plan = qe.executedPlan + } + override def onFailure( + funcName: String, + qe: QueryExecution, + exception: Exception): Unit = {} + } + spark.listenerManager.register(listener) + withTable("t1") { + val format = classOf[NoopDataSource].getName + Seq((0, 1)).toDF("x", "y").write.format(format).mode("overwrite").save() + + sparkContext.listenerBus.waitUntilEmpty() + assert(plan.isInstanceOf[V2TableWriteExec]) + val childPlan = plan.asInstanceOf[V2TableWriteExec].child + assert(childPlan.isInstanceOf[FakeRowAdaptor]) + assert(childPlan.asInstanceOf[FakeRowAdaptor].child.isInstanceOf[AdaptiveSparkPlanExec]) + + spark.listenerManager.unregister(listener) + } + } + } + testGluten("SPARK-35650: Coalesce number of partitions by AEQ") { withSQLConf(SQLConf.COALESCE_PARTITIONS_MIN_PARTITION_NUM.key -> "1") { Seq("REPARTITION", "REBALANCE(key)") diff --git a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/adaptive/velox/VeloxAdaptiveQueryExecSuite.scala b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/adaptive/velox/VeloxAdaptiveQueryExecSuite.scala index f8b6092a46f7..ce9513c8cc9b 100644 --- a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/adaptive/velox/VeloxAdaptiveQueryExecSuite.scala +++ b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/adaptive/velox/VeloxAdaptiveQueryExecSuite.scala @@ -16,6 +16,7 @@ */ package org.apache.spark.sql.execution.adaptive.velox +import org.apache.gluten.config.GlutenConfig import org.apache.gluten.execution.{BroadcastHashJoinExecTransformerBase, ShuffledHashJoinExecTransformerBase, SortExecTransformer, SortMergeJoinExecTransformer} import org.apache.spark.SparkConf @@ -25,14 +26,20 @@ import org.apache.spark.sql.GlutenTestConstants.GLUTEN_TEST import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight} import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.adaptive._ +import org.apache.spark.sql.execution.command.DataWritingCommandExec +import org.apache.spark.sql.execution.datasources.FakeRowAdaptor +import org.apache.spark.sql.execution.datasources.noop.NoopDataSource +import org.apache.spark.sql.execution.datasources.v2.V2TableWriteExec import org.apache.spark.sql.execution.exchange._ import org.apache.spark.sql.execution.joins.{BaseJoinExec, BroadcastHashJoinExec, ShuffledHashJoinExec, SortMergeJoinExec} import org.apache.spark.sql.execution.metric.SQLShuffleReadMetricsReporter import org.apache.spark.sql.execution.ui.SparkListenerSQLAdaptiveExecutionUpdate import org.apache.spark.sql.functions.when import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.SQLConf.PartitionOverwriteMode import org.apache.spark.sql.test.SQLTestData.TestData import org.apache.spark.sql.types.{IntegerType, StructType} +import org.apache.spark.sql.util.QueryExecutionListener import org.apache.log4j.Level @@ -41,7 +48,7 @@ class VeloxAdaptiveQueryExecSuite extends AdaptiveQueryExecSuite with GlutenSQLT override def sparkConf: SparkConf = { super.sparkConf - .set("spark.gluten.sql.columnar.forceShuffledHashJoin", "false") + .set(GlutenConfig.COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED.key, "false") .set(SQLConf.SHUFFLE_PARTITIONS.key, "5") } @@ -1175,6 +1182,86 @@ class VeloxAdaptiveQueryExecSuite extends AdaptiveQueryExecSuite with GlutenSQLT } } + testGluten("SPARK-32932: Do not use local shuffle read at final stage on write command") { + withSQLConf( + SQLConf.PARTITION_OVERWRITE_MODE.key -> PartitionOverwriteMode.DYNAMIC.toString, + SQLConf.SHUFFLE_PARTITIONS.key -> "5", + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true" + ) { + val data = + for ( + i <- 1L to 10L; + j <- 1L to 3L + ) yield (i, j) + + val df = data.toDF("i", "j").repartition($"j") + var noLocalread: Boolean = false + val listener = new QueryExecutionListener { + override def onSuccess(funcName: String, qe: QueryExecution, durationNs: Long): Unit = { + qe.executedPlan match { + case plan @ (_: DataWritingCommandExec | _: V2TableWriteExec) => + noLocalread = collect(plan) { + case exec: AQEShuffleReadExec if exec.isLocalRead => exec + }.isEmpty + case _ => // ignore other events + } + } + override def onFailure( + funcName: String, + qe: QueryExecution, + exception: Exception): Unit = {} + } + spark.listenerManager.register(listener) + + withTable("t") { + df.write.partitionBy("j").saveAsTable("t") + sparkContext.listenerBus.waitUntilEmpty() + assert(noLocalread) + noLocalread = false + } + + // Test DataSource v2 + val format = classOf[NoopDataSource].getName + df.write.format(format).mode("overwrite").save() + sparkContext.listenerBus.waitUntilEmpty() + assert(noLocalread) + noLocalread = false + + spark.listenerManager.unregister(listener) + } + } + + testGluten( + "SPARK-30953: InsertAdaptiveSparkPlan should apply AQE on child plan of v2 write commands") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.ADAPTIVE_EXECUTION_FORCE_APPLY.key -> "true") { + var plan: SparkPlan = null + val listener = new QueryExecutionListener { + override def onSuccess(funcName: String, qe: QueryExecution, durationNs: Long): Unit = { + plan = qe.executedPlan + } + override def onFailure( + funcName: String, + qe: QueryExecution, + exception: Exception): Unit = {} + } + spark.listenerManager.register(listener) + withTable("t1") { + val format = classOf[NoopDataSource].getName + Seq((0, 1)).toDF("x", "y").write.format(format).mode("overwrite").save() + + sparkContext.listenerBus.waitUntilEmpty() + assert(plan.isInstanceOf[V2TableWriteExec]) + val childPlan = plan.asInstanceOf[V2TableWriteExec].child + assert(childPlan.isInstanceOf[FakeRowAdaptor]) + assert(childPlan.asInstanceOf[FakeRowAdaptor].child.isInstanceOf[AdaptiveSparkPlanExec]) + + spark.listenerManager.unregister(listener) + } + } + } + testGluten("SPARK-35650: Coalesce number of partitions by AEQ") { withSQLConf(SQLConf.COALESCE_PARTITIONS_MIN_PARTITION_NUM.key -> "1") { Seq("REPARTITION", "REBALANCE(key)") diff --git a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenWriterColumnarRulesSuite.scala b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenWriterColumnarRulesSuite.scala deleted file mode 100644 index 10abca1c6dd3..000000000000 --- a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenWriterColumnarRulesSuite.scala +++ /dev/null @@ -1,54 +0,0 @@ -/* - * 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.datasources - -import org.apache.gluten.config.GlutenConfig - -import org.apache.spark.sql.{GlutenSQLTestsBaseTrait, SaveMode} -import org.apache.spark.sql.execution.QueryExecution -import org.apache.spark.sql.util.QueryExecutionListener - -class GlutenWriterColumnarRulesSuite extends GlutenSQLTestsBaseTrait { - - class WriterColumnarListener extends QueryExecutionListener { - var fakeRowAdaptor: Option[FakeRowAdaptor] = None - - override def onSuccess(funcName: String, qe: QueryExecution, durationNs: Long): Unit = { - fakeRowAdaptor = qe.executedPlan.collectFirst { case f: FakeRowAdaptor => f } - } - - override def onFailure(funcName: String, qe: QueryExecution, exception: Exception): Unit = {} - } - - testGluten("writing to noop") { - withTempDir { - dir => - withSQLConf(GlutenConfig.NATIVE_WRITER_ENABLED.key -> "true") { - spark.range(0, 100).write.mode(SaveMode.Overwrite).parquet(dir.getPath) - val listener = new WriterColumnarListener - spark.listenerManager.register(listener) - try { - spark.read.parquet(dir.getPath).write.format("noop").mode(SaveMode.Overwrite).save() - spark.sparkContext.listenerBus.waitUntilEmpty() - assert(listener.fakeRowAdaptor.isDefined, "FakeRowAdaptor is not found.") - } finally { - spark.listenerManager.unregister(listener) - } - } - } - } -} diff --git a/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala b/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala index 219eb0d0b97e..f91841b991c7 100644 --- a/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala +++ b/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala @@ -1119,6 +1119,9 @@ class ClickHouseTestSettings extends BackendTestSettings { .exclude("Change merge join to broadcast join without local shuffle read") .exclude( "Avoid changing merge join to broadcast join if too many empty partitions on build plan") + .exclude("SPARK-32932: Do not use local shuffle read at final stage on write command") + .exclude( + "SPARK-30953: InsertAdaptiveSparkPlan should apply AQE on child plan of v2 write commands") .exclude("SPARK-37753: Allow changing outer join to broadcast join even if too many empty partitions on broadcast side") .exclude("SPARK-29544: adaptive skew join with different join types") .exclude("SPARK-34682: AQEShuffleReadExec operating on canonicalized plan") diff --git a/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala b/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala index d3bc3846d80f..72b77ae1f95b 100644 --- a/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala +++ b/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala @@ -70,7 +70,6 @@ class VeloxTestSettings extends BackendTestSettings { enableSuite[GlutenSupportsCatalogOptionsSuite] enableSuite[GlutenTableCapabilityCheckSuite] enableSuite[GlutenWriteDistributionAndOrderingSuite] - enableSuite[GlutenWriterColumnarRulesSuite] enableSuite[GlutenQueryCompilationErrorsDSv2Suite] @@ -191,14 +190,12 @@ class VeloxTestSettings extends BackendTestSettings { "SPARK-30403", "SPARK-30719", "SPARK-31384", - "SPARK-30953", "SPARK-31658", "SPARK-32717", "SPARK-32649", "SPARK-34533", "SPARK-34781", "SPARK-35585", - "SPARK-32932", "SPARK-33494", "SPARK-33933", "SPARK-31220", diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/adaptive/clickhouse/ClickHouseAdaptiveQueryExecSuite.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/adaptive/clickhouse/ClickHouseAdaptiveQueryExecSuite.scala index 441f3a60a3a9..779d264114cb 100644 --- a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/adaptive/clickhouse/ClickHouseAdaptiveQueryExecSuite.scala +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/adaptive/clickhouse/ClickHouseAdaptiveQueryExecSuite.scala @@ -16,6 +16,7 @@ */ package org.apache.spark.sql.execution.adaptive.clickhouse +import org.apache.gluten.config.GlutenConfig import org.apache.gluten.execution.{BroadcastHashJoinExecTransformerBase, ShuffledHashJoinExecTransformerBase, SortExecTransformer, SortMergeJoinExecTransformerBase} import org.apache.spark.SparkConf @@ -24,14 +25,20 @@ import org.apache.spark.sql.{Dataset, GlutenSQLTestsTrait, Row} import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight} import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.adaptive._ +import org.apache.spark.sql.execution.command.DataWritingCommandExec +import org.apache.spark.sql.execution.datasources.FakeRowAdaptor +import org.apache.spark.sql.execution.datasources.noop.NoopDataSource +import org.apache.spark.sql.execution.datasources.v2.V2TableWriteExec import org.apache.spark.sql.execution.exchange._ import org.apache.spark.sql.execution.joins.{BaseJoinExec, BroadcastHashJoinExec, ShuffledHashJoinExec, SortMergeJoinExec} import org.apache.spark.sql.execution.metric.SQLShuffleReadMetricsReporter import org.apache.spark.sql.execution.ui.SparkListenerSQLAdaptiveExecutionUpdate import org.apache.spark.sql.functions.when import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.SQLConf.PartitionOverwriteMode import org.apache.spark.sql.test.SQLTestData.TestData import org.apache.spark.sql.types.{IntegerType, StructType} +import org.apache.spark.sql.util.QueryExecutionListener import org.apache.logging.log4j.Level @@ -40,7 +47,7 @@ class ClickHouseAdaptiveQueryExecSuite extends AdaptiveQueryExecSuite with Glute override def sparkConf: SparkConf = { super.sparkConf - .set("spark.gluten.sql.columnar.forceShuffledHashJoin", "false") + .set(GlutenConfig.COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED.key, "false") .set(SQLConf.SHUFFLE_PARTITIONS.key, "5") } @@ -1193,6 +1200,86 @@ class ClickHouseAdaptiveQueryExecSuite extends AdaptiveQueryExecSuite with Glute } } + testGluten("SPARK-32932: Do not use local shuffle read at final stage on write command") { + withSQLConf( + SQLConf.PARTITION_OVERWRITE_MODE.key -> PartitionOverwriteMode.DYNAMIC.toString, + SQLConf.SHUFFLE_PARTITIONS.key -> "5", + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true" + ) { + val data = + for ( + i <- 1L to 10L; + j <- 1L to 3L + ) yield (i, j) + + val df = data.toDF("i", "j").repartition($"j") + var noLocalread: Boolean = false + val listener = new QueryExecutionListener { + override def onSuccess(funcName: String, qe: QueryExecution, durationNs: Long): Unit = { + qe.executedPlan match { + case plan @ (_: DataWritingCommandExec | _: V2TableWriteExec) => + noLocalread = collect(plan) { + case exec: AQEShuffleReadExec if exec.isLocalRead => exec + }.isEmpty + case _ => // ignore other events + } + } + override def onFailure( + funcName: String, + qe: QueryExecution, + exception: Exception): Unit = {} + } + spark.listenerManager.register(listener) + + withTable("t") { + df.write.partitionBy("j").saveAsTable("t") + sparkContext.listenerBus.waitUntilEmpty() + assert(noLocalread) + noLocalread = false + } + + // Test DataSource v2 + val format = classOf[NoopDataSource].getName + df.write.format(format).mode("overwrite").save() + sparkContext.listenerBus.waitUntilEmpty() + assert(noLocalread) + noLocalread = false + + spark.listenerManager.unregister(listener) + } + } + + testGluten( + "SPARK-30953: InsertAdaptiveSparkPlan should apply AQE on child plan of v2 write commands") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.ADAPTIVE_EXECUTION_FORCE_APPLY.key -> "true") { + var plan: SparkPlan = null + val listener = new QueryExecutionListener { + override def onSuccess(funcName: String, qe: QueryExecution, durationNs: Long): Unit = { + plan = qe.executedPlan + } + override def onFailure( + funcName: String, + qe: QueryExecution, + exception: Exception): Unit = {} + } + spark.listenerManager.register(listener) + withTable("t1") { + val format = classOf[NoopDataSource].getName + Seq((0, 1)).toDF("x", "y").write.format(format).mode("overwrite").save() + + sparkContext.listenerBus.waitUntilEmpty() + assert(plan.isInstanceOf[V2TableWriteExec]) + val childPlan = plan.asInstanceOf[V2TableWriteExec].child + assert(childPlan.isInstanceOf[FakeRowAdaptor]) + assert(childPlan.asInstanceOf[FakeRowAdaptor].child.isInstanceOf[AdaptiveSparkPlanExec]) + + spark.listenerManager.unregister(listener) + } + } + } + testGluten("SPARK-35650: Coalesce number of partitions by AEQ") { withSQLConf(SQLConf.COALESCE_PARTITIONS_MIN_PARTITION_NUM.key -> "1") { Seq("REPARTITION", "REBALANCE(key)") diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/adaptive/velox/VeloxAdaptiveQueryExecSuite.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/adaptive/velox/VeloxAdaptiveQueryExecSuite.scala index 729a12f56cc5..f9f0723e00cc 100644 --- a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/adaptive/velox/VeloxAdaptiveQueryExecSuite.scala +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/adaptive/velox/VeloxAdaptiveQueryExecSuite.scala @@ -16,6 +16,7 @@ */ package org.apache.spark.sql.execution.adaptive.velox +import org.apache.gluten.config.GlutenConfig import org.apache.gluten.execution.{BroadcastHashJoinExecTransformerBase, ShuffledHashJoinExecTransformerBase, SortExecTransformer, SortMergeJoinExecTransformer} import org.apache.spark.SparkConf @@ -25,14 +26,20 @@ import org.apache.spark.sql.GlutenTestConstants.GLUTEN_TEST import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight} import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.adaptive._ +import org.apache.spark.sql.execution.command.DataWritingCommandExec +import org.apache.spark.sql.execution.datasources.FakeRowAdaptor +import org.apache.spark.sql.execution.datasources.noop.NoopDataSource +import org.apache.spark.sql.execution.datasources.v2.V2TableWriteExec import org.apache.spark.sql.execution.exchange._ import org.apache.spark.sql.execution.joins.{BaseJoinExec, BroadcastHashJoinExec, ShuffledHashJoinExec, SortMergeJoinExec} import org.apache.spark.sql.execution.metric.SQLShuffleReadMetricsReporter import org.apache.spark.sql.execution.ui.SparkListenerSQLAdaptiveExecutionUpdate import org.apache.spark.sql.functions.when import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.SQLConf.PartitionOverwriteMode import org.apache.spark.sql.test.SQLTestData.TestData import org.apache.spark.sql.types.{IntegerType, StructType} +import org.apache.spark.sql.util.QueryExecutionListener import org.apache.logging.log4j.Level @@ -41,7 +48,7 @@ class VeloxAdaptiveQueryExecSuite extends AdaptiveQueryExecSuite with GlutenSQLT override def sparkConf: SparkConf = { super.sparkConf - .set("spark.gluten.sql.columnar.forceShuffledHashJoin", "false") + .set(GlutenConfig.COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED.key, "false") .set(SQLConf.SHUFFLE_PARTITIONS.key, "5") } @@ -1179,6 +1186,86 @@ class VeloxAdaptiveQueryExecSuite extends AdaptiveQueryExecSuite with GlutenSQLT } } + testGluten("SPARK-32932: Do not use local shuffle read at final stage on write command") { + withSQLConf( + SQLConf.PARTITION_OVERWRITE_MODE.key -> PartitionOverwriteMode.DYNAMIC.toString, + SQLConf.SHUFFLE_PARTITIONS.key -> "5", + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true" + ) { + val data = + for ( + i <- 1L to 10L; + j <- 1L to 3L + ) yield (i, j) + + val df = data.toDF("i", "j").repartition($"j") + var noLocalread: Boolean = false + val listener = new QueryExecutionListener { + override def onSuccess(funcName: String, qe: QueryExecution, durationNs: Long): Unit = { + qe.executedPlan match { + case plan @ (_: DataWritingCommandExec | _: V2TableWriteExec) => + noLocalread = collect(plan) { + case exec: AQEShuffleReadExec if exec.isLocalRead => exec + }.isEmpty + case _ => // ignore other events + } + } + override def onFailure( + funcName: String, + qe: QueryExecution, + exception: Exception): Unit = {} + } + spark.listenerManager.register(listener) + + withTable("t") { + df.write.partitionBy("j").saveAsTable("t") + sparkContext.listenerBus.waitUntilEmpty() + assert(noLocalread) + noLocalread = false + } + + // Test DataSource v2 + val format = classOf[NoopDataSource].getName + df.write.format(format).mode("overwrite").save() + sparkContext.listenerBus.waitUntilEmpty() + assert(noLocalread) + noLocalread = false + + spark.listenerManager.unregister(listener) + } + } + + testGluten( + "SPARK-30953: InsertAdaptiveSparkPlan should apply AQE on child plan of v2 write commands") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.ADAPTIVE_EXECUTION_FORCE_APPLY.key -> "true") { + var plan: SparkPlan = null + val listener = new QueryExecutionListener { + override def onSuccess(funcName: String, qe: QueryExecution, durationNs: Long): Unit = { + plan = qe.executedPlan + } + override def onFailure( + funcName: String, + qe: QueryExecution, + exception: Exception): Unit = {} + } + spark.listenerManager.register(listener) + withTable("t1") { + val format = classOf[NoopDataSource].getName + Seq((0, 1)).toDF("x", "y").write.format(format).mode("overwrite").save() + + sparkContext.listenerBus.waitUntilEmpty() + assert(plan.isInstanceOf[V2TableWriteExec]) + val childPlan = plan.asInstanceOf[V2TableWriteExec].child + assert(childPlan.isInstanceOf[FakeRowAdaptor]) + assert(childPlan.asInstanceOf[FakeRowAdaptor].child.isInstanceOf[AdaptiveSparkPlanExec]) + + spark.listenerManager.unregister(listener) + } + } + } + testGluten("SPARK-35650: Coalesce number of partitions by AEQ") { withSQLConf(SQLConf.COALESCE_PARTITIONS_MIN_PARTITION_NUM.key -> "1") { Seq("REPARTITION", "REBALANCE(key)") diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenWriterColumnarRulesSuite.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenWriterColumnarRulesSuite.scala deleted file mode 100644 index 10abca1c6dd3..000000000000 --- a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenWriterColumnarRulesSuite.scala +++ /dev/null @@ -1,54 +0,0 @@ -/* - * 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.datasources - -import org.apache.gluten.config.GlutenConfig - -import org.apache.spark.sql.{GlutenSQLTestsBaseTrait, SaveMode} -import org.apache.spark.sql.execution.QueryExecution -import org.apache.spark.sql.util.QueryExecutionListener - -class GlutenWriterColumnarRulesSuite extends GlutenSQLTestsBaseTrait { - - class WriterColumnarListener extends QueryExecutionListener { - var fakeRowAdaptor: Option[FakeRowAdaptor] = None - - override def onSuccess(funcName: String, qe: QueryExecution, durationNs: Long): Unit = { - fakeRowAdaptor = qe.executedPlan.collectFirst { case f: FakeRowAdaptor => f } - } - - override def onFailure(funcName: String, qe: QueryExecution, exception: Exception): Unit = {} - } - - testGluten("writing to noop") { - withTempDir { - dir => - withSQLConf(GlutenConfig.NATIVE_WRITER_ENABLED.key -> "true") { - spark.range(0, 100).write.mode(SaveMode.Overwrite).parquet(dir.getPath) - val listener = new WriterColumnarListener - spark.listenerManager.register(listener) - try { - spark.read.parquet(dir.getPath).write.format("noop").mode(SaveMode.Overwrite).save() - spark.sparkContext.listenerBus.waitUntilEmpty() - assert(listener.fakeRowAdaptor.isDefined, "FakeRowAdaptor is not found.") - } finally { - spark.listenerManager.unregister(listener) - } - } - } - } -} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala b/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala index 56c95ae1bd05..9ebcadf53118 100644 --- a/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala +++ b/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala @@ -983,6 +983,8 @@ class ClickHouseTestSettings extends BackendTestSettings { .exclude("Change merge join to broadcast join without local shuffle read") .exclude( "Avoid changing merge join to broadcast join if too many empty partitions on build plan") + .exclude( + "SPARK-30953: InsertAdaptiveSparkPlan should apply AQE on child plan of v2 write commands") .exclude("SPARK-37753: Allow changing outer join to broadcast join even if too many empty partitions on broadcast side") .exclude("SPARK-29544: adaptive skew join with different join types") .exclude("SPARK-34682: AQEShuffleReadExec operating on canonicalized plan") diff --git a/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala b/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala index cc9746dcdb53..94d3a1f6e890 100644 --- a/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala +++ b/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.connector.{GlutenDataSourceV2DataFrameSessionCatalog import org.apache.spark.sql.errors.{GlutenQueryCompilationErrorsDSv2Suite, GlutenQueryCompilationErrorsSuite, GlutenQueryExecutionErrorsSuite, GlutenQueryParsingErrorsSuite} import org.apache.spark.sql.execution.{FallbackStrategiesSuite, GlutenBroadcastExchangeSuite, GlutenCoalesceShufflePartitionsSuite, GlutenExchangeSuite, GlutenLocalBroadcastExchangeSuite, GlutenReplaceHashWithSortAggSuite, GlutenReuseExchangeAndSubquerySuite, GlutenSameResultSuite, GlutenSortSuite, GlutenSQLAggregateFunctionSuite, GlutenSQLWindowFunctionSuite, GlutenTakeOrderedAndProjectSuite} import org.apache.spark.sql.execution.adaptive.velox.VeloxAdaptiveQueryExecSuite -import org.apache.spark.sql.execution.datasources.{GlutenBucketingUtilsSuite, GlutenCSVReadSchemaSuite, GlutenDataSourceStrategySuite, GlutenDataSourceSuite, GlutenFileFormatWriterSuite, GlutenFileIndexSuite, GlutenFileMetadataStructRowIndexSuite, GlutenFileMetadataStructSuite, GlutenFileSourceStrategySuite, GlutenHadoopFileLinesReaderSuite, GlutenHeaderCSVReadSchemaSuite, GlutenJsonReadSchemaSuite, GlutenMergedOrcReadSchemaSuite, GlutenMergedParquetReadSchemaSuite, GlutenOrcCodecSuite, GlutenOrcReadSchemaSuite, GlutenOrcV1AggregatePushDownSuite, GlutenOrcV2AggregatePushDownSuite, GlutenParquetCodecSuite, GlutenParquetReadSchemaSuite, GlutenParquetV1AggregatePushDownSuite, GlutenParquetV2AggregatePushDownSuite, GlutenPathFilterStrategySuite, GlutenPathFilterSuite, GlutenPruneFileSourcePartitionsSuite, GlutenV1WriteCommandSuite, GlutenVectorizedOrcReadSchemaSuite, GlutenVectorizedParquetReadSchemaSuite} +import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.binaryfile.GlutenBinaryFileFormatSuite import org.apache.spark.sql.execution.datasources.csv.{GlutenCSVLegacyTimeParserSuite, GlutenCSVv1Suite, GlutenCSVv2Suite} import org.apache.spark.sql.execution.datasources.exchange.GlutenValidateRequirementsSuite @@ -182,7 +182,6 @@ class VeloxTestSettings extends BackendTestSettings { "SPARK-30403", "SPARK-30719", "SPARK-31384", - "SPARK-30953", "SPARK-31658", "SPARK-32717", "SPARK-32649", diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/adaptive/clickhouse/ClickHouseAdaptiveQueryExecSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/adaptive/clickhouse/ClickHouseAdaptiveQueryExecSuite.scala index 49d47fa65b1f..2bd5a96dadee 100644 --- a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/adaptive/clickhouse/ClickHouseAdaptiveQueryExecSuite.scala +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/adaptive/clickhouse/ClickHouseAdaptiveQueryExecSuite.scala @@ -16,6 +16,7 @@ */ package org.apache.spark.sql.execution.adaptive.clickhouse +import org.apache.gluten.config.GlutenConfig import org.apache.gluten.execution.{BroadcastHashJoinExecTransformerBase, ShuffledHashJoinExecTransformerBase, SortExecTransformer, SortMergeJoinExecTransformer} import org.apache.spark.SparkConf @@ -25,6 +26,9 @@ import org.apache.spark.sql.GlutenTestConstants.GLUTEN_TEST import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight} import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.adaptive._ +import org.apache.spark.sql.execution.datasources.FakeRowAdaptor +import org.apache.spark.sql.execution.datasources.noop.NoopDataSource +import org.apache.spark.sql.execution.datasources.v2.V2TableWriteExec import org.apache.spark.sql.execution.exchange._ import org.apache.spark.sql.execution.joins.{BaseJoinExec, BroadcastHashJoinExec, ShuffledHashJoinExec, SortMergeJoinExec} import org.apache.spark.sql.execution.metric.SQLShuffleReadMetricsReporter @@ -33,6 +37,7 @@ import org.apache.spark.sql.functions.when import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SQLTestData.TestData import org.apache.spark.sql.types.{IntegerType, StructType} +import org.apache.spark.sql.util.QueryExecutionListener import org.apache.logging.log4j.Level @@ -41,7 +46,7 @@ class ClickHouseAdaptiveQueryExecSuite extends AdaptiveQueryExecSuite with Glute override def sparkConf: SparkConf = { super.sparkConf - .set("spark.gluten.sql.columnar.forceShuffledHashJoin", "false") + .set(GlutenConfig.COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED.key, "false") .set(SQLConf.SHUFFLE_PARTITIONS.key, "5") } @@ -1182,6 +1187,37 @@ class ClickHouseAdaptiveQueryExecSuite extends AdaptiveQueryExecSuite with Glute } } + testGluten( + "SPARK-30953: InsertAdaptiveSparkPlan should apply AQE on child plan of v2 write commands") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.ADAPTIVE_EXECUTION_FORCE_APPLY.key -> "true") { + var plan: SparkPlan = null + val listener = new QueryExecutionListener { + override def onSuccess(funcName: String, qe: QueryExecution, durationNs: Long): Unit = { + plan = qe.executedPlan + } + override def onFailure( + funcName: String, + qe: QueryExecution, + exception: Exception): Unit = {} + } + spark.listenerManager.register(listener) + withTable("t1") { + val format = classOf[NoopDataSource].getName + Seq((0, 1)).toDF("x", "y").write.format(format).mode("overwrite").save() + + sparkContext.listenerBus.waitUntilEmpty() + assert(plan.isInstanceOf[V2TableWriteExec]) + val childPlan = plan.asInstanceOf[V2TableWriteExec].child + assert(childPlan.isInstanceOf[FakeRowAdaptor]) + assert(childPlan.asInstanceOf[FakeRowAdaptor].child.isInstanceOf[AdaptiveSparkPlanExec]) + + spark.listenerManager.unregister(listener) + } + } + } + testGluten("SPARK-35650: Coalesce number of partitions by AEQ") { withSQLConf(SQLConf.COALESCE_PARTITIONS_MIN_PARTITION_NUM.key -> "1") { Seq("REPARTITION", "REBALANCE(key)") diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/adaptive/velox/VeloxAdaptiveQueryExecSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/adaptive/velox/VeloxAdaptiveQueryExecSuite.scala index 729a12f56cc5..6a3d6da27cef 100644 --- a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/adaptive/velox/VeloxAdaptiveQueryExecSuite.scala +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/adaptive/velox/VeloxAdaptiveQueryExecSuite.scala @@ -16,6 +16,7 @@ */ package org.apache.spark.sql.execution.adaptive.velox +import org.apache.gluten.config.GlutenConfig import org.apache.gluten.execution.{BroadcastHashJoinExecTransformerBase, ShuffledHashJoinExecTransformerBase, SortExecTransformer, SortMergeJoinExecTransformer} import org.apache.spark.SparkConf @@ -25,6 +26,9 @@ import org.apache.spark.sql.GlutenTestConstants.GLUTEN_TEST import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight} import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.adaptive._ +import org.apache.spark.sql.execution.datasources.FakeRowAdaptor +import org.apache.spark.sql.execution.datasources.noop.NoopDataSource +import org.apache.spark.sql.execution.datasources.v2.V2TableWriteExec import org.apache.spark.sql.execution.exchange._ import org.apache.spark.sql.execution.joins.{BaseJoinExec, BroadcastHashJoinExec, ShuffledHashJoinExec, SortMergeJoinExec} import org.apache.spark.sql.execution.metric.SQLShuffleReadMetricsReporter @@ -33,6 +37,7 @@ import org.apache.spark.sql.functions.when import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SQLTestData.TestData import org.apache.spark.sql.types.{IntegerType, StructType} +import org.apache.spark.sql.util.QueryExecutionListener import org.apache.logging.log4j.Level @@ -41,7 +46,7 @@ class VeloxAdaptiveQueryExecSuite extends AdaptiveQueryExecSuite with GlutenSQLT override def sparkConf: SparkConf = { super.sparkConf - .set("spark.gluten.sql.columnar.forceShuffledHashJoin", "false") + .set(GlutenConfig.COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED.key, "false") .set(SQLConf.SHUFFLE_PARTITIONS.key, "5") } @@ -1179,6 +1184,37 @@ class VeloxAdaptiveQueryExecSuite extends AdaptiveQueryExecSuite with GlutenSQLT } } + testGluten( + "SPARK-30953: InsertAdaptiveSparkPlan should apply AQE on child plan of v2 write commands") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.ADAPTIVE_EXECUTION_FORCE_APPLY.key -> "true") { + var plan: SparkPlan = null + val listener = new QueryExecutionListener { + override def onSuccess(funcName: String, qe: QueryExecution, durationNs: Long): Unit = { + plan = qe.executedPlan + } + override def onFailure( + funcName: String, + qe: QueryExecution, + exception: Exception): Unit = {} + } + spark.listenerManager.register(listener) + withTable("t1") { + val format = classOf[NoopDataSource].getName + Seq((0, 1)).toDF("x", "y").write.format(format).mode("overwrite").save() + + sparkContext.listenerBus.waitUntilEmpty() + assert(plan.isInstanceOf[V2TableWriteExec]) + val childPlan = plan.asInstanceOf[V2TableWriteExec].child + assert(childPlan.isInstanceOf[FakeRowAdaptor]) + assert(childPlan.asInstanceOf[FakeRowAdaptor].child.isInstanceOf[AdaptiveSparkPlanExec]) + + spark.listenerManager.unregister(listener) + } + } + } + testGluten("SPARK-35650: Coalesce number of partitions by AEQ") { withSQLConf(SQLConf.COALESCE_PARTITIONS_MIN_PARTITION_NUM.key -> "1") { Seq("REPARTITION", "REBALANCE(key)") diff --git a/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala b/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala index bdb160ad24be..f482ad921ee3 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala @@ -983,6 +983,8 @@ class ClickHouseTestSettings extends BackendTestSettings { .exclude("Change merge join to broadcast join without local shuffle read") .exclude( "Avoid changing merge join to broadcast join if too many empty partitions on build plan") + .exclude( + "SPARK-30953: InsertAdaptiveSparkPlan should apply AQE on child plan of v2 write commands") .exclude("SPARK-37753: Allow changing outer join to broadcast join even if too many empty partitions on broadcast side") .exclude("SPARK-29544: adaptive skew join with different join types") .exclude("SPARK-34682: AQEShuffleReadExec operating on canonicalized plan") diff --git a/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala b/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala index 71786c91322b..73c4d43ced53 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala @@ -185,7 +185,6 @@ class VeloxTestSettings extends BackendTestSettings { "SPARK-30403", "SPARK-30719", "SPARK-31384", - "SPARK-30953", "SPARK-31658", "SPARK-32717", "SPARK-32649", diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/adaptive/clickhouse/ClickHouseAdaptiveQueryExecSuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/adaptive/clickhouse/ClickHouseAdaptiveQueryExecSuite.scala index 2e5df7b859e3..bd941586d73c 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/adaptive/clickhouse/ClickHouseAdaptiveQueryExecSuite.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/adaptive/clickhouse/ClickHouseAdaptiveQueryExecSuite.scala @@ -16,6 +16,7 @@ */ package org.apache.spark.sql.execution.adaptive.clickhouse +import org.apache.gluten.config.GlutenConfig import org.apache.gluten.execution.{BroadcastHashJoinExecTransformerBase, ShuffledHashJoinExecTransformerBase, SortExecTransformer, SortMergeJoinExecTransformer} import org.apache.spark.SparkConf @@ -25,6 +26,9 @@ import org.apache.spark.sql.GlutenTestConstants.GLUTEN_TEST import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight} import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.adaptive._ +import org.apache.spark.sql.execution.datasources.FakeRowAdaptor +import org.apache.spark.sql.execution.datasources.noop.NoopDataSource +import org.apache.spark.sql.execution.datasources.v2.V2TableWriteExec import org.apache.spark.sql.execution.exchange._ import org.apache.spark.sql.execution.joins.{BaseJoinExec, BroadcastHashJoinExec, ShuffledHashJoinExec, SortMergeJoinExec} import org.apache.spark.sql.execution.metric.SQLShuffleReadMetricsReporter @@ -33,6 +37,7 @@ import org.apache.spark.sql.functions.when import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SQLTestData.TestData import org.apache.spark.sql.types.{IntegerType, StructType} +import org.apache.spark.sql.util.QueryExecutionListener import org.apache.logging.log4j.Level @@ -41,7 +46,7 @@ class ClickHouseAdaptiveQueryExecSuite extends AdaptiveQueryExecSuite with Glute override def sparkConf: SparkConf = { super.sparkConf - .set("spark.gluten.sql.columnar.forceShuffledHashJoin", "false") + .set(GlutenConfig.COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED.key, "false") .set(SQLConf.SHUFFLE_PARTITIONS.key, "5") } @@ -1197,6 +1202,37 @@ class ClickHouseAdaptiveQueryExecSuite extends AdaptiveQueryExecSuite with Glute } } + testGluten( + "SPARK-30953: InsertAdaptiveSparkPlan should apply AQE on child plan of v2 write commands") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.ADAPTIVE_EXECUTION_FORCE_APPLY.key -> "true") { + var plan: SparkPlan = null + val listener = new QueryExecutionListener { + override def onSuccess(funcName: String, qe: QueryExecution, durationNs: Long): Unit = { + plan = qe.executedPlan + } + override def onFailure( + funcName: String, + qe: QueryExecution, + exception: Exception): Unit = {} + } + spark.listenerManager.register(listener) + withTable("t1") { + val format = classOf[NoopDataSource].getName + Seq((0, 1)).toDF("x", "y").write.format(format).mode("overwrite").save() + + sparkContext.listenerBus.waitUntilEmpty() + assert(plan.isInstanceOf[V2TableWriteExec]) + val childPlan = plan.asInstanceOf[V2TableWriteExec].child + assert(childPlan.isInstanceOf[FakeRowAdaptor]) + assert(childPlan.asInstanceOf[FakeRowAdaptor].child.isInstanceOf[AdaptiveSparkPlanExec]) + + spark.listenerManager.unregister(listener) + } + } + } + testGluten("SPARK-35650: Coalesce number of partitions by AEQ") { withSQLConf(SQLConf.COALESCE_PARTITIONS_MIN_PARTITION_NUM.key -> "1") { Seq("REPARTITION", "REBALANCE(key)") diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/adaptive/velox/VeloxAdaptiveQueryExecSuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/adaptive/velox/VeloxAdaptiveQueryExecSuite.scala index 729a12f56cc5..6a3d6da27cef 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/adaptive/velox/VeloxAdaptiveQueryExecSuite.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/adaptive/velox/VeloxAdaptiveQueryExecSuite.scala @@ -16,6 +16,7 @@ */ package org.apache.spark.sql.execution.adaptive.velox +import org.apache.gluten.config.GlutenConfig import org.apache.gluten.execution.{BroadcastHashJoinExecTransformerBase, ShuffledHashJoinExecTransformerBase, SortExecTransformer, SortMergeJoinExecTransformer} import org.apache.spark.SparkConf @@ -25,6 +26,9 @@ import org.apache.spark.sql.GlutenTestConstants.GLUTEN_TEST import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight} import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.adaptive._ +import org.apache.spark.sql.execution.datasources.FakeRowAdaptor +import org.apache.spark.sql.execution.datasources.noop.NoopDataSource +import org.apache.spark.sql.execution.datasources.v2.V2TableWriteExec import org.apache.spark.sql.execution.exchange._ import org.apache.spark.sql.execution.joins.{BaseJoinExec, BroadcastHashJoinExec, ShuffledHashJoinExec, SortMergeJoinExec} import org.apache.spark.sql.execution.metric.SQLShuffleReadMetricsReporter @@ -33,6 +37,7 @@ import org.apache.spark.sql.functions.when import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SQLTestData.TestData import org.apache.spark.sql.types.{IntegerType, StructType} +import org.apache.spark.sql.util.QueryExecutionListener import org.apache.logging.log4j.Level @@ -41,7 +46,7 @@ class VeloxAdaptiveQueryExecSuite extends AdaptiveQueryExecSuite with GlutenSQLT override def sparkConf: SparkConf = { super.sparkConf - .set("spark.gluten.sql.columnar.forceShuffledHashJoin", "false") + .set(GlutenConfig.COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED.key, "false") .set(SQLConf.SHUFFLE_PARTITIONS.key, "5") } @@ -1179,6 +1184,37 @@ class VeloxAdaptiveQueryExecSuite extends AdaptiveQueryExecSuite with GlutenSQLT } } + testGluten( + "SPARK-30953: InsertAdaptiveSparkPlan should apply AQE on child plan of v2 write commands") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.ADAPTIVE_EXECUTION_FORCE_APPLY.key -> "true") { + var plan: SparkPlan = null + val listener = new QueryExecutionListener { + override def onSuccess(funcName: String, qe: QueryExecution, durationNs: Long): Unit = { + plan = qe.executedPlan + } + override def onFailure( + funcName: String, + qe: QueryExecution, + exception: Exception): Unit = {} + } + spark.listenerManager.register(listener) + withTable("t1") { + val format = classOf[NoopDataSource].getName + Seq((0, 1)).toDF("x", "y").write.format(format).mode("overwrite").save() + + sparkContext.listenerBus.waitUntilEmpty() + assert(plan.isInstanceOf[V2TableWriteExec]) + val childPlan = plan.asInstanceOf[V2TableWriteExec].child + assert(childPlan.isInstanceOf[FakeRowAdaptor]) + assert(childPlan.asInstanceOf[FakeRowAdaptor].child.isInstanceOf[AdaptiveSparkPlanExec]) + + spark.listenerManager.unregister(listener) + } + } + } + testGluten("SPARK-35650: Coalesce number of partitions by AEQ") { withSQLConf(SQLConf.COALESCE_PARTITIONS_MIN_PARTITION_NUM.key -> "1") { Seq("REPARTITION", "REBALANCE(key)") diff --git a/gluten-ut/test/src/test/scala/org/apache/spark/sql/datasources/GlutenNoopWriterRuleSuite.scala b/gluten-ut/test/src/test/scala/org/apache/spark/sql/datasources/GlutenNoopWriterRuleSuite.scala new file mode 100644 index 000000000000..ebf17444e623 --- /dev/null +++ b/gluten-ut/test/src/test/scala/org/apache/spark/sql/datasources/GlutenNoopWriterRuleSuite.scala @@ -0,0 +1,70 @@ +/* + * 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.datasources + +import org.apache.gluten.config.GlutenConfig +import org.apache.gluten.utils.{BackendTestUtils, SystemParameters} + +import org.apache.spark.SparkConf +import org.apache.spark.sql.{GlutenQueryTest, SaveMode} +import org.apache.spark.sql.execution.QueryExecution +import org.apache.spark.sql.execution.datasources.FakeRowAdaptor +import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.util.QueryExecutionListener + +class GlutenNoopWriterRuleSuite extends GlutenQueryTest with SharedSparkSession { + + override def sparkConf: SparkConf = { + val conf = super.sparkConf + .set("spark.plugins", "org.apache.gluten.GlutenPlugin") + .set("spark.default.parallelism", "1") + .set("spark.memory.offHeap.enabled", "true") + .set("spark.memory.offHeap.size", "1024MB") + .set("spark.ui.enabled", "false") + .set("spark.gluten.ui.enabled", "false") + if (BackendTestUtils.isCHBackendLoaded()) { + conf.set(GlutenConfig.GLUTEN_LIB_PATH, SystemParameters.getClickHouseLibPath) + } + conf + } + + class WriterColumnarListener extends QueryExecutionListener { + var fakeRowAdaptor: Option[FakeRowAdaptor] = None + + override def onSuccess(funcName: String, qe: QueryExecution, durationNs: Long): Unit = { + fakeRowAdaptor = qe.executedPlan.collectFirst { case f: FakeRowAdaptor => f } + } + + override def onFailure(funcName: String, qe: QueryExecution, exception: Exception): Unit = {} + } + + test("writing to noop") { + withTempDir { + dir => + spark.range(0, 100).write.mode(SaveMode.Overwrite).parquet(dir.getPath) + val listener = new WriterColumnarListener + spark.listenerManager.register(listener) + try { + spark.read.parquet(dir.getPath).write.format("noop").mode(SaveMode.Overwrite).save() + spark.sparkContext.listenerBus.waitUntilEmpty() + assert(listener.fakeRowAdaptor.isDefined, "FakeRowAdaptor is not found.") + } finally { + spark.listenerManager.unregister(listener) + } + } + } +} From d6a58dcb9eafa76bacf2aafc4e0232d891bf61c0 Mon Sep 17 00:00:00 2001 From: Guoliang Sun Date: Thu, 9 Jan 2025 17:16:26 +0800 Subject: [PATCH 04/28] [GLUTEN-8462][CH] Fixed the loading of Components and Backend (#8464) Closes #8462 --- .../org/apache/gluten/utils/ResourceUtil.java | 25 +++++++++++++++---- 1 file changed, 20 insertions(+), 5 deletions(-) diff --git a/gluten-core/src/main/java/org/apache/gluten/utils/ResourceUtil.java b/gluten-core/src/main/java/org/apache/gluten/utils/ResourceUtil.java index 692a91af2667..ac66c57a8c3a 100644 --- a/gluten-core/src/main/java/org/apache/gluten/utils/ResourceUtil.java +++ b/gluten-core/src/main/java/org/apache/gluten/utils/ResourceUtil.java @@ -22,6 +22,7 @@ import java.io.File; import java.io.IOException; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.Enumeration; import java.util.List; @@ -47,14 +48,28 @@ public class ResourceUtil { */ public static List getResources(final Pattern pattern) { final List buffer = new ArrayList<>(); - final String classPath = System.getProperty("java.class.path"); - final String[] classPathElements = classPath.split(File.pathSeparator); - for (final String element : classPathElements) { - getResources(element, pattern, buffer); - } + String classPath = System.getProperty("java.class.path"); + processClassPathElements(classPath, pattern, buffer); return Collections.unmodifiableList(buffer); } + private static void processClassPathElements( + String classPath, Pattern pattern, List buffer) { + if (classPath == null || classPath.isEmpty()) { + return; + } + String[] classPathElements = classPath.split(File.pathSeparator); + Arrays.stream(classPathElements).forEach(element -> getResources(element, pattern, buffer)); + // the Gluten project may wrapped by the other service to use the Native Engine. + // As a result, the java.class.path points to xxx/other.jar instead of xxx/gluten.jar. + // This will result in the failure to properly load the required Components. + if (buffer.isEmpty()) { + classPath = ResourceUtil.class.getProtectionDomain().getCodeSource().getLocation().getPath(); + classPathElements = classPath.split(File.pathSeparator); + Arrays.stream(classPathElements).forEach(element -> getResources(element, pattern, buffer)); + } + } + private static void getResources( final String element, final Pattern pattern, final List buffer) { final File file = new File(element); From 8109665a1ca5acd975ffb61b49c4b5fde52988ae Mon Sep 17 00:00:00 2001 From: moslifulong Date: Thu, 9 Jan 2025 20:38:02 +0800 Subject: [PATCH 05/28] [GLUTEN-8414][VL] Override doCanonicalize in ColumnarPartialProjectExec node (#8415) Closes #8414 --- .../gluten/execution/ColumnarPartialProjectExec.scala | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/backends-velox/src/main/scala/org/apache/gluten/execution/ColumnarPartialProjectExec.scala b/backends-velox/src/main/scala/org/apache/gluten/execution/ColumnarPartialProjectExec.scala index 056d19e55070..435fd239b364 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/execution/ColumnarPartialProjectExec.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/execution/ColumnarPartialProjectExec.scala @@ -30,6 +30,7 @@ import org.apache.gluten.vectorized.{ArrowColumnarRow, ArrowWritableColumnVector import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeReference, CaseWhen, Coalesce, Expression, If, LambdaFunction, NamedExpression, NaNvl, ScalaUDF} +import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.execution.{ExplainUtils, ProjectExec, SparkPlan, UnaryExecNode} import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.hive.HiveUdfUtil @@ -75,6 +76,14 @@ case class ColumnarPartialProjectExec(original: ProjectExec, child: SparkPlan)( override def output: Seq[Attribute] = child.output ++ replacedAliasUdf.map(_.toAttribute) + override def doCanonicalize(): ColumnarPartialProjectExec = { + val canonicalized = original.canonicalized.asInstanceOf[ProjectExec] + this.copy( + original = canonicalized, + child = child.canonicalized + )(replacedAliasUdf.map(QueryPlan.normalizeExpressions(_, child.output))) + } + override def batchType(): Convention.BatchType = BackendsApiManager.getSettings.primaryBatchType override def rowType0(): Convention.RowType = Convention.RowType.None From f153ceeb980a1a5d1ec097a93211d048e888d9a1 Mon Sep 17 00:00:00 2001 From: Yan Xin Date: Thu, 9 Jan 2025 20:39:41 +0800 Subject: [PATCH 06/28] [GLUTEN-8397][CH][Part-2] Fix statica_cast failed on macos (#8485) --- cpp-ch/local-engine/Parser/CHColumnToSparkRow.cpp | 4 ++-- cpp-ch/local-engine/local_engine_jni.cpp | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/cpp-ch/local-engine/Parser/CHColumnToSparkRow.cpp b/cpp-ch/local-engine/Parser/CHColumnToSparkRow.cpp index e88db198953e..b5ac3aedf548 100644 --- a/cpp-ch/local-engine/Parser/CHColumnToSparkRow.cpp +++ b/cpp-ch/local-engine/Parser/CHColumnToSparkRow.cpp @@ -964,10 +964,10 @@ jobject create(JNIEnv * env, const SparkRowInfo & spark_row_info) { auto * offsets_arr = env->NewLongArray(spark_row_info.getNumRows()); const auto * offsets_src = spark_row_info.getOffsets().data(); - env->SetLongArrayRegion(offsets_arr, 0, spark_row_info.getNumRows(), static_cast(offsets_src)); + env->SetLongArrayRegion(offsets_arr, 0, spark_row_info.getNumRows(), reinterpret_cast(offsets_src)); auto * lengths_arr = env->NewLongArray(spark_row_info.getNumRows()); const auto * lengths_src = spark_row_info.getLengths().data(); - env->SetLongArrayRegion(lengths_arr, 0, spark_row_info.getNumRows(), static_cast(lengths_src)); + env->SetLongArrayRegion(lengths_arr, 0, spark_row_info.getNumRows(), reinterpret_cast(lengths_src)); int64_t address = reinterpret_cast(spark_row_info.getBufferAddress()); int64_t column_number = spark_row_info.getNumCols(); int64_t total_size = spark_row_info.getTotalBytes(); diff --git a/cpp-ch/local-engine/local_engine_jni.cpp b/cpp-ch/local-engine/local_engine_jni.cpp index 855d2b8371ef..6f3df78fb61f 100644 --- a/cpp-ch/local-engine/local_engine_jni.cpp +++ b/cpp-ch/local-engine/local_engine_jni.cpp @@ -1039,7 +1039,7 @@ JNIEXPORT jobject Java_org_apache_spark_sql_execution_datasources_CHDatasourceJn local_engine::BlockStripes bs = local_engine::BlockStripeSplitter::split(*block, partition_col_indice_vec, hasBucket, reserve_); auto * addresses = env->NewLongArray(bs.block_addresses.size()); - env->SetLongArrayRegion(addresses, 0, bs.block_addresses.size(), static_cast(bs.block_addresses.data())); + env->SetLongArrayRegion(addresses, 0, bs.block_addresses.size(), reinterpret_cast(bs.block_addresses.data())); auto * indices = env->NewIntArray(bs.heading_row_indice.size()); env->SetIntArrayRegion(indices, 0, bs.heading_row_indice.size(), bs.heading_row_indice.data()); From 66e816fb10e821f5544a4e5cb296f456039b022f Mon Sep 17 00:00:00 2001 From: kevinyhzou <37431499+KevinyhZou@users.noreply.github.com> Date: Fri, 10 Jan 2025 09:39:00 +0800 Subject: [PATCH 07/28] [GLUTEN-8343][CH]Fix cast number to decimal (#8351) * fix cast number to decimal * simply code * fix ci --- ...enClickHouseTPCHSaltNullParquetSuite.scala | 15 +++ .../SparkFunctionCheckDecimalOverflow.cpp | 94 +++++++++++++------ .../local-engine/Parser/ExpressionParser.cpp | 13 ++- 3 files changed, 88 insertions(+), 34 deletions(-) diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHSaltNullParquetSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHSaltNullParquetSuite.scala index 1ebdded5e189..dc01cf7c409c 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHSaltNullParquetSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHSaltNullParquetSuite.scala @@ -3362,5 +3362,20 @@ class GlutenClickHouseTPCHSaltNullParquetSuite extends GlutenClickHouseTPCHAbstr val sql = "select * from test_filter where (c1, c2) in (('a1', 'b1'), ('a2', 'b2'))" compareResultsAgainstVanillaSpark(sql, true, { _ => }) } + + test("GLUTEN-8343: Cast number to decimal") { + val create_table_sql = "create table test_tbl_8343(id bigint, d bigint, f double) using parquet" + val insert_data_sql = + "insert into test_tbl_8343 values(1, 55, 55.12345), (2, 137438953483, 137438953483.12345), (3, -12, -12.123), (4, 0, 0.0001), (5, NULL, NULL), (6, %d, NULL), (7, %d, NULL)" + .format(Double.MaxValue.longValue(), Double.MinValue.longValue()) + val query_sql = + "select cast(d as decimal(1, 0)), cast(d as decimal(9, 1)), cast((f-55.12345) as decimal(9,1)), cast(f as decimal(4,2)), " + + "cast(f as decimal(32, 3)), cast(f as decimal(2, 1)), cast(d as decimal(38,3)) from test_tbl_8343" + spark.sql(create_table_sql); + spark.sql(insert_data_sql); + compareResultsAgainstVanillaSpark(query_sql, true, { _ => }) + spark.sql("drop table test_tbl_8343") + } + } // scalastyle:on line.size.limit diff --git a/cpp-ch/local-engine/Functions/SparkFunctionCheckDecimalOverflow.cpp b/cpp-ch/local-engine/Functions/SparkFunctionCheckDecimalOverflow.cpp index 8b5a7eff65db..73fc7b5d4e40 100644 --- a/cpp-ch/local-engine/Functions/SparkFunctionCheckDecimalOverflow.cpp +++ b/cpp-ch/local-engine/Functions/SparkFunctionCheckDecimalOverflow.cpp @@ -25,6 +25,7 @@ #include #include #include +#include namespace DB { @@ -34,6 +35,7 @@ extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int ILLEGAL_COLUMN; extern const int TYPE_MISMATCH; +extern const int NOT_IMPLEMENTED; } } @@ -78,7 +80,7 @@ class FunctionCheckDecimalOverflow : public IFunction DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - if (!isDecimal(arguments[0].type) || !isInteger(arguments[1].type) || !isInteger(arguments[2].type)) + if ((!isDecimal(arguments[0].type) && !isNativeNumber(arguments[0].type)) || !isInteger(arguments[1].type) || !isInteger(arguments[2].type)) throw Exception( ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} {} {} of argument of function {}", @@ -86,17 +88,14 @@ class FunctionCheckDecimalOverflow : public IFunction arguments[1].type->getName(), arguments[2].type->getName(), getName()); - UInt32 precision = extractArgument(arguments[1]); UInt32 scale = extractArgument(arguments[2]); - auto return_type = createDecimal(precision, scale); if constexpr (exception_mode == CheckExceptionMode::Null) { if (!arguments[0].type->isNullable()) return std::make_shared(return_type); } - return return_type; } @@ -113,19 +112,15 @@ class FunctionCheckDecimalOverflow : public IFunction using Types = std::decay_t; using FromDataType = typename Types::LeftType; using ToDataType = typename Types::RightType; - - if constexpr (IsDataTypeDecimal) + if constexpr (IsDataTypeDecimal || IsDataTypeNumber) { using FromFieldType = typename FromDataType::FieldType; - using ColVecType = ColumnDecimal; - - if (const ColVecType * col_vec = checkAndGetColumn(src_column.column.get())) + if (const ColumnVectorOrDecimal * col_vec = checkAndGetColumn>(src_column.column.get())) { - executeInternal(*col_vec, result_column, input_rows_count, precision, scale); + executeInternal(*col_vec, result_column, input_rows_count, precision, scale); return true; } } - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal column while execute function {}", getName()); }; @@ -146,17 +141,28 @@ class FunctionCheckDecimalOverflow : public IFunction } private: - template + template + requires(IsDataTypeDecimal && (IsDataTypeDecimal || IsDataTypeNumber)) static void executeInternal( - const ColumnDecimal & col_source, ColumnPtr & result_column, size_t input_rows_count, UInt32 precision, UInt32 scale_to) + const ColumnVectorOrDecimal & col_source, ColumnPtr & result_column, size_t input_rows_count, UInt32 precision, UInt32 scale_to) { using ToFieldType = typename ToDataType::FieldType; using ToColumnType = typename ToDataType::ColumnType; + using T = typename FromDataType::FieldType; ColumnUInt8::MutablePtr col_null_map_to; ColumnUInt8::Container * vec_null_map_to [[maybe_unused]] = nullptr; - auto scale_from = col_source.getScale(); - + UInt32 scale_from = 0; + using ToFieldNativeType = typename ToFieldType::NativeType; + ToFieldNativeType decimal_int_part_max = 0; + ToFieldNativeType decimal_int_part_min = 0; + if constexpr (IsDataTypeDecimal) + scale_from = col_source.getScale(); + else + { + decimal_int_part_max = DecimalUtils::scaleMultiplier(precision - scale_to) - 1; + decimal_int_part_min = 1 - DecimalUtils::scaleMultiplier(precision - scale_to); + } if constexpr (exception_mode == CheckExceptionMode::Null) { col_null_map_to = ColumnUInt8::create(input_rows_count, false); @@ -170,17 +176,17 @@ class FunctionCheckDecimalOverflow : public IFunction auto & datas = col_source.getData(); for (size_t i = 0; i < input_rows_count; ++i) { - // bool overflow = outOfDigits(datas[i], precision, scale_from, scale_to); ToFieldType result; - bool success = convertToDecimalImpl(datas[i], precision, scale_from, scale_to, result); - - if (success) + bool success = convertToDecimalImpl(datas[i], precision, scale_from, scale_to, decimal_int_part_max, decimal_int_part_min, result); + if constexpr (exception_mode == CheckExceptionMode::Null) + { vec_to[i] = static_cast(result); + (*vec_null_map_to)[i] = !success; + } else { - vec_to[i] = static_cast(0); - if constexpr (exception_mode == CheckExceptionMode::Null) - (*vec_null_map_to)[i] = static_cast(1); + if (success) + vec_to[i] = static_cast(result); else throw Exception(ErrorCodes::DECIMAL_OVERFLOW, "Decimal value is overflow."); } @@ -192,20 +198,50 @@ class FunctionCheckDecimalOverflow : public IFunction result_column = std::move(col_to); } - template + template requires(IsDataTypeDecimal) static bool convertToDecimalImpl( - const FromFieldType & decimal, UInt32 precision_to, UInt32 scale_from, UInt32 scale_to, typename ToDataType::FieldType & result) + const FromDataType::FieldType & value, + UInt32 precision_to, + UInt32 scale_from, + UInt32 scale_to, + typename ToDataType::FieldType::NativeType decimal_int_part_max, + typename ToDataType::FieldType::NativeType decimal_int_part_min, + typename ToDataType::FieldType & result) { + using FromFieldType = typename FromDataType::FieldType; if constexpr (std::is_same_v) - return convertDecimalsImpl, ToDataType>(decimal, precision_to, scale_from, scale_to, result); - + return convertDecimalsImpl, ToDataType>(value, precision_to, scale_from, scale_to, result); else if constexpr (std::is_same_v) - return convertDecimalsImpl, ToDataType>(decimal, precision_to, scale_from, scale_to, result); + return convertDecimalsImpl, ToDataType>(value, precision_to, scale_from, scale_to, result); else if constexpr (std::is_same_v) - return convertDecimalsImpl, ToDataType>(decimal, precision_to, scale_from, scale_to, result); + return convertDecimalsImpl, ToDataType>(value, precision_to, scale_from, scale_to, result); + else if constexpr (std::is_same_v) + return convertDecimalsImpl, ToDataType>(value, precision_to, scale_from, scale_to, result); + else if constexpr (IsDataTypeNumber && !std::is_same_v) + return convertNumberToDecimalImpl, ToDataType>(value, scale_to, decimal_int_part_max, decimal_int_part_min, result); else - return convertDecimalsImpl, ToDataType>(decimal, precision_to, scale_from, scale_to, result); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Convert from {} type to decimal type is not implemented.", typeid(value).name()); + } + + template + requires(IsDataTypeNumber && IsDataTypeDecimal) + static inline bool convertNumberToDecimalImpl( + const typename FromDataType::FieldType & value, + UInt32 scale, + typename ToDataType::FieldType::NativeType decimal_int_part_max, + typename ToDataType::FieldType::NativeType decimal_int_part_min, + typename ToDataType::FieldType & result) + { + using FromFieldType = typename FromDataType::FieldType; + using ToFieldNativeType = typename ToDataType::FieldType::NativeType; + ToFieldNativeType int_part = 0; + if constexpr (std::is_same_v || std::is_same_v) + int_part = static_cast(value); + else + int_part = value; + + return int_part >= decimal_int_part_min && int_part <= decimal_int_part_max && tryConvertToDecimal(value, scale, result); } template diff --git a/cpp-ch/local-engine/Parser/ExpressionParser.cpp b/cpp-ch/local-engine/Parser/ExpressionParser.cpp index 400d8c28df40..a50590aaf778 100644 --- a/cpp-ch/local-engine/Parser/ExpressionParser.cpp +++ b/cpp-ch/local-engine/Parser/ExpressionParser.cpp @@ -313,7 +313,6 @@ ExpressionParser::NodeRawConstPtr ExpressionParser::parseExpression(ActionsDAG & DataTypePtr denull_input_type = removeNullable(input_type); DataTypePtr output_type = TypeParser::parseType(substrait_type); DataTypePtr denull_output_type = removeNullable(output_type); - const ActionsDAG::Node * result_node = nullptr; if (substrait_type.has_binary()) { @@ -336,11 +335,15 @@ ExpressionParser::NodeRawConstPtr ExpressionParser::parseExpression(ActionsDAG & String function_name = "sparkCastFloatTo" + denull_output_type->getName(); result_node = toFunctionNode(actions_dag, function_name, args); } - else if ((isDecimal(denull_input_type) && substrait_type.has_decimal())) + else if ((isDecimal(denull_input_type) || isNativeNumber(denull_input_type)) && substrait_type.has_decimal()) { - args.emplace_back(addConstColumn(actions_dag, std::make_shared(), substrait_type.decimal().precision())); - args.emplace_back(addConstColumn(actions_dag, std::make_shared(), substrait_type.decimal().scale())); - result_node = toFunctionNode(actions_dag, "checkDecimalOverflowSparkOrNull", args); + int decimal_precision = substrait_type.decimal().precision(); + if (decimal_precision) + { + args.emplace_back(addConstColumn(actions_dag, std::make_shared(), decimal_precision)); + args.emplace_back(addConstColumn(actions_dag, std::make_shared(), substrait_type.decimal().scale())); + result_node = toFunctionNode(actions_dag, "checkDecimalOverflowSparkOrNull", args); + } } else if (isMap(denull_input_type) && isString(denull_output_type)) { From 7af129cba65f3987fa899f83d7560c19014a6943 Mon Sep 17 00:00:00 2001 From: Rong Ma Date: Fri, 10 Jan 2025 10:04:53 +0800 Subject: [PATCH 08/28] [GLUTEN-8481][VL] Clean up shuffle reader cpp code --- cpp/core/CMakeLists.txt | 1 - cpp/core/jni/JniWrapper.cc | 1 - cpp/core/shuffle/ShuffleReader.cc | 55 ------------------- cpp/core/shuffle/ShuffleReader.h | 49 ++--------------- cpp/velox/compute/VeloxRuntime.cc | 2 +- cpp/velox/shuffle/VeloxShuffleReader.cc | 40 +++++++++----- cpp/velox/shuffle/VeloxShuffleReader.h | 33 ++++++----- .../utils/tests/VeloxShuffleWriterTestBase.h | 2 +- 8 files changed, 52 insertions(+), 131 deletions(-) delete mode 100644 cpp/core/shuffle/ShuffleReader.cc diff --git a/cpp/core/CMakeLists.txt b/cpp/core/CMakeLists.txt index 5a5eeac35480..d52ad74b9912 100644 --- a/cpp/core/CMakeLists.txt +++ b/cpp/core/CMakeLists.txt @@ -120,7 +120,6 @@ set(SPARK_COLUMNAR_PLUGIN_SRCS shuffle/RandomPartitioner.cc shuffle/RoundRobinPartitioner.cc shuffle/ShuffleMemoryPool.cc - shuffle/ShuffleReader.cc shuffle/ShuffleWriter.cc shuffle/SinglePartitioner.cc shuffle/Spill.cc diff --git a/cpp/core/jni/JniWrapper.cc b/cpp/core/jni/JniWrapper.cc index b4359bab4160..4e1d1f09fd1e 100644 --- a/cpp/core/jni/JniWrapper.cc +++ b/cpp/core/jni/JniWrapper.cc @@ -1132,7 +1132,6 @@ JNIEXPORT void JNICALL Java_org_apache_gluten_vectorized_ShuffleReaderJniWrapper jlong shuffleReaderHandle) { JNI_METHOD_START auto reader = ObjectStore::retrieve(shuffleReaderHandle); - GLUTEN_THROW_NOT_OK(reader->close()); ObjectStore::release(shuffleReaderHandle); JNI_METHOD_END() } diff --git a/cpp/core/shuffle/ShuffleReader.cc b/cpp/core/shuffle/ShuffleReader.cc deleted file mode 100644 index ced80b3de13f..000000000000 --- a/cpp/core/shuffle/ShuffleReader.cc +++ /dev/null @@ -1,55 +0,0 @@ -/* - * 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. - */ - -#include "ShuffleReader.h" -#include "arrow/ipc/reader.h" -#include "arrow/record_batch.h" -#include "utils/Macros.h" - -#include - -#include "ShuffleSchema.h" - -namespace gluten { - -ShuffleReader::ShuffleReader(std::unique_ptr factory) : factory_(std::move(factory)) {} - -std::shared_ptr ShuffleReader::readStream(std::shared_ptr in) { - return std::make_shared(factory_->createDeserializer(in)); -} - -arrow::Status ShuffleReader::close() { - return arrow::Status::OK(); -} - -arrow::MemoryPool* ShuffleReader::getPool() const { - return factory_->getPool(); -} - -int64_t ShuffleReader::getDecompressTime() const { - return factory_->getDecompressTime(); -} - -ShuffleWriterType ShuffleReader::getShuffleWriterType() const { - return factory_->getShuffleWriterType(); -} - -int64_t ShuffleReader::getDeserializeTime() const { - return factory_->getDeserializeTime(); -} - -} // namespace gluten diff --git a/cpp/core/shuffle/ShuffleReader.h b/cpp/core/shuffle/ShuffleReader.h index 0f985c7da939..6e2b079fc73b 100644 --- a/cpp/core/shuffle/ShuffleReader.h +++ b/cpp/core/shuffle/ShuffleReader.h @@ -17,63 +17,22 @@ #pragma once -#include "memory/ColumnarBatch.h" - -#include -#include - -#include "Options.h" #include "compute/ResultIterator.h" -#include "utils/Compression.h" namespace gluten { -class DeserializerFactory { - public: - virtual ~DeserializerFactory() = default; - - virtual std::unique_ptr createDeserializer(std::shared_ptr in) = 0; - - virtual arrow::MemoryPool* getPool() = 0; - - virtual int64_t getDecompressTime() = 0; - - virtual int64_t getDeserializeTime() = 0; - - virtual ShuffleWriterType getShuffleWriterType() = 0; -}; - class ShuffleReader { public: - explicit ShuffleReader(std::unique_ptr factory); - virtual ~ShuffleReader() = default; // FIXME iterator should be unique_ptr or un-copyable singleton - virtual std::shared_ptr readStream(std::shared_ptr in); - - arrow::Status close(); - - int64_t getDecompressTime() const; - - int64_t getIpcTime() const; - - int64_t getDeserializeTime() const; - - arrow::MemoryPool* getPool() const; - - ShuffleWriterType getShuffleWriterType() const; + virtual std::shared_ptr readStream(std::shared_ptr in) = 0; - protected: - arrow::MemoryPool* pool_; - int64_t decompressTime_ = 0; - int64_t deserializeTime_ = 0; + virtual int64_t getDecompressTime() const = 0; - ShuffleWriterType shuffleWriterType_; + virtual int64_t getDeserializeTime() const = 0; - private: - std::shared_ptr schema_; - std::unique_ptr factory_; + virtual arrow::MemoryPool* getPool() const = 0; }; } // namespace gluten diff --git a/cpp/velox/compute/VeloxRuntime.cc b/cpp/velox/compute/VeloxRuntime.cc index 20c3dec939a0..2a2ea929c1ce 100644 --- a/cpp/velox/compute/VeloxRuntime.cc +++ b/cpp/velox/compute/VeloxRuntime.cc @@ -251,7 +251,7 @@ std::shared_ptr VeloxRuntime::createShuffleReader( auto codec = gluten::createArrowIpcCodec(options.compressionType, options.codecBackend); auto ctxVeloxPool = memoryManager()->getLeafMemoryPool(); auto veloxCompressionType = facebook::velox::common::stringToCompressionKind(options.compressionTypeStr); - auto deserializerFactory = std::make_unique( + auto deserializerFactory = std::make_unique( schema, std::move(codec), veloxCompressionType, diff --git a/cpp/velox/shuffle/VeloxShuffleReader.cc b/cpp/velox/shuffle/VeloxShuffleReader.cc index 0407be736a70..3aba7cf0fc3c 100644 --- a/cpp/velox/shuffle/VeloxShuffleReader.cc +++ b/cpp/velox/shuffle/VeloxShuffleReader.cc @@ -15,13 +15,13 @@ * limitations under the License. */ -#include "VeloxShuffleReader.h" -#include "GlutenByteStream.h" +#include "shuffle/VeloxShuffleReader.h" #include #include #include "memory/VeloxColumnarBatch.h" +#include "shuffle/GlutenByteStream.h" #include "shuffle/Payload.h" #include "shuffle/Utils.h" #include "utils/Common.h" @@ -576,7 +576,7 @@ std::shared_ptr VeloxRssSortShuffleReaderDeserializer::next() { return std::make_shared(std::move(rowVector)); } -VeloxColumnarBatchDeserializerFactory::VeloxColumnarBatchDeserializerFactory( +VeloxShuffleReaderDeserializerFactory::VeloxShuffleReaderDeserializerFactory( const std::shared_ptr& schema, const std::shared_ptr& codec, const facebook::velox::common::CompressionKind veloxCompressionType, @@ -598,7 +598,7 @@ VeloxColumnarBatchDeserializerFactory::VeloxColumnarBatchDeserializerFactory( initFromSchema(); } -std::unique_ptr VeloxColumnarBatchDeserializerFactory::createDeserializer( +std::unique_ptr VeloxShuffleReaderDeserializerFactory::createDeserializer( std::shared_ptr in) { switch (shuffleWriterType_) { case ShuffleWriterType::kHashShuffle: @@ -635,23 +635,19 @@ std::unique_ptr VeloxColumnarBatchDeserializerFactory::cr } } -arrow::MemoryPool* VeloxColumnarBatchDeserializerFactory::getPool() { +arrow::MemoryPool* VeloxShuffleReaderDeserializerFactory::getPool() { return memoryPool_; } -ShuffleWriterType VeloxColumnarBatchDeserializerFactory::getShuffleWriterType() { - return shuffleWriterType_; -} - -int64_t VeloxColumnarBatchDeserializerFactory::getDecompressTime() { +int64_t VeloxShuffleReaderDeserializerFactory::getDecompressTime() { return decompressTime_; } -int64_t VeloxColumnarBatchDeserializerFactory::getDeserializeTime() { +int64_t VeloxShuffleReaderDeserializerFactory::getDeserializeTime() { return deserializeTime_; } -void VeloxColumnarBatchDeserializerFactory::initFromSchema() { +void VeloxShuffleReaderDeserializerFactory::initFromSchema() { GLUTEN_ASSIGN_OR_THROW(auto arrowColumnTypes, toShuffleTypeId(schema_->fields())); isValidityBuffer_.reserve(arrowColumnTypes.size()); for (size_t i = 0; i < arrowColumnTypes.size(); ++i) { @@ -681,7 +677,23 @@ void VeloxColumnarBatchDeserializerFactory::initFromSchema() { } } -VeloxShuffleReader::VeloxShuffleReader(std::unique_ptr factory) - : ShuffleReader(std::move(factory)) {} +VeloxShuffleReader::VeloxShuffleReader(std::unique_ptr factory) + : factory_(std::move(factory)) {} + +std::shared_ptr VeloxShuffleReader::readStream(std::shared_ptr in) { + return std::make_shared(factory_->createDeserializer(in)); +} + +arrow::MemoryPool* VeloxShuffleReader::getPool() const { + return factory_->getPool(); +} + +int64_t VeloxShuffleReader::getDecompressTime() const { + return factory_->getDecompressTime(); +} + +int64_t VeloxShuffleReader::getDeserializeTime() const { + return factory_->getDeserializeTime(); +} } // namespace gluten diff --git a/cpp/velox/shuffle/VeloxShuffleReader.h b/cpp/velox/shuffle/VeloxShuffleReader.h index af35f977127f..8ebdbf2bacab 100644 --- a/cpp/velox/shuffle/VeloxShuffleReader.h +++ b/cpp/velox/shuffle/VeloxShuffleReader.h @@ -17,16 +17,14 @@ #pragma once -#include "operators/serializer/VeloxColumnarBatchSerializer.h" #include "shuffle/Payload.h" #include "shuffle/ShuffleReader.h" #include "shuffle/VeloxSortShuffleWriter.h" -#include "utils/Timer.h" + +#include "velox/serializers/PrestoSerializer.h" #include "velox/type/Type.h" #include "velox/vector/ComplexVector.h" -#include - namespace gluten { class VeloxHashShuffleReaderDeserializer final : public ColumnarBatchIterator { @@ -134,9 +132,9 @@ class VeloxRssSortShuffleReaderDeserializer : public ColumnarBatchIterator { std::shared_ptr in_; }; -class VeloxColumnarBatchDeserializerFactory : public DeserializerFactory { +class VeloxShuffleReaderDeserializerFactory { public: - VeloxColumnarBatchDeserializerFactory( + VeloxShuffleReaderDeserializerFactory( const std::shared_ptr& schema, const std::shared_ptr& codec, const facebook::velox::common::CompressionKind veloxCompressionType, @@ -147,15 +145,13 @@ class VeloxColumnarBatchDeserializerFactory : public DeserializerFactory { std::shared_ptr veloxPool, ShuffleWriterType shuffleWriterType); - std::unique_ptr createDeserializer(std::shared_ptr in) override; - - arrow::MemoryPool* getPool() override; + std::unique_ptr createDeserializer(std::shared_ptr in); - int64_t getDecompressTime() override; + arrow::MemoryPool* getPool(); - int64_t getDeserializeTime() override; + int64_t getDecompressTime(); - ShuffleWriterType getShuffleWriterType() override; + int64_t getDeserializeTime(); private: void initFromSchema(); @@ -180,6 +176,17 @@ class VeloxColumnarBatchDeserializerFactory : public DeserializerFactory { class VeloxShuffleReader final : public ShuffleReader { public: - VeloxShuffleReader(std::unique_ptr factory); + VeloxShuffleReader(std::unique_ptr factory); + + std::shared_ptr readStream(std::shared_ptr in) override; + + int64_t getDecompressTime() const override; + + int64_t getDeserializeTime() const override; + + arrow::MemoryPool* getPool() const override; + + private: + std::unique_ptr factory_; }; } // namespace gluten diff --git a/cpp/velox/utils/tests/VeloxShuffleWriterTestBase.h b/cpp/velox/utils/tests/VeloxShuffleWriterTestBase.h index 9331a7207858..4fcab8f24271 100644 --- a/cpp/velox/utils/tests/VeloxShuffleWriterTestBase.h +++ b/cpp/velox/utils/tests/VeloxShuffleWriterTestBase.h @@ -360,7 +360,7 @@ class VeloxShuffleWriterTest : public ::testing::TestWithParam( + auto deserializerFactory = std::make_unique( schema, std::move(codec), veloxCompressionType, From ab5a691ededcf770c8e18766045eda88d02ce255 Mon Sep 17 00:00:00 2001 From: Wei-Ting Chen Date: Fri, 10 Jan 2025 10:49:17 +0800 Subject: [PATCH 09/28] [CORE] Bump version to 1.4.0-SNAPSHOT (#8452) --- backends-clickhouse/pom.xml | 2 +- backends-velox/pom.xml | 2 +- dev/info.sh | 2 +- gluten-arrow/pom.xml | 2 +- gluten-celeborn/pom.xml | 2 +- gluten-core/pom.xml | 2 +- gluten-delta/pom.xml | 2 +- gluten-hudi/pom.xml | 2 +- gluten-iceberg/pom.xml | 2 +- gluten-ras/common/pom.xml | 2 +- gluten-ras/planner/pom.xml | 2 +- gluten-ras/pom.xml | 2 +- gluten-substrait/pom.xml | 2 +- gluten-ui/pom.xml | 2 +- gluten-uniffle/pom.xml | 2 +- gluten-ut/common/pom.xml | 2 +- gluten-ut/pom.xml | 2 +- gluten-ut/spark32/pom.xml | 2 +- gluten-ut/spark33/pom.xml | 2 +- gluten-ut/spark34/pom.xml | 2 +- gluten-ut/spark35/pom.xml | 2 +- gluten-ut/test/pom.xml | 2 +- package/pom.xml | 2 +- pom.xml | 2 +- shims/common/pom.xml | 2 +- shims/pom.xml | 2 +- shims/spark32/pom.xml | 2 +- shims/spark33/pom.xml | 2 +- shims/spark34/pom.xml | 2 +- shims/spark35/pom.xml | 2 +- tools/gluten-it/common/pom.xml | 4 ++-- tools/gluten-it/package/pom.xml | 2 +- tools/gluten-it/pom.xml | 4 ++-- tools/workload/benchmark_velox/params.yaml.template | 2 +- tools/workload/benchmark_velox/sample/tpch_q1.html | 2 +- tools/workload/benchmark_velox/sample/tpch_q1.ipynb | 6 +++--- tools/workload/benchmark_velox/tpc_workload.ipynb | 2 +- tools/workload/tpcds/run_tpcds/run_tpcds.sh | 2 +- 38 files changed, 42 insertions(+), 42 deletions(-) diff --git a/backends-clickhouse/pom.xml b/backends-clickhouse/pom.xml index e34f571961d6..fa6863b8774e 100644 --- a/backends-clickhouse/pom.xml +++ b/backends-clickhouse/pom.xml @@ -5,7 +5,7 @@ gluten-parent org.apache.gluten - 1.3.0-SNAPSHOT + 1.4.0-SNAPSHOT 4.0.0 diff --git a/backends-velox/pom.xml b/backends-velox/pom.xml index 240b2218641c..5c96fa6246b3 100755 --- a/backends-velox/pom.xml +++ b/backends-velox/pom.xml @@ -5,7 +5,7 @@ gluten-parent org.apache.gluten - 1.3.0-SNAPSHOT + 1.4.0-SNAPSHOT 4.0.0 diff --git a/dev/info.sh b/dev/info.sh index 1ced70db3e7c..fe259ea4389e 100755 --- a/dev/info.sh +++ b/dev/info.sh @@ -14,7 +14,7 @@ # limitations under the License. set -e -version='1.3.0-SNAPSHOT' +version='1.4.0-SNAPSHOT' cb='```' if [ ! -x "$(command -v cmake)" ]; then diff --git a/gluten-arrow/pom.xml b/gluten-arrow/pom.xml index ffba2682e9dc..8aa3cd9d489c 100644 --- a/gluten-arrow/pom.xml +++ b/gluten-arrow/pom.xml @@ -5,7 +5,7 @@ gluten-parent org.apache.gluten - 1.3.0-SNAPSHOT + 1.4.0-SNAPSHOT ../pom.xml 4.0.0 diff --git a/gluten-celeborn/pom.xml b/gluten-celeborn/pom.xml index 0eca5da979e1..6b8d5ea2304a 100755 --- a/gluten-celeborn/pom.xml +++ b/gluten-celeborn/pom.xml @@ -5,7 +5,7 @@ gluten-parent org.apache.gluten - 1.3.0-SNAPSHOT + 1.4.0-SNAPSHOT ../pom.xml 4.0.0 diff --git a/gluten-core/pom.xml b/gluten-core/pom.xml index 5e077a8b7db5..977977098ab2 100644 --- a/gluten-core/pom.xml +++ b/gluten-core/pom.xml @@ -4,7 +4,7 @@ gluten-parent org.apache.gluten - 1.3.0-SNAPSHOT + 1.4.0-SNAPSHOT 4.0.0 diff --git a/gluten-delta/pom.xml b/gluten-delta/pom.xml index cf2fdd7518fa..ef30a4f5631f 100755 --- a/gluten-delta/pom.xml +++ b/gluten-delta/pom.xml @@ -5,7 +5,7 @@ gluten-parent org.apache.gluten - 1.3.0-SNAPSHOT + 1.4.0-SNAPSHOT ../pom.xml 4.0.0 diff --git a/gluten-hudi/pom.xml b/gluten-hudi/pom.xml index 5865f1f6ece8..5d20f9e8d943 100755 --- a/gluten-hudi/pom.xml +++ b/gluten-hudi/pom.xml @@ -5,7 +5,7 @@ gluten-parent org.apache.gluten - 1.3.0-SNAPSHOT + 1.4.0-SNAPSHOT ../pom.xml 4.0.0 diff --git a/gluten-iceberg/pom.xml b/gluten-iceberg/pom.xml index c8c811674a54..b2bf5f9ebe07 100644 --- a/gluten-iceberg/pom.xml +++ b/gluten-iceberg/pom.xml @@ -5,7 +5,7 @@ gluten-parent org.apache.gluten - 1.3.0-SNAPSHOT + 1.4.0-SNAPSHOT ../pom.xml 4.0.0 diff --git a/gluten-ras/common/pom.xml b/gluten-ras/common/pom.xml index 6990a855508d..761a1c0b6153 100644 --- a/gluten-ras/common/pom.xml +++ b/gluten-ras/common/pom.xml @@ -5,7 +5,7 @@ org.apache.gluten gluten-ras - 1.3.0-SNAPSHOT + 1.4.0-SNAPSHOT gluten-ras-common Gluten Ras Common diff --git a/gluten-ras/planner/pom.xml b/gluten-ras/planner/pom.xml index 652977451a75..61c215b0125b 100644 --- a/gluten-ras/planner/pom.xml +++ b/gluten-ras/planner/pom.xml @@ -5,7 +5,7 @@ org.apache.gluten gluten-ras - 1.3.0-SNAPSHOT + 1.4.0-SNAPSHOT gluten-ras-planner Gluten Ras Planner diff --git a/gluten-ras/pom.xml b/gluten-ras/pom.xml index 570b772235f6..50494e1e9d83 100644 --- a/gluten-ras/pom.xml +++ b/gluten-ras/pom.xml @@ -17,7 +17,7 @@ org.apache.gluten gluten-parent - 1.3.0-SNAPSHOT + 1.4.0-SNAPSHOT gluten-ras pom diff --git a/gluten-substrait/pom.xml b/gluten-substrait/pom.xml index 8a790aa15337..7e73b4850cbc 100644 --- a/gluten-substrait/pom.xml +++ b/gluten-substrait/pom.xml @@ -5,7 +5,7 @@ gluten-parent org.apache.gluten - 1.3.0-SNAPSHOT + 1.4.0-SNAPSHOT 4.0.0 diff --git a/gluten-ui/pom.xml b/gluten-ui/pom.xml index a55d104a0b7a..2c92daff85da 100644 --- a/gluten-ui/pom.xml +++ b/gluten-ui/pom.xml @@ -6,7 +6,7 @@ org.apache.gluten gluten-parent - 1.3.0-SNAPSHOT + 1.4.0-SNAPSHOT gluten-ui diff --git a/gluten-uniffle/pom.xml b/gluten-uniffle/pom.xml index efc8ce6555c5..402f7a94d74f 100644 --- a/gluten-uniffle/pom.xml +++ b/gluten-uniffle/pom.xml @@ -5,7 +5,7 @@ gluten-parent org.apache.gluten - 1.3.0-SNAPSHOT + 1.4.0-SNAPSHOT ../pom.xml 4.0.0 diff --git a/gluten-ut/common/pom.xml b/gluten-ut/common/pom.xml index a61a8c47d147..11caaa4dd4ff 100644 --- a/gluten-ut/common/pom.xml +++ b/gluten-ut/common/pom.xml @@ -5,7 +5,7 @@ gluten-ut org.apache.gluten - 1.3.0-SNAPSHOT + 1.4.0-SNAPSHOT ../pom.xml 4.0.0 diff --git a/gluten-ut/pom.xml b/gluten-ut/pom.xml index 89d1ff9bc9ae..ca41d580aa46 100644 --- a/gluten-ut/pom.xml +++ b/gluten-ut/pom.xml @@ -20,7 +20,7 @@ org.apache.gluten gluten-parent - 1.3.0-SNAPSHOT + 1.4.0-SNAPSHOT ../pom.xml diff --git a/gluten-ut/spark32/pom.xml b/gluten-ut/spark32/pom.xml index 1ef95e0ea73d..8d0d0f4fcae2 100644 --- a/gluten-ut/spark32/pom.xml +++ b/gluten-ut/spark32/pom.xml @@ -5,7 +5,7 @@ gluten-ut org.apache.gluten - 1.3.0-SNAPSHOT + 1.4.0-SNAPSHOT ../pom.xml 4.0.0 diff --git a/gluten-ut/spark33/pom.xml b/gluten-ut/spark33/pom.xml index 539f60a63f1b..f8377195eec9 100644 --- a/gluten-ut/spark33/pom.xml +++ b/gluten-ut/spark33/pom.xml @@ -5,7 +5,7 @@ gluten-ut org.apache.gluten - 1.3.0-SNAPSHOT + 1.4.0-SNAPSHOT ../pom.xml 4.0.0 diff --git a/gluten-ut/spark34/pom.xml b/gluten-ut/spark34/pom.xml index a9a2f1c8a51e..e61142b79df8 100644 --- a/gluten-ut/spark34/pom.xml +++ b/gluten-ut/spark34/pom.xml @@ -5,7 +5,7 @@ gluten-ut org.apache.gluten - 1.3.0-SNAPSHOT + 1.4.0-SNAPSHOT ../pom.xml 4.0.0 diff --git a/gluten-ut/spark35/pom.xml b/gluten-ut/spark35/pom.xml index 1750a5e27816..d95e4aeeee05 100644 --- a/gluten-ut/spark35/pom.xml +++ b/gluten-ut/spark35/pom.xml @@ -5,7 +5,7 @@ gluten-ut org.apache.gluten - 1.3.0-SNAPSHOT + 1.4.0-SNAPSHOT ../pom.xml 4.0.0 diff --git a/gluten-ut/test/pom.xml b/gluten-ut/test/pom.xml index fb637d548937..a641a19b2faf 100644 --- a/gluten-ut/test/pom.xml +++ b/gluten-ut/test/pom.xml @@ -5,7 +5,7 @@ gluten-ut org.apache.gluten - 1.3.0-SNAPSHOT + 1.4.0-SNAPSHOT ../pom.xml 4.0.0 diff --git a/package/pom.xml b/package/pom.xml index b9c114181bcd..a1df1edc4eb8 100644 --- a/package/pom.xml +++ b/package/pom.xml @@ -20,7 +20,7 @@ org.apache.gluten gluten-parent - 1.3.0-SNAPSHOT + 1.4.0-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index 9cdc3f2c492d..8fa2f355030b 100644 --- a/pom.xml +++ b/pom.xml @@ -16,7 +16,7 @@ 4.0.0 org.apache.gluten gluten-parent - 1.3.0-SNAPSHOT + 1.4.0-SNAPSHOT pom Gluten Parent Pom diff --git a/shims/common/pom.xml b/shims/common/pom.xml index 92cee60b07bd..f09f196b4011 100644 --- a/shims/common/pom.xml +++ b/shims/common/pom.xml @@ -20,7 +20,7 @@ org.apache.gluten spark-sql-columnar-shims - 1.3.0-SNAPSHOT + 1.4.0-SNAPSHOT ../pom.xml diff --git a/shims/pom.xml b/shims/pom.xml index 4727cf474dcb..9ca5048cd17f 100644 --- a/shims/pom.xml +++ b/shims/pom.xml @@ -20,7 +20,7 @@ org.apache.gluten gluten-parent - 1.3.0-SNAPSHOT + 1.4.0-SNAPSHOT ../pom.xml diff --git a/shims/spark32/pom.xml b/shims/spark32/pom.xml index 9c6322ac5891..fee18fdc60e2 100644 --- a/shims/spark32/pom.xml +++ b/shims/spark32/pom.xml @@ -20,7 +20,7 @@ org.apache.gluten spark-sql-columnar-shims - 1.3.0-SNAPSHOT + 1.4.0-SNAPSHOT ../pom.xml diff --git a/shims/spark33/pom.xml b/shims/spark33/pom.xml index edab2c8705a1..1e1e1c537e76 100644 --- a/shims/spark33/pom.xml +++ b/shims/spark33/pom.xml @@ -20,7 +20,7 @@ org.apache.gluten spark-sql-columnar-shims - 1.3.0-SNAPSHOT + 1.4.0-SNAPSHOT ../pom.xml diff --git a/shims/spark34/pom.xml b/shims/spark34/pom.xml index dd4b8d156f1b..825ec2015b11 100644 --- a/shims/spark34/pom.xml +++ b/shims/spark34/pom.xml @@ -20,7 +20,7 @@ org.apache.gluten spark-sql-columnar-shims - 1.3.0-SNAPSHOT + 1.4.0-SNAPSHOT ../pom.xml diff --git a/shims/spark35/pom.xml b/shims/spark35/pom.xml index ed1b59991063..47af80194f84 100644 --- a/shims/spark35/pom.xml +++ b/shims/spark35/pom.xml @@ -20,7 +20,7 @@ org.apache.gluten spark-sql-columnar-shims - 1.3.0-SNAPSHOT + 1.4.0-SNAPSHOT ../pom.xml diff --git a/tools/gluten-it/common/pom.xml b/tools/gluten-it/common/pom.xml index e6ec5ffe1992..0e97be740b28 100644 --- a/tools/gluten-it/common/pom.xml +++ b/tools/gluten-it/common/pom.xml @@ -7,11 +7,11 @@ org.apache.gluten gluten-it - 1.3.0-SNAPSHOT + 1.4.0-SNAPSHOT gluten-it-common - 1.3.0-SNAPSHOT + 1.4.0-SNAPSHOT jar diff --git a/tools/gluten-it/package/pom.xml b/tools/gluten-it/package/pom.xml index 6a11b987f12c..9b42b4edf5ba 100644 --- a/tools/gluten-it/package/pom.xml +++ b/tools/gluten-it/package/pom.xml @@ -5,7 +5,7 @@ org.apache.gluten gluten-it - 1.3.0-SNAPSHOT + 1.4.0-SNAPSHOT gluten-it-package http://maven.apache.org diff --git a/tools/gluten-it/pom.xml b/tools/gluten-it/pom.xml index 488ff1c54cd1..89738668039a 100644 --- a/tools/gluten-it/pom.xml +++ b/tools/gluten-it/pom.xml @@ -6,7 +6,7 @@ org.apache.gluten gluten-it - 1.3.0-SNAPSHOT + 1.4.0-SNAPSHOT pom common @@ -23,7 +23,7 @@ 3 0.3.2-incubating 0.9.1 - 1.3.0-SNAPSHOT + 1.4.0-SNAPSHOT 32.0.1-jre 1.1 1.4 diff --git a/tools/workload/benchmark_velox/params.yaml.template b/tools/workload/benchmark_velox/params.yaml.template index 285afb11cb7c..e206cc3b8ccf 100644 --- a/tools/workload/benchmark_velox/params.yaml.template +++ b/tools/workload/benchmark_velox/params.yaml.template @@ -2,7 +2,7 @@ gluten_home: /home/sparkuser/gluten # Local path to gluten jar. -gluten_target_jar: /home/sparkuser/gluten-velox-bundle-spark3.3_2.12-centos_7_x86_64-1.3.0-SNAPSHOT.jar +gluten_target_jar: /home/sparkuser/gluten-velox-bundle-spark3.3_2.12-centos_7_x86_64-1.4.0-SNAPSHOT.jar # Spark app master. master: yarn diff --git a/tools/workload/benchmark_velox/sample/tpch_q1.html b/tools/workload/benchmark_velox/sample/tpch_q1.html index c401c35ec87c..7782ca198288 100644 --- a/tools/workload/benchmark_velox/sample/tpch_q1.html +++ b/tools/workload/benchmark_velox/sample/tpch_q1.html @@ -16208,7 +16208,7 @@

Config compare\n", " \n", " spark.executor.extraClassPath\n", - " file:///data0/home/sparkuser/jars/6600a164407ae0e4f5ea5b33dc4b902f23a27730/gluten-velox-bundle-spark3.3_2.12-centos_7_x86_64-1.3.0-snapshot.jar\n", + " file:///data0/home/sparkuser/jars/6600a164407ae0e4f5ea5b33dc4b902f23a27730/gluten-velox-bundle-spark3.3_2.12-centos_7_x86_64-1.4.0-snapshot.jar\n", " \n", " False\n", " \n", @@ -4673,7 +4673,7 @@ " 0851_0048 \\\n", "callSite.short collect at /tmp/ipykernel_265482/1936321720.py:117 \n", "spark.app.submitTime 1733464301669 \n", - "spark.executor.extraClassPath file:///data0/home/sparkuser/jars/6600a164407ae0e4f5ea5b33dc4b902f23a27730/gluten-velox-bundle-spark3.3_2.12-centos_7_x86_64-1.3.0-snapshot.jar \n", + "spark.executor.extraClassPath file:///data0/home/sparkuser/jars/6600a164407ae0e4f5ea5b33dc4b902f23a27730/gluten-velox-bundle-spark3.3_2.12-centos_7_x86_64-1.4.0-snapshot.jar \n", "spark.executor.extraJavaOptions -xx:+ignoreunrecognizedvmoptions --add-opens=java.base/java.lang=all-unnamed --add-opens=java.base/java.lang.invoke=all-unnamed --add-opens=java.base/java.lang.reflect=all-unnamed --add-opens=java.base/java.io=all-unnamed --add-opens=java.base/java.net=all-unnamed --add-opens=java.base/java.nio=all-unnamed --add-opens=java.base/java.util=all-unnamed --add-opens=java.base/java.util.concurrent=all-unnamed --add-opens=java.base/java.util.concurrent.atomic=all-unnamed --add-opens=java.base/sun.nio.ch=all-unnamed --add-opens=java.base/sun.nio.cs=all-unnamed --add-opens=java.base/sun.security.action=all-unnamed --add-opens=java.base/sun.util.calendar=all-unnamed --add-opens=java.security.jgss/sun.security.krb5=all-unnamed -xx:+useparalleloldgc -xx:parallelgcthreads=2 -xx:newratio=1 -xx:survivorratio=1 -xx:+usecompressedoops -verbose:gc -xx:+printgcdetails -xx:+printgctimestamps -xx:errorfile=/home/sparkuser/logs/java/hs_err_pid%p.log \n", "spark.executor.memory 10944m \n", "spark.gluten.memory.conservative.task.offHeap.size.in.bytes 10041163776 \n", @@ -4981,4 +4981,4 @@ }, "nbformat": 4, "nbformat_minor": 5 -} \ No newline at end of file +} diff --git a/tools/workload/benchmark_velox/tpc_workload.ipynb b/tools/workload/benchmark_velox/tpc_workload.ipynb index 3f6a24e9b1c8..601deb8eb117 100644 --- a/tools/workload/benchmark_velox/tpc_workload.ipynb +++ b/tools/workload/benchmark_velox/tpc_workload.ipynb @@ -21,7 +21,7 @@ "gluten_home='/home/sparkuser/gluten'\n", "\n", "# Local path to gluten jar.\n", - "gluten_target_jar='/home/sparkuser/gluten-velox-bundle-spark3.3_2.12-centos_7_x86_64-1.3.0-SNAPSHOT.jar'\n", + "gluten_target_jar='/home/sparkuser/gluten-velox-bundle-spark3.3_2.12-centos_7_x86_64-1.4.0-SNAPSHOT.jar'\n", "\n", "# Spark app master. e.g. 'yarn'\n", "master='yarn'\n", diff --git a/tools/workload/tpcds/run_tpcds/run_tpcds.sh b/tools/workload/tpcds/run_tpcds/run_tpcds.sh index 813184307eee..cc7aec34f210 100755 --- a/tools/workload/tpcds/run_tpcds/run_tpcds.sh +++ b/tools/workload/tpcds/run_tpcds/run_tpcds.sh @@ -37,4 +37,4 @@ cat tpcds_parquet.scala | ${SPARK_HOME}/bin/spark-shell \ # e.g. # --conf spark.gluten.loadLibFromJar=true \ # --jars /PATH_TO_GLUTEN_HOME/package/target/thirdparty-lib/gluten-thirdparty-lib-ubuntu-22.04-x86_64.jar, - # /PATH_TO_GLUTEN_HOME/package/target/gluten-velox-bundle-spark3.3_2.12-ubuntu_22.04_x86_64-1.3.0-SNAPSHOT.jar + # /PATH_TO_GLUTEN_HOME/package/target/gluten-velox-bundle-spark3.3_2.12-ubuntu_22.04_x86_64-1.x.x-SNAPSHOT.jar From bdc0fbf1be1b83f90db403675a9b0d4e0203a709 Mon Sep 17 00:00:00 2001 From: Hongze Zhang Date: Fri, 10 Jan 2025 13:52:14 +0800 Subject: [PATCH 10/28] [GLUTEN-8483][CORE] A stable and universal way to find component files (#8486) Closes #8483 --- .../org/apache/gluten/utils/ResourceUtil.java | 100 +++++++++++------- .../apache/gluten/component/Discovery.scala | 10 +- .../apache/gluten/util/ResourceUtilTest.java | 46 ++++++++ 3 files changed, 111 insertions(+), 45 deletions(-) create mode 100644 gluten-core/src/test/java/org/apache/gluten/util/ResourceUtilTest.java diff --git a/gluten-core/src/main/java/org/apache/gluten/utils/ResourceUtil.java b/gluten-core/src/main/java/org/apache/gluten/utils/ResourceUtil.java index ac66c57a8c3a..89f48d4984f6 100644 --- a/gluten-core/src/main/java/org/apache/gluten/utils/ResourceUtil.java +++ b/gluten-core/src/main/java/org/apache/gluten/utils/ResourceUtil.java @@ -16,16 +16,20 @@ */ package org.apache.gluten.utils; +import org.apache.gluten.exception.GlutenException; + +import com.google.common.base.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.File; import java.io.IOException; +import java.net.URL; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; import java.util.Enumeration; import java.util.List; +import java.util.regex.Matcher; import java.util.regex.Pattern; import java.util.zip.ZipEntry; import java.util.zip.ZipException; @@ -37,58 +41,73 @@ * and then modified for Gluten's use. */ public class ResourceUtil { - private static final Logger LOG = LoggerFactory.getLogger(ResourceUtil.class); /** - * Get a collection of resource paths by the input RegEx pattern. + * Get a collection of resource paths by the input RegEx pattern in a certain container folder. * - * @param pattern The pattern to match. + * @param container The container folder. E.g., `META-INF`. Should not be left empty, because + * Classloader requires for at a meaningful file name to search inside the loaded jar files. + * @param pattern The pattern to match on the file names. * @return The relative resource paths in the order they are found. */ - public static List getResources(final Pattern pattern) { + public static List getResources(final String container, final Pattern pattern) { + Preconditions.checkArgument( + !container.isEmpty(), + "Resource search should only be used under a certain container folder"); + Preconditions.checkArgument( + !container.startsWith("/") && !container.endsWith("/"), + "Resource container should not start or end with\"/\""); final List buffer = new ArrayList<>(); - String classPath = System.getProperty("java.class.path"); - processClassPathElements(classPath, pattern, buffer); - return Collections.unmodifiableList(buffer); - } - - private static void processClassPathElements( - String classPath, Pattern pattern, List buffer) { - if (classPath == null || classPath.isEmpty()) { - return; + final Enumeration containerUrls; + try { + containerUrls = Thread.currentThread().getContextClassLoader().getResources(container); + } catch (IOException e) { + throw new GlutenException(e); } - String[] classPathElements = classPath.split(File.pathSeparator); - Arrays.stream(classPathElements).forEach(element -> getResources(element, pattern, buffer)); - // the Gluten project may wrapped by the other service to use the Native Engine. - // As a result, the java.class.path points to xxx/other.jar instead of xxx/gluten.jar. - // This will result in the failure to properly load the required Components. - if (buffer.isEmpty()) { - classPath = ResourceUtil.class.getProtectionDomain().getCodeSource().getLocation().getPath(); - classPathElements = classPath.split(File.pathSeparator); - Arrays.stream(classPathElements).forEach(element -> getResources(element, pattern, buffer)); + while (containerUrls.hasMoreElements()) { + final URL containerUrl = containerUrls.nextElement(); + getResources(containerUrl, pattern, buffer); } + return Collections.unmodifiableList(buffer); } private static void getResources( - final String element, final Pattern pattern, final List buffer) { - final File file = new File(element); - if (!file.exists()) { - LOG.info("Skip non-existing classpath: {}", element); - return; - } - if (file.isDirectory()) { - getResourcesFromDirectory(file, file, pattern, buffer); - } else { - getResourcesFromJarFile(file, pattern, buffer); + final URL containerUrl, final Pattern pattern, final List buffer) { + final String protocol = containerUrl.getProtocol(); + switch (protocol) { + case "file": + final File fileContainer = new File(containerUrl.getPath()); + Preconditions.checkState( + fileContainer.exists() && fileContainer.isDirectory(), + "Specified file container " + containerUrl + " is not a directory or not a file"); + getResourcesFromDirectory(fileContainer, fileContainer, pattern, buffer); + break; + case "jar": + final String jarContainerPath = containerUrl.getPath(); + final Pattern jarContainerPattern = Pattern.compile("file:([^!]+)!/(.+)"); + final Matcher m = jarContainerPattern.matcher(jarContainerPath); + if (!m.matches()) { + throw new GlutenException("Illegal Jar container URL: " + containerUrl); + } + final String jarPath = m.group(1); + final File jarFile = new File(jarPath); + Preconditions.checkState( + jarFile.exists() && jarFile.isFile(), + "Specified Jar container " + containerUrl + " is not a Jar file"); + final String dir = m.group(2); + getResourcesFromJarFile(jarFile, dir, pattern, buffer); + break; + default: + throw new GlutenException("Unrecognizable resource protocol: " + protocol); } } private static void getResourcesFromJarFile( - final File file, final Pattern pattern, final List buffer) { - ZipFile zf; + final File jarFile, final String dir, final Pattern pattern, final List buffer) { + final ZipFile zf; try { - zf = new ZipFile(file); + zf = new ZipFile(jarFile); } catch (final ZipException e) { throw new RuntimeException(e); } catch (final IOException e) { @@ -98,9 +117,14 @@ private static void getResourcesFromJarFile( while (e.hasMoreElements()) { final ZipEntry ze = (ZipEntry) e.nextElement(); final String fileName = ze.getName(); - final boolean accept = pattern.matcher(fileName).matches(); + if (!fileName.startsWith(dir)) { + continue; + } + final String relativeFileName = + new File(dir).toURI().relativize(new File(fileName).toURI()).getPath(); + final boolean accept = pattern.matcher(relativeFileName).matches(); if (accept) { - buffer.add(fileName); + buffer.add(relativeFileName); } } try { diff --git a/gluten-core/src/main/scala/org/apache/gluten/component/Discovery.scala b/gluten-core/src/main/scala/org/apache/gluten/component/Discovery.scala index 2b8f060a69f7..ffa2ba9b7706 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/component/Discovery.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/component/Discovery.scala @@ -14,7 +14,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.gluten.component import org.apache.gluten.exception.GlutenException @@ -26,11 +25,8 @@ import org.apache.spark.util.SparkReflectionUtil import scala.collection.JavaConverters._ import scala.collection.mutable import scala.util.matching.Regex - - - - // format: off + /** * Gluten's global discovery to find all [[Component]] definitions in the classpath. * @@ -54,12 +50,12 @@ import scala.util.matching.Regex // format: on private object Discovery extends Logging { private val container: String = "META-INF/gluten-components" - private val componentFilePattern: Regex = s"^$container/(.+)$$".r + private val componentFilePattern: Regex = s"^(.+)$$".r def discoverAll(): Seq[Component] = { logInfo("Start discovering components in the current classpath... ") val prev = System.currentTimeMillis() - val allFiles = ResourceUtil.getResources(componentFilePattern.pattern).asScala + val allFiles = ResourceUtil.getResources(container, componentFilePattern.pattern).asScala val duration = System.currentTimeMillis() - prev logInfo(s"Discovered component files: ${allFiles.mkString(", ")}. Duration: $duration ms.") val deDup = mutable.Set[String]() diff --git a/gluten-core/src/test/java/org/apache/gluten/util/ResourceUtilTest.java b/gluten-core/src/test/java/org/apache/gluten/util/ResourceUtilTest.java new file mode 100644 index 000000000000..570e5a6e4d03 --- /dev/null +++ b/gluten-core/src/test/java/org/apache/gluten/util/ResourceUtilTest.java @@ -0,0 +1,46 @@ +/* + * 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.util; + +import org.apache.gluten.utils.ResourceUtil; + +import org.junit.Assert; +import org.junit.Test; + +import java.util.List; +import java.util.regex.Pattern; + +public class ResourceUtilTest { + @Test + public void testFile() { + // Use the class file of this test to verify the sanity of ResourceUtil. + List classes = + ResourceUtil.getResources( + "org", Pattern.compile("apache/gluten/util/ResourceUtilTest\\.class")); + Assert.assertEquals(1, classes.size()); + Assert.assertEquals("apache/gluten/util/ResourceUtilTest.class", classes.get(0)); + } + + @Test + public void testJar() { + // Use the class file of Spark code to verify the sanity of ResourceUtil. + List classes = + ResourceUtil.getResources("org", Pattern.compile("apache/spark/SparkContext\\.class")); + Assert.assertEquals(1, classes.size()); + Assert.assertEquals("apache/spark/SparkContext.class", classes.get(0)); + } +} From 6d71d27884dd6bda38848cb012a63fb37ee2d298 Mon Sep 17 00:00:00 2001 From: Rong Ma Date: Fri, 10 Jan 2025 15:05:38 +0800 Subject: [PATCH 11/28] [DOC][VL] Fix typo in microbenchmark.md (#8495) --- docs/developers/MicroBenchmarks.md | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/docs/developers/MicroBenchmarks.md b/docs/developers/MicroBenchmarks.md index c59d6be3a631..e274d5e39fc9 100644 --- a/docs/developers/MicroBenchmarks.md +++ b/docs/developers/MicroBenchmarks.md @@ -361,7 +361,11 @@ Developers can use `--debug-mode` command line flag to turn on debug mode when n ## Enable HDFS support -After enabling the dynamic loading of libhdfs.so at runtime to support HDFS, if you run the benchmark with an HDFS file, you need to set the classpath for Hadoop. You can do this by running `export CLASSPATH=`$HADOOP_HOME/bin/hdfs classpath --glob``. Otherwise, the HDFS connection will fail. If you have replaced ${HADOOP_HOME}/lib/native/libhdfs.so with libhdfs3.so, there is no need to set the `CLASSPATH`. +After enabling the dynamic loading of libhdfs.so at runtime to support HDFS, if you run the benchmark with an HDFS file, you need to set the classpath for Hadoop. You can do this by running +``` +export CLASSPATH=`$HADOOP_HOME/bin/hdfs classpath --glob` +``` +Otherwise, the HDFS connection will fail. If you have replaced ${HADOOP_HOME}/lib/native/libhdfs.so with libhdfs3.so, there is no need to set the `CLASSPATH`. ## Simulate write tasks From 6c60fd3af69b444c4a8ec2c02e2894b6c567570a Mon Sep 17 00:00:00 2001 From: Kyligence Git Date: Fri, 10 Jan 2025 08:50:55 -0600 Subject: [PATCH 12/28] [GLUTEN-1632][CH]Daily Update Clickhouse Version (20250110) (#8490) * [GLUTEN-1632][CH]Daily Update Clickhouse Version (20250110) * fix build due to https://github.com/ClickHouse/ClickHouse/pull/74271 --------- Co-authored-by: kyligence-git Co-authored-by: Chang Chen --- cpp-ch/clickhouse.version | 4 +-- .../MergeTree/SparkStorageMergeTree.cpp | 28 +++++++++---------- 2 files changed, 15 insertions(+), 17 deletions(-) diff --git a/cpp-ch/clickhouse.version b/cpp-ch/clickhouse.version index 3e28ad21e9ad..79ca4eb69bb1 100644 --- a/cpp-ch/clickhouse.version +++ b/cpp-ch/clickhouse.version @@ -1,3 +1,3 @@ CH_ORG=Kyligence -CH_BRANCH=rebase_ch/20250107 -CH_COMMIT=01d2a08fb01 +CH_BRANCH=rebase_ch/20250110 +CH_COMMIT=eafc5ef70b3 diff --git a/cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.cpp b/cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.cpp index b4e0bb9ee8a3..3cc9cb69db93 100644 --- a/cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.cpp +++ b/cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.cpp @@ -469,24 +469,22 @@ MergeTreeDataWriter::TemporaryPart SparkMergeTreeDataWriter::writeTempPart( new_data_part->uuid = UUIDHelpers::generateV4(); SyncGuardPtr sync_guard; - if (new_data_part->isStoredOnDisk()) - { - /// The name could be non-unique in case of stale files from previous runs. - String full_path = new_data_part->getDataPartStorage().getFullPath(); - if (new_data_part->getDataPartStorage().exists()) - { - // LOG_WARNING(log, "Removing old temporary directory {}", full_path); - data_part_storage->removeRecursive(); - } + /// The name could be non-unique in case of stale files from previous runs. + String full_path = new_data_part->getDataPartStorage().getFullPath(); - data_part_storage->createDirectories(); + if (new_data_part->getDataPartStorage().exists()) + { + LOG_WARNING(log, "Removing old temporary directory {}", full_path); + data_part_storage->removeRecursive(); + } - if ((*data.getSettings())[MergeTreeSetting::fsync_part_directory]) - { - const auto disk = data_part_volume->getDisk(); - sync_guard = disk->getDirectorySyncGuard(full_path); - } + data_part_storage->createDirectories(); + + if ((*data.getSettings())[MergeTreeSetting::fsync_part_directory]) + { + const auto disk = data_part_volume->getDisk(); + sync_guard = disk->getDirectorySyncGuard(full_path); } /// This effectively chooses minimal compression method: From cd327f950799b6e104dfb3ee0df36bf72d6fd9ba Mon Sep 17 00:00:00 2001 From: Gluten Performance Bot <137994563+GlutenPerfBot@users.noreply.github.com> Date: Sat, 11 Jan 2025 02:55:14 +0800 Subject: [PATCH 13/28] [GLUTEN-6887][VL] Daily Update Velox Version (2025_01_10) (#8489) 02b0f0389 by Deepak Majeti, build(cmake): Remove unused targets (12045) ff943fd49 by aditi-pandit, refactor: Change C style casts to C++ style (Part 6) (11728) --- ep/build-velox/src/get_velox.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh index 676c062c1bf4..9c4bc68793c1 100755 --- a/ep/build-velox/src/get_velox.sh +++ b/ep/build-velox/src/get_velox.sh @@ -17,7 +17,7 @@ set -exu VELOX_REPO=https://github.com/oap-project/velox.git -VELOX_BRANCH=2025_01_09 +VELOX_BRANCH=2025_01_10 VELOX_HOME="" OS=`uname -s` From 4bdda17e4fe1bbed1dc7994fb080fd983a338204 Mon Sep 17 00:00:00 2001 From: jkhaliqi <60749291+jkhaliqi@users.noreply.github.com> Date: Fri, 10 Jan 2025 11:31:24 -0800 Subject: [PATCH 14/28] [GLUTEN-8476][VL] Fix allocate and free memory (#8477) fix security volations --- cpp/core/benchmarks/CompressionBenchmark.cc | 4 +++- cpp/core/memory/MemoryAllocator.cc | 2 ++ 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/cpp/core/benchmarks/CompressionBenchmark.cc b/cpp/core/benchmarks/CompressionBenchmark.cc index e1aa69ff39a7..da8c71c044ba 100644 --- a/cpp/core/benchmarks/CompressionBenchmark.cc +++ b/cpp/core/benchmarks/CompressionBenchmark.cc @@ -48,7 +48,9 @@ void printTrace(void) { for (i = 0; i < size; i++) printf(" %s\n", strings[i]); puts(""); - free(strings); + if (strings != nullptr) { + free(strings); + } } using arrow::RecordBatchReader; diff --git a/cpp/core/memory/MemoryAllocator.cc b/cpp/core/memory/MemoryAllocator.cc index dd5fb8e1974b..84708962d01c 100644 --- a/cpp/core/memory/MemoryAllocator.cc +++ b/cpp/core/memory/MemoryAllocator.cc @@ -157,6 +157,7 @@ bool StdMemoryAllocator::reallocate(void* p, int64_t size, int64_t newSize, void } bool StdMemoryAllocator::reallocateAligned(void* p, uint64_t alignment, int64_t size, int64_t newSize, void** out) { + GLUTEN_CHECK(p != nullptr, "reallocate with nullptr"); if (newSize <= 0) { return false; } @@ -179,6 +180,7 @@ bool StdMemoryAllocator::reallocateAligned(void* p, uint64_t alignment, int64_t } bool StdMemoryAllocator::free(void* p, int64_t size) { + GLUTEN_CHECK(p != nullptr, "free with nullptr"); std::free(p); bytes_ -= size; return true; From c5aecbda9dc2a8da572dcfcdd0d1ad641fe07d52 Mon Sep 17 00:00:00 2001 From: jkhaliqi <60749291+jkhaliqi@users.noreply.github.com> Date: Fri, 10 Jan 2025 15:33:37 -0800 Subject: [PATCH 15/28] [GLUTEN-8503][VL] Fix macro parenthesis CVE (#8504) Macro replacement lists should be parenthesized --- cpp/core/benchmarks/CompressionBenchmark.cc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cpp/core/benchmarks/CompressionBenchmark.cc b/cpp/core/benchmarks/CompressionBenchmark.cc index da8c71c044ba..e46556214529 100644 --- a/cpp/core/benchmarks/CompressionBenchmark.cc +++ b/cpp/core/benchmarks/CompressionBenchmark.cc @@ -60,7 +60,7 @@ using gluten::ShuffleWriterOptions; namespace gluten { -#define ALIGNMENT 2 * 1024 * 1024 +#define ALIGNMENT (2 * 1024 * 1024) const int32_t kQatGzip = 0; const int32_t kQatZstd = 1; From a7301e199dae989f85a60336d5b459dbce98cd4b Mon Sep 17 00:00:00 2001 From: jkhaliqi <60749291+jkhaliqi@users.noreply.github.com> Date: Fri, 10 Jan 2025 17:11:57 -0800 Subject: [PATCH 16/28] [GLUTEN-8471][VL] Fix usage of uninitialized variables (#8470) fix security volation --- cpp/velox/memory/VeloxMemoryManager.cc | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/cpp/velox/memory/VeloxMemoryManager.cc b/cpp/velox/memory/VeloxMemoryManager.cc index 597ce1c9da37..46c5186f5b20 100644 --- a/cpp/velox/memory/VeloxMemoryManager.cc +++ b/cpp/velox/memory/VeloxMemoryManager.cc @@ -126,7 +126,7 @@ class ListenableArbitrator : public velox::memory::MemoryArbitrator { uint64_t shrinkCapacity(uint64_t targetBytes, bool allowSpill, bool allowAbort) override { velox::memory::ScopedMemoryArbitrationContext ctx{}; facebook::velox::exec::MemoryReclaimer::Stats status; - velox::memory::MemoryPool* pool; + velox::memory::MemoryPool* pool = nullptr; { std::unique_lock guard{mutex_}; VELOX_CHECK_EQ(candidates_.size(), 1, "ListenableArbitrator should only be used within a single root pool"); @@ -178,7 +178,7 @@ class ListenableArbitrator : public velox::memory::MemoryArbitrator { return freeBytes; } - gluten::AllocationListener* listener_; + gluten::AllocationListener* listener_ = nullptr; const uint64_t memoryPoolInitialCapacity_; // FIXME: Unused. const uint64_t memoryPoolTransferCapacity_; const uint64_t memoryReclaimMaxWaitMs_; From d70de0bb5203c32a438bf9ff2296409919770e8f Mon Sep 17 00:00:00 2001 From: Gluten Performance Bot <137994563+GlutenPerfBot@users.noreply.github.com> Date: Sun, 12 Jan 2025 09:30:32 +0800 Subject: [PATCH 17/28] [GLUTEN-6887][VL] Daily Update Velox Version (2025_01_11) (#8507) Upstream Velox's New Commits: 9dcfd3934 by Jimmy Lu, fix: Reuse vector in LocalPartition (12002) ec2740ff4 by PHILO-HE, feat: Register Spark date_format function (11953) 923dcc81f by Jia Ke, fix(hadoop): Remove the schema for hdfs path when reading file (11963) 315bf18bb by Xiao Du, Support for Sum Aggregation Signature with Interval (12044) a53bc0124 by Jimmy Lu, fix(wave): NVRTC remote execution (12051) b13e209b5 by Darren Fu, refactor(dwio): Rewrite partition file path unescape utility functions using folly::uriUnescape() (12036) c965e7c74 by Kevin Wilfong, feat: Add fast path for single bit in ByteOutputStream::appendBits (12047) --- ep/build-velox/src/get_velox.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh index 9c4bc68793c1..fa30e8c747b4 100755 --- a/ep/build-velox/src/get_velox.sh +++ b/ep/build-velox/src/get_velox.sh @@ -17,7 +17,7 @@ set -exu VELOX_REPO=https://github.com/oap-project/velox.git -VELOX_BRANCH=2025_01_10 +VELOX_BRANCH=2025_01_11 VELOX_HOME="" OS=`uname -s` From 318bb2177349802312b5e94f608b53df33db39b6 Mon Sep 17 00:00:00 2001 From: Gluten Performance Bot <137994563+GlutenPerfBot@users.noreply.github.com> Date: Mon, 13 Jan 2025 07:51:11 +0800 Subject: [PATCH 18/28] [GLUTEN-6887][VL] Daily Update Velox Version (2025_01_12) (#8508) Upstream Velox's New Commits: 022cd8757 by Bryan Cutler, feat: Add support for UUID comparison functions (10791) --- ep/build-velox/src/get_velox.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh index fa30e8c747b4..06ce5b96068c 100755 --- a/ep/build-velox/src/get_velox.sh +++ b/ep/build-velox/src/get_velox.sh @@ -17,7 +17,7 @@ set -exu VELOX_REPO=https://github.com/oap-project/velox.git -VELOX_BRANCH=2025_01_11 +VELOX_BRANCH=2025_01_12 VELOX_HOME="" OS=`uname -s` From ecda35bf5314c9d15515133309747ebe69f000e5 Mon Sep 17 00:00:00 2001 From: Hongze Zhang Date: Mon, 13 Jan 2025 10:02:44 +0800 Subject: [PATCH 19/28] [GLUTEN-8497][VL] A bad test case that fails columnar table cache query (#8498) --- .../execution/VeloxColumnarCacheSuite.scala | 43 +++++++++++++++++-- .../columnar/transition/TransitionGraph.scala | 1 + 2 files changed, 41 insertions(+), 3 deletions(-) diff --git a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxColumnarCacheSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxColumnarCacheSuite.scala index e9151ad84ab6..8c7be883bbbc 100644 --- a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxColumnarCacheSuite.scala +++ b/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxColumnarCacheSuite.scala @@ -24,8 +24,11 @@ import org.apache.spark.sql.Row import org.apache.spark.sql.execution.{ColumnarToRowExec, SparkPlan} import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec +import org.apache.spark.sql.types.{LongType, Metadata, MetadataBuilder, StructType} import org.apache.spark.storage.StorageLevel +import scala.collection.JavaConverters._ + class VeloxColumnarCacheSuite extends VeloxWholeStageTransformerSuite with AdaptiveSparkPlanHelper { override protected val resourcePath: String = "/tpch-data-parquet" override protected val fileFormat: String = "parquet" @@ -55,7 +58,7 @@ class VeloxColumnarCacheSuite extends VeloxWholeStageTransformerSuite with Adapt ) } - test("input columnar batch") { + test("Input columnar batch") { TPCHTables.map(_.name).foreach { table => runQueryAndCompare(s"SELECT * FROM $table", cache = true) { @@ -64,7 +67,7 @@ class VeloxColumnarCacheSuite extends VeloxWholeStageTransformerSuite with Adapt } } - test("input columnar batch and column pruning") { + test("Input columnar batch and column pruning") { val expected = sql("SELECT l_partkey FROM lineitem").collect() val cached = sql("SELECT * FROM lineitem").cache() try { @@ -85,7 +88,7 @@ class VeloxColumnarCacheSuite extends VeloxWholeStageTransformerSuite with Adapt } } - test("input vanilla Spark columnar batch") { + test("Input vanilla Spark columnar batch") { withSQLConf(GlutenConfig.COLUMNAR_FILESCAN_ENABLED.key -> "false") { val df = spark.table("lineitem") val expected = df.collect() @@ -98,6 +101,40 @@ class VeloxColumnarCacheSuite extends VeloxWholeStageTransformerSuite with Adapt } } + // TODO: Fix this case. See https://github.com/apache/incubator-gluten/issues/8497. + testWithSpecifiedSparkVersion("Input fallen back vanilla Spark columnar scan", Some("3.3")) { + def withId(id: Int): Metadata = + new MetadataBuilder().putLong("parquet.field.id", id).build() + + withTempDir { + dir => + val readSchema = + new StructType() + .add("l_orderkey_read", LongType, true, withId(1)) + val writeSchema = + new StructType() + .add("l_orderkey_write", LongType, true, withId(1)) + withSQLConf("spark.sql.parquet.fieldId.read.enabled" -> "true") { + // Write a table with metadata information that Gluten Velox backend doesn't support, + // to emulate the scenario that a Spark columnar scan is not offload-able so fallen back, + // then user tries to cache it. + spark + .createDataFrame( + spark.sql("select l_orderkey from lineitem").collect().toList.asJava, + writeSchema) + .write + .mode("overwrite") + .parquet(dir.getCanonicalPath) + val df = spark.read.schema(readSchema).parquet(dir.getCanonicalPath) + df.cache() + // FIXME: The following call will throw since ColumnarCachedBatchSerializer will be + // confused by the input vanilla Parquet scan when its #convertColumnarBatchToCachedBatch + // method is called. + assertThrows[Exception](df.collect()) + } + } + } + test("CachedColumnarBatch serialize and deserialize") { val df = spark.table("lineitem") val expected = df.collect() diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/TransitionGraph.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/TransitionGraph.scala index 2733ed9f4ff6..ef08a34d5615 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/TransitionGraph.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/TransitionGraph.scala @@ -77,6 +77,7 @@ object TransitionGraph { } } + // TODO: Consolidate transition graph's cost model with RAS cost model. private object TransitionCostModel extends FloydWarshallGraph.CostModel[Transition] { override def zero(): TransitionCost = TransitionCost(0, Nil) override def costOf(transition: Transition): TransitionCost = { From adf092d1c09342ba2b9e9afd7f9e82aecde954ba Mon Sep 17 00:00:00 2001 From: PHILO-HE Date: Mon, 13 Jan 2025 10:53:51 +0800 Subject: [PATCH 20/28] [DOC] Update README.md (#8444) --- README.md | 139 ++++++++++++++++++++---------------------------------- 1 file changed, 52 insertions(+), 87 deletions(-) diff --git a/README.md b/README.md index 17e0c380289e..74b51af20e00 100644 --- a/README.md +++ b/README.md @@ -6,46 +6,43 @@ *This project is still under active development now, and doesn't have a stable release. Welcome to evaluate it.* -# 1 Introduction - -## 1.1 Problem Statement - -Apache Spark is a stable, mature project that has been developed for many years. It is one of the best frameworks to scale out for processing petabyte-scale datasets. However, the Spark community has had to address performance challenges that require various optimizations over time. As a key optimization in Spark 2.0, Whole Stage Code Generation is introduced to replace Volcano Model, which achieves 2x speedup. Henceforth, most optimizations are at query plan level. Single operator's performance almost stops growing. +# 1. Introduction +## Problem Statement +Apache Spark is a stable, mature project that has been developed for many years. It is one of the best frameworks to scale out for processing petabyte-scale datasets. However, the Spark community has had to address +performance challenges that require various optimizations over time. As a key optimization in Spark 2.0, Whole Stage Code Generation is introduced to replace Volcano Model, which achieves 2x speedup. Henceforth, most +optimizations are at query plan level. Single operator's performance almost stops growing.

-On the other side, SQL engines have been researched for many years. There are a few libraries like Clickhouse, Arrow and Velox, etc. By using features like native implementation, columnar data format and vectorized data processing, these libraries can outperform Spark's JVM based SQL engine. However, these libraries only support single node execution. - -## 1.2 Gluten's Solution - -“Gluten” is Latin for glue. The main goal of Gluten project is to “glue" native libraries with SparkSQL. Thus, we can benefit from high scalability of Spark SQL framework and high performance of native libraries. +On the other side, native SQL engines have been developed for a few years, such as Clickhouse, Arrow and Velox, etc. With features like native execution, columnar data format and vectorized +data processing, these native engines can outperform Spark's JVM based SQL engine. However, they only support single node execution. -The basic rule of Gluten's design is that we would reuse spark's whole control flow and as many JVM code as possible but offload the compute-intensive data processing part to native code. Here is what Gluten does: -* Transform Spark’s whole stage physical plan to Substrait plan and send to native -* Offload performance-critical data processing to native library -* Define clear JNI interfaces for native libraries -* Switch available native backends easily -* Reuse Spark’s distributed control flow -* Manage data sharing between JVM and native -* Extensible to support more native accelerators +## Gluten's Basic Design +“Gluten” is Latin for "glue". The main goal of Gluten project is to glue native engines with SparkSQL. Thus, we can benefit from high scalability of Spark SQL framework and high performance of native engines. -## 1.3 Target User +The basic design rule is that we would reuse Spark's whole control flow and as much JVM code as possible but offload the compute-intensive data processing to native side. Here is what Gluten does basically: +* Transform Spark’s physical plan to Substrait plan, then transform it to native engine's plan. +* Offload performance-critical data processing to native engine. +* Define clear JNI interfaces for native SQL engines. +* Switch available native backends easily. +* Reuse Spark’s distributed control flow. +* Manage data sharing between JVM and native. +* Extensible to support more native engines. -Gluten's target user is anyone who wants to accelerate SparkSQL fundamentally. As a plugin to Spark, Gluten doesn't require any change for dataframe API or SQL query, but only requires user to make correct configuration. +## Target User +Gluten's target user is anyone who aspires to accelerate SparkSQL fundamentally. As a plugin to Spark, Gluten doesn't require any change for dataframe API or SQL query, but only requires user to make correct configuration. See Gluten configuration properties [here](https://github.com/apache/incubator-gluten/blob/main/docs/Configuration.md). -## 1.4 References - +## References You can click below links for more related information. - [Gluten Intro Video at Data AI Summit 2022](https://www.youtube.com/watch?v=0Q6gHT_N-1U) - [Gluten Intro Article at Medium.com](https://medium.com/intel-analytics-software/accelerate-spark-sql-queries-with-gluten-9000b65d1b4e) - [Gluten Intro Article at Kyligence.io(in Chinese)](https://cn.kyligence.io/blog/gluten-spark/) - [Velox Intro from Meta](https://engineering.fb.com/2023/03/09/open-source/velox-open-source-execution-engine/) -# 2 Architecture - +# 2. Architecture The overview chart is like below. Substrait provides a well-defined cross-language specification for data compute operations (see more details [here](https://substrait.io/)). Spark physical plan is transformed to Substrait plan. Then Substrait plan is passed to native through JNI call. On native side, the native operator chain will be built out and offloaded to native engine. Gluten will return Columnar Batch to Spark and Spark Columnar API (since Spark-3.0) will be used at execution time. Gluten uses Apache Arrow data format as its basic data format, so the returned data to Spark JVM is ArrowColumnarBatch.

@@ -61,84 +58,60 @@ There are several key components in Gluten: * **Metrics**: collected from Gluten native engine to help identify bugs, performance bottlenecks, etc. The metrics are displayed in Spark UI. * **Shim Layer**: supports multiple Spark versions. We plan to only support Spark's latest 2 or 3 releases. Currently, Spark-3.2, Spark-3.3 & Spark-3.4 (experimental) are supported. -# 3 How to Use - -There are two ways to use Gluten. - -# 3.1 Use Released Jar - -One way is to use released jar. Here is a simple example. Currently, only centos7/8 and ubuntu20.04/22.04 are well supported. - -``` -spark-shell \ - --master yarn --deploy-mode client \ - --conf spark.plugins=org.apache.gluten.GlutenPlugin \ - --conf spark.memory.offHeap.enabled=true \ - --conf spark.memory.offHeap.size=20g \ - --conf spark.shuffle.manager=org.apache.spark.shuffle.sort.ColumnarShuffleManager \ - --jars https://github.com/apache/incubator-gluten/releases/download/v1.1.1/gluten-velox-bundle-spark3.2_2.12-ubuntu_20.04_x86_64-1.1.1.jar -``` - -# 3.2 Custom Build - -Alternatively, you can build gluten from source, then do some configurations to enable Gluten plugin for Spark. Here is a simple example. Please refer to the corresponding backend part below for more details. +# 3. User Guide +Here is a basic configuration to enable Gluten in Spark. ``` -export gluten_jar = /PATH/TO/GLUTEN/backends-velox/target/ +export GLUTEN_JAR=/PATH/TO/GLUTEN_JAR spark-shell \ --master yarn --deploy-mode client \ --conf spark.plugins=org.apache.gluten.GlutenPlugin \ --conf spark.memory.offHeap.enabled=true \ --conf spark.memory.offHeap.size=20g \ - --conf spark.driver.extraClassPath=${gluten_jar} \ - --conf spark.executor.extraClassPath=${gluten_jar} \ + --conf spark.driver.extraClassPath=${GLUTEN_JAR} \ + --conf spark.executor.extraClassPath=${GLUTEN_JAR} \ --conf spark.shuffle.manager=org.apache.spark.shuffle.sort.ColumnarShuffleManager ... ``` -### 3.2.1 Build and install Gluten with Velox backend - -If you want to use Gluten **Velox** backend, see [Build with Velox](./docs/get-started/Velox.md) to build and install the necessary libraries. +There are two ways to acquire Gluten jar for the above configuration. -### 3.2.2 Build and install Gluten with ClickHouse backend +### Use Released Jar +Please download a tar package [here](https://downloads.apache.org/incubator/gluten/), then extract out Gluten jar from it. +It was verified on Centos-7, Centos-8, Ubuntu-20.04 and Ubuntu-22.04. -If you want to use Gluten **ClickHouse** backend, see [Build with ClickHouse Backend](./docs/get-started/ClickHouse.md). ClickHouse backend is developed by [Kyligence](https://kyligence.io/), please visit https://github.com/Kyligence/ClickHouse for more infomation. +### Build From Source +For **Velox** backend, please refer to [Velox.md](./docs/get-started/Velox.md) and [build-guide.md](./docs/get-started/build-guide.md). -### 3.2.3 Build options +For **ClickHouse** backend, please refer to [ClickHouse.md](./docs/get-started/ClickHouse.md). ClickHouse backend is developed by [Kyligence](https://kyligence.io/), please visit https://github.com/Kyligence/ClickHouse for more information. -See [Gluten build guide](./docs/get-started/build-guide.md). +Gluten jar will be generated under `/PATH/TO/GLUTEN/package/target/` after the build. -# 4 Contribution +# 4. Gluten Website +https://gluten.apache.org/ -Welcome to contribute to Gluten project! See [contributing guide](CONTRIBUTING.md) about how to make contributions. +# 5. Contribution +Welcome to contribute to Gluten project! See [CONTRIBUTING.md](CONTRIBUTING.md) about how to make contributions. -## 4.1 Community +# 6. Community +Gluten successfully became Apache incubator project in March 2024. Here are several ways to contact us: -Gluten successfully joined Apache Incubator since March'24. We welcome developers and users who are interested in Gluten project. Here are several ways to contact us: +## GitHub +Welcome to report any issue or create any discussion related to Gluten in GitHub. Please do a search from GitHub issue list before creating a new one to avoid repetition. -### Gluten website -https://gluten.apache.org/ - -### Mailing lists +## Mail Lists For any technical discussion, please send email to [dev@gluten.apache.org](mailto:dev@gluten.apache.org). You can go to [archives](https://lists.apache.org/list.html?dev@gluten.apache.org) for getting historical discussions. Please click [here](mailto:dev-subscribe@gluten.apache.org) to subscribe the mail list. -### Wechat group -We also have a Wechat group (in Chinese) which may be more friendly for PRC developers/users. Due to the limitation of wechat group, please contact with weitingchen at apache.org or zhangzc at apache.org to be invited to the group. - -### Slack channel -There's also a Spark channel in Velox Slack group (in English) for community communication for Velox backend. Please check Velox document here: https://github.com/facebookincubator/velox?tab=readme-ov-file#community - -## 4.2 Issue Report +## Slack Channel (English communication) +Please click [here](https://github.com/apache/incubator-gluten/discussions/8429) to get invitation for ASF Slack workspace where you can find "incubator-gluten" channel. -Please feel free to create Github issue for reporting bug or proposing enhancement. For contributing code, please submit an issue firstly and mention that issue in your PR. +The ASF Slack login entry: https://the-asf.slack.com/. -## 4.3 Documentation - -Currently, all gluten documents are held at [docs](https://github.com/apache/incubator-gluten/tree/main/docs). The documents may not reflect the latest designs. Please feel free to contact us for getting design details or sharing your design ideas. - -# 5 Performance +## WeChat Group (Chinese communication) +For PRC developers/users, please contact weitingchen at apache.org or zhangzc at apache.org for getting invited to the WeChat group. +# 7. Performance We use Decision Support Benchmark1 (TPC-H like) to evaluate Gluten's performance. Decision Support Benchmark1 is a query set modified from [TPC-H benchmark](http://tpc.org/tpch/default5.asp). We use Parquet file format for Velox testing & MergeTree file format for Clickhouse testing, compared to Parquet file format as baseline. See [Decision Support Benchmark1](./tools/workload/tpch). @@ -150,22 +123,14 @@ The below testing environment: a 8-nodes AWS cluster with 1TB data; Spark-3.1.1 ![Performance](./docs/image/clickhouse_decision_support_bench1_22queries_performance.png) -# 6 License - +# 8. License Gluten is licensed under [Apache 2.0 license](https://www.apache.org/licenses/LICENSE-2.0). -# 7 Contact - -Gluten was initiated by Intel and Kyligence in 2022. Several companies are also actively participating in the development, such as BIGO, Meituan, Alibaba Cloud, NetEase, Baidu, Microsoft, etc. If you are interested in Gluten project, please contact and subscribe below mailing lists for further discussion. - -* For community activity: dev@gluten.apache.org -* For code repository activity: commits@gluten.apache.org - -# 8 Thanks to our contributors +# 9. Acknowledgements +Gluten was initiated by Intel and Kyligence in 2022. Several companies are also actively participating in the development, such as BIGO, Meituan, Alibaba Cloud, NetEase, Baidu, Microsoft, IBM, Google, etc. ##### \* LEGAL NOTICE: Your use of this software and any required dependent software (the "Software Package") is subject to the terms and conditions of the software license agreements for the Software Package, which may also include notices, disclaimers, or license terms for third party or open source software included in or with the Software Package, and your use indicates your acceptance of all such terms. Please refer to the "TPP.txt" or other similarly-named text file included with the Software Package for additional details. - From f79f0528bf620ec375365fc8000370e67d492b9d Mon Sep 17 00:00:00 2001 From: PHILO-HE Date: Mon, 13 Jan 2025 12:59:06 +0800 Subject: [PATCH 21/28] [GLUTEN-8319][VL] Support date_format Spark function (#8323) [GLUTEN-8319][VL] Support date_format Spark function. facebookincubator/velox#11953. --- .../execution/ScalarFunctionsValidateSuite.scala | 15 +++++++++++++++ cpp/velox/substrait/SubstraitParser.cc | 1 - 2 files changed, 15 insertions(+), 1 deletion(-) diff --git a/backends-velox/src/test/scala/org/apache/gluten/execution/ScalarFunctionsValidateSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/execution/ScalarFunctionsValidateSuite.scala index 0d0607d3b93e..94c69abd8d32 100644 --- a/backends-velox/src/test/scala/org/apache/gluten/execution/ScalarFunctionsValidateSuite.scala +++ b/backends-velox/src/test/scala/org/apache/gluten/execution/ScalarFunctionsValidateSuite.scala @@ -1515,4 +1515,19 @@ abstract class ScalarFunctionsValidateSuite extends FunctionsValidateSuite { } } } + + test("date_format") { + withTempPath { + path => + val t1 = Timestamp.valueOf("2024-08-22 10:10:10.010") + val t2 = Timestamp.valueOf("2014-12-31 00:00:00.012") + val t3 = Timestamp.valueOf("1968-12-31 23:59:59.001") + Seq(t1, t2, t3).toDF("c0").write.parquet(path.getCanonicalPath) + + spark.read.parquet(path.getCanonicalPath).createOrReplaceTempView("t") + runQueryAndCompare("SELECT date_format(c0, 'yyyy') FROM t") { + checkGlutenOperatorMatch[ProjectExecTransformer] + } + } + } } diff --git a/cpp/velox/substrait/SubstraitParser.cc b/cpp/velox/substrait/SubstraitParser.cc index 006a20c232ea..bdbcc2785ded 100644 --- a/cpp/velox/substrait/SubstraitParser.cc +++ b/cpp/velox/substrait/SubstraitParser.cc @@ -404,7 +404,6 @@ std::unordered_map SubstraitParser::substraitVeloxFunc {"murmur3hash", "hash_with_seed"}, {"xxhash64", "xxhash64_with_seed"}, {"modulus", "remainder"}, - {"date_format", "format_datetime"}, {"negative", "unaryminus"}, {"get_array_item", "get"}}; From d12deaa6419c976409f23df0c69e7e10e823263a Mon Sep 17 00:00:00 2001 From: Yuan Date: Tue, 14 Jan 2025 07:35:06 +0800 Subject: [PATCH 22/28] [GLUTEN-8487][VL] adding JDK11 based Centos8 image (#8513) This patch added JDK11 based Centos 8 image --- .github/workflows/docker_image.yml | 12 ++++++++++-- dev/docker/Dockerfile.centos8-dynamic-build-jdk11 | 12 ++++++++++++ 2 files changed, 22 insertions(+), 2 deletions(-) create mode 100644 dev/docker/Dockerfile.centos8-dynamic-build-jdk11 diff --git a/.github/workflows/docker_image.yml b/.github/workflows/docker_image.yml index f6f0cd51b1f5..58b7549adb70 100644 --- a/.github/workflows/docker_image.yml +++ b/.github/workflows/docker_image.yml @@ -69,10 +69,18 @@ jobs: username: ${{ secrets.DOCKERHUB_USER }} password: ${{ secrets.DOCKERHUB_TOKEN }} - - name: Build and push Docker image + - name: Build and push Docker image Centos8 uses: docker/build-push-action@v2 with: context: . file: dev/docker/Dockerfile.centos8-dynamic-build push: true - tags: apache/gluten:centos-8 + tags: apache/gluten:centos-8 # JDK8 based + + - name: Build and push Docker image Centos8 + JDK11 + uses: docker/build-push-action@v2 + with: + context: . + file: dev/docker/Dockerfile.centos8-dynamic-build-jdk11 + push: true + tags: apache/gluten:centos-8-jdk11 diff --git a/dev/docker/Dockerfile.centos8-dynamic-build-jdk11 b/dev/docker/Dockerfile.centos8-dynamic-build-jdk11 new file mode 100644 index 000000000000..0c2d8cc0c686 --- /dev/null +++ b/dev/docker/Dockerfile.centos8-dynamic-build-jdk11 @@ -0,0 +1,12 @@ +FROM apache/gluten:centos-8 + + +RUN yum install -y java-11-openjdk-devel patch wget git perl +ENV JAVA_HOME=/usr/lib/jvm/java-11-openjdk +ENV PATH=$JAVA_HOME/bin:$PATH + +ENV PATH=${PATH}:/usr/lib/maven/bin + +RUN git clone --depth=1 https://github.com/apache/incubator-gluten /opt/gluten + +RUN cd /opt/gluten && source /opt/rh/gcc-toolset-11/enable && ./dev/builddeps-veloxbe.sh --run_setup_script=ON build_arrow && rm -rf /opt/gluten From 575536840af66229383a2f4f01f1cc1974ad8229 Mon Sep 17 00:00:00 2001 From: Gluten Performance Bot <137994563+GlutenPerfBot@users.noreply.github.com> Date: Tue, 14 Jan 2025 10:22:36 +0800 Subject: [PATCH 23/28] [GLUTEN-6887][VL] Daily Update Velox Version (2025_01_14) (#8522) Upstream Velox's New Commits: 5d547b7c9 by Krishna Pai, Reduce complexElementsMaxSize for Window Fuzzers (12074) d1ec9c014 by Ke Wang, fix: Fix config name for stats-based-filter-reorder-disabaled (12075) --- ep/build-velox/src/get_velox.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh index 06ce5b96068c..b3fa3ed5d3aa 100755 --- a/ep/build-velox/src/get_velox.sh +++ b/ep/build-velox/src/get_velox.sh @@ -17,7 +17,7 @@ set -exu VELOX_REPO=https://github.com/oap-project/velox.git -VELOX_BRANCH=2025_01_12 +VELOX_BRANCH=2025_01_14 VELOX_HOME="" OS=`uname -s` From ea58aab7348c97a28b545e2df4c59649fda40325 Mon Sep 17 00:00:00 2001 From: JiaKe Date: Tue, 14 Jan 2025 17:29:58 +0800 Subject: [PATCH 24/28] [GLUTEN-8020][VL] Remove the libhdfs3 installation script required for static linking (#8013) - We will only retain the dynamic libraries libhdfs.so or libhdfs3.so at runtime based on this benchmark here. So there is no need to keep the libhdfs3 installation script required for static linking. - Some customers still use libhdfs3. We provide a script to compile libhdfs3.so in the dev folder --- dev/build-thirdparty.sh | 14 +++---- dev/build_helper_functions.sh | 6 --- dev/build_libhdfs3.sh | 37 +++++++++++++++++++ dev/builddeps-veloxbe.sh | 9 ----- dev/vcpkg/init.sh | 6 --- dev/vcpkg/ports/libhdfs3/libhdfs3Config.cmake | 26 ------------- dev/vcpkg/ports/libhdfs3/portfile.cmake | 27 -------------- dev/vcpkg/ports/libhdfs3/usage | 4 -- dev/vcpkg/ports/libhdfs3/vcpkg.json | 34 ----------------- dev/vcpkg/vcpkg.json | 6 --- docs/get-started/Velox.md | 4 ++ ep/build-velox/src/setup-centos7.sh | 11 ------ 12 files changed, 48 insertions(+), 136 deletions(-) create mode 100755 dev/build_libhdfs3.sh delete mode 100644 dev/vcpkg/ports/libhdfs3/libhdfs3Config.cmake delete mode 100644 dev/vcpkg/ports/libhdfs3/portfile.cmake delete mode 100644 dev/vcpkg/ports/libhdfs3/usage delete mode 100644 dev/vcpkg/ports/libhdfs3/vcpkg.json diff --git a/dev/build-thirdparty.sh b/dev/build-thirdparty.sh index ee827ef197f7..109392f4f4a7 100755 --- a/dev/build-thirdparty.sh +++ b/dev/build-thirdparty.sh @@ -12,40 +12,40 @@ ARCH=`uname -m` mkdir -p $THIRDPARTY_LIB function process_setup_ubuntu_2004 { cp /usr/lib/${ARCH}-linux-gnu/{libroken.so.18,libasn1.so.8,libcrypto.so.1.1,libnghttp2.so.14,libnettle.so.7,libhogweed.so.5,librtmp.so.1,libssh.so.4,libssl.so.1.1,liblber-2.4.so.2,libsasl2.so.2,libwind.so.0,libheimbase.so.1,libhcrypto.so.4,libhx509.so.5,libkrb5.so.26,libheimntlm.so.0,libgssapi.so.3,libldap_r-2.4.so.2,libcurl.so.4,libdouble-conversion.so.3,libevent-2.1.so.7,libgflags.so.2.2,libunwind.so.8,libglog.so.0,libidn.so.11,libntlm.so.0,libgsasl.so.7,libicudata.so.66,libicuuc.so.66,libxml2.so.2,libre2.so.5,libsnappy.so.1,libpsl.so.5,libbrotlidec.so.1,libbrotlicommon.so.1,libthrift-0.13.0.so} $THIRDPARTY_LIB/ - cp /usr/local/lib/{libprotobuf.so.32,libhdfs3.so.1,libboost_context.so.1.84.0,libboost_regex.so.1.84.0} $THIRDPARTY_LIB/ + cp /usr/local/lib/{libprotobuf.so.32,libboost_context.so.1.84.0,libboost_regex.so.1.84.0} $THIRDPARTY_LIB/ } function process_setup_ubuntu_2204 { cp /usr/lib/${ARCH}-linux-gnu/{libre2.so.9,libdouble-conversion.so.3,libidn.so.12,libglog.so.0,libgflags.so.2.2,libevent-2.1.so.7,libsnappy.so.1,libunwind.so.8,libcurl.so.4,libxml2.so.2,libgsasl.so.7,libicui18n.so.70,libicuuc.so.70,libnghttp2.so.14,libldap-2.5.so.0,liblber-2.5.so.0,libntlm.so.0,librtmp.so.1,libsasl2.so.2,libssh.so.4,libicudata.so.70,libthrift-0.16.0.so} $THIRDPARTY_LIB/ - cp /usr/local/lib/{libhdfs3.so.1,libprotobuf.so.32,libboost_context.so.1.84.0,libboost_regex.so.1.84.0} $THIRDPARTY_LIB/ + cp /usr/local/lib/{libprotobuf.so.32,libboost_context.so.1.84.0,libboost_regex.so.1.84.0} $THIRDPARTY_LIB/ } function process_setup_centos_9 { cp /lib64/{libre2.so.9,libdouble-conversion.so.3,libevent-2.1.so.7,libdwarf.so.0,libgsasl.so.7,libicudata.so.67,libicui18n.so.67,libicuuc.so.67,libidn.so.12,libntlm.so.0,libsodium.so.23} $THIRDPARTY_LIB/ - cp /usr/local/lib/{libhdfs3.so.1,libboost_context.so.1.84.0,libboost_filesystem.so.1.84.0,libboost_program_options.so.1.84.0,libboost_regex.so.1.84.0,libboost_system.so.1.84.0,libboost_thread.so.1.84.0,libboost_atomic.so.1.84.0,libprotobuf.so.32} $THIRDPARTY_LIB/ + cp /usr/local/lib/{libboost_context.so.1.84.0,libboost_filesystem.so.1.84.0,libboost_program_options.so.1.84.0,libboost_regex.so.1.84.0,libboost_system.so.1.84.0,libboost_thread.so.1.84.0,libboost_atomic.so.1.84.0,libprotobuf.so.32} $THIRDPARTY_LIB/ cp /usr/local/lib64/{libgflags.so.2.2,libglog.so.1} $THIRDPARTY_LIB/ } function process_setup_centos_8 { cp /usr/lib64/{libre2.so.0,libdouble-conversion.so.3,libevent-2.1.so.6,libdwarf.so.1,libgsasl.so.7,libicudata.so.60,libicui18n.so.60,libicuuc.so.60,libidn.so.11,libntlm.so.0,libsodium.so.23} $THIRDPARTY_LIB/ - cp /usr/local/lib/{libhdfs3.so.1,libboost_context.so.1.84.0,libboost_filesystem.so.1.84.0,libboost_program_options.so.1.84.0,libboost_regex.so.1.84.0,libboost_system.so.1.84.0,libboost_thread.so.1.84.0,libboost_atomic.so.1.84.0,libprotobuf.so.32} $THIRDPARTY_LIB/ + cp /usr/local/lib/{libboost_context.so.1.84.0,libboost_filesystem.so.1.84.0,libboost_program_options.so.1.84.0,libboost_regex.so.1.84.0,libboost_system.so.1.84.0,libboost_thread.so.1.84.0,libboost_atomic.so.1.84.0,libprotobuf.so.32} $THIRDPARTY_LIB/ cp /usr/local/lib64/{libgflags.so.2.2,libglog.so.1} $THIRDPARTY_LIB/ } function process_setup_centos_7 { cp /usr/local/lib64/{libgflags.so.2.2,libglog.so.0} $THIRDPARTY_LIB/ cp /usr/lib64/{libdouble-conversion.so.1,libevent-2.0.so.5,libzstd.so.1,libntlm.so.0,libgsasl.so.7,liblz4.so.1} $THIRDPARTY_LIB/ - cp /usr/local/lib/{libre2.so.10,libhdfs3.so.1,libboost_context.so.1.84.0,libboost_filesystem.so.1.84.0,libboost_program_options.so.1.84.0,libboost_system.so.1.84.0,libboost_thread.so.1.84.0,libboost_regex.so.1.84.0,libboost_atomic.so.1.84.0,libprotobuf.so.32} $THIRDPARTY_LIB/ + cp /usr/local/lib/{libre2.so.10,libboost_context.so.1.84.0,libboost_filesystem.so.1.84.0,libboost_program_options.so.1.84.0,libboost_system.so.1.84.0,libboost_thread.so.1.84.0,libboost_regex.so.1.84.0,libboost_atomic.so.1.84.0,libprotobuf.so.32} $THIRDPARTY_LIB/ } function process_setup_debian_11 { cp /usr/lib/x86_64-linux-gnu/{libre2.so.9,libthrift-0.13.0.so,libdouble-conversion.so.3,libevent-2.1.so.7,libgflags.so.2.2,libglog.so.0,libsnappy.so.1,libunwind.so.8,libcurl.so.4,libicui18n.so.67,libicuuc.so.67,libnghttp2.so.14,librtmp.so.1,libssh2.so.1,libpsl.so.5,libldap_r-2.4.so.2,liblber-2.4.so.2,libbrotlidec.so.1,libicudata.so.67,libsasl2.so.2,libbrotlicommon.so.1} $THIRDPARTY_LIB/ - cp /usr/local/lib/{libhdfs3.so.1,libprotobuf.so.32,libboost_context.so.1.84.0,libboost_regex.so.1.84.0} $THIRDPARTY_LIB/ + cp /usr/local/lib/{libprotobuf.so.32,libboost_context.so.1.84.0,libboost_regex.so.1.84.0} $THIRDPARTY_LIB/ } function process_setup_debian_12 { cp /usr/lib/x86_64-linux-gnu/{libthrift-0.17.0.so,libdouble-conversion.so.3,libevent-2.1.so.7,libgflags.so.2.2,libglog.so.1,libsnappy.so.1,libunwind.so.8,libcurl.so.4,libicui18n.so.72,libicuuc.so.72,libnghttp2.so.14,librtmp.so.1,libssh2.so.1,libpsl.so.5,libldap-2.5.so.0,liblber-2.5.so.0,libbrotlidec.so.1,libicudata.so.72,libsasl2.so.2,libbrotlicommon.so.1,libcrypto.so.3,libssl.so.3,libgssapi_krb5.so.2,libkrb5.so.3,libk5crypto.so.3,libkrb5support.so.0,libkeyutils.so.1} $THIRDPARTY_LIB/ - cp /usr/local/lib/{libprotobuf.so.32,libhdfs3.so.1,libboost_context.so.1.84.0,libboost_regex.so.1.84.0} $THIRDPARTY_LIB/ + cp /usr/local/lib/{libprotobuf.so.32,libboost_context.so.1.84.0,libboost_regex.so.1.84.0} $THIRDPARTY_LIB/ } if [[ "$LINUX_OS" == "ubuntu" || "$LINUX_OS" == "pop" ]]; then diff --git a/dev/build_helper_functions.sh b/dev/build_helper_functions.sh index 97e3a0993520..836ed6ca4e55 100644 --- a/dev/build_helper_functions.sh +++ b/dev/build_helper_functions.sh @@ -208,9 +208,3 @@ function setup_linux { exit 1 fi } - -function install_libhdfs3 { - github_checkout oap-project/libhdfs3 master - cmake_install -} - diff --git a/dev/build_libhdfs3.sh b/dev/build_libhdfs3.sh new file mode 100755 index 000000000000..b001a121c69b --- /dev/null +++ b/dev/build_libhdfs3.sh @@ -0,0 +1,37 @@ +#!/bin/bash +# 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. + +set -exu + +CURRENT_DIR=$(cd "$(dirname "$BASH_SOURCE")"; pwd) +export SUDO=sudo +source ${CURRENT_DIR}/build_helper_functions.sh +DEPENDENCY_DIR=${DEPENDENCY_DIR:-$CURRENT_DIR/../ep/_ep} + +function build_libhdfs3 { + cd "${DEPENDENCY_DIR}" + github_checkout apache/hawq master + cd depends/libhdfs3 + sed -i "/FIND_PACKAGE(GoogleTest REQUIRED)/d" ./CMakeLists.txt + sed -i "s/dumpversion/dumpfullversion/" ./CMake/Platform.cmake + sed -i "s/dfs.domain.socket.path\", \"\"/dfs.domain.socket.path\", \"\/var\/lib\/hadoop-hdfs\/dn_socket\"/g" src/common/SessionConfig.cpp + sed -i "s/pos < endOfCurBlock/pos \< endOfCurBlock \&\& pos \- cursor \<\= 128 \* 1024/g" src/client/InputStreamImpl.cpp + cmake_install +} + +echo "Start to build Libhdfs3" +build_libhdfs3 +echo "Finished building Libhdfs3. You can find the libhdfs3.so in /usr/local/lib/libhdfs3.so.1" diff --git a/dev/builddeps-veloxbe.sh b/dev/builddeps-veloxbe.sh index 8eb4cf4edf2c..2180db9f8dca 100755 --- a/dev/builddeps-veloxbe.sh +++ b/dev/builddeps-veloxbe.sh @@ -247,15 +247,6 @@ if [ -z "${GLUTEN_VCPKG_ENABLED:-}" ] && [ $RUN_SETUP_SCRIPT == "ON" ]; then fi ${VELOX_HOME}/scripts/setup-adapters.sh aws fi - if [ $ENABLE_HDFS == "ON" ]; then - if [ $OS == 'Darwin' ]; then - echo "HDFS is not supported on MacOS." - exit 1 - fi - pushd $VELOX_HOME - install_libhdfs3 - popd - fi if [ $ENABLE_GCS == "ON" ]; then ${VELOX_HOME}/scripts/setup-adapters.sh gcs fi diff --git a/dev/vcpkg/init.sh b/dev/vcpkg/init.sh index bae1a8ad32eb..4e7b16af821c 100755 --- a/dev/vcpkg/init.sh +++ b/dev/vcpkg/init.sh @@ -70,9 +70,6 @@ fi if [ "$ENABLE_GCS" = "ON" ]; then EXTRA_FEATURES+="--x-feature=velox-gcs " fi -if [ "$ENABLE_HDFS" = "ON" ]; then - EXTRA_FEATURES+="--x-feature=velox-hdfs " -fi if [ "$ENABLE_ABFS" = "ON" ]; then EXTRA_FEATURES+="--x-feature=velox-abfs" fi @@ -90,6 +87,3 @@ cp $VCPKG_TRIPLET_INSTALL_DIR/lib/libssl.a $VCPKG_TRIPLET_INSTALL_DIR/debug/lib cp $VCPKG_TRIPLET_INSTALL_DIR/lib/libcrypto.a $VCPKG_TRIPLET_INSTALL_DIR/debug/lib cp $VCPKG_TRIPLET_INSTALL_DIR/lib/liblzma.a $VCPKG_TRIPLET_INSTALL_DIR/debug/lib cp $VCPKG_TRIPLET_INSTALL_DIR/lib/libdwarf.a $VCPKG_TRIPLET_INSTALL_DIR/debug/lib -# Allow libhdfs3.a is not installed as build option may not enable hdfs. -cp $VCPKG_TRIPLET_INSTALL_DIR/lib/libhdfs3.a $VCPKG_TRIPLET_INSTALL_DIR/debug/lib || true - diff --git a/dev/vcpkg/ports/libhdfs3/libhdfs3Config.cmake b/dev/vcpkg/ports/libhdfs3/libhdfs3Config.cmake deleted file mode 100644 index 93ef72da56b2..000000000000 --- a/dev/vcpkg/ports/libhdfs3/libhdfs3Config.cmake +++ /dev/null @@ -1,26 +0,0 @@ -include(CMakeFindDependencyMacro) -include(FindPkgConfig) - -find_dependency(Boost COMPONENTS thread chrono system atomic iostreams) -find_dependency(LibXml2) -find_dependency(Protobuf) -pkg_check_modules(Gsasl REQUIRED libgsasl mit-krb5-gssapi) -pkg_check_modules(UUID REQUIRED uuid) - -FUNCTION(SET_LIBRARY_TARGET NAMESPACE LIB_NAME DEBUG_LIB_FILE_NAME RELEASE_LIB_FILE_NAME INCLUDE_DIR) - ADD_LIBRARY(${NAMESPACE}::${LIB_NAME} STATIC IMPORTED) - SET_TARGET_PROPERTIES(${NAMESPACE}::${LIB_NAME} PROPERTIES - IMPORTED_CONFIGURATIONS "RELEASE;DEBUG" - IMPORTED_LOCATION_RELEASE "${RELEASE_LIB_FILE_NAME}" - IMPORTED_LOCATION_DEBUG "${DEBUG_LIB_FILE_NAME}" - INTERFACE_INCLUDE_DIRECTORIES "${INCLUDE_DIR}" - INTERFACE_LINK_LIBRARIES "protobuf::libprotobuf;LibXml2::LibXml2;${Gsasl_LINK_LIBRARIES};${UUID_LINK_LIBRARIES}" - ) - SET(${NAMESPACE}_${LIB_NAME}_FOUND 1) -ENDFUNCTION() - -GET_FILENAME_COMPONENT(ROOT "${CMAKE_CURRENT_LIST_FILE}" PATH) -GET_FILENAME_COMPONENT(ROOT "${ROOT}" PATH) -GET_FILENAME_COMPONENT(ROOT "${ROOT}" PATH) - -SET_LIBRARY_TARGET("HDFS" "hdfs3" "${ROOT}/debug/lib/libhdfs3.a" "${ROOT}/lib/libhdfs3.a" "${ROOT}/include/hdfs") \ No newline at end of file diff --git a/dev/vcpkg/ports/libhdfs3/portfile.cmake b/dev/vcpkg/ports/libhdfs3/portfile.cmake deleted file mode 100644 index 4c59c57ef9a8..000000000000 --- a/dev/vcpkg/ports/libhdfs3/portfile.cmake +++ /dev/null @@ -1,27 +0,0 @@ -vcpkg_from_github( - OUT_SOURCE_PATH SOURCE_PATH - REPO oap-project/libhdfs3 - HEAD_REF master - REF 9f234edb354ebcc99179cc6f72aefd66865f4154 - SHA512 a1a587fdca60a39f77d36b281ad15fefd7cb4b353c982274ef3d7702e84c834525cd5a3ec2bbc4154fce58f1c7054a17789f08485eaacfbb672544398a277951 -) - -vcpkg_configure_cmake( - SOURCE_PATH ${SOURCE_PATH} - PREFER_NINJA - OPTIONS - -DCMAKE_PROGRAM_PATH=${CURRENT_HOST_INSTALLED_DIR}/tools/yasm - -DWITH_KERBEROS=on -) - -vcpkg_install_cmake() - -vcpkg_copy_pdbs() - -file(GLOB HDFS3_SHARED_LIBS ${CURRENT_PACKAGES_DIR}/debug/lib/libhdfs3.so* ${CURRENT_PACKAGES_DIR}/lib/libhdfs3.so*) -file(REMOVE ${HDFS3_SHARED_LIBS}) - -file(REMOVE_RECURSE ${CURRENT_PACKAGES_DIR}/debug/include ${CURRENT_PACKAGES_DIR}/debug/share) -file(INSTALL ${SOURCE_PATH}/LICENSE.txt DESTINATION ${CURRENT_PACKAGES_DIR}/share/${PORT} RENAME copyright) -FILE(INSTALL ${CMAKE_CURRENT_LIST_DIR}/libhdfs3Config.cmake DESTINATION ${CURRENT_PACKAGES_DIR}/share/${PORT}) -FILE(INSTALL ${CMAKE_CURRENT_LIST_DIR}/usage DESTINATION ${CURRENT_PACKAGES_DIR}/share/${PORT}) diff --git a/dev/vcpkg/ports/libhdfs3/usage b/dev/vcpkg/ports/libhdfs3/usage deleted file mode 100644 index 780d82d25683..000000000000 --- a/dev/vcpkg/ports/libhdfs3/usage +++ /dev/null @@ -1,4 +0,0 @@ -The package libhdfs3 is compatible with built-in CMake targets: - - FIND_PACKAGE(libhdfs3 REQUIRED) - TARGET_LINK_LIBRARIES(main PRIVATE HDFS::hdfs3) diff --git a/dev/vcpkg/ports/libhdfs3/vcpkg.json b/dev/vcpkg/ports/libhdfs3/vcpkg.json deleted file mode 100644 index 495615cf9077..000000000000 --- a/dev/vcpkg/ports/libhdfs3/vcpkg.json +++ /dev/null @@ -1,34 +0,0 @@ -{ - "name": "libhdfs3", - "version-date": "2019-11-05", - "port-version": 3, - "description": "Native Hadoop RPC protocol and HDFS data transfer protocol implementation", - "homepage": "https://github.com/erikmuttersbach/libhdfs3", - "supports": "!windows", - "dependencies": [ - "boost-thread", - "boost-chrono", - "boost-system", - "boost-atomic", - "boost-iostreams", - "boost-function", - "boost-bind", - "boost-crc", - "boost-functional", - { - "name": "libuuid", - "platform": "!windows & !osx" - }, - "libxml2", - "protobuf", - "krb5", - "gsasl", - { - "name": "yasm", - "host": true, - "features": [ - "tools" - ] - } - ] -} diff --git a/dev/vcpkg/vcpkg.json b/dev/vcpkg/vcpkg.json index c0123cfbe998..a6a70ec913bf 100644 --- a/dev/vcpkg/vcpkg.json +++ b/dev/vcpkg/vcpkg.json @@ -80,12 +80,6 @@ } ] }, - "velox-hdfs": { - "description": "Velox HDFS Support", - "dependencies": [ - "libhdfs3" - ] - }, "velox-abfs": { "description": "Velox ABFS Support", "dependencies": [ diff --git a/docs/get-started/Velox.md b/docs/get-started/Velox.md index 81bb88c75aec..8c8cb7bbd818 100644 --- a/docs/get-started/Velox.md +++ b/docs/get-started/Velox.md @@ -127,6 +127,10 @@ libraries list from the third-party jar. Gluten supports dynamically loading both libhdfs.so and libhdfs3.so at runtime by using dlopen, allowing the JVM to load the appropriate shared library file as needed. This means you do not need to set the library path during the compilation phase. To enable this functionality, you must set the JAVA_HOME and HADOOP_HOME environment variables. Gluten will then locate and load the ${HADOOP_HOME}/lib/native/libhdfs.so file at runtime. If you prefer to use libhdfs3.so instead, simply replace the ${HADOOP_HOME}/lib/native/libhdfs.so file with libhdfs3.so. +### Build libhdfs3 + +If you want to run Gluten with libhdfs3.so, you need to manually compile libhdfs3 to obtain the libhdfs3.so file. We provide the script dev/build_libhdfs3.sh in Gluten to help you compile libhdfs3.so. + ### Build with HDFS support To build Gluten with HDFS support, below command is suggested: diff --git a/ep/build-velox/src/setup-centos7.sh b/ep/build-velox/src/setup-centos7.sh index 45880161a4a5..dbac575fbb47 100755 --- a/ep/build-velox/src/setup-centos7.sh +++ b/ep/build-velox/src/setup-centos7.sh @@ -166,17 +166,6 @@ function install_boost { $SUDO ./b2 "-j$(nproc)" -d0 install threading=multi } -function install_libhdfs3 { - cd "${DEPENDENCY_DIR}" - github_checkout apache/hawq master - cd depends/libhdfs3 - sed -i "/FIND_PACKAGE(GoogleTest REQUIRED)/d" ./CMakeLists.txt - sed -i "s/dumpversion/dumpfullversion/" ./CMake/Platform.cmake - sed -i "s/dfs.domain.socket.path\", \"\"/dfs.domain.socket.path\", \"\/var\/lib\/hadoop-hdfs\/dn_socket\"/g" src/common/SessionConfig.cpp - sed -i "s/pos < endOfCurBlock/pos \< endOfCurBlock \&\& pos \- cursor \<\= 128 \* 1024/g" src/client/InputStreamImpl.cpp - cmake_install -} - function install_protobuf { cd "${DEPENDENCY_DIR}" wget https://github.com/protocolbuffers/protobuf/releases/download/v21.4/protobuf-all-21.4.tar.gz From ed6cdb34e45b957b63c38531fd709394fc308c33 Mon Sep 17 00:00:00 2001 From: jkhaliqi <60749291+jkhaliqi@users.noreply.github.com> Date: Tue, 14 Jan 2025 14:46:11 -0800 Subject: [PATCH 25/28] [GLUTEN-8532][VL] Fix parenthesis within macro (#8533) Use parentheses within macros around parameter names --- cpp/core/utils/qat/QatCodec.cc | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/cpp/core/utils/qat/QatCodec.cc b/cpp/core/utils/qat/QatCodec.cc index 1a5fc8ea6db1..e01fa96988bd 100644 --- a/cpp/core/utils/qat/QatCodec.cc +++ b/cpp/core/utils/qat/QatCodec.cc @@ -26,9 +26,9 @@ #include "QatCodec.h" -#define QZ_INIT_FAIL(rc) (QZ_OK != rc && QZ_DUPLICATE != rc) +#define QZ_INIT_FAIL(rc) ((QZ_OK != (rc)) && (QZ_DUPLICATE != (rc))) -#define QZ_SETUP_SESSION_FAIL(rc) (QZ_PARAMS == rc || QZ_NOSW_NO_HW == rc || QZ_NOSW_LOW_MEM == rc) +#define QZ_SETUP_SESSION_FAIL(rc) (QZ_PARAMS == (rc) || QZ_NOSW_NO_HW == (rc) || QZ_NOSW_LOW_MEM == (rc)) namespace gluten { namespace qat { From 9171124483eb4935cace5979d1373d4c5f78ad1b Mon Sep 17 00:00:00 2001 From: Gluten Performance Bot <137994563+GlutenPerfBot@users.noreply.github.com> Date: Wed, 15 Jan 2025 09:05:38 +0800 Subject: [PATCH 26/28] [GLUTEN-6887][VL] Daily Update Velox Version (2025_01_15) (#8536) Upstream Velox's New Commits: 88170133e by rui-mo, feat(fuzzer): Add custom special form signatures for Presto and Spark (12032) 61e737c59 by Pedro Eugenio Rocha Pedreira, fix(plan-builder): Avoid advancing plan id on error (12081) 93523f823 by Deepak Majeti, feat(s3): Support S3 Region (12063) 50f9a5455 by Kevin Wilfong, refactor: Break PrestoSerializer.cpp into components (11922) 2b74a93bc by Pramod Satya, feat: Increase peer rows in partitions generated by window fuzzer (10293) --- ep/build-velox/src/get_velox.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh index b3fa3ed5d3aa..f68c9b9980b7 100755 --- a/ep/build-velox/src/get_velox.sh +++ b/ep/build-velox/src/get_velox.sh @@ -17,7 +17,7 @@ set -exu VELOX_REPO=https://github.com/oap-project/velox.git -VELOX_BRANCH=2025_01_14 +VELOX_BRANCH=2025_01_15 VELOX_HOME="" OS=`uname -s` From 0b5a46a23a5a404eb466f40b6bbacb057c7b67a7 Mon Sep 17 00:00:00 2001 From: Hongze Zhang Date: Wed, 15 Jan 2025 10:12:49 +0800 Subject: [PATCH 27/28] [CORE] Use RAS's cost model for legacy transition planner to evaluate cost of transitions (#8527) --- .../clickhouse/CHListenerApi.scala | 4 +- .../backendsapi/clickhouse/CHRuleApi.scala | 4 + .../backendsapi/velox/VeloxListenerApi.scala | 6 +- .../backendsapi/velox/VeloxRuleApi.scala | 11 +- .../execution/VeloxRoughCostModel2Suite.scala | 65 ----------- .../enumerated/planner/VeloxRasSuite.scala | 25 +++-- .../transition/VeloxTransitionSuite.scala | 4 +- .../enumerated/EnumeratedTransform.scala | 4 +- .../planner/cost/GlutenCostModel.scala | 30 +++++ .../planner/cost/LongCostModel.scala | 42 ++++--- .../planner/cost/LongCosterChain.scala | 2 +- .../columnar/transition/Convention.scala | 32 +++--- .../columnar/transition/ConventionFunc.scala | 22 ---- .../transition/FloydWarshallGraph.scala | 43 +++++--- .../columnar/transition/TransitionGraph.scala | 104 +++++++++++++----- .../extension/injector/GlutenInjector.scala | 7 +- .../transition/FloydWarshallGraphSuite.scala | 13 ++- .../planner/cost/LegacyCoster.scala | 4 +- .../enumerated/planner/cost/RoughCoster.scala | 2 - .../planner/cost/RoughCoster2.scala | 83 -------------- .../columnar/transition/TransitionSuite.scala | 47 +++++--- .../apache/gluten/config/GlutenConfig.scala | 32 +----- 22 files changed, 255 insertions(+), 331 deletions(-) delete mode 100644 backends-velox/src/test/scala/org/apache/gluten/execution/VeloxRoughCostModel2Suite.scala create mode 100644 gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/GlutenCostModel.scala delete mode 100644 gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/RoughCoster2.scala diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHListenerApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHListenerApi.scala index fbaf9e37c15f..48ef66ca74a8 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHListenerApi.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHListenerApi.scala @@ -23,6 +23,7 @@ import org.apache.gluten.execution.CHBroadcastBuildSideCache import org.apache.gluten.execution.datasource.GlutenFormatFactory import org.apache.gluten.expression.UDFMappings import org.apache.gluten.extension.ExpressionExtensionTrait +import org.apache.gluten.extension.columnar.transition.Convention import org.apache.gluten.jni.JniLibLoader import org.apache.gluten.vectorized.CHNativeExpressionEvaluator @@ -70,7 +71,8 @@ class CHListenerApi extends ListenerApi with Logging { override def onExecutorShutdown(): Unit = shutdown() private def initialize(conf: SparkConf, isDriver: Boolean): Unit = { - // Force batch type initializations. + // Do row / batch type initializations. + Convention.ensureSparkRowAndBatchTypesRegistered() CHBatch.ensureRegistered() SparkDirectoryUtil.init(conf) val libPath = conf.get(GlutenConfig.GLUTEN_LIB_PATH, StringUtils.EMPTY) diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala index 21ae342a2263..426c88c9073f 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala @@ -22,6 +22,7 @@ import org.apache.gluten.config.GlutenConfig import org.apache.gluten.extension._ import org.apache.gluten.extension.columnar._ import org.apache.gluten.extension.columnar.MiscColumnarRules.{RemoveGlutenTableCacheColumnarToRow, RemoveTopmostColumnarToRow, RewriteSubqueryBroadcast} +import org.apache.gluten.extension.columnar.enumerated.planner.cost.LegacyCoster import org.apache.gluten.extension.columnar.heuristic.{ExpandFallbackPolicy, HeuristicTransform} import org.apache.gluten.extension.columnar.offload.{OffloadExchange, OffloadJoin, OffloadOthers} import org.apache.gluten.extension.columnar.rewrite._ @@ -142,6 +143,9 @@ object CHRuleApi { } private def injectRas(injector: RasInjector): Unit = { + // Register legacy coster for transition planner. + injector.injectCoster(_ => LegacyCoster) + // CH backend doesn't work with RAS at the moment. Inject a rule that aborts any // execution calls. injector.injectPreTransform( diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxListenerApi.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxListenerApi.scala index 5d75521b8473..0453558d1af7 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxListenerApi.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxListenerApi.scala @@ -22,6 +22,7 @@ import org.apache.gluten.columnarbatch.VeloxBatch import org.apache.gluten.config.GlutenConfig import org.apache.gluten.execution.datasource.GlutenFormatFactory import org.apache.gluten.expression.UDFMappings +import org.apache.gluten.extension.columnar.transition.Convention import org.apache.gluten.init.NativeBackendInitializer import org.apache.gluten.jni.{JniLibLoader, JniWorkspace} import org.apache.gluten.udf.UdfJniWrapper @@ -126,10 +127,11 @@ class VeloxListenerApi extends ListenerApi with Logging { override def onExecutorShutdown(): Unit = shutdown() private def initialize(conf: SparkConf, isDriver: Boolean): Unit = { - // Force batch type initializations. - VeloxBatch.ensureRegistered() + // Do row / batch type initializations. + Convention.ensureSparkRowAndBatchTypesRegistered() ArrowJavaBatch.ensureRegistered() ArrowNativeBatch.ensureRegistered() + VeloxBatch.ensureRegistered() // Register columnar shuffle so can be considered when // `org.apache.spark.shuffle.GlutenShuffleManager` is set as Spark shuffle manager. diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxRuleApi.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxRuleApi.scala index f3c75cd98318..6c60ab7d537f 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxRuleApi.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxRuleApi.scala @@ -23,7 +23,7 @@ import org.apache.gluten.extension._ import org.apache.gluten.extension.columnar._ import org.apache.gluten.extension.columnar.MiscColumnarRules.{RemoveGlutenTableCacheColumnarToRow, RemoveTopmostColumnarToRow, RewriteSubqueryBroadcast} import org.apache.gluten.extension.columnar.enumerated.{RasOffload, RemoveSort} -import org.apache.gluten.extension.columnar.enumerated.planner.cost.{LegacyCoster, RoughCoster, RoughCoster2} +import org.apache.gluten.extension.columnar.enumerated.planner.cost.{LegacyCoster, RoughCoster} import org.apache.gluten.extension.columnar.heuristic.{ExpandFallbackPolicy, HeuristicTransform} import org.apache.gluten.extension.columnar.offload.{OffloadExchange, OffloadJoin, OffloadOthers} import org.apache.gluten.extension.columnar.rewrite._ @@ -120,6 +120,10 @@ object VeloxRuleApi { } private def injectRas(injector: RasInjector): Unit = { + // Gluten RAS: Costers. + injector.injectCoster(_ => LegacyCoster) + injector.injectCoster(_ => RoughCoster) + // Gluten RAS: Pre rules. injector.injectPreTransform(_ => RemoveTransitions) injector.injectPreTransform(_ => PushDownInputFileExpression.PreOffload) @@ -131,6 +135,7 @@ object VeloxRuleApi { // Gluten RAS: The RAS rule. val validatorBuilder: GlutenConfig => Validator = conf => Validators.newValidator(conf) + injector.injectRasRule(_ => RemoveSort) val rewrites = Seq( RewriteIn, @@ -139,10 +144,6 @@ object VeloxRuleApi { PullOutPreProject, PullOutPostProject, ProjectColumnPruning) - injector.injectCoster(_ => LegacyCoster) - injector.injectCoster(_ => RoughCoster) - injector.injectCoster(_ => RoughCoster2) - injector.injectRasRule(_ => RemoveSort) val offloads: Seq[RasOffload] = Seq( RasOffload.from[Exchange](OffloadExchange()), RasOffload.from[BaseJoinExec](OffloadJoin()), diff --git a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxRoughCostModel2Suite.scala b/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxRoughCostModel2Suite.scala deleted file mode 100644 index cf61a7323665..000000000000 --- a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxRoughCostModel2Suite.scala +++ /dev/null @@ -1,65 +0,0 @@ -/* - * 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.config.GlutenConfig - -import org.apache.spark.SparkConf -import org.apache.spark.sql.execution.ProjectExec - -class VeloxRoughCostModel2Suite extends VeloxWholeStageTransformerSuite { - override protected val resourcePath: String = "/tpch-data-parquet-velox" - override protected val fileFormat: String = "parquet" - - override def beforeAll(): Unit = { - super.beforeAll() - spark - .range(100) - .selectExpr("cast(id % 3 as int) as c1", "id as c2", "array(id, id + 1) as c3") - .write - .format("parquet") - .saveAsTable("tmp1") - } - - override protected def afterAll(): Unit = { - spark.sql("drop table tmp1") - super.afterAll() - } - - override protected def sparkConf: SparkConf = super.sparkConf - .set(GlutenConfig.RAS_ENABLED.key, "true") - .set(GlutenConfig.RAS_COST_MODEL.key, "rough2") - .set(GlutenConfig.VANILLA_VECTORIZED_READERS_ENABLED.key, "false") - - test("fallback trivial project if its neighbor nodes fell back") { - withSQLConf(GlutenConfig.COLUMNAR_FILESCAN_ENABLED.key -> "false") { - runQueryAndCompare("select c1 as c3 from tmp1") { - checkSparkOperatorMatch[ProjectExec] - } - } - } - - test("avoid adding r2c if r2c cost greater than native") { - withSQLConf( - GlutenConfig.COLUMNAR_FILESCAN_ENABLED.key -> "false", - GlutenConfig.RAS_ROUGH2_SIZEBYTES_THRESHOLD.key -> "1") { - runQueryAndCompare("select array_contains(c3, 0) as list from tmp1") { - checkSparkOperatorMatch[ProjectExec] - } - } - } -} diff --git a/backends-velox/src/test/scala/org/apache/gluten/extension/columnar/enumerated/planner/VeloxRasSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/extension/columnar/enumerated/planner/VeloxRasSuite.scala index 65d32ebf6162..e7de629b39e3 100644 --- a/backends-velox/src/test/scala/org/apache/gluten/extension/columnar/enumerated/planner/VeloxRasSuite.scala +++ b/backends-velox/src/test/scala/org/apache/gluten/extension/columnar/enumerated/planner/VeloxRasSuite.scala @@ -18,10 +18,10 @@ package org.apache.gluten.extension.columnar.enumerated.planner import org.apache.gluten.config.GlutenConfig import org.apache.gluten.extension.columnar.enumerated.EnumeratedTransform -import org.apache.gluten.extension.columnar.enumerated.planner.cost.{LegacyCoster, LongCostModel} +import org.apache.gluten.extension.columnar.enumerated.planner.cost.{GlutenCostModel, LegacyCoster, LongCostModel} import org.apache.gluten.extension.columnar.enumerated.planner.property.Conv -import org.apache.gluten.extension.columnar.transition.ConventionReq -import org.apache.gluten.ras.{Cost, CostModel, Ras} +import org.apache.gluten.extension.columnar.transition.{Convention, ConventionReq} +import org.apache.gluten.ras.{Cost, Ras} import org.apache.gluten.ras.RasSuiteBase._ import org.apache.gluten.ras.path.RasPath import org.apache.gluten.ras.property.PropertySet @@ -37,6 +37,11 @@ import org.apache.spark.sql.types.StringType class VeloxRasSuite extends SharedSparkSession { import VeloxRasSuite._ + override protected def beforeAll(): Unit = { + super.beforeAll() + Convention.ensureSparkRowAndBatchTypesRegistered() + } + test("C2R, R2C - basic") { val in = RowUnary(RowLeaf(TRIVIAL_SCHEMA)) val planner = newRas().newPlanner(in) @@ -153,14 +158,14 @@ object VeloxRasSuite { .asInstanceOf[Ras[SparkPlan]] } - private def legacyCostModel(): CostModel[SparkPlan] = { + private def legacyCostModel(): GlutenCostModel = { val registry = LongCostModel.registry() val coster = LegacyCoster registry.register(coster) registry.get(coster.kind()) } - private def sessionCostModel(): CostModel[SparkPlan] = { + private def sessionCostModel(): GlutenCostModel = { val transform = EnumeratedTransform.static() transform.costModel } @@ -198,7 +203,7 @@ object VeloxRasSuite { override def shape(): Shape[SparkPlan] = Shapes.fixedHeight(1) } - class UserCostModel1 extends CostModel[SparkPlan] { + class UserCostModel1 extends GlutenCostModel { private val base = legacyCostModel() override def costOf(node: SparkPlan): Cost = node match { case _: RowUnary => base.makeInfCost() @@ -206,9 +211,12 @@ object VeloxRasSuite { } override def costComparator(): Ordering[Cost] = base.costComparator() override def makeInfCost(): Cost = base.makeInfCost() + override def sum(one: Cost, other: Cost): Cost = base.sum(one, other) + override def diff(one: Cost, other: Cost): Cost = base.diff(one, other) + override def makeZeroCost(): Cost = base.makeZeroCost() } - class UserCostModel2 extends CostModel[SparkPlan] { + class UserCostModel2 extends GlutenCostModel { private val base = legacyCostModel() override def costOf(node: SparkPlan): Cost = node match { case _: ColumnarUnary => base.makeInfCost() @@ -216,5 +224,8 @@ object VeloxRasSuite { } override def costComparator(): Ordering[Cost] = base.costComparator() override def makeInfCost(): Cost = base.makeInfCost() + override def sum(one: Cost, other: Cost): Cost = base.sum(one, other) + override def diff(one: Cost, other: Cost): Cost = base.diff(one, other) + override def makeZeroCost(): Cost = base.makeZeroCost() } } diff --git a/backends-velox/src/test/scala/org/apache/gluten/extension/columnar/transition/VeloxTransitionSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/extension/columnar/transition/VeloxTransitionSuite.scala index e14ffd43d82d..335844782a44 100644 --- a/backends-velox/src/test/scala/org/apache/gluten/extension/columnar/transition/VeloxTransitionSuite.scala +++ b/backends-velox/src/test/scala/org/apache/gluten/extension/columnar/transition/VeloxTransitionSuite.scala @@ -200,13 +200,13 @@ class VeloxTransitionSuite extends SharedSparkSession { } override protected def beforeAll(): Unit = { - api.onExecutorStart(MockVeloxBackend.mockPluginContext()) super.beforeAll() + api.onExecutorStart(MockVeloxBackend.mockPluginContext()) } override protected def afterAll(): Unit = { - super.afterAll() api.onExecutorShutdown() + super.afterAll() } } diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedTransform.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedTransform.scala index 34b4005a756d..59e829e17936 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedTransform.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedTransform.scala @@ -20,11 +20,11 @@ import org.apache.gluten.component.Component import org.apache.gluten.exception.GlutenException import org.apache.gluten.extension.columnar.ColumnarRuleApplier.ColumnarRuleCall import org.apache.gluten.extension.columnar.enumerated.planner.GlutenOptimization +import org.apache.gluten.extension.columnar.enumerated.planner.cost.GlutenCostModel import org.apache.gluten.extension.columnar.enumerated.planner.property.Conv import org.apache.gluten.extension.injector.Injector import org.apache.gluten.extension.util.AdaptiveContext import org.apache.gluten.logging.LogLevelUtil -import org.apache.gluten.ras.CostModel import org.apache.gluten.ras.property.PropertySet import org.apache.gluten.ras.rule.RasRule @@ -44,7 +44,7 @@ import org.apache.spark.sql.execution._ * * The feature requires enabling RAS to function. */ -case class EnumeratedTransform(costModel: CostModel[SparkPlan], rules: Seq[RasRule[SparkPlan]]) +case class EnumeratedTransform(costModel: GlutenCostModel, rules: Seq[RasRule[SparkPlan]]) extends Rule[SparkPlan] with LogLevelUtil { diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/GlutenCostModel.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/GlutenCostModel.scala new file mode 100644 index 000000000000..41e5529d2eba --- /dev/null +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/GlutenCostModel.scala @@ -0,0 +1,30 @@ +/* + * 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.extension.columnar.enumerated.planner.cost + +import org.apache.gluten.ras.{Cost, CostModel} + +import org.apache.spark.sql.execution.SparkPlan + +trait GlutenCostModel extends CostModel[SparkPlan] { + // Returns cost value of one + other. + def sum(one: Cost, other: Cost): Cost + // Returns cost value of one - other. + def diff(one: Cost, other: Cost): Cost + + def makeZeroCost(): Cost +} diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/LongCostModel.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/LongCostModel.scala index 393ac35de42f..0d11541b73dd 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/LongCostModel.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/LongCostModel.scala @@ -18,34 +18,46 @@ package org.apache.gluten.extension.columnar.enumerated.planner.cost import org.apache.gluten.exception.GlutenException import org.apache.gluten.extension.columnar.enumerated.planner.plan.GlutenPlanModel.GroupLeafExec -import org.apache.gluten.ras.{Cost, CostModel} +import org.apache.gluten.ras.Cost import org.apache.spark.internal.Logging import org.apache.spark.sql.execution.SparkPlan import scala.collection.mutable -abstract class LongCostModel extends CostModel[SparkPlan] { +abstract class LongCostModel extends GlutenCostModel { private val infLongCost = Long.MaxValue + private val zeroLongCost = 0 override def costOf(node: SparkPlan): LongCost = node match { case _: GroupLeafExec => throw new IllegalStateException() case _ => LongCost(longCostOf(node)) } + // Sum with ceil to avoid overflow. + private def safeSum(a: Long, b: Long): Long = { + assert(a >= 0) + assert(b >= 0) + val sum = a + b + if (sum < a || sum < b) Long.MaxValue else sum + } + + override def sum(one: Cost, other: Cost): LongCost = (one, other) match { + case (LongCost(value), LongCost(otherValue)) => LongCost(safeSum(value, otherValue)) + } + + // Returns cost value of one - other. + override def diff(one: Cost, other: Cost): Cost = (one, other) match { + case (LongCost(value), LongCost(otherValue)) => + val d = Math.subtractExact(value, otherValue) + require(d >= zeroLongCost, s"Difference between cost $one and $other should not be negative") + LongCost(d) + } + private def longCostOf(node: SparkPlan): Long = node match { case n => val selfCost = selfLongCostOf(n) - - // Sum with ceil to avoid overflow. - def safeSum(a: Long, b: Long): Long = { - assert(a >= 0) - assert(b >= 0) - val sum = a + b - if (sum < a || sum < b) Long.MaxValue else sum - } - - (n.children.map(longCostOf).toList :+ selfCost).reduce(safeSum) + (n.children.map(longCostOf).toSeq :+ selfCost).reduce[Long](safeSum) } def selfLongCostOf(node: SparkPlan): Long @@ -56,6 +68,7 @@ abstract class LongCostModel extends CostModel[SparkPlan] { } override def makeInfCost(): Cost = LongCost(infLongCost) + override def makeZeroCost(): Cost = LongCost(zeroLongCost) } object LongCostModel extends Logging { @@ -98,11 +111,6 @@ object LongCostModel extends Logging { override def name(): String = "rough" } - /** Compared with rough, rough2 can be more precise to avoid the costly r2c. */ - case object Rough2 extends Kind { - override def name(): String = "rough2" - } - class Registry private[LongCostModel] { private val lookup: mutable.Map[Kind, LongCosterChain.Builder] = mutable.Map() diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/LongCosterChain.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/LongCosterChain.scala index 8b0c8b9f2d8a..00980e7712a4 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/LongCosterChain.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/LongCosterChain.scala @@ -37,7 +37,7 @@ private class LongCosterChain private (costers: Seq[LongCoster]) extends LongCos case (c @ Some(_), _) => c } - .getOrElse(throw new GlutenException(s"Cost node found for node: $node")) + .getOrElse(throw new GlutenException(s"Cost not found for node: $node")) } } diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/Convention.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/Convention.scala index 0e5387559674..ff0f29585299 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/Convention.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/Convention.scala @@ -19,8 +19,6 @@ package org.apache.gluten.extension.columnar.transition import org.apache.spark.sql.execution.{ColumnarToRowExec, RowToColumnarExec, SparkPlan} import org.apache.spark.util.SparkVersionUtil -import java.util.concurrent.atomic.AtomicBoolean - import scala.collection.mutable /** @@ -33,6 +31,13 @@ sealed trait Convention { } object Convention { + def ensureSparkRowAndBatchTypesRegistered(): Unit = { + RowType.None.ensureRegistered() + RowType.VanillaRow.ensureRegistered() + BatchType.None.ensureRegistered() + BatchType.VanillaBatch.ensureRegistered() + } + implicit class ConventionOps(val conv: Convention) extends AnyVal { def isNone: Boolean = { conv.rowType == RowType.None && conv.batchType == BatchType.None @@ -80,10 +85,17 @@ object Convention { } sealed trait RowType extends TransitionGraph.Vertex with Serializable { - Transition.graph.addVertex(this) + import RowType._ + + final protected[this] def register0(): Unit = BatchType.synchronized { + assert(all.add(this)) + } } object RowType { + private val all: mutable.Set[RowType] = mutable.Set() + def values(): Set[RowType] = all.toSet + // None indicates that the plan doesn't support row-based processing. final case object None extends RowType final case object VanillaRow extends RowType @@ -91,24 +103,12 @@ object Convention { trait BatchType extends TransitionGraph.Vertex with Serializable { import BatchType._ - private val initialized: AtomicBoolean = new AtomicBoolean(false) - final def ensureRegistered(): Unit = { - if (!initialized.compareAndSet(false, true)) { - // Already registered. - return - } - register() - } - - final private def register(): Unit = BatchType.synchronized { + final protected[this] def register0(): Unit = BatchType.synchronized { assert(all.add(this)) - Transition.graph.addVertex(this) registerTransitions() } - ensureRegistered() - /** * User batch type could override this method to define transitions from/to this batch type by * calling the subsequent protected APIs. diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/ConventionFunc.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/ConventionFunc.scala index c4405aeb8d0a..3105713d989d 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/ConventionFunc.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/ConventionFunc.scala @@ -24,7 +24,6 @@ import org.apache.spark.sql.execution.{ColumnarToRowExec, SparkPlan, UnionExec} import org.apache.spark.sql.execution.adaptive.QueryStageExec import org.apache.spark.sql.execution.command.DataWritingCommandExec import org.apache.spark.sql.execution.exchange.ReusedExchangeExec -import org.apache.spark.util.SparkTestUtil /** ConventionFunc is a utility to derive [[Convention]] or [[ConventionReq]] from a query plan. */ sealed trait ConventionFunc { @@ -43,33 +42,12 @@ object ConventionFunc { object Empty extends Override } - // For testing, to make things work without a backend loaded. - private var ignoreBackend: Boolean = false - - // Visible for testing. - def ignoreBackend[T](body: => T): T = synchronized { - assert(SparkTestUtil.isTesting) - assert(!ignoreBackend) - ignoreBackend = true - try { - body - } finally { - ignoreBackend = false - } - } - def create(): ConventionFunc = { val batchOverride = newOverride() new BuiltinFunc(batchOverride) } private def newOverride(): Override = { - synchronized { - if (ignoreBackend) { - // For testing - return Override.Empty - } - } // Components should override Backend's convention function. Hence, reversed injection order // is applied. val overrides = Component.sorted().reverse.map(_.convFuncOverride()) diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/FloydWarshallGraph.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/FloydWarshallGraph.scala index 2a4e1f422517..b05e93968711 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/FloydWarshallGraph.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/FloydWarshallGraph.scala @@ -30,31 +30,32 @@ trait FloydWarshallGraph[V <: AnyRef, E <: AnyRef] { } object FloydWarshallGraph { - trait Cost { - def +(other: Cost): Cost - } + trait Cost trait CostModel[E <: AnyRef] { def zero(): Cost + def sum(one: Cost, other: Cost): Cost def costOf(edge: E): Cost def costComparator(): Ordering[Cost] } trait Path[E <: AnyRef] { def edges(): Seq[E] - def cost(): Cost + def cost(costModel: CostModel[E]): Cost } - def builder[V <: AnyRef, E <: AnyRef](costModel: CostModel[E]): Builder[V, E] = { - Builder.create(costModel) + def builder[V <: AnyRef, E <: AnyRef](costModelFactory: () => CostModel[E]): Builder[V, E] = { + Builder.create(costModelFactory) } private object Path { - def apply[E <: AnyRef](costModel: CostModel[E], edges: Seq[E]): Path[E] = Impl(edges)(costModel) - private case class Impl[E <: AnyRef](override val edges: Seq[E])(costModel: CostModel[E]) - extends Path[E] { - override val cost: Cost = { - edges.map(costModel.costOf).reduceOption(_ + _).getOrElse(costModel.zero()) + def apply[E <: AnyRef](edges: Seq[E]): Path[E] = Impl(edges) + private case class Impl[E <: AnyRef](override val edges: Seq[E]) extends Path[E] { + override def cost(costModel: CostModel[E]): Cost = { + edges + .map(costModel.costOf) + .reduceOption((c1, c2) => costModel.sum(c1, c2)) + .getOrElse(costModel.zero()) } } } @@ -87,13 +88,14 @@ object FloydWarshallGraph { private object Builder { // Thread safe. - private class Impl[V <: AnyRef, E <: AnyRef](costModel: CostModel[E]) extends Builder[V, E] { + private class Impl[V <: AnyRef, E <: AnyRef](costModelFactory: () => CostModel[E]) + extends Builder[V, E] { private val pathTable: mutable.Map[V, mutable.Map[V, Path[E]]] = mutable.Map() private var graph: Option[FloydWarshallGraph[V, E]] = None override def addVertex(v: V): Builder[V, E] = synchronized { assert(!pathTable.contains(v), s"Vertex $v already exists in graph") - pathTable.getOrElseUpdate(v, mutable.Map()).getOrElseUpdate(v, Path(costModel, Nil)) + pathTable.getOrElseUpdate(v, mutable.Map()).getOrElseUpdate(v, Path(Nil)) graph = None this } @@ -103,7 +105,7 @@ object FloydWarshallGraph { assert(pathTable.contains(from), s"Vertex $from not exists in graph") assert(pathTable.contains(to), s"Vertex $to not exists in graph") assert(!hasPath(from, to), s"Path from $from to $to already exists in graph") - pathTable(from) += to -> Path(costModel, Seq(edge)) + pathTable(from) += to -> Path(Seq(edge)) graph = None this } @@ -127,6 +129,7 @@ object FloydWarshallGraph { } private def compile(): FloydWarshallGraph[V, E] = { + val costModel = costModelFactory() val vertices = pathTable.keys for (k <- vertices) { for (i <- vertices) { @@ -134,12 +137,16 @@ object FloydWarshallGraph { if (hasPath(i, k) && hasPath(k, j)) { val pathIk = pathTable(i)(k) val pathKj = pathTable(k)(j) - val newPath = Path(costModel, pathIk.edges() ++ pathKj.edges()) + val newPath = Path(pathIk.edges() ++ pathKj.edges()) if (!hasPath(i, j)) { pathTable(i) += j -> newPath } else { val path = pathTable(i)(j) - if (costModel.costComparator().compare(newPath.cost(), path.cost()) < 0) { + if ( + costModel + .costComparator() + .compare(newPath.cost(costModel), path.cost(costModel)) < 0 + ) { pathTable(i) += j -> newPath } } @@ -151,8 +158,8 @@ object FloydWarshallGraph { } } - def create[V <: AnyRef, E <: AnyRef](costModel: CostModel[E]): Builder[V, E] = { - new Impl(costModel) + def create[V <: AnyRef, E <: AnyRef](costModelFactory: () => CostModel[E]): Builder[V, E] = { + new Impl(costModelFactory) } } } diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/TransitionGraph.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/TransitionGraph.scala index ef08a34d5615..8e9744383107 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/TransitionGraph.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/TransitionGraph.scala @@ -16,18 +16,43 @@ */ package org.apache.gluten.extension.columnar.transition +import org.apache.gluten.extension.columnar.enumerated.EnumeratedTransform +import org.apache.gluten.extension.columnar.transition.Convention.BatchType +import org.apache.gluten.ras.Cost + import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.util.SparkReflectionUtil +import java.util.concurrent.atomic.AtomicBoolean + +import scala.collection.mutable + object TransitionGraph { trait Vertex { + private val initialized: AtomicBoolean = new AtomicBoolean(false) + + final def ensureRegistered(): Unit = { + if (!initialized.compareAndSet(false, true)) { + // Already registered. + return + } + register() + } + + final private def register(): Unit = BatchType.synchronized { + Transition.graph.addVertex(this) + register0() + } + + protected[this] def register0(): Unit + override def toString: String = SparkReflectionUtil.getSimpleClassName(this.getClass) } type Builder = FloydWarshallGraph.Builder[TransitionGraph.Vertex, Transition] def builder(): Builder = { - FloydWarshallGraph.builder(TransitionCostModel) + FloydWarshallGraph.builder(() => new TransitionCostModel()) } implicit class TransitionGraphOps(val graph: TransitionGraph) { @@ -67,54 +92,77 @@ object TransitionGraph { } } - private case class TransitionCost(count: Int, nodeNames: Seq[String]) - extends FloydWarshallGraph.Cost { - override def +(other: FloydWarshallGraph.Cost): TransitionCost = { - other match { - case TransitionCost(otherCount, otherNodeNames) => - TransitionCost(count + otherCount, nodeNames ++ otherNodeNames) - } - } - } + /** Reuse RAS cost to represent transition cost. */ + private case class TransitionCost(value: Cost, nodeNames: Seq[String]) + extends FloydWarshallGraph.Cost - // TODO: Consolidate transition graph's cost model with RAS cost model. - private object TransitionCostModel extends FloydWarshallGraph.CostModel[Transition] { - override def zero(): TransitionCost = TransitionCost(0, Nil) + /** + * The cost model reuses RAS's cost model to evaluate cost of transitions. + * + * Note the transition graph is built once for all subsequent Spark sessions created on the same + * driver, so any access to Spark dynamic SQL config in RAS cost model will not take effect for + * the transition cost evaluation. Hence, it's not recommended to access Spark dynamic + * configurations in RAS cost model as well. + */ + private class TransitionCostModel() extends FloydWarshallGraph.CostModel[Transition] { + private val rasCostModel = EnumeratedTransform.static().costModel + + override def zero(): TransitionCost = TransitionCost(rasCostModel.makeZeroCost(), Nil) override def costOf(transition: Transition): TransitionCost = { costOf0(transition) } + override def sum( + one: FloydWarshallGraph.Cost, + other: FloydWarshallGraph.Cost): FloydWarshallGraph.Cost = (one, other) match { + case (TransitionCost(c1, p1), TransitionCost(c2, p2)) => + TransitionCost(rasCostModel.sum(c1, c2), p1 ++ p2) + } override def costComparator(): Ordering[FloydWarshallGraph.Cost] = { (x: FloydWarshallGraph.Cost, y: FloydWarshallGraph.Cost) => (x, y) match { - case (TransitionCost(count, nodeNames), TransitionCost(otherCount, otherNodeNames)) => - if (count != otherCount) { - count - otherCount + case (TransitionCost(v1, nodeNames1), TransitionCost(v2, nodeNames2)) => + val diff = rasCostModel.costComparator().compare(v1, v2) + if (diff != 0) { + diff } else { // To make the output order stable. - nodeNames.mkString.hashCode - otherNodeNames.mkString.hashCode + nodeNames1.mkString.hashCode - nodeNames2.mkString.hashCode } } } private def costOf0(transition: Transition): TransitionCost = { val leaf = DummySparkPlan() + val transited = transition.apply(leaf) /** * The calculation considers C2C's cost as half of C2R / R2C's cost. So query planner prefers * C2C than C2R / R2C. */ - def costOfPlan(plan: SparkPlan): TransitionCost = plan - .map { - case p if p == leaf => TransitionCost(0, Nil) - case node @ RowToColumnarLike(_) => TransitionCost(2, Seq(node.nodeName)) - case node @ ColumnarToRowLike(_) => TransitionCost(2, Seq(node.nodeName)) - case node @ ColumnarToColumnarLike(_) => TransitionCost(1, Seq(node.nodeName)) - } - .reduce((l, r) => l + r) + def rasCostOfPlan(plan: SparkPlan): Cost = rasCostModel.costOf(plan) + def nodeNamesOfPlan(plan: SparkPlan): Seq[String] = { + plan.map(_.nodeName).reverse + } + + val leafCost = rasCostOfPlan(leaf) + val accumulatedCost = rasCostOfPlan(transited) + val costDiff = rasCostModel.diff(accumulatedCost, leafCost) + + val leafNodeNames = nodeNamesOfPlan(leaf) + val accumulatedNodeNames = nodeNamesOfPlan(transited) + require( + accumulatedNodeNames.startsWith(leafNodeNames), + s"Transition should only add unary nodes on the input plan or leave it unchanged. " + + s"Before: $leaf, after: $transited" + ) + val nodeNamesDiff = mutable.ListBuffer[String]() + nodeNamesDiff ++= accumulatedNodeNames + leafNodeNames.foreach(n => assert(nodeNamesDiff.remove(0) == n)) + assert( + nodeNamesDiff.size == accumulatedNodeNames.size - leafNodeNames.size, + s"Dummy leaf node not found in the transited plan: $transited") - val plan = transition.apply(leaf) - val cost = costOfPlan(plan) - cost + TransitionCost(costDiff, nodeNamesDiff.toSeq) } } } diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/injector/GlutenInjector.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/injector/GlutenInjector.scala index 11172a9b3636..23db1c436da8 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/injector/GlutenInjector.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/injector/GlutenInjector.scala @@ -21,9 +21,8 @@ import org.apache.gluten.extension.GlutenColumnarRule import org.apache.gluten.extension.columnar.ColumnarRuleApplier import org.apache.gluten.extension.columnar.ColumnarRuleApplier.ColumnarRuleCall import org.apache.gluten.extension.columnar.enumerated.{EnumeratedApplier, EnumeratedTransform} -import org.apache.gluten.extension.columnar.enumerated.planner.cost.{LongCoster, LongCostModel} +import org.apache.gluten.extension.columnar.enumerated.planner.cost.{GlutenCostModel, LongCoster, LongCostModel} import org.apache.gluten.extension.columnar.heuristic.{HeuristicApplier, HeuristicTransform} -import org.apache.gluten.ras.CostModel import org.apache.gluten.ras.rule.RasRule import org.apache.spark.internal.Logging @@ -149,7 +148,7 @@ object GlutenInjector { private def findCostModel( registry: LongCostModel.Registry, - aliasOrClass: String): CostModel[SparkPlan] = { + aliasOrClass: String): GlutenCostModel = { if (LongCostModel.Kind.values().contains(aliasOrClass)) { val kind = LongCostModel.Kind.values()(aliasOrClass) val model = registry.get(kind) @@ -159,7 +158,7 @@ object GlutenInjector { logInfo(s"Using user cost model: $aliasOrClass") val ctor = clazz.getDeclaredConstructor() ctor.setAccessible(true) - val model: CostModel[SparkPlan] = ctor.newInstance() + val model: GlutenCostModel = ctor.newInstance().asInstanceOf[GlutenCostModel] model } } diff --git a/gluten-core/src/test/scala/org/apache/gluten/extension/columnar/transition/FloydWarshallGraphSuite.scala b/gluten-core/src/test/scala/org/apache/gluten/extension/columnar/transition/FloydWarshallGraphSuite.scala index 6bc4ab804f1d..7b60940a1ae2 100644 --- a/gluten-core/src/test/scala/org/apache/gluten/extension/columnar/transition/FloydWarshallGraphSuite.scala +++ b/gluten-core/src/test/scala/org/apache/gluten/extension/columnar/transition/FloydWarshallGraphSuite.scala @@ -36,7 +36,7 @@ class FloydWarshallGraphSuite extends AnyFunSuite { val e42 = Edge(3) val graph = FloydWarshallGraph - .builder(CostModel) + .builder(() => CostModel) .addVertex(v0) .addVertex(v1) .addVertex(v2) @@ -87,14 +87,15 @@ private object FloydWarshallGraphSuite { } } - private case class LongCost(c: Long) extends FloydWarshallGraph.Cost { - override def +(other: FloydWarshallGraph.Cost): FloydWarshallGraph.Cost = other match { - case LongCost(o) => LongCost(c + o) - } - } + private case class LongCost(c: Long) extends FloydWarshallGraph.Cost private object CostModel extends FloydWarshallGraph.CostModel[Edge] { override def zero(): FloydWarshallGraph.Cost = LongCost(0) + override def sum( + one: FloydWarshallGraph.Cost, + other: FloydWarshallGraph.Cost): FloydWarshallGraph.Cost = { + LongCost(one.asInstanceOf[LongCost].c + other.asInstanceOf[LongCost].c) + } override def costOf(edge: Edge): FloydWarshallGraph.Cost = LongCost(edge.distance * 10) override def costComparator(): Ordering[FloydWarshallGraph.Cost] = Ordering.Long.on { case LongCost(c) => c diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/LegacyCoster.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/LegacyCoster.scala index 5cf9b87f2ac1..bb89d0035bf8 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/LegacyCoster.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/LegacyCoster.scala @@ -19,7 +19,7 @@ package org.apache.gluten.extension.columnar.enumerated.planner.cost import org.apache.gluten.extension.columnar.transition.{ColumnarToColumnarLike, ColumnarToRowLike, RowToColumnarLike} import org.apache.gluten.utils.PlanUtil -import org.apache.spark.sql.execution.{ColumnarToRowExec, ColumnarWriteFilesExec, ProjectExec, RowToColumnarExec, SparkPlan} +import org.apache.spark.sql.execution.{ColumnarWriteFilesExec, ProjectExec, SparkPlan} object LegacyCoster extends LongCoster { override def kind(): LongCostModel.Kind = LongCostModel.Legacy @@ -34,8 +34,6 @@ object LegacyCoster extends LongCoster { private def selfCostOf0(node: SparkPlan): Long = { node match { case ColumnarWriteFilesExec.OnNoopLeafPath(_) => 0 - case ColumnarToRowExec(_) => 10L - case RowToColumnarExec(_) => 10L case ColumnarToRowLike(_) => 10L case RowToColumnarLike(_) => 10L case ColumnarToColumnarLike(_) => 5L diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/RoughCoster.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/RoughCoster.scala index d2959d46a13c..ab893265ec42 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/RoughCoster.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/RoughCoster.scala @@ -42,8 +42,6 @@ object RoughCoster extends LongCoster { // Avoid moving computation back to native when transition has complex types in schema. // Such transitions are observed to be extremely expensive as of now. Long.MaxValue - case ColumnarToRowExec(_) => 10L - case RowToColumnarExec(_) => 10L case ColumnarToRowLike(_) => 10L case RowToColumnarLike(_) => 10L case ColumnarToColumnarLike(_) => 5L diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/RoughCoster2.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/RoughCoster2.scala deleted file mode 100644 index e46274a79f69..000000000000 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/RoughCoster2.scala +++ /dev/null @@ -1,83 +0,0 @@ -/* - * 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.extension.columnar.enumerated.planner.cost - -import org.apache.gluten.config.GlutenConfig -import org.apache.gluten.extension.columnar.transition.{ColumnarToRowLike, RowToColumnarLike} -import org.apache.gluten.utils.PlanUtil - -import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, NamedExpression} -import org.apache.spark.sql.execution._ -import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec -import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanExecBase - -// Since https://github.com/apache/incubator-gluten/pull/7686. -object RoughCoster2 extends LongCoster { - override def kind(): LongCostModel.Kind = LongCostModel.Rough2 - - override def selfCostOf(node: SparkPlan): Option[Long] = { - Some(selfCostOf0(node)) - } - - private def selfCostOf0(node: SparkPlan): Long = { - val sizeFactor = getSizeFactor(node) - val opCost = node match { - case ProjectExec(projectList, _) if projectList.forall(isCheapExpression) => - // Make trivial ProjectExec has the same cost as ProjectExecTransform to reduce unnecessary - // c2r and r2c. - 1L - case ColumnarToRowExec(_) => 1L - case RowToColumnarExec(_) => 1L - case ColumnarToRowLike(_) => 1L - case RowToColumnarLike(_) => - // If sizeBytes is less than the threshold, the cost of RowToColumnarLike is ignored. - if (sizeFactor == 0) 1L else GlutenConfig.get.rasRough2R2cCost - case p if PlanUtil.isGlutenColumnarOp(p) => 1L - case p if PlanUtil.isVanillaColumnarOp(p) => GlutenConfig.get.rasRough2VanillaCost - // Other row ops. Usually a vanilla row op. - case _ => GlutenConfig.get.rasRough2VanillaCost - } - opCost * Math.max(1, sizeFactor) - } - - private def getSizeFactor(plan: SparkPlan): Long = { - // Get the bytes size that the plan needs to consume. - val sizeBytes = plan match { - case _: DataSourceScanExec | _: DataSourceV2ScanExecBase => getStatSizeBytes(plan) - case _: LeafExecNode => 0L - case p => p.children.map(getStatSizeBytes).sum - } - sizeBytes / GlutenConfig.get.rasRough2SizeBytesThreshold - } - - private def getStatSizeBytes(plan: SparkPlan): Long = { - plan match { - case a: AdaptiveSparkPlanExec => getStatSizeBytes(a.inputPlan) - case _ => - plan.logicalLink match { - case Some(logicalPlan) => logicalPlan.stats.sizeInBytes.toLong - case _ => plan.children.map(getStatSizeBytes).sum - } - } - } - - private def isCheapExpression(ne: NamedExpression): Boolean = ne match { - case Alias(_: Attribute, _) => true - case _: Attribute => true - case _ => false - } -} diff --git a/gluten-substrait/src/test/scala/org/apache/gluten/extension/columnar/transition/TransitionSuite.scala b/gluten-substrait/src/test/scala/org/apache/gluten/extension/columnar/transition/TransitionSuite.scala index fec36ac1acfa..2c423783fdcc 100644 --- a/gluten-substrait/src/test/scala/org/apache/gluten/extension/columnar/transition/TransitionSuite.scala +++ b/gluten-substrait/src/test/scala/org/apache/gluten/extension/columnar/transition/TransitionSuite.scala @@ -16,8 +16,12 @@ */ package org.apache.gluten.extension.columnar.transition +import org.apache.gluten.backend.Backend +import org.apache.gluten.component.Component import org.apache.gluten.exception.GlutenException import org.apache.gluten.execution.{ColumnarToColumnarExec, GlutenPlan} +import org.apache.gluten.extension.columnar.enumerated.planner.cost.LegacyCoster +import org.apache.gluten.extension.injector.Injector import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow @@ -28,35 +32,38 @@ import org.apache.spark.sql.vectorized.ColumnarBatch class TransitionSuite extends SharedSparkSession { import TransitionSuite._ + + override protected def beforeAll(): Unit = { + super.beforeAll() + new DummyBackend().ensureRegistered() + Convention.ensureSparkRowAndBatchTypesRegistered() + TypeA.ensureRegistered() + TypeB.ensureRegistered() + TypeC.ensureRegistered() + TypeD.ensureRegistered() + } + test("Trivial C2R") { val in = BatchLeaf(TypeA) - val out = ConventionFunc.ignoreBackend { - Transitions.insert(in, outputsColumnar = false) - } + val out = Transitions.insert(in, outputsColumnar = false) assert(out == BatchToRow(TypeA, BatchLeaf(TypeA))) } test("Insert C2R") { val in = RowUnary(BatchLeaf(TypeA)) - val out = ConventionFunc.ignoreBackend { - Transitions.insert(in, outputsColumnar = false) - } + val out = Transitions.insert(in, outputsColumnar = false) assert(out == RowUnary(BatchToRow(TypeA, BatchLeaf(TypeA)))) } test("Insert R2C") { val in = BatchUnary(TypeA, RowLeaf()) - val out = ConventionFunc.ignoreBackend { - Transitions.insert(in, outputsColumnar = false) - } + val out = Transitions.insert(in, outputsColumnar = false) assert(out == BatchToRow(TypeA, BatchUnary(TypeA, RowToBatch(TypeA, RowLeaf())))) } test("Insert C2R2C") { val in = BatchUnary(TypeA, BatchLeaf(TypeB)) - val out = ConventionFunc.ignoreBackend { - Transitions.insert(in, outputsColumnar = false) - } + val out = Transitions.insert(in, outputsColumnar = false) assert( out == BatchToRow( TypeA, @@ -65,9 +72,7 @@ class TransitionSuite extends SharedSparkSession { test("Insert C2C") { val in = BatchUnary(TypeA, BatchLeaf(TypeC)) - val out = ConventionFunc.ignoreBackend { - Transitions.insert(in, outputsColumnar = false) - } + val out = Transitions.insert(in, outputsColumnar = false) assert( out == BatchToRow( TypeA, @@ -77,9 +82,7 @@ class TransitionSuite extends SharedSparkSession { test("No transitions found") { val in = BatchUnary(TypeA, BatchLeaf(TypeD)) assertThrows[GlutenException] { - ConventionFunc.ignoreBackend { - Transitions.insert(in, outputsColumnar = false) - } + Transitions.insert(in, outputsColumnar = false) } } } @@ -145,4 +148,12 @@ object TransitionSuite extends TransitionSuiteBase { throw new UnsupportedOperationException() } + class DummyBackend extends Backend { + override def name(): String = "dummy-backend" + override def buildInfo(): Component.BuildInfo = + Component.BuildInfo("DUMMY_BACKEND", "N/A", "N/A", "N/A") + override def injectRules(injector: Injector): Unit = { + injector.gluten.ras.injectCoster(_ => LegacyCoster) + } + } } diff --git a/shims/common/src/main/scala/org/apache/gluten/config/GlutenConfig.scala b/shims/common/src/main/scala/org/apache/gluten/config/GlutenConfig.scala index d4083d5896eb..eb9071badb14 100644 --- a/shims/common/src/main/scala/org/apache/gluten/config/GlutenConfig.scala +++ b/shims/common/src/main/scala/org/apache/gluten/config/GlutenConfig.scala @@ -295,12 +295,6 @@ class GlutenConfig(conf: SQLConf) extends Logging { def rasCostModel: String = getConf(RAS_COST_MODEL) - def rasRough2SizeBytesThreshold: Long = getConf(RAS_ROUGH2_SIZEBYTES_THRESHOLD) - - def rasRough2R2cCost: Long = getConf(RAS_ROUGH2_R2C_COST) - - def rasRough2VanillaCost: Long = getConf(RAS_ROUGH2_VANILLA_COST) - def enableVeloxCache: Boolean = getConf(COLUMNAR_VELOX_CACHE_ENABLED) def veloxMemCacheSize: Long = getConf(COLUMNAR_VELOX_MEM_CACHE_SIZE) @@ -1459,32 +1453,12 @@ object GlutenConfig { val RAS_COST_MODEL = buildConf("spark.gluten.ras.costModel") .doc( - "Experimental: The class name of user-defined cost model that will be used by RAS. If " + - "not specified, a legacy built-in cost model that exhaustively offloads computations " + - "will be used.") + "The class name of user-defined cost model that will be used by Gluten's transition " + + "planner as well as by RAS. If not specified, a legacy built-in cost model that " + + "exhaustively offloads computations will be used.") .stringConf .createWithDefaultString("legacy") - val RAS_ROUGH2_SIZEBYTES_THRESHOLD = - buildConf("spark.gluten.ras.rough2.sizeBytesThreshold") - .doc( - "Experimental: Threshold of the byte size consumed by sparkPlan, coefficient used " + - "to calculate cost in RAS rough2 model") - .longConf - .createWithDefault(1073741824L) - - val RAS_ROUGH2_R2C_COST = - buildConf("spark.gluten.ras.rough2.r2c.cost") - .doc("Experimental: Cost of RowToVeloxColumnarExec in RAS rough2 model") - .longConf - .createWithDefault(100L) - - val RAS_ROUGH2_VANILLA_COST = - buildConf("spark.gluten.ras.rough2.vanilla.cost") - .doc("Experimental: Cost of vanilla spark operater in RAS rough model") - .longConf - .createWithDefault(20L) - // velox caching options. val COLUMNAR_VELOX_CACHE_ENABLED = buildStaticConf("spark.gluten.sql.columnar.backend.velox.cacheEnabled") From a96e0d2c644be8b211f3ef09b2c9610724a6daa4 Mon Sep 17 00:00:00 2001 From: Yuan Date: Wed, 15 Jan 2025 11:18:00 +0800 Subject: [PATCH 28/28] [GLUTEN-8487][VL] adding JDK17 based Centos8 image (#8513) (#8539) This patch added JDK17 based Centos 8 image --- .github/workflows/docker_image.yml | 11 +++++++++++ dev/docker/Dockerfile.centos8-dynamic-build-jdk17 | 13 +++++++++++++ 2 files changed, 24 insertions(+) create mode 100644 dev/docker/Dockerfile.centos8-dynamic-build-jdk17 diff --git a/.github/workflows/docker_image.yml b/.github/workflows/docker_image.yml index 58b7549adb70..727dc3cd057b 100644 --- a/.github/workflows/docker_image.yml +++ b/.github/workflows/docker_image.yml @@ -23,6 +23,8 @@ on: - '.github/workflows/docker_image.yml' - 'dev/docker/Dockerfile.centos7-static-build' - 'dev/docker/Dockerfile.centos8-dynamic-build' + - 'dev/docker/Dockerfile.centos8-dynamic-build-jdk11' + - 'dev/docker/Dockerfile.centos8-dynamic-build-jdk17' schedule: - cron: '0 20 * * 0' @@ -84,3 +86,12 @@ jobs: file: dev/docker/Dockerfile.centos8-dynamic-build-jdk11 push: true tags: apache/gluten:centos-8-jdk11 + + - name: Build and push Docker image Centos8 + JDK17 + uses: docker/build-push-action@v2 + with: + context: . + file: dev/docker/Dockerfile.centos8-dynamic-build-jdk17 + push: true + tags: apache/gluten:centos-8-jdk17 + diff --git a/dev/docker/Dockerfile.centos8-dynamic-build-jdk17 b/dev/docker/Dockerfile.centos8-dynamic-build-jdk17 new file mode 100644 index 000000000000..e6817a8d605c --- /dev/null +++ b/dev/docker/Dockerfile.centos8-dynamic-build-jdk17 @@ -0,0 +1,13 @@ +FROM apache/gluten:centos-8 + + +RUN yum install -y java-17-openjdk-devel patch wget git perl +ENV JAVA_HOME=/usr/lib/jvm/java-17-openjdk +ENV PATH=$JAVA_HOME/bin:$PATH + +ENV PATH=${PATH}:/usr/lib/maven/bin + +RUN git clone --depth=1 https://github.com/apache/incubator-gluten /opt/gluten + +RUN cd /opt/gluten && source /opt/rh/gcc-toolset-11/enable && ./dev/builddeps-veloxbe.sh --run_setup_script=ON build_arrow && rm -rf /opt/gluten +