diff --git a/.github/workflows/clickhouse_be_trigger.yml b/.github/workflows/clickhouse_be_trigger.yml index 7bd33528c0a4..fbc3eac86393 100644 --- a/.github/workflows/clickhouse_be_trigger.yml +++ b/.github/workflows/clickhouse_be_trigger.yml @@ -48,7 +48,11 @@ jobs: script: | const issueNumber = context.payload.number; let body; - body = "Run Gluten Clickhouse CI on x86"; + if (issueNumber % 10 === 0) { + body = "Run Gluten ClickHouse CI on ARM"; + } else { + body = "Run Gluten Clickhouse CI on x86"; + } await github.rest.issues.createComment({ owner: context.repo.owner, repo: context.repo.repo, diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala index 924f3b31704e..aa9e3e553c17 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala @@ -227,6 +227,7 @@ object CHBackendSettings extends BackendSettingsApi with Logging { format: FileFormat, fields: Array[StructField], bucketSpec: Option[BucketSpec], + isPartitionedTable: Boolean, options: Map[String, String]): ValidationResult = { def validateCompressionCodec(): Option[String] = { diff --git a/backends-velox/src/main/java/org/apache/gluten/columnarbatch/VeloxColumnarBatches.java b/backends-velox/src/main/java/org/apache/gluten/columnarbatch/VeloxColumnarBatches.java index db2d08e31435..33f02be08980 100644 --- a/backends-velox/src/main/java/org/apache/gluten/columnarbatch/VeloxColumnarBatches.java +++ b/backends-velox/src/main/java/org/apache/gluten/columnarbatch/VeloxColumnarBatches.java @@ -17,6 +17,7 @@ package org.apache.gluten.columnarbatch; import org.apache.gluten.backendsapi.BackendsApiManager; +import org.apache.gluten.memory.arrow.alloc.ArrowBufferAllocators; import org.apache.gluten.runtime.Runtime; import org.apache.gluten.runtime.Runtimes; @@ -56,6 +57,7 @@ public static void checkNonVeloxBatch(ColumnarBatch batch) { } public static ColumnarBatch toVeloxBatch(ColumnarBatch input) { + ColumnarBatches.checkOffloaded(input); if (ColumnarBatches.isZeroColumnBatch(input)) { return input; } @@ -86,6 +88,26 @@ public static ColumnarBatch toVeloxBatch(ColumnarBatch input) { return input; } + /** + * Check if a columnar batch is in Velox format. If not, convert it to Velox format then return. + * If already in Velox format, return the batch directly. + * + *

Should only be used for certain conditions when unable to insert explicit to-Velox + * transitions through query planner. + * + *

For example, used by {@link org.apache.spark.sql.execution.ColumnarCachedBatchSerializer} as + * Spark directly calls API ColumnarCachedBatchSerializer#convertColumnarBatchToCachedBatch for + * query plan that returns supportsColumnar=true without generating a cache-write query plan node. + */ + public static ColumnarBatch ensureVeloxBatch(ColumnarBatch input) { + final ColumnarBatch light = + ColumnarBatches.ensureOffloaded(ArrowBufferAllocators.contextInstance(), input); + if (isVeloxBatch(light)) { + return light; + } + return toVeloxBatch(light); + } + /** * Combine multiple columnar batches horizontally, assuming each of them is already offloaded. * Otherwise {@link UnsupportedOperationException} will be thrown. 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 18c9efab39e0..6c51ad484c7a 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 @@ -155,18 +155,12 @@ object VeloxBackendSettings extends BackendSettingsApi { format match { case ParquetReadFormat => - val typeValidator: PartialFunction[StructField, String] = { - // Parquet timestamp is not fully supported yet - case StructField(_, TimestampType, _, _) - if GlutenConfig.get.forceParquetTimestampTypeScanFallbackEnabled => - "TimestampType(force fallback)" - } val parquetOptions = new ParquetOptions(CaseInsensitiveMap(properties), SQLConf.get) if (parquetOptions.mergeSchema) { // https://github.com/apache/incubator-gluten/issues/7174 Some(s"not support when merge schema is true") } else { - validateTypes(typeValidator) + None } case DwrfReadFormat => None case OrcReadFormat => @@ -238,6 +232,7 @@ object VeloxBackendSettings extends BackendSettingsApi { format: FileFormat, fields: Array[StructField], bucketSpec: Option[BucketSpec], + isPartitionedTable: Boolean, options: Map[String, String]): ValidationResult = { // Validate if HiveFileFormat write is supported based on output file type @@ -337,10 +332,17 @@ object VeloxBackendSettings extends BackendSettingsApi { } def validateBucketSpec(): Option[String] = { - if (bucketSpec.nonEmpty) { - Some("Unsupported native write: bucket write is not supported.") - } else { + val isHiveCompatibleBucketTable = bucketSpec.nonEmpty && options + .getOrElse("__hive_compatible_bucketed_table_insertion__", "false") + .equals("true") + // Currently, the velox backend only supports bucketed tables compatible with Hive and + // is limited to partitioned tables. Therefore, we should add this condition restriction. + // After velox supports bucketed non-partitioned tables, we can remove the restriction on + // partitioned tables. + if (bucketSpec.isEmpty || (isHiveCompatibleBucketTable && isPartitionedTable)) { None + } else { + Some("Unsupported native write: non-compatible hive bucket write is not supported.") } } 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 45a30f8cf30a..5d75521b8473 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 @@ -67,10 +67,11 @@ class VeloxListenerApi extends ListenerApi with Logging { ByteUnit.BYTE.toMiB(desiredOverheadSize).toString) } val overheadSize: Long = SparkResourceUtil.getMemoryOverheadSize(conf) - if (overheadSize < desiredOverheadSize) { + if (ByteUnit.BYTE.toMiB(overheadSize) < ByteUnit.BYTE.toMiB(desiredOverheadSize)) { logWarning( - s"Memory overhead is set to $overheadSize which is smaller than the recommended size" + - s" $desiredOverheadSize. This may cause OOM.") + s"Memory overhead is set to ${ByteUnit.BYTE.toMiB(overheadSize)}MiB which is smaller than" + + s" the recommended size ${ByteUnit.BYTE.toMiB(desiredOverheadSize)}MiB." + + s" This may cause OOM.") } conf.set(GlutenConfig.GLUTEN_OVERHEAD_SIZE_IN_BYTES_KEY, overheadSize.toString) diff --git a/backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarCachedBatchSerializer.scala b/backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarCachedBatchSerializer.scala index 9982fa36c762..16004737ea7f 100644 --- a/backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarCachedBatchSerializer.scala +++ b/backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarCachedBatchSerializer.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution import org.apache.gluten.backendsapi.BackendsApiManager -import org.apache.gluten.columnarbatch.ColumnarBatches +import org.apache.gluten.columnarbatch.{ColumnarBatches, VeloxColumnarBatches} import org.apache.gluten.config.GlutenConfig import org.apache.gluten.execution.{RowToVeloxColumnarExec, VeloxColumnarToRowExec} import org.apache.gluten.iterator.Iterators @@ -28,7 +28,7 @@ import org.apache.gluten.vectorized.ColumnarBatchSerializerJniWrapper import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.{InternalRow, SQLConfHelper} +import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} import org.apache.spark.sql.columnar.{CachedBatch, CachedBatchSerializer} import org.apache.spark.sql.execution.columnar.DefaultCachedBatchSerializer @@ -76,9 +76,11 @@ case class CachedColumnarBatch( * -> Convert DefaultCachedBatch to InternalRow using vanilla Spark serializer */ // format: on -class ColumnarCachedBatchSerializer extends CachedBatchSerializer with SQLConfHelper with Logging { +class ColumnarCachedBatchSerializer extends CachedBatchSerializer with Logging { private lazy val rowBasedCachedBatchSerializer = new DefaultCachedBatchSerializer + private def glutenConf: GlutenConfig = GlutenConfig.get + private def toStructType(schema: Seq[Attribute]): StructType = { StructType(schema.map(a => StructField(a.name, a.dataType, a.nullable, a.metadata))) } @@ -108,14 +110,14 @@ class ColumnarCachedBatchSerializer extends CachedBatchSerializer with SQLConfHe // `convertColumnarBatchToCachedBatch`, but the inside ColumnarBatch is not arrow-based. // See: `InMemoryRelation.apply()`. // So we should disallow columnar input if using vanilla Spark columnar scan. - val noVanillaSparkColumnarScan = conf.getConf(GlutenConfig.COLUMNAR_FILESCAN_ENABLED) || - !conf.getConf(GlutenConfig.VANILLA_VECTORIZED_READERS_ENABLED) - conf.getConf(GlutenConfig.GLUTEN_ENABLED) && validateSchema( + val noVanillaSparkColumnarScan = glutenConf.getConf(GlutenConfig.COLUMNAR_FILESCAN_ENABLED) || + !glutenConf.getConf(GlutenConfig.VANILLA_VECTORIZED_READERS_ENABLED) + glutenConf.getConf(GlutenConfig.GLUTEN_ENABLED) && validateSchema( schema) && noVanillaSparkColumnarScan } override def supportsColumnarOutput(schema: StructType): Boolean = { - conf.getConf(GlutenConfig.GLUTEN_ENABLED) && validateSchema(schema) + glutenConf.getConf(GlutenConfig.GLUTEN_ENABLED) && validateSchema(schema) } override def convertInternalRowToCachedBatch( @@ -169,11 +171,16 @@ class ColumnarCachedBatchSerializer extends CachedBatchSerializer with SQLConfHe conf: SQLConf): RDD[CachedBatch] = { input.mapPartitions { it => + val veloxBatches = it.map { + /* Native code needs a Velox offloaded batch, making sure to offload + if heavy batch is encountered */ + batch => VeloxColumnarBatches.ensureVeloxBatch(batch) + } new Iterator[CachedBatch] { - override def hasNext: Boolean = it.hasNext + override def hasNext: Boolean = veloxBatches.hasNext override def next(): CachedBatch = { - val batch = it.next() + val batch = veloxBatches.next() val results = ColumnarBatchSerializerJniWrapper .create( diff --git a/backends-velox/src/test/java/org/apache/gluten/test/MockVeloxBackend.java b/backends-velox/src/test/java/org/apache/gluten/test/MockVeloxBackend.java index 912aa09bcb64..efb3903d2324 100644 --- a/backends-velox/src/test/java/org/apache/gluten/test/MockVeloxBackend.java +++ b/backends-velox/src/test/java/org/apache/gluten/test/MockVeloxBackend.java @@ -17,6 +17,7 @@ package org.apache.gluten.test; import org.apache.gluten.config.GlutenConfig; +import org.apache.gluten.config.GlutenConfig$; import com.codahale.metrics.MetricRegistry; import org.apache.spark.SparkConf; @@ -71,7 +72,7 @@ public Object ask(Object message) throws Exception { private static SparkConf newSparkConf() { final SparkConf conf = new SparkConf(); conf.set(GlutenConfig.SPARK_OFFHEAP_SIZE_KEY(), "1g"); - conf.set(GlutenConfig.COLUMNAR_VELOX_CONNECTOR_IO_THREADS(), "0"); + conf.set(GlutenConfig$.MODULE$.COLUMNAR_VELOX_CONNECTOR_IO_THREADS().key(), "0"); return conf; } } diff --git a/backends-velox/src/test/scala/org/apache/gluten/execution/ArrowCsvScanSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/execution/ArrowCsvScanSuite.scala index 374fa543af10..c59936a927c1 100644 --- a/backends-velox/src/test/scala/org/apache/gluten/execution/ArrowCsvScanSuite.scala +++ b/backends-velox/src/test/scala/org/apache/gluten/execution/ArrowCsvScanSuite.scala @@ -67,33 +67,26 @@ class ArrowCsvScanSuiteV2 extends ArrowCsvScanSuite { } } -/** Since https://github.com/apache/incubator-gluten/pull/5850. */ -abstract class ArrowCsvScanSuite extends VeloxWholeStageTransformerSuite { - override protected val resourcePath: String = "N/A" - override protected val fileFormat: String = "N/A" - - protected val rootPath: String = getClass.getResource("/").getPath - - override def beforeAll(): Unit = { - super.beforeAll() - createCsvTables() - } - - override def afterAll(): Unit = { - super.afterAll() - } - +class ArrowCsvScanWithTableCacheSuite extends ArrowCsvScanSuiteBase { override protected def sparkConf: SparkConf = { super.sparkConf - .set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager") - .set("spark.sql.files.maxPartitionBytes", "1g") - .set("spark.sql.shuffle.partitions", "1") - .set("spark.memory.offHeap.size", "2g") - .set("spark.unsafe.exceptionOnMemoryLeak", "true") - .set("spark.sql.autoBroadcastJoinThreshold", "-1") - .set(GlutenConfig.NATIVE_ARROW_READER_ENABLED.key, "true") + .set("spark.sql.sources.useV1SourceList", "csv") + .set(GlutenConfig.COLUMNAR_TABLE_CACHE_ENABLED.key, "true") + } + + /** + * Test for GLUTEN-8453: https://github.com/apache/incubator-gluten/issues/8453. To make sure no + * error is thrown when caching an Arrow Java query plan. + */ + test("csv scan v1 with table cache") { + val df = spark.sql("select * from student") + df.cache() + assert(df.collect().length == 3) } +} +/** Since https://github.com/apache/incubator-gluten/pull/5850. */ +abstract class ArrowCsvScanSuite extends ArrowCsvScanSuiteBase { test("csv scan with option string as null") { val df = runAndCompare("select * from student_option_str")() val plan = df.queryExecution.executedPlan @@ -152,6 +145,33 @@ abstract class ArrowCsvScanSuite extends VeloxWholeStageTransformerSuite { val df = runAndCompare("select count(1) from student")() checkLengthAndPlan(df, 1) } +} + +abstract class ArrowCsvScanSuiteBase extends VeloxWholeStageTransformerSuite { + override protected val resourcePath: String = "N/A" + override protected val fileFormat: String = "N/A" + + protected val rootPath: String = getClass.getResource("/").getPath + + override def beforeAll(): Unit = { + super.beforeAll() + createCsvTables() + } + + override def afterAll(): Unit = { + super.afterAll() + } + + override protected def sparkConf: SparkConf = { + super.sparkConf + .set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager") + .set("spark.sql.files.maxPartitionBytes", "1g") + .set("spark.sql.shuffle.partitions", "1") + .set("spark.memory.offHeap.size", "2g") + .set("spark.unsafe.exceptionOnMemoryLeak", "true") + .set("spark.sql.autoBroadcastJoinThreshold", "-1") + .set(GlutenConfig.NATIVE_ARROW_READER_ENABLED.key, "true") + } private def createCsvTables(): Unit = { spark.read diff --git a/backends-velox/src/test/scala/org/apache/spark/sql/execution/BucketWriteUtils.scala b/backends-velox/src/test/scala/org/apache/spark/sql/execution/BucketWriteUtils.scala new file mode 100644 index 000000000000..a9fe8269e9b9 --- /dev/null +++ b/backends-velox/src/test/scala/org/apache/spark/sql/execution/BucketWriteUtils.scala @@ -0,0 +1,95 @@ +/* + * 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 + +import org.apache.spark.sql.{DataFrame, GlutenQueryTest} +import org.apache.spark.sql.catalyst.expressions.{BitwiseAnd, Expression, HiveHash, Literal, Pmod, UnsafeProjection} +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.test.SQLTestUtils + +import java.io.File + +trait BucketWriteUtils extends GlutenQueryTest with SQLTestUtils { + + def tableDir(table: String): File = { + val identifier = spark.sessionState.sqlParser.parseTableIdentifier(table) + new File(spark.sessionState.catalog.defaultTablePath(identifier)) + } + + protected def testBucketing( + dataDir: File, + source: String = "parquet", + numBuckets: Int, + bucketCols: Seq[String], + sortCols: Seq[String] = Nil, + inputDF: DataFrame, + bucketIdExpression: (Seq[Expression], Int) => Expression, + getBucketIdFromFileName: String => Option[Int]): Unit = { + val allBucketFiles = + dataDir.listFiles().filterNot(f => f.getName.startsWith(".") || f.getName.startsWith("_")) + + for (bucketFile <- allBucketFiles) { + val bucketId = getBucketIdFromFileName(bucketFile.getName).getOrElse { + fail(s"Unable to find the related bucket files.") + } + + // Remove the duplicate columns in bucketCols and sortCols; + // Otherwise, we got analysis errors due to duplicate names + val selectedColumns = (bucketCols ++ sortCols).distinct + // We may lose the type information after write(e.g. json format doesn't keep schema + // information), here we get the types from the original dataframe. + val types = inputDF.select(selectedColumns.map(col): _*).schema.map(_.dataType) + val columns = selectedColumns.zip(types).map { case (colName, dt) => col(colName).cast(dt) } + + // Read the bucket file into a dataframe, so that it's easier to test. + val readBack = spark.read + .format(source) + .load(bucketFile.getAbsolutePath) + .select(columns: _*) + + // If we specified sort columns while writing bucket table, make sure the data in this + // bucket file is already sorted. + if (sortCols.nonEmpty) { + checkAnswer(readBack.sort(sortCols.map(col): _*), readBack.collect()) + } + + // Go through all rows in this bucket file, calculate bucket id according to bucket column + // values, and make sure it equals to the expected bucket id that inferred from file name. + val qe = readBack.select(bucketCols.map(col): _*).queryExecution + val rows = qe.toRdd.map(_.copy()).collect() + val getBucketId = UnsafeProjection.create( + bucketIdExpression(qe.analyzed.output, numBuckets) :: Nil, + qe.analyzed.output) + + for (row <- rows) { + val actualBucketId = getBucketId(row).getInt(0) + assert(actualBucketId == bucketId) + } + } + } + + def bucketIdExpression(expressions: Seq[Expression], numBuckets: Int): Expression = + Pmod(BitwiseAnd(HiveHash(expressions), Literal(Int.MaxValue)), Literal(numBuckets)) + + def getBucketIdFromFileName(fileName: String): Option[Int] = { + val hiveBucketedFileName = """^(\d+)_0_.*$""".r + fileName match { + case hiveBucketedFileName(bucketId) => Some(bucketId.toInt) + case _ => None + } + } +} diff --git a/backends-velox/src/test/scala/org/apache/spark/sql/execution/VeloxParquetWriteForHiveSuite.scala b/backends-velox/src/test/scala/org/apache/spark/sql/execution/VeloxParquetWriteForHiveSuite.scala index 5932f4e5a741..11efdccfcf72 100644 --- a/backends-velox/src/test/scala/org/apache/spark/sql/execution/VeloxParquetWriteForHiveSuite.scala +++ b/backends-velox/src/test/scala/org/apache/spark/sql/execution/VeloxParquetWriteForHiveSuite.scala @@ -20,6 +20,7 @@ import org.apache.spark.SparkConf import org.apache.spark.internal.config import org.apache.spark.internal.config.UI.UI_ENABLED import org.apache.spark.sql.{GlutenQueryTest, Row, SparkSession} +import org.apache.spark.sql.SaveMode import org.apache.spark.sql.catalyst.expressions.CodegenObjectFactoryMode import org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation import org.apache.spark.sql.execution.datasources.FakeRowAdaptor @@ -33,8 +34,14 @@ import org.apache.hadoop.fs.Path import org.apache.parquet.hadoop.ParquetFileReader import org.apache.parquet.hadoop.util.HadoopInputFile -class VeloxParquetWriteForHiveSuite extends GlutenQueryTest with SQLTestUtils { +import java.io.File + +class VeloxParquetWriteForHiveSuite + extends GlutenQueryTest + with SQLTestUtils + with BucketWriteUtils { private var _spark: SparkSession = _ + import testImplicits._ override protected def beforeAll(): Unit = { super.beforeAll() @@ -222,4 +229,105 @@ class VeloxParquetWriteForHiveSuite extends GlutenQueryTest with SQLTestUtils { } } } + + test("Native writer support compatible hive bucket write with dynamic partition") { + if (isSparkVersionGE("3.4")) { + Seq("true", "false").foreach { + enableConvertMetastore => + withSQLConf("spark.sql.hive.convertMetastoreParquet" -> enableConvertMetastore) { + val source = "hive_source_table" + val target = "hive_bucketed_table" + withTable(source, target) { + sql(s""" + |CREATE TABLE IF NOT EXISTS $target (i int, j string) + |PARTITIONED BY(k string) + |CLUSTERED BY (i, j) SORTED BY (i) INTO 8 BUCKETS + |STORED AS PARQUET + """.stripMargin) + + val df = + (0 until 50).map(i => (i % 13, i.toString, i % 5)).toDF("i", "j", "k") + df.write.mode(SaveMode.Overwrite).saveAsTable(source) + + withSQLConf("hive.exec.dynamic.partition.mode" -> "nonstrict") { + checkNativeWrite(s"INSERT INTO $target SELECT * FROM $source", checkNative = true) + } + + for (k <- 0 until 5) { + testBucketing( + new File(tableDir(target), s"k=$k"), + "parquet", + 8, + Seq("i", "j"), + Seq("i"), + df, + bucketIdExpression, + getBucketIdFromFileName) + } + } + } + } + } + } + + test("bucket writer with non-dynamic partition should fallback") { + if (isSparkVersionGE("3.4")) { + Seq("true", "false").foreach { + enableConvertMetastore => + withSQLConf("spark.sql.hive.convertMetastoreParquet" -> enableConvertMetastore) { + val source = "hive_source_table" + val target = "hive_bucketed_table" + withTable(source, target) { + sql(s""" + |CREATE TABLE IF NOT EXISTS $target (i int, j string) + |PARTITIONED BY(k string) + |CLUSTERED BY (i, j) SORTED BY (i) INTO 8 BUCKETS + |STORED AS PARQUET + """.stripMargin) + + val df = + (0 until 50).map(i => (i % 13, i.toString, i % 5)).toDF("i", "j", "k") + df.write.mode(SaveMode.Overwrite).saveAsTable(source) + + // hive relation convert always use dynamic, so it will offload to native. + checkNativeWrite( + s"INSERT INTO $target PARTITION(k='0') SELECT i, j FROM $source", + checkNative = enableConvertMetastore.toBoolean) + val files = tableDir(target) + .listFiles() + .filterNot(f => f.getName.startsWith(".") || f.getName.startsWith("_")) + assert(files.length == 1 && files.head.getName.contains("k=0")) + checkAnswer(spark.table(target).select("i", "j"), df.select("i", "j")) + } + } + } + } + } + + test("bucket writer with non-partition table should fallback") { + if (isSparkVersionGE("3.4")) { + Seq("true", "false").foreach { + enableConvertMetastore => + withSQLConf("spark.sql.hive.convertMetastoreParquet" -> enableConvertMetastore) { + val source = "hive_source_table" + val target = "hive_bucketed_table" + withTable(source, target) { + sql(s""" + |CREATE TABLE IF NOT EXISTS $target (i int, j string) + |CLUSTERED BY (i, j) SORTED BY (i) INTO 8 BUCKETS + |STORED AS PARQUET + """.stripMargin) + + val df = + (0 until 50).map(i => (i % 13, i.toString)).toDF("i", "j") + df.write.mode(SaveMode.Overwrite).saveAsTable(source) + + checkNativeWrite(s"INSERT INTO $target SELECT i, j FROM $source", checkNative = false) + + checkAnswer(spark.table(target), df) + } + } + } + } + } } diff --git a/backends-velox/src/test/scala/org/apache/spark/sql/execution/benchmark/ColumnarTableCacheBenchmark.scala b/backends-velox/src/test/scala/org/apache/spark/sql/execution/benchmark/ColumnarTableCacheBenchmark.scala index 10c0b01ead6c..0cecae47a9a2 100644 --- a/backends-velox/src/test/scala/org/apache/spark/sql/execution/benchmark/ColumnarTableCacheBenchmark.scala +++ b/backends-velox/src/test/scala/org/apache/spark/sql/execution/benchmark/ColumnarTableCacheBenchmark.scala @@ -33,11 +33,16 @@ object ColumnarTableCacheBenchmark extends SqlBasedBenchmark { private def doBenchmark(name: String, cardinality: Long)(f: => Unit): Unit = { val benchmark = new Benchmark(name, cardinality, output = output) - val flag = if (spark.sessionState.conf.getConf(GlutenConfig.COLUMNAR_TABLE_CACHE_ENABLED)) { - "enable" - } else { - "disable" - } + val flag = + if ( + spark.sessionState.conf + .getConfString(GlutenConfig.COLUMNAR_TABLE_CACHE_ENABLED.key) + .toBoolean + ) { + "enable" + } else { + "disable" + } benchmark.addCase(s"$flag columnar table cache", 3)(_ => f) benchmark.run() } diff --git a/cpp-ch/local-engine/Common/ArrayJoinHelper.cpp b/cpp-ch/local-engine/Common/ArrayJoinHelper.cpp index acefad0aea2a..aa88b42db1c6 100644 --- a/cpp-ch/local-engine/Common/ArrayJoinHelper.cpp +++ b/cpp-ch/local-engine/Common/ArrayJoinHelper.cpp @@ -26,6 +26,7 @@ #include #include #include +#include #include namespace DB @@ -110,11 +111,11 @@ addArrayJoinStep(DB::ContextPtr context, DB::QueryPlan & plan, const DB::Actions { /// If generator in generate rel is explode/posexplode, transform arrayJoin function to ARRAY JOIN STEP to apply max_block_size /// which avoids OOM when several lateral view explode/posexplode is used in spark sqls - LOG_DEBUG(logger, "original actions_dag:{}", actions_dag.dumpDAG()); + LOG_TEST(logger, "original actions_dag:\n{}", debug::dumpActionsDAG(actions_dag)); auto splitted_actions_dags = splitActionsDAGInGenerate(actions_dag); - LOG_DEBUG(logger, "actions_dag before arrayJoin:{}", splitted_actions_dags.before_array_join.dumpDAG()); - LOG_DEBUG(logger, "actions_dag during arrayJoin:{}", splitted_actions_dags.array_join.dumpDAG()); - LOG_DEBUG(logger, "actions_dag after arrayJoin:{}", splitted_actions_dags.after_array_join.dumpDAG()); + LOG_TEST(logger, "actions_dag before arrayJoin:\n{}", debug::dumpActionsDAG(splitted_actions_dags.before_array_join)); + LOG_TEST(logger, "actions_dag during arrayJoin:\n{}", debug::dumpActionsDAG(splitted_actions_dags.array_join)); + LOG_TEST(logger, "actions_dag after arrayJoin:\n{}", debug::dumpActionsDAG(splitted_actions_dags.after_array_join)); auto ignore_actions_dag = [](const DB::ActionsDAG & actions_dag_) -> bool { diff --git a/cpp-ch/local-engine/Common/DebugUtils.cpp b/cpp-ch/local-engine/Common/DebugUtils.cpp index 4d3a0f55d280..513a4ee557d2 100644 --- a/cpp-ch/local-engine/Common/DebugUtils.cpp +++ b/cpp-ch/local-engine/Common/DebugUtils.cpp @@ -32,6 +32,8 @@ #include #include #include +#include "Functions/IFunction.h" +#include namespace pb_util = google::protobuf::util; @@ -397,4 +399,74 @@ std::string showString(const DB::Block & block, size_t numRows, size_t truncate, [](const DB::ColumnWithTypeAndName & col) { return std::make_pair(col.name, col.column); }); return Utils::showString(name_and_columns, numRows, truncate, vertical); } + +std::string dumpActionsDAG(const DB::ActionsDAG & dag) +{ + std::stringstream ss; + ss << "digraph ActionsDAG {\n"; + ss << " rankdir=BT;\n"; // Invert the vertical direction + ss << " nodesep=0.1;\n"; // Reduce space between nodes + ss << " ranksep=0.1;\n"; // Reduce space between ranks + ss << " margin=0.1;\n"; // Reduce graph margin + + std::unordered_map node_to_id; + size_t id = 0; + for (const auto & node : dag.getNodes()) + node_to_id[&node] = id++; + + std::unordered_set output_nodes(dag.getOutputs().begin(), dag.getOutputs().end()); + + for (const auto & node : dag.getNodes()) + { + ss << " n" << node_to_id[&node] << " [label=\""; + + ss << "id:" << node_to_id[&node] << "\\l"; + switch (node.type) + { + case DB::ActionsDAG::ActionType::COLUMN: + ss << "column:" + << (node.column && DB::isColumnConst(*node.column) + ? toString(assert_cast(*node.column).getField()) + : "null") + << "\\l"; + break; + case DB::ActionsDAG::ActionType::ALIAS: + ss << "alias" << "\\l"; + break; + case DB::ActionsDAG::ActionType::FUNCTION: + ss << "function: " << (node.function_base ? node.function_base->getName() : "null"); + if (node.is_function_compiled) + ss << " [compiled]"; + ss << "\\l"; + break; + case DB::ActionsDAG::ActionType::ARRAY_JOIN: + ss << "array join" << "\\l"; + break; + case DB::ActionsDAG::ActionType::INPUT: + ss << "input" << "\\l"; + break; + } + + ss << "result type: " << (node.result_type ? node.result_type->getName() : "null") << "\\l"; + + ss << "children:"; + for (const auto * child : node.children) + ss << " " << node_to_id[child]; + ss << "\\l"; + + ss << "\""; + if (output_nodes.contains(&node)) + ss << ", shape=doublecircle"; + + ss << "];\n"; + } + + for (const auto & node : dag.getNodes()) + for (const auto * child : node.children) + ss << " n" << node_to_id[child] << " -> n" << node_to_id[&node] << ";\n"; + + ss << "}\n"; + return ss.str(); +} + } \ No newline at end of file diff --git a/cpp-ch/local-engine/Common/DebugUtils.h b/cpp-ch/local-engine/Common/DebugUtils.h index 850da408fb22..dd29f93d8d31 100644 --- a/cpp-ch/local-engine/Common/DebugUtils.h +++ b/cpp-ch/local-engine/Common/DebugUtils.h @@ -25,6 +25,7 @@ class Message; namespace DB { class QueryPlan; +class ActionsDAG; } namespace debug { @@ -41,5 +42,6 @@ inline std::string verticalShowString(const DB::Block & block, size_t numRows = { return showString(block, numRows, truncate, true); } +std::string dumpActionsDAG(const DB::ActionsDAG & dag); } diff --git a/cpp-ch/local-engine/Functions/SparkFunctionDecimalBinaryArithmetic.h b/cpp-ch/local-engine/Functions/SparkFunctionDecimalBinaryArithmetic.h index cec492e477ba..beef342e0825 100644 --- a/cpp-ch/local-engine/Functions/SparkFunctionDecimalBinaryArithmetic.h +++ b/cpp-ch/local-engine/Functions/SparkFunctionDecimalBinaryArithmetic.h @@ -14,6 +14,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +#pragma once + #include "SparkFunctionDecimalBinaryOperator.h" #include @@ -27,7 +29,6 @@ #include #include #include -#include #if USE_EMBEDDED_COMPILER #include diff --git a/cpp-ch/local-engine/Functions/SparkFunctionDecimalBinaryOperator.h b/cpp-ch/local-engine/Functions/SparkFunctionDecimalBinaryOperator.h index 73feccf2dd3b..fa846a51cf0a 100644 --- a/cpp-ch/local-engine/Functions/SparkFunctionDecimalBinaryOperator.h +++ b/cpp-ch/local-engine/Functions/SparkFunctionDecimalBinaryOperator.h @@ -15,9 +15,15 @@ * limitations under the License. */ #pragma once +#include "config.h" #include +#if USE_EMBEDDED_COMPILER +#include +#include +#endif + #pragma clang diagnostic push #pragma clang diagnostic ignored "-Wbit-int-extension" @@ -27,11 +33,11 @@ using BitUInt128 = unsigned _BitInt(128); using BitInt256 = signed _BitInt(256); using BitUInt256 = unsigned _BitInt(256); #else -// up to version 18, clang supports large _Bitint sizes on x86 and x86-64; +// up to version 18, clang supports large _Bitint sizes on x86 and x86-64; // but on arm and aarch64, they are currently only supported up to 128 bits. // https://stackoverflow.com/questions/78614816/why-am-i-getting-a-256-bit-arithmetic-error-unsigined-bitint-of-bit-sizes-gre - using BitInt256 = Int256; - using BitUInt256 = UInt256; +using BitInt256 = Int256; +using BitUInt256 = UInt256; #endif namespace local_engine diff --git a/cpp/velox/compute/VeloxBackend.cc b/cpp/velox/compute/VeloxBackend.cc index 3beda675398b..328cd724858a 100644 --- a/cpp/velox/compute/VeloxBackend.cc +++ b/cpp/velox/compute/VeloxBackend.cc @@ -280,10 +280,6 @@ void VeloxBackend::initConnector() { ioThreads >= 0, kVeloxIOThreads + " was set to negative number " + std::to_string(ioThreads) + ", this should not happen."); if (ioThreads > 0) { - LOG(WARNING) - << "Velox background IO threads is enabled. Which is highly unrecommended as of now, since it may cause" - << " some unexpected issues like query crash or hanging. Please turn it off if you are unsure about" - << " this option."; ioExecutor_ = std::make_unique(ioThreads); } velox::connector::registerConnector(std::make_shared( diff --git a/cpp/velox/substrait/SubstraitToVeloxPlan.cc b/cpp/velox/substrait/SubstraitToVeloxPlan.cc index dab98379363f..e01d2d89856b 100644 --- a/cpp/velox/substrait/SubstraitToVeloxPlan.cc +++ b/cpp/velox/substrait/SubstraitToVeloxPlan.cc @@ -514,11 +514,12 @@ std::shared_ptr makeLocationHandle( const std::string& targetDirectory, dwio::common::FileFormat fileFormat, common::CompressionKind compression, + const bool& isBucketed, const std::optional& writeDirectory = std::nullopt, const connector::hive::LocationHandle::TableType& tableType = connector::hive::LocationHandle::TableType::kExisting) { std::string targetFileName = ""; - if (fileFormat == dwio::common::FileFormat::PARQUET) { + if (fileFormat == dwio::common::FileFormat::PARQUET && !isBucketed) { targetFileName = fmt::format("gluten-part-{}{}{}", makeUuid(), compressionFileNameSuffix(compression), ".parquet"); } return std::make_shared( @@ -607,6 +608,35 @@ core::PlanNodePtr SubstraitToVeloxPlanConverter::toVeloxPlan(const ::substrait:: } } + std::shared_ptr bucketProperty = nullptr; + if (writeRel.has_bucket_spec()) { + const auto& bucketSpec = writeRel.bucket_spec(); + const auto& numBuckets = bucketSpec.num_buckets(); + + std::vector bucketedBy; + for (const auto& name : bucketSpec.bucket_column_names()) { + bucketedBy.emplace_back(name); + } + + std::vector bucketedTypes; + bucketedTypes.reserve(bucketedBy.size()); + std::vector tableColumnTypes = inputType->children(); + for (const auto& name : bucketedBy) { + auto it = std::find(tableColumnNames.begin(), tableColumnNames.end(), name); + VELOX_CHECK(it != tableColumnNames.end(), "Invalid bucket {}", name); + std::size_t index = std::distance(tableColumnNames.begin(), it); + bucketedTypes.emplace_back(tableColumnTypes[index]); + } + + std::vector> sortedBy; + for (const auto& name : bucketSpec.sort_column_names()) { + sortedBy.emplace_back(std::make_shared(name, core::SortOrder{true, true})); + } + + bucketProperty = std::make_shared( + connector::hive::HiveBucketProperty::Kind::kHiveCompatible, numBuckets, bucketedBy, bucketedTypes, sortedBy); + } + std::string writePath; if (writeFilesTempPath_.has_value()) { writePath = writeFilesTempPath_.value(); @@ -652,8 +682,8 @@ core::PlanNodePtr SubstraitToVeloxPlanConverter::toVeloxPlan(const ::substrait:: tableColumnNames, /*inputType->names() clolumn name is different*/ inputType->children(), partitionedKey, - nullptr /*bucketProperty*/, - makeLocationHandle(writePath, fileFormat, compressionCodec), + bucketProperty, + makeLocationHandle(writePath, fileFormat, compressionCodec, bucketProperty != nullptr), fileFormat, compressionCodec)), (!partitionedKey.empty()), diff --git a/cpp/velox/tests/VeloxSubstraitRoundTripTest.cc b/cpp/velox/tests/VeloxSubstraitRoundTripTest.cc index a1cd59c6c05a..de338a3c4385 100644 --- a/cpp/velox/tests/VeloxSubstraitRoundTripTest.cc +++ b/cpp/velox/tests/VeloxSubstraitRoundTripTest.cc @@ -34,6 +34,7 @@ using namespace facebook::velox; using namespace facebook::velox::test; +using namespace facebook::velox::exec; using namespace facebook::velox::exec::test; namespace gluten { diff --git a/docs/developers/SubstraitModifications.md b/docs/developers/SubstraitModifications.md index 24a9c1a2128d..3db2b5869c5e 100644 --- a/docs/developers/SubstraitModifications.md +++ b/docs/developers/SubstraitModifications.md @@ -28,6 +28,7 @@ changed `Unbounded` in `WindowFunction` into `Unbounded_Preceding` and `Unbounde * Added `WriteRel` ([#3690](https://github.com/apache/incubator-gluten/pull/3690)). * Added `TopNRel` ([#5409](https://github.com/apache/incubator-gluten/pull/5409)). * Added `ref` field in window bound `Preceding` and `Following` ([#5626](https://github.com/apache/incubator-gluten/pull/5626)). +* Added `BucketSpec` field in `WriteRel`([#8386](https://github.com/apache/incubator-gluten/pull/8386)) ## Modifications to type.proto diff --git a/docs/developers/velox-backend-build-in-docker.md b/docs/developers/velox-backend-build-in-docker.md index 4820c7cdc779..4d5a32767f61 100755 --- a/docs/developers/velox-backend-build-in-docker.md +++ b/docs/developers/velox-backend-build-in-docker.md @@ -5,17 +5,64 @@ nav_order: 7 parent: Developer Overview --- -Currently, Centos-7/8/9 and Ubuntu 20.04/22.04 are supported to build Gluten Velox backend. Please refer to -`.github/workflows/velox_weekly.yml` to install required tools before the build. +Currently, we have two way to build Gluten, static link or dynamic link. -There are two docker images with almost all dependencies installed, respective for static build and dynamic build. -The according Dockerfiles are respectively `Dockerfile.centos7-static-build` and `Dockerfile.centos8-dynamic-build` -under `dev/docker/`. +# Static link +The static link approach builds all dependency libraries in vcpkg for both Velox and Gluten. It then statically links these libraries into libvelox.so and libgluten.so, enabling the build of Gluten on *any* Linux OS on x86 platforms with 64G memory. However we only verified on Centos-7/8/9 and Ubuntu 20.04/22.04. Please submit an issue if it fails on your OS. -```shell -# For static build on centos-7. -docker pull apache/gluten:vcpkg-centos-7 +Here is the dependency libraries required on target system, they are the essential libraries pre-installed in every Linux OS. +``` +linux-vdso.so.1 +librt.so.1 +libpthread.so.0 +libdl.so.2 +libm.so.6 +libc.so.6 +/lib64/ld-linux-x86-64.so.2 +``` + +The 'dockerfile' to build Gluten jar: + +``` +FROM apache/gluten:vcpkg-centos-7 -# For dynamic build on centos-8. -docker pull apache/gluten:centos-8 (dynamic build) +# Build Gluten Jar +RUN source /opt/rh/devtoolset-11/enable && \ + git clone https://github.com/apache/incubator-gluten.git && \ + cd incubator-gluten && \ + ./dev/builddeps-veloxbe.sh --run_setup_script=OFF --enable_s3=ON --enable_gcs=ON --enable_abfs=ON --enable_vcpkg=ON --build_arrow=OFF && \ + mvn clean package -Pbackends-velox -Pceleborn -Piceberg -Pdelta -Pspark-3.4 -DskipTests +``` +`enable_vcpkg=ON` enables the static link. Vcpkg packages are already pre-installed in the vcpkg-centos-7 image and can be reused automatically. The image is maintained by Gluten community. + +The command builds Gluten jar in 'glutenimage': +``` +docker build -t glutenimage -f dockerfile +``` +The gluten jar can be copied from glutenimage:/incubator-gluten/package/target/gluten-velox-bundle-*.jar + +# Dynamic link +The dynamic link approach needs to install the dependencies libraries. It then dynamically link the .so files into libvelox.so and libgluten.so. Currently, Centos-7/8/9 and + Ubuntu 20.04/22.04 are supported to build Gluten Velox backend dynamically. + +The 'dockerfile' to build Gluten jar: + +``` +FROM apache/gluten:centos-8 + +# Build Gluten Jar +RUN source /opt/rh/devtoolset-11/enable && \ + git clone https://github.com/apache/incubator-gluten.git && \ + cd incubator-gluten && \ + ./dev/builddeps-veloxbe.sh --run_setup_script=ON --enable_hdfs=ON --enable_vcpkg=OFF --build_arrow=OFF && \ + mvn clean package -Pbackends-velox -Pceleborn -Piceberg -Pdelta -Pspark-3.4 -DskipTests && \ + ./dev/build-thirdparty.sh +``` +`enable_vcpkg=OFF` enables the dynamic link. Part of shared libraries are pre-installed in the image. You need to specify `--run_setup_script=ON` to install the rest of them. It then packages all dependency libraries into a jar by `build-thirdparty.sh`. +Please note the image is built based on centos-8. It has risk to build and deploy the jar on other OSes. + +The command builds Gluten jar in 'glutenimage': +``` +docker build -t glutenimage -f dockerfile ``` +The gluten jar can be copied from glutenimage:/incubator-gluten/package/target/gluten-velox-bundle-*.jar and glutenimage:/incubator-gluten/package/target/gluten-thirdparty-lib-*.jar diff --git a/docs/get-started/Velox.md b/docs/get-started/Velox.md index 48bca9a6d31e..81bb88c75aec 100644 --- a/docs/get-started/Velox.md +++ b/docs/get-started/Velox.md @@ -16,8 +16,7 @@ parent: Getting-Started # Prerequisite -Currently, Gluten+Velox backend is only tested on **Ubuntu20.04/Ubuntu22.04/Centos7/Centos8**. -Other kinds of OS support are still in progress. The long term goal is to support several common OS and conda env deployment. +Currently, with static build Gluten+Velox backend supports all the Linux OSes, but is only tested on **Ubuntu20.04/Ubuntu22.04/Centos7/Centos8**. With dynamic build, Gluten+Velox backend support **Ubuntu20.04/Ubuntu22.04/Centos7/Centos8** and their variants. Currently, the officially supported Spark versions are 3.2.2, 3.3.1, 3.4.3 and 3.5.1. @@ -103,20 +102,25 @@ mvn clean package -Pbackends-velox -Pceleborn -Puniffle -Pspark-3.4 -DskipTests mvn clean package -Pbackends-velox -Pceleborn -Puniffle -Pspark-3.5 -DskipTests ``` -Notes: Building Velox may fail caused by OOM. You can prevent this failure by adjusting `NUM_THREADS` (e.g., `export NUM_THREADS=4`) before building Gluten/Velox. +Notes: Building Velox may fail caused by OOM. You can prevent this failure by adjusting `NUM_THREADS` (e.g., `export NUM_THREADS=4`) before building Gluten/Velox. The recommended minimal memory size is 64G. After the above build process, the Jar file will be generated under `package/target/`. +Alternatively you may refer to [build in docker](docs/developers/velox-backend-build-in-docker.md) to build the gluten jar in docker. + ## Dependency library deployment With build option `enable_vcpkg=ON`, all dependency libraries will be statically linked to `libvelox.so` and `libgluten.so` which are packed into the gluten-jar. In this way, only the gluten-jar is needed to add to `spark..extraClassPath` and spark will deploy the jar to each worker node. It's better to build -the static version using a clean docker image without any extra libraries installed. On host with some libraries like jemalloc installed, the script may crash with -odd message. You may need to uninstall those libraries to get a clean host. We strongly recommend user to build Gluten in this way to avoid dependency lacking issue. +the static version using a clean docker image without any extra libraries installed ( [build in docker](docs/developers/velox-backend-build-in-docker.md) ). On host with +some libraries like jemalloc installed, the script may crash with odd message. You may need to uninstall those libraries to get a clean host. We ** strongly recommend ** user to build Gluten in this way to avoid dependency lacking issue. + +With build option `enable_vcpkg=OFF`, not all dependency libraries will be dynamically linked. After building, you need to separately execute `./dev/build-thirdparty.sh` to +pack required shared libraries into another jar named `gluten-thirdparty-lib-$LINUX_OS-$VERSION-$ARCH.jar`. Then you need to add the jar to Spark config `extraClassPath` and +set `spark.gluten.loadLibFromJar=true`. Otherwise, you need to install required shared libraries with ** exactly the same versions ** on each worker node . You may find the +libraries list from the third-party jar. -With build option `enable_vcpkg=OFF`, not all dependency libraries will be statically linked. You need to separately execute `./dev/build-thirdparty.sh` to pack required -shared libraries into another jar named `gluten-thirdparty-lib-$LINUX_OS-$VERSION-$ARCH.jar`. Then you need to add the jar to Spark config `extraClassPath` and set -`spark.gluten.loadLibFromJar=true`. Otherwise, you need to install required shared libraries on each worker node. You may find the libraries list from the third-party jar. +# Remote storage support ## HDFS support @@ -219,6 +223,12 @@ cd /path/to/gluten Currently there are several ways to access S3 in Spark. Please refer [Velox S3](VeloxS3.md) part for more detailed configurations +## GCS support + +Please refer [GCS](VeloxGCS.md) + +# Remote Shuffle Service Support + ## Celeborn support Gluten with velox backend supports [Celeborn](https://github.com/apache/celeborn) as remote shuffle service. Currently, the supported Celeborn versions are `0.3.x`, `0.4.x` and `0.5.x`. @@ -308,6 +318,8 @@ spark.rss.storage.type LOCALFILE_HDFS spark.dynamicAllocation.enabled false ``` +# Datalake Framework Support + ## DeltaLake Support Gluten with velox backend supports [DeltaLake](https://delta.io/) table. @@ -393,7 +405,7 @@ native validation failed due to: in ProjectRel, Scalar function name not registe In the above, the symbol `^` indicates a plan is offloaded to Velox in a stage. In Spark DAG, all such pipelined plans (consecutive plans marked with `^`) are plotted inside an umbrella node named `WholeStageCodegenTransformer` (It's not codegen node. The naming is just for making it well plotted like Spark Whole Stage Codegen). -# Spill (Experimental) +# Spill Velox backend supports spilling-to-disk. @@ -417,251 +429,9 @@ Using the following configuration options to customize spilling: Please check the [VeloxNativeUDF.md](../developers/VeloxNativeUDF.md) for more detailed usage and configurations. -# High-Bandwidth Memory (HBM) support - -Gluten supports allocating memory on HBM. This feature is optional and is disabled by default. It is implemented on top of [Memkind library](http://memkind.github.io/memkind/). You can refer to memkind's [readme](https://github.com/memkind/memkind#memkind) for more details. - -## Build Gluten with HBM - -Gluten will internally build and link to a specific version of Memkind library and [hwloc](https://github.com/open-mpi/hwloc). Other dependencies should be installed on Driver and Worker node first: - -```bash -sudo apt install -y autoconf automake g++ libnuma-dev libtool numactl unzip libdaxctl-dev -``` - -After the set-up, you can now build Gluten with HBM. Below command is used to enable this feature - -```bash -cd /path/to/gluten - -## The script builds four jars for spark 3.2.2, 3.3.1, 3.4.3 and 3.5.1. -./dev/buildbundle-veloxbe.sh --enable_hbm=ON -``` - -## Configure and enable HBM in Spark Application - -At runtime, `MEMKIND_HBW_NODES` enviroment variable is detected for configuring HBM NUMA nodes. For the explaination to this variable, please refer to memkind's manual page. This can be set for all executors through spark conf, e.g. `--conf spark.executorEnv.MEMKIND_HBW_NODES=8-15`. Note that memory allocation fallback is also supported and cannot be turned off. If HBM is unavailable or fills up, the allocator will use default(DDR) memory. - -# Intel® QuickAssist Technology (QAT) support - -Gluten supports using Intel® QuickAssist Technology (QAT) for data compression during Spark Shuffle. It benefits from QAT Hardware-based acceleration on compression/decompression, and uses Gzip as compression format for higher compression ratio to reduce the pressure on disks and network transmission. - -This feature is based on QAT driver library and [QATzip](https://github.com/intel/QATzip) library. Please manually download QAT driver for your system, and follow its README to build and install on all Driver and Worker node: [Intel® QuickAssist Technology Driver for Linux* – HW Version 2.0](https://www.intel.com/content/www/us/en/download/765501/intel-quickassist-technology-driver-for-linux-hw-version-2-0.html?wapkw=quickassist). - -## Software Requirements - -- Download QAT driver for your system, and follow its README to build and install on all Driver and Worker nodes: [Intel® QuickAssist Technology Driver for Linux* – HW Version 2.0](https://www.intel.com/content/www/us/en/download/765501/intel-quickassist-technology-driver-for-linux-hw-version-2-0.html?wapkw=quickassist). -- Below compression libraries need to be installed on all Driver and Worker nodes: - - Zlib* library of version 1.2.7 or higher - - ZSTD* library of version 1.5.4 or higher - - LZ4* library - -## Build Gluten with QAT - -1. Setup ICP_ROOT environment variable to the directory where QAT driver is extracted. This environment variable is required during building Gluten and running Spark applications. It's recommended to put it in .bashrc on Driver and Worker nodes. - -```bash -echo "export ICP_ROOT=/path/to/QAT_driver" >> ~/.bashrc -source ~/.bashrc - -# Also set for root if running as non-root user -sudo su - -echo "export ICP_ROOT=/path/to/QAT_driver" >> ~/.bashrc -exit -``` - -2. **This step is required if your application is running as Non-root user.** - The users must be added to the 'qat' group after QAT drvier is installed. And change the amount of max locked memory for the username that is included in the group name. This can be done by specifying the limit in /etc/security/limits.conf. - -```bash -sudo su - -usermod -aG qat username # need relogin to take effect - -# To set 500MB add a line like this in /etc/security/limits.conf -echo "@qat - memlock 500000" >> /etc/security/limits.conf - -exit -``` - -3. Enable huge page. This step is required to execute each time after system reboot. We recommend using systemctl to manage at system startup. You change the values for "max_huge_pages" and "max_huge_pages_per_process" to make sure there are enough resources for your workload. As for Spark applications, one process matches one executor. Within the executor, every task is allocated a maximum of 5 huge pages. - -```bash -sudo su - - -cat << EOF > /usr/local/bin/qat_startup.sh -#!/bin/bash -echo 1024 > /sys/kernel/mm/hugepages/hugepages-2048kB/nr_hugepages -rmmod usdm_drv -insmod $ICP_ROOT/build/usdm_drv.ko max_huge_pages=1024 max_huge_pages_per_process=32 -EOF - -chmod +x /usr/local/bin/qat_startup.sh - -cat << EOF > /etc/systemd/system/qat_startup.service -[Unit] -Description=Configure QAT - -[Service] -ExecStart=/usr/local/bin/qat_startup.sh - -[Install] -WantedBy=multi-user.target -EOF - -systemctl enable qat_startup.service -systemctl start qat_startup.service # setup immediately -systemctl status qat_startup.service - -exit -``` - -4. After the setup, you are now ready to build Gluten with QAT. Use the command below to enable this feature: - -```bash -cd /path/to/gluten - -## The script builds four jars for spark 3.2.2, 3.3.1, 3.4.3 and 3.5.1. -./dev/buildbundle-veloxbe.sh --enable_qat=ON -``` - -## Enable QAT with Gzip/Zstd for shuffle compression - -1. To offload shuffle compression into QAT, first make sure you have the right QAT configuration file at /etc/4xxx_devX.conf. We provide a [example configuration file](../qat/4x16.conf). This configuration sets up to 4 processes that can bind to 1 QAT, and each process can use up to 16 QAT DC instances. - -```bash -## run as root -## Overwrite QAT configuration file. -cd /etc -for i in {0..7}; do echo "4xxx_dev$i.conf"; done | xargs -i cp -f /path/to/gluten/docs/qat/4x16.conf {} -## Restart QAT after updating configuration files. -adf_ctl restart -``` - -2. Check QAT status and make sure the status is up - -```bash -adf_ctl status -``` - -The output should be like: - -``` -Checking status of all devices. -There is 8 QAT acceleration device(s) in the system: - qat_dev0 - type: 4xxx, inst_id: 0, node_id: 0, bsf: 0000:6b:00.0, #accel: 1 #engines: 9 state: up - qat_dev1 - type: 4xxx, inst_id: 1, node_id: 1, bsf: 0000:70:00.0, #accel: 1 #engines: 9 state: up - qat_dev2 - type: 4xxx, inst_id: 2, node_id: 2, bsf: 0000:75:00.0, #accel: 1 #engines: 9 state: up - qat_dev3 - type: 4xxx, inst_id: 3, node_id: 3, bsf: 0000:7a:00.0, #accel: 1 #engines: 9 state: up - qat_dev4 - type: 4xxx, inst_id: 4, node_id: 4, bsf: 0000:e8:00.0, #accel: 1 #engines: 9 state: up - qat_dev5 - type: 4xxx, inst_id: 5, node_id: 5, bsf: 0000:ed:00.0, #accel: 1 #engines: 9 state: up - qat_dev6 - type: 4xxx, inst_id: 6, node_id: 6, bsf: 0000:f2:00.0, #accel: 1 #engines: 9 state: up - qat_dev7 - type: 4xxx, inst_id: 7, node_id: 7, bsf: 0000:f7:00.0, #accel: 1 #engines: 9 state: up -``` - -3. Extra Gluten configurations are required when starting Spark application - -``` ---conf spark.gluten.sql.columnar.shuffle.codec=gzip # Valid options are gzip and zstd ---conf spark.gluten.sql.columnar.shuffle.codecBackend=qat -``` - -4. You can use below command to check whether QAT is working normally at run-time. The value of fw_counters should continue to increase during shuffle. - -``` -while :; do cat /sys/kernel/debug/qat_4xxx_0000:6b:00.0/fw_counters; sleep 1; done -``` - -## QAT driver references - -**Documentation** - -[README Text Files (README_QAT20.L.1.0.0-00021.txt)](https://downloadmirror.intel.com/765523/README_QAT20.L.1.0.0-00021.txt) - -**Release Notes** - -Check out the [Intel® QuickAssist Technology Software for Linux*](https://www.intel.com/content/www/us/en/content-details/632507/intel-quickassist-technology-intel-qat-software-for-linux-release-notes-hardware-version-2-0.html) - Release Notes for the latest changes in this release. - -**Getting Started Guide** - -Check out the [Intel® QuickAssist Technology Software for Linux*](https://www.intel.com/content/www/us/en/content-details/632506/intel-quickassist-technology-intel-qat-software-for-linux-getting-started-guide-hardware-version-2-0.html) - Getting Started Guide for detailed installation instructions. - -**Programmer's Guide** - -Check out the [Intel® QuickAssist Technology Software for Linux*](https://www.intel.com/content/www/us/en/content-details/743912/intel-quickassist-technology-intel-qat-software-for-linux-programmers-guide-hardware-version-2-0.html) - Programmer's Guide for software usage guidelines. - -For more Intel® QuickAssist Technology resources go to [Intel® QuickAssist Technology (Intel® QAT)](https://developer.intel.com/quickassist) - -# Intel® In-memory Analytics Accelerator (IAA/IAX) support - -Similar to Intel® QAT, Gluten supports using Intel® In-memory Analytics Accelerator (IAA, also called IAX) for data compression during Spark Shuffle. It benefits from IAA Hardware-based acceleration on compression/decompression, and uses Gzip as compression format for higher compression ratio to reduce the pressure on disks and network transmission. - -This feature is based on Intel® [QPL](https://github.com/intel/qpl). - -## Build Gluten with IAA - -Gluten will internally build and link to a specific version of QPL library, but extra environment setup is still required. Please refer to [QPL Installation Guide](https://intel.github.io/qpl/documentation/get_started_docs/installation.html) to install dependencies and configure accelerators. - -**This step is required if your application is running as Non-root user.** -Create a group for the users who have privilege to use IAA, and grant group iaa read/write access to the IAA Work-Queues. - -```bash -sudo groupadd iaa -sudo usermod -aG iaa username # need to relogin -sudo chgrp -R iaa /dev/iax -sudo chmod -R g+rw /dev/iax -``` - -After the set-up, you can now build Gluten with QAT. Below command is used to enable this feature - -```bash -cd /path/to/gluten - -## The script builds four jars for spark 3.2.2, 3.3.1, 3.4.3 and 3.5.1. -./dev/buildbundle-veloxbe.sh --enable_iaa=ON -``` - -## Enable IAA with Gzip Compression for shuffle compression - -1. To enable QAT at run-time, first make sure you have configured the IAA Work-Queues correctly, and the file permissions of /dev/iax/wqX.0 are correct. - -```bash -sudo ls -l /dev/iax -``` - -The output should be like: - -``` -total 0 -crw-rw---- 1 root iaa 509, 0 Apr 5 18:54 wq1.0 -crw-rw---- 1 root iaa 509, 5 Apr 5 18:54 wq11.0 -crw-rw---- 1 root iaa 509, 6 Apr 5 18:54 wq13.0 -crw-rw---- 1 root iaa 509, 7 Apr 5 18:54 wq15.0 -crw-rw---- 1 root iaa 509, 1 Apr 5 18:54 wq3.0 -crw-rw---- 1 root iaa 509, 2 Apr 5 18:54 wq5.0 -crw-rw---- 1 root iaa 509, 3 Apr 5 18:54 wq7.0 -crw-rw---- 1 root iaa 509, 4 Apr 5 18:54 wq9.0 -``` - -2. Extra Gluten configurations are required when starting Spark application - -``` ---conf spark.gluten.sql.columnar.shuffle.codec=gzip ---conf spark.gluten.sql.columnar.shuffle.codecBackend=iaa -``` - -## IAA references - -**Intel® IAA Enabling Guide** - -Check out the [Intel® In-Memory Analytics Accelerator (Intel® IAA) Enabling Guide](https://www.intel.com/content/www/us/en/developer/articles/technical/intel-iaa-enabling-guide.html) - -**Intel® QPL Documentation** - -Check out the [Intel® Query Processing Library (Intel® QPL) Documentation](https://intel.github.io/qpl/index.html) - # Test TPC-H or TPC-DS on Gluten with Velox backend -All TPC-H and TPC-DS queries are supported in Gluten Velox backend. +All TPC-H and TPC-DS queries are supported in Gluten Velox backend. You may refer to the [notebook](../../tools/workload/benchmark_velox) we used to do the performance test. ## Data preparation @@ -724,118 +494,9 @@ Both Parquet and ORC datasets are sf1024. | TPC-H Q6 | 13.6 | 21.6 | 34.9 | | TPC-H Q1 | 26.1 | 76.7 | 84.9 | -# External reference setup - -TO ease your first-hand experience of using Gluten, we have set up an external reference cluster. If you are interested, please contact Weiting.Chen@intel.com. - # Gluten UI -## Gluten event - -Gluten provides two events `GlutenBuildInfoEvent` and `GlutenPlanFallbackEvent`: - -- GlutenBuildInfoEvent, it contains the Gluten build information so that we are able to be aware of the environment when doing some debug. - It includes `Java Version`, `Scala Version`, `GCC Version`, `Gluten Version`, `Spark Version`, `Hadoop Version`, `Gluten Revision`, `Backend`, `Backend Revision`, etc. - -- GlutenPlanFallbackEvent, it contains the fallback information for each query execution. - Note, if the query execution is in AQE, then Gluten will post it for each stage. - -Developers can register `SparkListener` to handle these two Gluten events. - -## SQL tab - -Gluten provides a tab based on Spark UI, named `Gluten SQL / DataFrame` - -![Gluten-UI](../image/gluten-ui.png) - -This tab contains two parts: - -1. The Gluten build information. -2. SQL/Dataframe queries fallback information. - -If you want to disable Gluten UI, add a config when submitting `--conf spark.gluten.ui.enabled=false`. - -## History server - -Gluten UI also supports Spark history server. Add gluten-ui jar into the history server classpath, e.g., $SPARK_HOME/jars, then restart history server. - -## Native plan string - -Gluten supports inject native plan string into Spark explain with formatted mode by setting `--conf spark.gluten.sql.injectNativePlanStringToExplain=true`. -Here is an example, how Gluten show the native plan string. - -``` -(9) WholeStageCodegenTransformer (2) -Input [6]: [c1#0L, c2#1L, c3#2L, c1#3L, c2#4L, c3#5L] -Arguments: false -Native Plan: --- Project[expressions: (n3_6:BIGINT, "n0_0"), (n3_7:BIGINT, "n0_1"), (n3_8:BIGINT, "n0_2"), (n3_9:BIGINT, "n1_0"), (n3_10:BIGINT, "n1_1"), (n3_11:BIGINT, "n1_2")] -> n3_6:BIGINT, n3_7:BIGINT, n3_8:BIGINT, n3_9:BIGINT, n3_10:BIGINT, n3_11:BIGINT - -- HashJoin[INNER n1_1=n0_1] -> n1_0:BIGINT, n1_1:BIGINT, n1_2:BIGINT, n0_0:BIGINT, n0_1:BIGINT, n0_2:BIGINT - -- TableScan[table: hive_table, range filters: [(c2, Filter(IsNotNull, deterministic, null not allowed))]] -> n1_0:BIGINT, n1_1:BIGINT, n1_2:BIGINT - -- ValueStream[] -> n0_0:BIGINT, n0_1:BIGINT, n0_2:BIGINT -``` - -## Native plan with stats - -Gluten supports print native plan with stats to executor system output stream by setting `--conf spark.gluten.sql.debug=true`. -Note that, the plan string with stats is task level which may cause executor log size big. Here is an example, how Gluten show the native plan string with stats. - -``` -I20231121 10:19:42.348845 90094332 WholeStageResultIterator.cc:220] Native Plan with stats for: [Stage: 1 TID: 16] --- Project[expressions: (n3_6:BIGINT, "n0_0"), (n3_7:BIGINT, "n0_1"), (n3_8:BIGINT, "n0_2"), (n3_9:BIGINT, "n1_0"), (n3_10:BIGINT, "n1_1"), (n3_11:BIGINT, "n1_2")] -> n3_6:BIGINT, n3_7:BIGINT, n3_8:BIGINT, n3_9:BIGINT, n3_10:BIGINT, n3_11:BIGINT - Output: 27 rows (3.56KB, 3 batches), Cpu time: 10.58us, Blocked wall time: 0ns, Peak memory: 0B, Memory allocations: 0, Threads: 1 - queuedWallNanos sum: 2.00us, count: 1, min: 2.00us, max: 2.00us - runningAddInputWallNanos sum: 626ns, count: 1, min: 626ns, max: 626ns - runningFinishWallNanos sum: 0ns, count: 1, min: 0ns, max: 0ns - runningGetOutputWallNanos sum: 5.54us, count: 1, min: 5.54us, max: 5.54us - -- HashJoin[INNER n1_1=n0_1] -> n1_0:BIGINT, n1_1:BIGINT, n1_2:BIGINT, n0_0:BIGINT, n0_1:BIGINT, n0_2:BIGINT - Output: 27 rows (3.56KB, 3 batches), Cpu time: 223.00us, Blocked wall time: 0ns, Peak memory: 93.12KB, Memory allocations: 15 - HashBuild: Input: 10 rows (960B, 10 batches), Output: 0 rows (0B, 0 batches), Cpu time: 185.67us, Blocked wall time: 0ns, Peak memory: 68.00KB, Memory allocations: 2, Threads: 1 - distinctKey0 sum: 4, count: 1, min: 4, max: 4 - hashtable.capacity sum: 4, count: 1, min: 4, max: 4 - hashtable.numDistinct sum: 10, count: 1, min: 10, max: 10 - hashtable.numRehashes sum: 1, count: 1, min: 1, max: 1 - queuedWallNanos sum: 0ns, count: 1, min: 0ns, max: 0ns - rangeKey0 sum: 4, count: 1, min: 4, max: 4 - runningAddInputWallNanos sum: 1.27ms, count: 1, min: 1.27ms, max: 1.27ms - runningFinishWallNanos sum: 0ns, count: 1, min: 0ns, max: 0ns - runningGetOutputWallNanos sum: 1.29us, count: 1, min: 1.29us, max: 1.29us - H23/11/21 10:19:42 INFO TaskSetManager: Finished task 3.0 in stage 1.0 (TID 13) in 335 ms on 10.221.97.35 (executor driver) (1/10) -ashProbe: Input: 9 rows (864B, 3 batches), Output: 27 rows (3.56KB, 3 batches), Cpu time: 37.33us, Blocked wall time: 0ns, Peak memory: 25.12KB, Memory allocations: 13, Threads: 1 - dynamicFiltersProduced sum: 1, count: 1, min: 1, max: 1 - queuedWallNanos sum: 0ns, count: 1, min: 0ns, max: 0ns - runningAddInputWallNanos sum: 4.54us, count: 1, min: 4.54us, max: 4.54us - runningFinishWallNanos sum: 83ns, count: 1, min: 83ns, max: 83ns - runningGetOutputWallNanos sum: 29.08us, count: 1, min: 29.08us, max: 29.08us - -- TableScan[table: hive_table, range filters: [(c2, Filter(IsNotNull, deterministic, null not allowed))]] -> n1_0:BIGINT, n1_1:BIGINT, n1_2:BIGINT - Input: 9 rows (864B, 3 batches), Output: 9 rows (864B, 3 batches), Cpu time: 630.75us, Blocked wall time: 0ns, Peak memory: 2.44KB, Memory allocations: 63, Threads: 1, Splits: 3 - dataSourceWallNanos sum: 102.00us, count: 1, min: 102.00us, max: 102.00us - dynamicFiltersAccepted sum: 1, count: 1, min: 1, max: 1 - flattenStringDictionaryValues sum: 0, count: 1, min: 0, max: 0 - ioWaitNanos sum: 312.00us, count: 1, min: 312.00us, max: 312.00us - localReadBytes sum: 0B, count: 1, min: 0B, max: 0B - numLocalRead sum: 0, count: 1, min: 0, max: 0 - numPrefetch sum: 0, count: 1, min: 0, max: 0 - numRamRead sum: 0, count: 1, min: 0, max: 0 - numStorageRead sum: 6, count: 1, min: 6, max: 6 - overreadBytes sum: 0B, count: 1, min: 0B, max: 0B - prefetchBytes sum: 0B, count: 1, min: 0B, max: 0B - queryThreadIoLatency sum: 12, count: 1, min: 12, max: 12 - ramReadBytes sum: 0B, count: 1, min: 0B, max: 0B - runningAddInputWallNanos sum: 0ns, count: 1, min: 0ns, max: 0ns - runningFinishWallNanos sum: 125ns, count: 1, min: 125ns, max: 125ns - runningGetOutputWallNanos sum: 1.07ms, count: 1, min: 1.07ms, max: 1.07ms - skippedSplitBytes sum: 0B, count: 1, min: 0B, max: 0B - skippedSplits sum: 0, count: 1, min: 0, max: 0 - skippedStrides sum: 0, count: 1, min: 0, max: 0 - storageReadBytes sum: 3.44KB, count: 1, min: 3.44KB, max: 3.44KB - totalScanTime sum: 0ns, count: 1, min: 0ns, max: 0ns - -- ValueStream[] -> n0_0:BIGINT, n0_1:BIGINT, n0_2:BIGINT - Input: 0 rows (0B, 0 batches), Output: 10 rows (960B, 10 batches), Cpu time: 1.03ms, Blocked wall time: 0ns, Peak memory: 0B, Memory allocations: 0, Threads: 1 - runningAddInputWallNanos sum: 0ns, count: 1, min: 0ns, max: 0ns - runningFinishWallNanos sum: 54.62us, count: 1, min: 54.62us, max: 54.62us - runningGetOutputWallNanos sum: 1.10ms, count: 1, min: 1.10ms, max: 1.10ms -``` +Please refer [Gluten UI](VeloxGlutenUI.md) # Gluten Implicits @@ -850,3 +511,7 @@ df.fallbackSummary Note that, if AQE is enabled, but the query is not materialized, then it will re-plan the query execution with disabled AQE. It is a workaround to get the final plan, and it may cause the inconsistent results with a materialized query. However, we have no choice. + +# Accelerators + +Please refer [HBM](VeloxHBM.md) [QAT](VeloxQAT.md) [IAA](VeloxIAA.md) for details diff --git a/docs/get-started/VeloxGlutenUI.md b/docs/get-started/VeloxGlutenUI.md new file mode 100644 index 000000000000..6f40e25b9e4c --- /dev/null +++ b/docs/get-started/VeloxGlutenUI.md @@ -0,0 +1,116 @@ +--- +layout: page +title: Gluten with Velox Backend +nav_order: 1 +parent: Getting-Started +--- + + +# Gluten UI + +## Gluten event + +Gluten provides two events `GlutenBuildInfoEvent` and `GlutenPlanFallbackEvent`: + +- GlutenBuildInfoEvent, it contains the Gluten build information so that we are able to be aware of the environment when doing some debug. + It includes `Java Version`, `Scala Version`, `GCC Version`, `Gluten Version`, `Spark Version`, `Hadoop Version`, `Gluten Revision`, `Backend`, `Backend Revision`, etc. + +- GlutenPlanFallbackEvent, it contains the fallback information for each query execution. + Note, if the query execution is in AQE, then Gluten will post it for each stage. + +Developers can register `SparkListener` to handle these two Gluten events. + +## SQL tab + +Gluten provides a tab based on Spark UI, named `Gluten SQL / DataFrame` + +![Gluten-UI](../image/gluten-ui.png) + +This tab contains two parts: + +1. The Gluten build information. +2. SQL/Dataframe queries fallback information. + +If you want to disable Gluten UI, add a config when submitting `--conf spark.gluten.ui.enabled=false`. + +## History server + +Gluten UI also supports Spark history server. Add gluten-ui jar into the history server classpath, e.g., $SPARK_HOME/jars, then restart history server. + +## Native plan string + +Gluten supports inject native plan string into Spark explain with formatted mode by setting `--conf spark.gluten.sql.injectNativePlanStringToExplain=true`. +Here is an example, how Gluten show the native plan string. + +``` +(9) WholeStageCodegenTransformer (2) +Input [6]: [c1#0L, c2#1L, c3#2L, c1#3L, c2#4L, c3#5L] +Arguments: false +Native Plan: +-- Project[expressions: (n3_6:BIGINT, "n0_0"), (n3_7:BIGINT, "n0_1"), (n3_8:BIGINT, "n0_2"), (n3_9:BIGINT, "n1_0"), (n3_10:BIGINT, "n1_1"), (n3_11:BIGINT, "n1_2")] -> n3_6:BIGINT, n3_7:BIGINT, n3_8:BIGINT, n3_9:BIGINT, n3_10:BIGINT, n3_11:BIGINT + -- HashJoin[INNER n1_1=n0_1] -> n1_0:BIGINT, n1_1:BIGINT, n1_2:BIGINT, n0_0:BIGINT, n0_1:BIGINT, n0_2:BIGINT + -- TableScan[table: hive_table, range filters: [(c2, Filter(IsNotNull, deterministic, null not allowed))]] -> n1_0:BIGINT, n1_1:BIGINT, n1_2:BIGINT + -- ValueStream[] -> n0_0:BIGINT, n0_1:BIGINT, n0_2:BIGINT +``` + +## Native plan with stats + +Gluten supports print native plan with stats to executor system output stream by setting `--conf spark.gluten.sql.debug=true`. +Note that, the plan string with stats is task level which may cause executor log size big. Here is an example, how Gluten show the native plan string with stats. + +``` +I20231121 10:19:42.348845 90094332 WholeStageResultIterator.cc:220] Native Plan with stats for: [Stage: 1 TID: 16] +-- Project[expressions: (n3_6:BIGINT, "n0_0"), (n3_7:BIGINT, "n0_1"), (n3_8:BIGINT, "n0_2"), (n3_9:BIGINT, "n1_0"), (n3_10:BIGINT, "n1_1"), (n3_11:BIGINT, "n1_2")] -> n3_6:BIGINT, n3_7:BIGINT, n3_8:BIGINT, n3_9:BIGINT, n3_10:BIGINT, n3_11:BIGINT + Output: 27 rows (3.56KB, 3 batches), Cpu time: 10.58us, Blocked wall time: 0ns, Peak memory: 0B, Memory allocations: 0, Threads: 1 + queuedWallNanos sum: 2.00us, count: 1, min: 2.00us, max: 2.00us + runningAddInputWallNanos sum: 626ns, count: 1, min: 626ns, max: 626ns + runningFinishWallNanos sum: 0ns, count: 1, min: 0ns, max: 0ns + runningGetOutputWallNanos sum: 5.54us, count: 1, min: 5.54us, max: 5.54us + -- HashJoin[INNER n1_1=n0_1] -> n1_0:BIGINT, n1_1:BIGINT, n1_2:BIGINT, n0_0:BIGINT, n0_1:BIGINT, n0_2:BIGINT + Output: 27 rows (3.56KB, 3 batches), Cpu time: 223.00us, Blocked wall time: 0ns, Peak memory: 93.12KB, Memory allocations: 15 + HashBuild: Input: 10 rows (960B, 10 batches), Output: 0 rows (0B, 0 batches), Cpu time: 185.67us, Blocked wall time: 0ns, Peak memory: 68.00KB, Memory allocations: 2, Threads: 1 + distinctKey0 sum: 4, count: 1, min: 4, max: 4 + hashtable.capacity sum: 4, count: 1, min: 4, max: 4 + hashtable.numDistinct sum: 10, count: 1, min: 10, max: 10 + hashtable.numRehashes sum: 1, count: 1, min: 1, max: 1 + queuedWallNanos sum: 0ns, count: 1, min: 0ns, max: 0ns + rangeKey0 sum: 4, count: 1, min: 4, max: 4 + runningAddInputWallNanos sum: 1.27ms, count: 1, min: 1.27ms, max: 1.27ms + runningFinishWallNanos sum: 0ns, count: 1, min: 0ns, max: 0ns + runningGetOutputWallNanos sum: 1.29us, count: 1, min: 1.29us, max: 1.29us + H23/11/21 10:19:42 INFO TaskSetManager: Finished task 3.0 in stage 1.0 (TID 13) in 335 ms on 10.221.97.35 (executor driver) (1/10) +ashProbe: Input: 9 rows (864B, 3 batches), Output: 27 rows (3.56KB, 3 batches), Cpu time: 37.33us, Blocked wall time: 0ns, Peak memory: 25.12KB, Memory allocations: 13, Threads: 1 + dynamicFiltersProduced sum: 1, count: 1, min: 1, max: 1 + queuedWallNanos sum: 0ns, count: 1, min: 0ns, max: 0ns + runningAddInputWallNanos sum: 4.54us, count: 1, min: 4.54us, max: 4.54us + runningFinishWallNanos sum: 83ns, count: 1, min: 83ns, max: 83ns + runningGetOutputWallNanos sum: 29.08us, count: 1, min: 29.08us, max: 29.08us + -- TableScan[table: hive_table, range filters: [(c2, Filter(IsNotNull, deterministic, null not allowed))]] -> n1_0:BIGINT, n1_1:BIGINT, n1_2:BIGINT + Input: 9 rows (864B, 3 batches), Output: 9 rows (864B, 3 batches), Cpu time: 630.75us, Blocked wall time: 0ns, Peak memory: 2.44KB, Memory allocations: 63, Threads: 1, Splits: 3 + dataSourceWallNanos sum: 102.00us, count: 1, min: 102.00us, max: 102.00us + dynamicFiltersAccepted sum: 1, count: 1, min: 1, max: 1 + flattenStringDictionaryValues sum: 0, count: 1, min: 0, max: 0 + ioWaitNanos sum: 312.00us, count: 1, min: 312.00us, max: 312.00us + localReadBytes sum: 0B, count: 1, min: 0B, max: 0B + numLocalRead sum: 0, count: 1, min: 0, max: 0 + numPrefetch sum: 0, count: 1, min: 0, max: 0 + numRamRead sum: 0, count: 1, min: 0, max: 0 + numStorageRead sum: 6, count: 1, min: 6, max: 6 + overreadBytes sum: 0B, count: 1, min: 0B, max: 0B + prefetchBytes sum: 0B, count: 1, min: 0B, max: 0B + queryThreadIoLatency sum: 12, count: 1, min: 12, max: 12 + ramReadBytes sum: 0B, count: 1, min: 0B, max: 0B + runningAddInputWallNanos sum: 0ns, count: 1, min: 0ns, max: 0ns + runningFinishWallNanos sum: 125ns, count: 1, min: 125ns, max: 125ns + runningGetOutputWallNanos sum: 1.07ms, count: 1, min: 1.07ms, max: 1.07ms + skippedSplitBytes sum: 0B, count: 1, min: 0B, max: 0B + skippedSplits sum: 0, count: 1, min: 0, max: 0 + skippedStrides sum: 0, count: 1, min: 0, max: 0 + storageReadBytes sum: 3.44KB, count: 1, min: 3.44KB, max: 3.44KB + totalScanTime sum: 0ns, count: 1, min: 0ns, max: 0ns + -- ValueStream[] -> n0_0:BIGINT, n0_1:BIGINT, n0_2:BIGINT + Input: 0 rows (0B, 0 batches), Output: 10 rows (960B, 10 batches), Cpu time: 1.03ms, Blocked wall time: 0ns, Peak memory: 0B, Memory allocations: 0, Threads: 1 + runningAddInputWallNanos sum: 0ns, count: 1, min: 0ns, max: 0ns + runningFinishWallNanos sum: 54.62us, count: 1, min: 54.62us, max: 54.62us + runningGetOutputWallNanos sum: 1.10ms, count: 1, min: 1.10ms, max: 1.10ms +``` diff --git a/docs/get-started/VeloxHBM.md b/docs/get-started/VeloxHBM.md new file mode 100644 index 000000000000..7056bb2fb076 --- /dev/null +++ b/docs/get-started/VeloxHBM.md @@ -0,0 +1,30 @@ +--- +layout: page +title: HBM Support in Velox Backend +nav_order: 1 +parent: Getting-Started +--- +# High-Bandwidth Memory (HBM) support + +Gluten supports allocating memory on HBM. This feature is optional and is disabled by default. It is implemented on top of [Memkind library](http://memkind.github.io/memkind/). You can refer to memkind's [readme](https://github.com/memkind/memkind#memkind) for more details. + +# Build Gluten with HBM + +Gluten will internally build and link to a specific version of Memkind library and [hwloc](https://github.com/open-mpi/hwloc). Other dependencies should be installed on Driver and Worker node first: + +```bash +sudo apt install -y autoconf automake g++ libnuma-dev libtool numactl unzip libdaxctl-dev +``` + +After the set-up, you can now build Gluten with HBM. Below command is used to enable this feature + +```bash +cd /path/to/gluten + +## The script builds four jars for spark 3.2.2, 3.3.1, 3.4.3 and 3.5.1. +./dev/buildbundle-veloxbe.sh --enable_hbm=ON +``` + +## Configure and enable HBM in Spark Application + +At runtime, `MEMKIND_HBW_NODES` enviroment variable is detected for configuring HBM NUMA nodes. For the explaination to this variable, please refer to memkind's manual page. This can be set for all executors through spark conf, e.g. `--conf spark.executorEnv.MEMKIND_HBW_NODES=8-15`. Note that memory allocation fallback is also supported and cannot be turned off. If HBM is unavailable or fills up, the allocator will use default(DDR) memory. diff --git a/docs/get-started/VeloxIAA.md b/docs/get-started/VeloxIAA.md new file mode 100644 index 000000000000..ffd74bf6c62e --- /dev/null +++ b/docs/get-started/VeloxIAA.md @@ -0,0 +1,75 @@ +--- +layout: page +title: IAA Support in Velox Backend +nav_order: 1 +parent: Getting-Started +--- + + +# Intel® In-memory Analytics Accelerator (IAA/IAX) support + +Similar to Intel® QAT, Gluten supports using Intel® In-memory Analytics Accelerator (IAA, also called IAX) for data compression during Spark Shuffle. It benefits from IAA Hardware-based acceleration on compression/decompression, and uses Gzip as compression format for higher compression ratio to reduce the pressure on disks and network transmission. + +This feature is based on Intel® [QPL](https://github.com/intel/qpl). + +## Build Gluten with IAA + +Gluten will internally build and link to a specific version of QPL library, but extra environment setup is still required. Please refer to [QPL Installation Guide](https://intel.github.io/qpl/documentation/get_started_docs/installation.html) to install dependencies and configure accelerators. + +**This step is required if your application is running as Non-root user.** +Create a group for the users who have privilege to use IAA, and grant group iaa read/write access to the IAA Work-Queues. + +```bash +sudo groupadd iaa +sudo usermod -aG iaa username # need to relogin +sudo chgrp -R iaa /dev/iax +sudo chmod -R g+rw /dev/iax +``` + +After the set-up, you can now build Gluten with QAT. Below command is used to enable this feature + +```bash +cd /path/to/gluten + +## The script builds four jars for spark 3.2.2, 3.3.1, 3.4.3 and 3.5.1. +./dev/buildbundle-veloxbe.sh --enable_iaa=ON +``` + +## Enable IAA with Gzip Compression for shuffle compression + +1. To enable QAT at run-time, first make sure you have configured the IAA Work-Queues correctly, and the file permissions of /dev/iax/wqX.0 are correct. + +```bash +sudo ls -l /dev/iax +``` + +The output should be like: + +``` +total 0 +crw-rw---- 1 root iaa 509, 0 Apr 5 18:54 wq1.0 +crw-rw---- 1 root iaa 509, 5 Apr 5 18:54 wq11.0 +crw-rw---- 1 root iaa 509, 6 Apr 5 18:54 wq13.0 +crw-rw---- 1 root iaa 509, 7 Apr 5 18:54 wq15.0 +crw-rw---- 1 root iaa 509, 1 Apr 5 18:54 wq3.0 +crw-rw---- 1 root iaa 509, 2 Apr 5 18:54 wq5.0 +crw-rw---- 1 root iaa 509, 3 Apr 5 18:54 wq7.0 +crw-rw---- 1 root iaa 509, 4 Apr 5 18:54 wq9.0 +``` + +2. Extra Gluten configurations are required when starting Spark application + +``` +--conf spark.gluten.sql.columnar.shuffle.codec=gzip +--conf spark.gluten.sql.columnar.shuffle.codecBackend=iaa +``` + +## IAA references + +**Intel® IAA Enabling Guide** + +Check out the [Intel® In-Memory Analytics Accelerator (Intel® IAA) Enabling Guide](https://www.intel.com/content/www/us/en/developer/articles/technical/intel-iaa-enabling-guide.html) + +**Intel® QPL Documentation** + +Check out the [Intel® Query Processing Library (Intel® QPL) Documentation](https://intel.github.io/qpl/index.html) diff --git a/docs/get-started/VeloxQAT.md b/docs/get-started/VeloxQAT.md new file mode 100644 index 000000000000..6d33654e59d5 --- /dev/null +++ b/docs/get-started/VeloxQAT.md @@ -0,0 +1,155 @@ +--- +layout: page +title: QAT Support in Velox Backend +nav_order: 1 +parent: Getting-Started +--- + +# Intel® QuickAssist Technology (QAT) support + +Gluten supports using Intel® QuickAssist Technology (QAT) for data compression during Spark Shuffle. It benefits from QAT Hardware-based acceleration on compression/decompression, and uses Gzip as compression format for higher compression ratio to reduce the pressure on disks and network transmission. + +This feature is based on QAT driver library and [QATzip](https://github.com/intel/QATzip) library. Please manually download QAT driver for your system, and follow its README to build and install on all Driver and Worker node: [Intel® QuickAssist Technology Driver for Linux* – HW Version 2.0](https://www.intel.com/content/www/us/en/download/765501/intel-quickassist-technology-driver-for-linux-hw-version-2-0.html?wapkw=quickassist). + +## Software Requirements + +- Download QAT driver for your system, and follow its README to build and install on all Driver and Worker nodes: [Intel® QuickAssist Technology Driver for Linux* – HW Version 2.0](https://www.intel.com/content/www/us/en/download/765501/intel-quickassist-technology-driver-for-linux-hw-version-2-0.html?wapkw=quickassist). +- Below compression libraries need to be installed on all Driver and Worker nodes: + - Zlib* library of version 1.2.7 or higher + - ZSTD* library of version 1.5.4 or higher + - LZ4* library + +## Build Gluten with QAT + +1. Setup ICP_ROOT environment variable to the directory where QAT driver is extracted. This environment variable is required during building Gluten and running Spark applications. It's recommended to put it in .bashrc on Driver and Worker nodes. + +```bash +echo "export ICP_ROOT=/path/to/QAT_driver" >> ~/.bashrc +source ~/.bashrc + +# Also set for root if running as non-root user +sudo su - +echo "export ICP_ROOT=/path/to/QAT_driver" >> ~/.bashrc +exit +``` + +2. **This step is required if your application is running as Non-root user.** + The users must be added to the 'qat' group after QAT drvier is installed. And change the amount of max locked memory for the username that is included in the group name. This can be done by specifying the limit in /etc/security/limits.conf. + +```bash +sudo su - +usermod -aG qat username # need relogin to take effect + +# To set 500MB add a line like this in /etc/security/limits.conf +echo "@qat - memlock 500000" >> /etc/security/limits.conf + +exit +``` + +3. Enable huge page. This step is required to execute each time after system reboot. We recommend using systemctl to manage at system startup. You change the values for "max_huge_pages" and "max_huge_pages_per_process" to make sure there are enough resources for your workload. As for Spark applications, one process matches one executor. Within the executor, every task is allocated a maximum of 5 huge pages. + +```bash +sudo su - + +cat << EOF > /usr/local/bin/qat_startup.sh +#!/bin/bash +echo 1024 > /sys/kernel/mm/hugepages/hugepages-2048kB/nr_hugepages +rmmod usdm_drv +insmod $ICP_ROOT/build/usdm_drv.ko max_huge_pages=1024 max_huge_pages_per_process=32 +EOF + +chmod +x /usr/local/bin/qat_startup.sh + +cat << EOF > /etc/systemd/system/qat_startup.service +[Unit] +Description=Configure QAT + +[Service] +ExecStart=/usr/local/bin/qat_startup.sh + +[Install] +WantedBy=multi-user.target +EOF + +systemctl enable qat_startup.service +systemctl start qat_startup.service # setup immediately +systemctl status qat_startup.service + +exit +``` + +4. After the setup, you are now ready to build Gluten with QAT. Use the command below to enable this feature: + +```bash +cd /path/to/gluten + +## The script builds four jars for spark 3.2.2, 3.3.1, 3.4.3 and 3.5.1. +./dev/buildbundle-veloxbe.sh --enable_qat=ON +``` + +## Enable QAT with Gzip/Zstd for shuffle compression + +1. To offload shuffle compression into QAT, first make sure you have the right QAT configuration file at /etc/4xxx_devX.conf. We provide a [example configuration file](../qat/4x16.conf). This configuration sets up to 4 processes that can bind to 1 QAT, and each process can use up to 16 QAT DC instances. + +```bash +## run as root +## Overwrite QAT configuration file. +cd /etc +for i in {0..7}; do echo "4xxx_dev$i.conf"; done | xargs -i cp -f /path/to/gluten/docs/qat/4x16.conf {} +## Restart QAT after updating configuration files. +adf_ctl restart +``` + +2. Check QAT status and make sure the status is up + +```bash +adf_ctl status +``` + +The output should be like: + +``` +Checking status of all devices. +There is 8 QAT acceleration device(s) in the system: + qat_dev0 - type: 4xxx, inst_id: 0, node_id: 0, bsf: 0000:6b:00.0, #accel: 1 #engines: 9 state: up + qat_dev1 - type: 4xxx, inst_id: 1, node_id: 1, bsf: 0000:70:00.0, #accel: 1 #engines: 9 state: up + qat_dev2 - type: 4xxx, inst_id: 2, node_id: 2, bsf: 0000:75:00.0, #accel: 1 #engines: 9 state: up + qat_dev3 - type: 4xxx, inst_id: 3, node_id: 3, bsf: 0000:7a:00.0, #accel: 1 #engines: 9 state: up + qat_dev4 - type: 4xxx, inst_id: 4, node_id: 4, bsf: 0000:e8:00.0, #accel: 1 #engines: 9 state: up + qat_dev5 - type: 4xxx, inst_id: 5, node_id: 5, bsf: 0000:ed:00.0, #accel: 1 #engines: 9 state: up + qat_dev6 - type: 4xxx, inst_id: 6, node_id: 6, bsf: 0000:f2:00.0, #accel: 1 #engines: 9 state: up + qat_dev7 - type: 4xxx, inst_id: 7, node_id: 7, bsf: 0000:f7:00.0, #accel: 1 #engines: 9 state: up +``` + +3. Extra Gluten configurations are required when starting Spark application + +``` +--conf spark.gluten.sql.columnar.shuffle.codec=gzip # Valid options are gzip and zstd +--conf spark.gluten.sql.columnar.shuffle.codecBackend=qat +``` + +4. You can use below command to check whether QAT is working normally at run-time. The value of fw_counters should continue to increase during shuffle. + +``` +while :; do cat /sys/kernel/debug/qat_4xxx_0000:6b:00.0/fw_counters; sleep 1; done +``` + +## QAT driver references + +**Documentation** + +[README Text Files (README_QAT20.L.1.0.0-00021.txt)](https://downloadmirror.intel.com/765523/README_QAT20.L.1.0.0-00021.txt) + +**Release Notes** + +Check out the [Intel® QuickAssist Technology Software for Linux*](https://www.intel.com/content/www/us/en/content-details/632507/intel-quickassist-technology-intel-qat-software-for-linux-release-notes-hardware-version-2-0.html) - Release Notes for the latest changes in this release. + +**Getting Started Guide** + +Check out the [Intel® QuickAssist Technology Software for Linux*](https://www.intel.com/content/www/us/en/content-details/632506/intel-quickassist-technology-intel-qat-software-for-linux-getting-started-guide-hardware-version-2-0.html) - Getting Started Guide for detailed installation instructions. + +**Programmer's Guide** + +Check out the [Intel® QuickAssist Technology Software for Linux*](https://www.intel.com/content/www/us/en/content-details/743912/intel-quickassist-technology-intel-qat-software-for-linux-programmers-guide-hardware-version-2-0.html) - Programmer's Guide for software usage guidelines. + +For more Intel® QuickAssist Technology resources go to [Intel® QuickAssist Technology (Intel® QAT)](https://developer.intel.com/quickassist) diff --git a/gluten-arrow/src/main/java/org/apache/gluten/columnarbatch/ColumnarBatches.java b/gluten-arrow/src/main/java/org/apache/gluten/columnarbatch/ColumnarBatches.java index 3914fb155ec4..156de4e0d84d 100644 --- a/gluten-arrow/src/main/java/org/apache/gluten/columnarbatch/ColumnarBatches.java +++ b/gluten-arrow/src/main/java/org/apache/gluten/columnarbatch/ColumnarBatches.java @@ -130,7 +130,7 @@ public static ColumnarBatch select(String backendName, ColumnarBatch batch, int[ * Ensure the input batch is offloaded as native-based columnar batch (See {@link IndicatorVector} * and {@link PlaceholderVector}). This method will close the input column batch after offloaded. */ - private static ColumnarBatch ensureOffloaded(BufferAllocator allocator, ColumnarBatch batch) { + static ColumnarBatch ensureOffloaded(BufferAllocator allocator, ColumnarBatch batch) { if (ColumnarBatches.isLightBatch(batch)) { return batch; } @@ -142,7 +142,7 @@ private static ColumnarBatch ensureOffloaded(BufferAllocator allocator, Columnar * take place if loading is required, which means when the input batch is not loaded yet. This * method will close the input column batch after loaded. */ - private static ColumnarBatch ensureLoaded(BufferAllocator allocator, ColumnarBatch batch) { + static ColumnarBatch ensureLoaded(BufferAllocator allocator, ColumnarBatch batch) { if (isHeavyBatch(batch)) { return batch; } @@ -230,7 +230,8 @@ public static ColumnarBatch offload(BufferAllocator allocator, ColumnarBatch inp if (input.numCols() == 0) { throw new IllegalArgumentException("batch with zero columns cannot be offloaded"); } - // Batch-offloading doesn't involve any backend-specific native code. Use the internal + // Batch-offloading doesn't involve any backend-specific native code. Use the + // internal // backend to store native batch references only. final Runtime runtime = Runtimes.contextInstance(INTERNAL_BACKEND_KIND, "ColumnarBatches#offload"); diff --git a/gluten-core/src/main/scala/org/apache/gluten/component/package.scala b/gluten-core/src/main/scala/org/apache/gluten/component/package.scala index 032a32d04121..cf0181c39cf1 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/component/package.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/component/package.scala @@ -37,6 +37,12 @@ package object component extends Logging { // Output log so user could view the component loading order. // Call #sortedUnsafe than on #sorted to avoid unnecessary recursion. val components = Component.sortedUnsafe() + require( + components.nonEmpty, + s"No component files found in container directories named with " + + s"'META-INF/gluten-components' from classpath. JVM classpath value " + + s"is: ${System.getProperty("java.class.path")}" + ) logInfo(s"Components registered within order: ${components.map(_.name()).mkString(", ")}") } } diff --git a/gluten-substrait/src/main/java/org/apache/gluten/substrait/rel/RelBuilder.java b/gluten-substrait/src/main/java/org/apache/gluten/substrait/rel/RelBuilder.java index 7d1931180847..86b27353183c 100644 --- a/gluten-substrait/src/main/java/org/apache/gluten/substrait/rel/RelBuilder.java +++ b/gluten-substrait/src/main/java/org/apache/gluten/substrait/rel/RelBuilder.java @@ -25,10 +25,7 @@ import org.apache.gluten.substrait.type.ColumnTypeNode; import org.apache.gluten.substrait.type.TypeNode; -import io.substrait.proto.CrossRel; -import io.substrait.proto.JoinRel; -import io.substrait.proto.SetRel; -import io.substrait.proto.SortField; +import io.substrait.proto.*; import org.apache.spark.sql.catalyst.expressions.Attribute; import java.util.List; @@ -191,10 +188,11 @@ public static RelNode makeWriteRel( List names, List columnTypeNodes, AdvancedExtensionNode extensionNode, + WriteRel.BucketSpec bucketSpec, SubstraitContext context, Long operatorId) { context.registerRelToOperator(operatorId); - return new WriteRelNode(input, types, names, columnTypeNodes, extensionNode); + return new WriteRelNode(input, types, names, columnTypeNodes, extensionNode, bucketSpec); } public static RelNode makeSortRel( diff --git a/gluten-substrait/src/main/java/org/apache/gluten/substrait/rel/WriteRelNode.java b/gluten-substrait/src/main/java/org/apache/gluten/substrait/rel/WriteRelNode.java index 45b4cd659eef..74ffc8282c60 100644 --- a/gluten-substrait/src/main/java/org/apache/gluten/substrait/rel/WriteRelNode.java +++ b/gluten-substrait/src/main/java/org/apache/gluten/substrait/rel/WriteRelNode.java @@ -21,10 +21,7 @@ import org.apache.gluten.substrait.type.TypeNode; import org.apache.gluten.utils.SubstraitUtil; -import io.substrait.proto.NamedObjectWrite; -import io.substrait.proto.NamedStruct; -import io.substrait.proto.Rel; -import io.substrait.proto.WriteRel; +import io.substrait.proto.*; import java.io.Serializable; import java.util.ArrayList; @@ -39,17 +36,21 @@ public class WriteRelNode implements RelNode, Serializable { private final AdvancedExtensionNode extensionNode; + private final WriteRel.BucketSpec bucketSpec; + WriteRelNode( RelNode input, List types, List names, List partitionColumnTypeNodes, - AdvancedExtensionNode extensionNode) { + AdvancedExtensionNode extensionNode, + WriteRel.BucketSpec bucketSpec) { this.input = input; this.types.addAll(types); this.names.addAll(names); this.columnTypeNodes.addAll(partitionColumnTypeNodes); this.extensionNode = extensionNode; + this.bucketSpec = bucketSpec; } @Override @@ -68,6 +69,10 @@ public Rel toProtobuf() { nameObjectWriter.setAdvancedExtension(extensionNode.toProtobuf()); } + if (bucketSpec != null) { + writeBuilder.setBucketSpec(bucketSpec); + } + writeBuilder.setNamedTable(nameObjectWriter); if (input != null) { diff --git a/gluten-substrait/src/main/resources/substrait/proto/substrait/algebra.proto b/gluten-substrait/src/main/resources/substrait/proto/substrait/algebra.proto index 0abb50b323ac..ca669c763906 100644 --- a/gluten-substrait/src/main/resources/substrait/proto/substrait/algebra.proto +++ b/gluten-substrait/src/main/resources/substrait/proto/substrait/algebra.proto @@ -608,6 +608,9 @@ message WriteRel { // Output mode determines what is the output of executing this rel OutputMode output = 6; + // The bucket spec for the writer. + BucketSpec bucket_spec = 7; + enum WriteOp { WRITE_OP_UNSPECIFIED = 0; // The insert of new tuples in a table @@ -631,6 +634,13 @@ message WriteRel { // subplans in the body of the Rel input) and return those with anounter PlanRel.relations. OUTPUT_MODE_MODIFIED_TUPLES = 2; } + + // A container for bucketing information. + message BucketSpec { + int32 num_buckets = 1; + repeated string bucket_column_names = 2; + repeated string sort_column_names = 3; + } } // The hash equijoin join operator will build a hash table out of the right input based on a set of join keys. diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/BackendSettingsApi.scala b/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/BackendSettingsApi.scala index 0c220ada64f9..04aa7fe7ca15 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/BackendSettingsApi.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/BackendSettingsApi.scala @@ -52,6 +52,7 @@ trait BackendSettingsApi { format: FileFormat, fields: Array[StructField], bucketSpec: Option[BucketSpec], + isPartitionedTable: Boolean, options: Map[String, String]): ValidationResult = ValidationResult.succeeded def supportNativeWrite(fields: Array[StructField]): Boolean = true diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/HashAggregateExecBaseTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/HashAggregateExecBaseTransformer.scala index 46c1ad00292d..35809be0be56 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/HashAggregateExecBaseTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/HashAggregateExecBaseTransformer.scala @@ -53,7 +53,7 @@ abstract class HashAggregateExecBaseTransformer( BackendsApiManager.getMetricsApiInstance.genHashAggregateTransformerMetrics(sparkContext) protected def isCapableForStreamingAggregation: Boolean = { - if (!conf.getConf(GlutenConfig.COLUMNAR_PREFER_STREAMING_AGGREGATE)) { + if (!glutenConf.getConf(GlutenConfig.COLUMNAR_PREFER_STREAMING_AGGREGATE)) { return false } if (groupingExpressions.isEmpty) { diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/WholeStageTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/WholeStageTransformer.scala index 5d37d53893f7..7a06709da097 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/WholeStageTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/WholeStageTransformer.scala @@ -271,7 +271,7 @@ case class WholeStageTransformer(child: SparkPlan, materializeInput: Boolean = f override def nodeName: String = s"WholeStageCodegenTransformer ($transformStageId)" override def verboseStringWithOperatorId(): String = { - val nativePlan = if (conf.getConf(GlutenConfig.INJECT_NATIVE_PLAN_STRING_TO_EXPLAIN)) { + val nativePlan = if (glutenConf.getConf(GlutenConfig.INJECT_NATIVE_PLAN_STRING_TO_EXPLAIN)) { s"Native Plan:\n${nativePlanString()}" } else { "" @@ -315,7 +315,7 @@ case class WholeStageTransformer(child: SparkPlan, materializeInput: Boolean = f def doWholeStageTransform(): WholeStageTransformContext = { val context = generateWholeStageTransformContext() - if (conf.getConf(GlutenConfig.CACHE_WHOLE_STAGE_TRANSFORMER_CONTEXT)) { + if (glutenConf.getConf(GlutenConfig.CACHE_WHOLE_STAGE_TRANSFORMER_CONTEXT)) { wholeStageTransformerContext = Some(context) } context diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/WriteFilesExecTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/WriteFilesExecTransformer.scala index 60259e00dcaa..3c1857236b25 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/WriteFilesExecTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/WriteFilesExecTransformer.scala @@ -39,7 +39,7 @@ import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{ArrayType, MapType, MetadataBuilder} -import io.substrait.proto.NamedStruct +import io.substrait.proto.{NamedStruct, WriteRel} import org.apache.parquet.hadoop.ParquetOutputFormat import java.util.Locale @@ -107,18 +107,29 @@ case class WriteFilesExecTransformer( ExtensionBuilder.makeAdvancedExtension( SubstraitUtil.createEnhancement(originalInputAttributes)) } + + val bucketSpecOption = bucketSpec.map { + bucketSpec => + val builder = WriteRel.BucketSpec.newBuilder() + builder.setNumBuckets(bucketSpec.numBuckets) + bucketSpec.bucketColumnNames.foreach(builder.addBucketColumnNames) + bucketSpec.sortColumnNames.foreach(builder.addSortColumnNames) + builder.build() + } + RelBuilder.makeWriteRel( input, typeNodes, nameList, columnTypeNodes, extensionNode, + bucketSpecOption.orNull, context, operatorId) } private def getFinalChildOutput: Seq[Attribute] = { - val metadataExclusionList = conf + val metadataExclusionList = glutenConf .getConf(GlutenConfig.NATIVE_WRITE_FILES_COLUMN_METADATA_EXCLUSION_LIST) .split(",") .map(_.trim) @@ -147,11 +158,13 @@ case class WriteFilesExecTransformer( "complex data type with constant") } + val childOutput = this.child.output.map(_.exprId) val validationResult = BackendsApiManager.getSettings.supportWriteFilesExec( fileFormat, finalChildOutput.toStructType.fields, bucketSpec, + partitionColumns.exists(c => childOutput.contains(c.exprId)), caseInsensitiveOptions) if (!validationResult.ok()) { return ValidationResult.failed("Unsupported native write: " + validationResult.reason()) diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/RemoveNativeWriteFilesSortAndProject.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/RemoveNativeWriteFilesSortAndProject.scala index ebab26144201..7fce082f8a95 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/RemoveNativeWriteFilesSortAndProject.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/RemoveNativeWriteFilesSortAndProject.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.execution.{ProjectExec, SortExec, SparkPlan} */ case class RemoveNativeWriteFilesSortAndProject() extends Rule[SparkPlan] { override def apply(plan: SparkPlan): SparkPlan = { - if (!conf.getConf(GlutenConfig.REMOVE_NATIVE_WRITE_FILES_SORT_AND_PROJECT)) { + if (!GlutenConfig.get.getConf(GlutenConfig.REMOVE_NATIVE_WRITE_FILES_SORT_AND_PROJECT)) { return plan } diff --git a/gluten-substrait/src/test/scala/org/apache/spark/softaffinity/SoftAffinityWithRDDInfoSuite.scala b/gluten-substrait/src/test/scala/org/apache/spark/softaffinity/SoftAffinityWithRDDInfoSuite.scala index 7915282e3e3e..2100518e741c 100644 --- a/gluten-substrait/src/test/scala/org/apache/spark/softaffinity/SoftAffinityWithRDDInfoSuite.scala +++ b/gluten-substrait/src/test/scala/org/apache/spark/softaffinity/SoftAffinityWithRDDInfoSuite.scala @@ -49,7 +49,7 @@ class SoftAffinityWithRDDInfoSuite extends QueryTest with SharedSparkSession wit .set(GlutenConfig.GLUTEN_SOFT_AFFINITY_DUPLICATE_READING_DETECT_ENABLED, "true") .set(GlutenConfig.GLUTEN_SOFT_AFFINITY_REPLICATIONS_NUM, "2") .set(GlutenConfig.GLUTEN_SOFT_AFFINITY_MIN_TARGET_HOSTS, "2") - .set(GlutenConfig.SOFT_AFFINITY_LOG_LEVEL, "INFO") + .set(GlutenConfig.SOFT_AFFINITY_LOG_LEVEL.key, "INFO") test("Soft Affinity Scheduler with duplicate reading detection") { if (SparkShimLoader.getSparkShims.supportDuplicateReadingTracking) { 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 baed98729bc3..f5071d2f3fc4 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 @@ -917,8 +917,6 @@ class VeloxTestSettings extends BackendTestSettings { .exclude("SPARK-35640: read binary as timestamp should throw schema incompatible error") // Exception msg. .exclude("SPARK-35640: int as long should throw schema incompatible error") - // Velox only support read Timestamp with INT96 for now. - .exclude("read dictionary and plain encoded timestamp_millis written as INT64") enableSuite[GlutenParquetV1PartitionDiscoverySuite] enableSuite[GlutenParquetV2PartitionDiscoverySuite] enableSuite[GlutenParquetProtobufCompatibilitySuite] @@ -927,9 +925,6 @@ class VeloxTestSettings extends BackendTestSettings { .exclude("Enabling/disabling ignoreCorruptFiles") // decimal failed ut .exclude("SPARK-34212 Parquet should read decimals correctly") - // Timestamp is read as INT96. - .exclude("Migration from INT96 to TIMESTAMP_MICROS timestamp type") - .exclude("SPARK-10365 timestamp written and read as INT64 - TIMESTAMP_MICROS") // Rewrite because the filter after datasource is not needed. .exclude( "SPARK-26677: negated null-safe equality comparison should not filter matched row groups") @@ -938,9 +933,6 @@ class VeloxTestSettings extends BackendTestSettings { .exclude("Enabling/disabling ignoreCorruptFiles") // decimal failed ut .exclude("SPARK-34212 Parquet should read decimals correctly") - // Timestamp is read as INT96. - .exclude("Migration from INT96 to TIMESTAMP_MICROS timestamp type") - .exclude("SPARK-10365 timestamp written and read as INT64 - TIMESTAMP_MICROS") // Rewrite because the filter after datasource is not needed. .exclude( "SPARK-26677: negated null-safe equality comparison should not filter matched row groups") diff --git a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/GlutenSparkSessionExtensionSuite.scala b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/GlutenSparkSessionExtensionSuite.scala index c5d528ddeca2..ae9b3901afb9 100644 --- a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/GlutenSparkSessionExtensionSuite.scala +++ b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/GlutenSparkSessionExtensionSuite.scala @@ -31,8 +31,10 @@ class GlutenSparkSessionExtensionSuite try { session.range(2).write.format("parquet").mode("overwrite").saveAsTable("a") def testWithFallbackSettings(scanFallback: Boolean, aggFallback: Boolean): Unit = { - session.sessionState.conf.setConf(GlutenConfig.COLUMNAR_FILESCAN_ENABLED, scanFallback) - session.sessionState.conf.setConf(GlutenConfig.COLUMNAR_HASHAGG_ENABLED, aggFallback) + session.sessionState.conf + .setConfString(GlutenConfig.COLUMNAR_FILESCAN_ENABLED.key, scanFallback.toString) + session.sessionState.conf + .setConfString(GlutenConfig.COLUMNAR_HASHAGG_ENABLED.key, aggFallback.toString) val df = session.sql("SELECT max(id) FROM a") val newDf = DummyFilterColmnarHelper.dfWithDummyFilterColumnar( session, 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 d1f8b5b0c492..d3bc3846d80f 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 @@ -720,8 +720,7 @@ class VeloxTestSettings extends BackendTestSettings { .exclude("SPARK-35640: read binary as timestamp should throw schema incompatible error") // Exception msg. .exclude("SPARK-35640: int as long should throw schema incompatible error") - // Velox only support read Timestamp with INT96 for now. - .exclude("read dictionary and plain encoded timestamp_millis written as INT64") + // Velox parquet reader not allow offset zero. .exclude("SPARK-40128 read DELTA_LENGTH_BYTE_ARRAY encoded strings") enableSuite[GlutenParquetV1PartitionDiscoverySuite] enableSuite[GlutenParquetV2PartitionDiscoverySuite] @@ -731,10 +730,6 @@ class VeloxTestSettings extends BackendTestSettings { .exclude("Enabling/disabling ignoreCorruptFiles") // decimal failed ut .exclude("SPARK-34212 Parquet should read decimals correctly") - // Timestamp is read as INT96. - .exclude("Migration from INT96 to TIMESTAMP_MICROS timestamp type") - .exclude("SPARK-10365 timestamp written and read as INT64 - TIMESTAMP_MICROS") - .exclude("SPARK-36182: read TimestampNTZ as TimestampLTZ") // new added in spark-3.3 and need fix later, random failure may caused by memory free .exclude("SPARK-39833: pushed filters with project without filter columns") .exclude("SPARK-39833: pushed filters with count()") @@ -746,10 +741,6 @@ class VeloxTestSettings extends BackendTestSettings { .exclude("Enabling/disabling ignoreCorruptFiles") // decimal failed ut .exclude("SPARK-34212 Parquet should read decimals correctly") - // Timestamp is read as INT96. - .exclude("Migration from INT96 to TIMESTAMP_MICROS timestamp type") - .exclude("SPARK-10365 timestamp written and read as INT64 - TIMESTAMP_MICROS") - .exclude("SPARK-36182: read TimestampNTZ as TimestampLTZ") // Rewrite because the filter after datasource is not needed. .exclude( "SPARK-26677: negated null-safe equality comparison should not filter matched row groups") diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/GlutenSparkSessionExtensionSuite.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/GlutenSparkSessionExtensionSuite.scala index c5d528ddeca2..ae9b3901afb9 100644 --- a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/GlutenSparkSessionExtensionSuite.scala +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/GlutenSparkSessionExtensionSuite.scala @@ -31,8 +31,10 @@ class GlutenSparkSessionExtensionSuite try { session.range(2).write.format("parquet").mode("overwrite").saveAsTable("a") def testWithFallbackSettings(scanFallback: Boolean, aggFallback: Boolean): Unit = { - session.sessionState.conf.setConf(GlutenConfig.COLUMNAR_FILESCAN_ENABLED, scanFallback) - session.sessionState.conf.setConf(GlutenConfig.COLUMNAR_HASHAGG_ENABLED, aggFallback) + session.sessionState.conf + .setConfString(GlutenConfig.COLUMNAR_FILESCAN_ENABLED.key, scanFallback.toString) + session.sessionState.conf + .setConfString(GlutenConfig.COLUMNAR_HASHAGG_ENABLED.key, aggFallback.toString) val df = session.sql("SELECT max(id) FROM a") val newDf = DummyFilterColmnarHelper.dfWithDummyFilterColumnar( session, 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 1de6961192f7..cc9746dcdb53 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 @@ -715,9 +715,7 @@ class VeloxTestSettings extends BackendTestSettings { .exclude("SPARK-35640: read binary as timestamp should throw schema incompatible error") // Exception msg. .exclude("SPARK-35640: int as long should throw schema incompatible error") - // Velox only support read Timestamp with INT96 for now. - .exclude("read dictionary and plain encoded timestamp_millis written as INT64") - .exclude("Read TimestampNTZ and TimestampLTZ for various logical TIMESTAMP types") + // Velox parquet reader not allow offset zero. .exclude("SPARK-40128 read DELTA_LENGTH_BYTE_ARRAY encoded strings") enableSuite[GlutenParquetV1PartitionDiscoverySuite] enableSuite[GlutenParquetV2PartitionDiscoverySuite] @@ -728,10 +726,6 @@ class VeloxTestSettings extends BackendTestSettings { .exclude("Enabling/disabling ignoreCorruptFiles") // decimal failed ut .exclude("SPARK-34212 Parquet should read decimals correctly") - // Timestamp is read as INT96. - .exclude("Migration from INT96 to TIMESTAMP_MICROS timestamp type") - .exclude("SPARK-10365 timestamp written and read as INT64 - TIMESTAMP_MICROS") - .exclude("SPARK-36182: read TimestampNTZ as TimestampLTZ") // new added in spark-3.3 and need fix later, random failure may caused by memory free .exclude("SPARK-39833: pushed filters with project without filter columns") .exclude("SPARK-39833: pushed filters with count()") @@ -744,10 +738,6 @@ class VeloxTestSettings extends BackendTestSettings { .exclude("Enabling/disabling ignoreCorruptFiles") // decimal failed ut .exclude("SPARK-34212 Parquet should read decimals correctly") - // Timestamp is read as INT96. - .exclude("Migration from INT96 to TIMESTAMP_MICROS timestamp type") - .exclude("SPARK-10365 timestamp written and read as INT64 - TIMESTAMP_MICROS") - .exclude("SPARK-36182: read TimestampNTZ as TimestampLTZ") // Rewrite because the filter after datasource is not needed. .exclude( "SPARK-26677: negated null-safe equality comparison should not filter matched row groups") diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenSparkSessionExtensionSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenSparkSessionExtensionSuite.scala index c5d528ddeca2..ae9b3901afb9 100644 --- a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenSparkSessionExtensionSuite.scala +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenSparkSessionExtensionSuite.scala @@ -31,8 +31,10 @@ class GlutenSparkSessionExtensionSuite try { session.range(2).write.format("parquet").mode("overwrite").saveAsTable("a") def testWithFallbackSettings(scanFallback: Boolean, aggFallback: Boolean): Unit = { - session.sessionState.conf.setConf(GlutenConfig.COLUMNAR_FILESCAN_ENABLED, scanFallback) - session.sessionState.conf.setConf(GlutenConfig.COLUMNAR_HASHAGG_ENABLED, aggFallback) + session.sessionState.conf + .setConfString(GlutenConfig.COLUMNAR_FILESCAN_ENABLED.key, scanFallback.toString) + session.sessionState.conf + .setConfString(GlutenConfig.COLUMNAR_HASHAGG_ENABLED.key, aggFallback.toString) val df = session.sql("SELECT max(id) FROM a") val newDf = DummyFilterColmnarHelper.dfWithDummyFilterColumnar( session, 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 2cf2f8ad3186..71786c91322b 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 @@ -726,9 +726,7 @@ class VeloxTestSettings extends BackendTestSettings { .exclude("SPARK-35640: read binary as timestamp should throw schema incompatible error") // Exception msg. .exclude("SPARK-35640: int as long should throw schema incompatible error") - // Velox only support read Timestamp with INT96 for now. - .exclude("read dictionary and plain encoded timestamp_millis written as INT64") - .exclude("Read TimestampNTZ and TimestampLTZ for various logical TIMESTAMP types") + // Velox parquet reader not allow offset zero. .exclude("SPARK-40128 read DELTA_LENGTH_BYTE_ARRAY encoded strings") enableSuite[GlutenParquetV1PartitionDiscoverySuite] enableSuite[GlutenParquetV2PartitionDiscoverySuite] @@ -739,10 +737,6 @@ class VeloxTestSettings extends BackendTestSettings { .exclude("Enabling/disabling ignoreCorruptFiles") // decimal failed ut .exclude("SPARK-34212 Parquet should read decimals correctly") - // Timestamp is read as INT96. - .exclude("Migration from INT96 to TIMESTAMP_MICROS timestamp type") - .exclude("SPARK-10365 timestamp written and read as INT64 - TIMESTAMP_MICROS") - .exclude("SPARK-36182: read TimestampNTZ as TimestampLTZ") // new added in spark-3.3 and need fix later, random failure may caused by memory free .exclude("SPARK-39833: pushed filters with project without filter columns") .exclude("SPARK-39833: pushed filters with count()") @@ -755,10 +749,6 @@ class VeloxTestSettings extends BackendTestSettings { .exclude("Enabling/disabling ignoreCorruptFiles") // decimal failed ut .exclude("SPARK-34212 Parquet should read decimals correctly") - // Timestamp is read as INT96. - .exclude("Migration from INT96 to TIMESTAMP_MICROS timestamp type") - .exclude("SPARK-10365 timestamp written and read as INT64 - TIMESTAMP_MICROS") - .exclude("SPARK-36182: read TimestampNTZ as TimestampLTZ") // Rewrite because the filter after datasource is not needed. .exclude( "SPARK-26677: negated null-safe equality comparison should not filter matched row groups") diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/GlutenSparkSessionExtensionSuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/GlutenSparkSessionExtensionSuite.scala index c5d528ddeca2..ae9b3901afb9 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/GlutenSparkSessionExtensionSuite.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/GlutenSparkSessionExtensionSuite.scala @@ -31,8 +31,10 @@ class GlutenSparkSessionExtensionSuite try { session.range(2).write.format("parquet").mode("overwrite").saveAsTable("a") def testWithFallbackSettings(scanFallback: Boolean, aggFallback: Boolean): Unit = { - session.sessionState.conf.setConf(GlutenConfig.COLUMNAR_FILESCAN_ENABLED, scanFallback) - session.sessionState.conf.setConf(GlutenConfig.COLUMNAR_HASHAGG_ENABLED, aggFallback) + session.sessionState.conf + .setConfString(GlutenConfig.COLUMNAR_FILESCAN_ENABLED.key, scanFallback.toString) + session.sessionState.conf + .setConfString(GlutenConfig.COLUMNAR_HASHAGG_ENABLED.key, aggFallback.toString) val df = session.sql("SELECT max(id) FROM a") val newDf = DummyFilterColmnarHelper.dfWithDummyFilterColumnar( session, diff --git a/shims/common/src/main/scala/org/apache/gluten/config/ConfigBuilder.scala b/shims/common/src/main/scala/org/apache/gluten/config/ConfigBuilder.scala new file mode 100644 index 000000000000..475fe69410a1 --- /dev/null +++ b/shims/common/src/main/scala/org/apache/gluten/config/ConfigBuilder.scala @@ -0,0 +1,229 @@ +/* + * 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.config + +import org.apache.spark.network.util.ByteUnit +import org.apache.spark.network.util.JavaUtils + +import java.util.concurrent.TimeUnit +import java.util.regex.Pattern; + +object BackendType extends Enumeration { + type BackendType = Value + val COMMON, VELOX, CLICKHOUSE = Value +} + +private[gluten] case class ConfigBuilder(key: String) { + import ConfigHelpers._ + + private[config] var _doc = "" + private[config] var _version = "" + private[config] var _backend = BackendType.COMMON + private[config] var _public = true + private[config] var _alternatives = List.empty[String] + private[config] var _onCreate: Option[ConfigEntry[_] => Unit] = None + + def doc(s: String): ConfigBuilder = { + _doc = s + this + } + + def version(s: String): ConfigBuilder = { + _version = s + this + } + + def backend(backend: BackendType.BackendType): ConfigBuilder = { + _backend = backend + this + } + + def internal(): ConfigBuilder = { + _public = false + this + } + + def onCreate(callback: ConfigEntry[_] => Unit): ConfigBuilder = { + _onCreate = Option(callback) + this + } + + def withAlternative(key: String): ConfigBuilder = { + _alternatives = _alternatives :+ key + this + } + + def intConf: TypedConfigBuilder[Int] = { + new TypedConfigBuilder(this, toNumber(_, _.toInt, key, "int")) + } + + def longConf: TypedConfigBuilder[Long] = { + new TypedConfigBuilder(this, toNumber(_, _.toLong, key, "long")) + } + + def doubleConf: TypedConfigBuilder[Double] = { + new TypedConfigBuilder(this, toNumber(_, _.toDouble, key, "double")) + } + + def booleanConf: TypedConfigBuilder[Boolean] = { + new TypedConfigBuilder(this, toBoolean(_, key)) + } + + def stringConf: TypedConfigBuilder[String] = { + new TypedConfigBuilder(this, identity) + } + + def timeConf(unit: TimeUnit): TypedConfigBuilder[Long] = { + new TypedConfigBuilder(this, timeFromString(_, unit), timeToString(_, unit)) + } + + def bytesConf(unit: ByteUnit): TypedConfigBuilder[Long] = { + new TypedConfigBuilder(this, byteFromString(_, unit), byteToString(_, unit)) + } + + def fallbackConf[T](fallback: ConfigEntry[T]): ConfigEntry[T] = { + val entry = + new ConfigEntryFallback[T](key, _doc, _version, _backend, _public, _alternatives, fallback) + _onCreate.foreach(_(entry)) + entry + } +} + +private object ConfigHelpers { + def toNumber[T](s: String, converter: String => T, key: String, configType: String): T = { + try { + converter(s.trim) + } catch { + case _: NumberFormatException => + throw new IllegalArgumentException(s"$key should be $configType, but was $s") + } + } + + def toBoolean(s: String, key: String): Boolean = { + try { + s.trim.toBoolean + } catch { + case _: IllegalArgumentException => + throw new IllegalArgumentException(s"$key should be boolean, but was $s") + } + } + + private val TIME_STRING_PATTERN = Pattern.compile("(-?[0-9]+)([a-z]+)?") + + def timeFromString(str: String, unit: TimeUnit): Long = JavaUtils.timeStringAs(str, unit) + + def timeToString(v: Long, unit: TimeUnit): String = s"${TimeUnit.MILLISECONDS.convert(v, unit)}ms" + + def byteFromString(str: String, unit: ByteUnit): Long = { + val (input, multiplier) = + if (str.length() > 0 && str.charAt(0) == '-') { + (str.substring(1), -1) + } else { + (str, 1) + } + multiplier * JavaUtils.byteStringAs(input, unit) + } + + def byteToString(v: Long, unit: ByteUnit): String = s"${unit.convertTo(v, ByteUnit.BYTE)}b" +} + +private[gluten] class TypedConfigBuilder[T]( + val parent: ConfigBuilder, + val converter: String => T, + val stringConverter: T => String) { + + def this(parent: ConfigBuilder, converter: String => T) = { + this(parent, converter, { v: T => v.toString }) + } + + def transform(fn: T => T): TypedConfigBuilder[T] = { + new TypedConfigBuilder(parent, s => fn(converter(s)), stringConverter) + } + + def checkValue(validator: T => Boolean, errorMsg: String): TypedConfigBuilder[T] = { + transform { + v => + if (!validator(v)) { + throw new IllegalArgumentException(s"'$v' in ${parent.key} is invalid. $errorMsg") + } + v + } + } + + def checkValues(validValues: Set[T]): TypedConfigBuilder[T] = { + transform { + v => + if (!validValues.contains(v)) { + throw new IllegalArgumentException( + s"The value of ${parent.key} should be one of ${validValues.mkString(", ")}, " + + s"but was $v") + } + v + } + } + + def createOptional: OptionalConfigEntry[T] = { + val entry = new OptionalConfigEntry[T]( + parent.key, + parent._doc, + parent._version, + parent._backend, + parent._public, + parent._alternatives, + converter, + stringConverter) + parent._onCreate.foreach(_(entry)) + entry + } + + def createWithDefault(default: T): ConfigEntry[T] = { + assert(default != null, "Use createOptional.") + default match { + case str: String => createWithDefaultString(str) + case _ => + val transformedDefault = converter(stringConverter(default)) + val entry = new ConfigEntryWithDefault[T]( + parent.key, + parent._doc, + parent._version, + parent._backend, + parent._public, + parent._alternatives, + converter, + stringConverter, + transformedDefault) + parent._onCreate.foreach(_(entry)) + entry + } + } + + def createWithDefaultString(default: String): ConfigEntry[T] = { + val entry = new ConfigEntryWithDefaultString[T]( + parent.key, + parent._doc, + parent._version, + parent._backend, + parent._public, + parent._alternatives, + converter, + stringConverter, + default + ) + parent._onCreate.foreach(_(entry)) + entry + } +} diff --git a/shims/common/src/main/scala/org/apache/gluten/config/ConfigEntry.scala b/shims/common/src/main/scala/org/apache/gluten/config/ConfigEntry.scala new file mode 100644 index 000000000000..5d087d33858c --- /dev/null +++ b/shims/common/src/main/scala/org/apache/gluten/config/ConfigEntry.scala @@ -0,0 +1,242 @@ +/* + * 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.config + +import org.apache.gluten.config.BackendType.BackendType + +import org.apache.spark.sql.internal.ConfigProvider + +/** + * An entry contains all meta information for a configuration. + * + * The code is similar to Spark's relevant config code but extended for Gluten's use, like adding + * backend type, etc. + * + * @tparam T + * the value type + */ +trait ConfigEntry[T] { + + /** The key for the configuration. */ + def key: String + + /** The documentation for the configuration. */ + def doc: String + + /** The gluten version when the configuration was released. */ + def version: String + + /** The backend type of the configuration. */ + def backend: BackendType.BackendType + + /** + * If this configuration is public to the user. If it's `false`, this configuration is only used + * internally and we should not expose it to users. + */ + def isPublic: Boolean + + /** the alternative keys for the configuration. */ + def alternatives: List[String] + + /** + * How to convert a string to the value. It should throw an exception if the string does not have + * the required format. + */ + def valueConverter: String => T + + /** How to convert a value to a string that the user can use it as a valid string value. */ + def stringConverter: T => String + + /** Read the configuration from the given ConfigProvider. */ + def readFrom(conf: ConfigProvider): T + + /** The default value of the configuration. */ + def defaultValue: Option[T] + + /** The string representation of the default value. */ + def defaultValueString: String + + final protected def readString(provider: ConfigProvider): Option[String] = { + alternatives.foldLeft(provider.get(key))((res, nextKey) => res.orElse(provider.get(nextKey))) + } + + override def toString: String = { + s"ConfigEntry(key=$key, defaultValue=$defaultValueString, doc=$doc, " + + s"public=$isPublic, version=$version)" + } + + ConfigEntry.registerEntry(this) +} + +private[gluten] class OptionalConfigEntry[T]( + _key: String, + _doc: String, + _version: String, + _backend: BackendType, + _isPublic: Boolean, + _alternatives: List[String], + _valueConverter: String => T, + _stringConverter: T => String) + extends ConfigEntry[Option[T]] { + override def key: String = _key + + override def doc: String = _doc + + override def version: String = _version + + override def backend: BackendType = _backend + + override def isPublic: Boolean = _isPublic + + override def alternatives: List[String] = _alternatives + + override def valueConverter: String => Option[T] = s => Option(_valueConverter(s)) + + override def stringConverter: Option[T] => String = v => v.map(_stringConverter).orNull + + override def readFrom(conf: ConfigProvider): Option[T] = readString(conf).map(_valueConverter) + + override def defaultValue: Option[Option[T]] = None + + override def defaultValueString: String = ConfigEntry.UNDEFINED +} + +private[gluten] class ConfigEntryWithDefault[T]( + _key: String, + _doc: String, + _version: String, + _backend: BackendType, + _isPublic: Boolean, + _alternatives: List[String], + _valueConverter: String => T, + _stringConverter: T => String, + _defaultVal: T) + extends ConfigEntry[T] { + override def key: String = _key + + override def doc: String = _doc + + override def version: String = _version + + override def backend: BackendType = _backend + + override def isPublic: Boolean = _isPublic + + override def alternatives: List[String] = _alternatives + + override def valueConverter: String => T = _valueConverter + + override def stringConverter: T => String = _stringConverter + + override def readFrom(conf: ConfigProvider): T = { + readString(conf).map(valueConverter).getOrElse(_defaultVal) + } + + override def defaultValue: Option[T] = Option(_defaultVal) + + override def defaultValueString: String = stringConverter(_defaultVal) +} + +private[gluten] class ConfigEntryWithDefaultString[T]( + _key: String, + _doc: String, + _version: String, + _backend: BackendType, + _isPublic: Boolean, + _alternatives: List[String], + _valueConverter: String => T, + _stringConverter: T => String, + _defaultVal: String) + extends ConfigEntry[T] { + override def key: String = _key + + override def doc: String = _doc + + override def version: String = _version + + override def backend: BackendType = _backend + + override def isPublic: Boolean = _isPublic + + override def alternatives: List[String] = _alternatives + + override def valueConverter: String => T = _valueConverter + + override def stringConverter: T => String = _stringConverter + + override def readFrom(conf: ConfigProvider): T = { + val value = readString(conf).getOrElse(_defaultVal) + valueConverter(value) + } + + override def defaultValue: Option[T] = Some(valueConverter(_defaultVal)) + + override def defaultValueString: String = _defaultVal +} + +private[gluten] class ConfigEntryFallback[T]( + _key: String, + _doc: String, + _version: String, + _backend: BackendType, + _isPublic: Boolean, + _alternatives: List[String], + fallback: ConfigEntry[T]) + extends ConfigEntry[T] { + override def key: String = _key + + override def doc: String = _doc + + override def version: String = _version + + override def backend: BackendType = _backend + + override def isPublic: Boolean = _isPublic + + override def alternatives: List[String] = _alternatives + + override def valueConverter: String => T = fallback.valueConverter + + override def stringConverter: T => String = fallback.stringConverter + + override def readFrom(conf: ConfigProvider): T = { + readString(conf).map(valueConverter).getOrElse(fallback.readFrom(conf)) + } + + override def defaultValue: Option[T] = fallback.defaultValue + + override def defaultValueString: String = fallback.defaultValueString +} + +object ConfigEntry { + + val UNDEFINED = "" + + private val knownConfigs = + new java.util.concurrent.ConcurrentHashMap[String, ConfigEntry[_]]() + + private def registerEntry(entry: ConfigEntry[_]): Unit = { + val existing = knownConfigs.putIfAbsent(entry.key, entry) + require(existing == null, s"Config entry ${entry.key} already registered!") + } + + def containsEntry(entry: ConfigEntry[_]): Boolean = { + Option(knownConfigs.get(entry.key)).isDefined + } + + def findEntry(key: String): ConfigEntry[_] = knownConfigs.get(key) +} 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 f6ed0327349d..cee4ea7d1645 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 @@ -18,7 +18,7 @@ package org.apache.gluten.config import org.apache.spark.internal.Logging import org.apache.spark.network.util.{ByteUnit, JavaUtils} -import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.{SQLConf, SQLConfProvider} import com.google.common.collect.ImmutableList import org.apache.hadoop.security.UserGroupInformation @@ -37,100 +37,104 @@ case class GlutenNumaBindingInfo( class GlutenConfig(conf: SQLConf) extends Logging { import GlutenConfig._ + private lazy val configProvider = new SQLConfProvider(conf) + + def getConf[T](entry: ConfigEntry[T]): T = { + require(ConfigEntry.containsEntry(entry), s"$entry is not registered") + entry.readFrom(configProvider) + } + def enableAnsiMode: Boolean = conf.ansiEnabled - def enableGluten: Boolean = conf.getConf(GLUTEN_ENABLED) + def enableGluten: Boolean = getConf(GLUTEN_ENABLED) // FIXME the option currently controls both JVM and native validation against a Substrait plan. - def enableNativeValidation: Boolean = conf.getConf(NATIVE_VALIDATION_ENABLED) + def enableNativeValidation: Boolean = getConf(NATIVE_VALIDATION_ENABLED) - def enableColumnarBatchScan: Boolean = conf.getConf(COLUMNAR_BATCHSCAN_ENABLED) + def enableColumnarBatchScan: Boolean = getConf(COLUMNAR_BATCHSCAN_ENABLED) - def enableColumnarFileScan: Boolean = conf.getConf(COLUMNAR_FILESCAN_ENABLED) + def enableColumnarFileScan: Boolean = getConf(COLUMNAR_FILESCAN_ENABLED) - def enableColumnarHiveTableScan: Boolean = conf.getConf(COLUMNAR_HIVETABLESCAN_ENABLED) + def enableColumnarHiveTableScan: Boolean = getConf(COLUMNAR_HIVETABLESCAN_ENABLED) def enableColumnarHiveTableScanNestedColumnPruning: Boolean = - conf.getConf(COLUMNAR_HIVETABLESCAN_NESTED_COLUMN_PRUNING_ENABLED) + getConf(COLUMNAR_HIVETABLESCAN_NESTED_COLUMN_PRUNING_ENABLED) - def enableVanillaVectorizedReaders: Boolean = conf.getConf(VANILLA_VECTORIZED_READERS_ENABLED) + def enableVanillaVectorizedReaders: Boolean = getConf(VANILLA_VECTORIZED_READERS_ENABLED) - def enableColumnarHashAgg: Boolean = conf.getConf(COLUMNAR_HASHAGG_ENABLED) + def enableColumnarHashAgg: Boolean = getConf(COLUMNAR_HASHAGG_ENABLED) - def forceToUseHashAgg: Boolean = conf.getConf(COLUMNAR_FORCE_HASHAGG_ENABLED) + def forceToUseHashAgg: Boolean = getConf(COLUMNAR_FORCE_HASHAGG_ENABLED) - def mergeTwoPhasesAggEnabled: Boolean = conf.getConf(MERGE_TWO_PHASES_ENABLED) + def mergeTwoPhasesAggEnabled: Boolean = getConf(MERGE_TWO_PHASES_ENABLED) - def enableColumnarProject: Boolean = conf.getConf(COLUMNAR_PROJECT_ENABLED) + def enableColumnarProject: Boolean = getConf(COLUMNAR_PROJECT_ENABLED) - def enableColumnarFilter: Boolean = conf.getConf(COLUMNAR_FILTER_ENABLED) + def enableColumnarFilter: Boolean = getConf(COLUMNAR_FILTER_ENABLED) - def enableColumnarSort: Boolean = conf.getConf(COLUMNAR_SORT_ENABLED) + def enableColumnarSort: Boolean = getConf(COLUMNAR_SORT_ENABLED) - def enableColumnarWindow: Boolean = conf.getConf(COLUMNAR_WINDOW_ENABLED) + def enableColumnarWindow: Boolean = getConf(COLUMNAR_WINDOW_ENABLED) - def enableColumnarWindowGroupLimit: Boolean = conf.getConf(COLUMNAR_WINDOW_GROUP_LIMIT_ENABLED) + def enableColumnarWindowGroupLimit: Boolean = getConf(COLUMNAR_WINDOW_GROUP_LIMIT_ENABLED) - def veloxColumnarWindowType: String = conf.getConfString(COLUMNAR_VELOX_WINDOW_TYPE.key) + def veloxColumnarWindowType: String = getConf(COLUMNAR_VELOX_WINDOW_TYPE) - def enableColumnarShuffledHashJoin: Boolean = conf.getConf(COLUMNAR_SHUFFLED_HASH_JOIN_ENABLED) + def enableColumnarShuffledHashJoin: Boolean = getConf(COLUMNAR_SHUFFLED_HASH_JOIN_ENABLED) def shuffledHashJoinOptimizeBuildSide: Boolean = - conf.getConf(COLUMNAR_SHUFFLED_HASH_JOIN_OPTIMIZE_BUILD_SIDE) + getConf(COLUMNAR_SHUFFLED_HASH_JOIN_OPTIMIZE_BUILD_SIDE) - def enableNativeColumnarToRow: Boolean = conf.getConf(COLUMNAR_COLUMNAR_TO_ROW_ENABLED) + def enableNativeColumnarToRow: Boolean = getConf(COLUMNAR_COLUMNAR_TO_ROW_ENABLED) - def forceShuffledHashJoin: Boolean = conf.getConf(COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED) + def forceShuffledHashJoin: Boolean = getConf(COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED) - def enableColumnarSortMergeJoin: Boolean = conf.getConf(COLUMNAR_SORTMERGEJOIN_ENABLED) + def enableColumnarSortMergeJoin: Boolean = getConf(COLUMNAR_SORTMERGEJOIN_ENABLED) - def enableColumnarUnion: Boolean = conf.getConf(COLUMNAR_UNION_ENABLED) + def enableColumnarUnion: Boolean = getConf(COLUMNAR_UNION_ENABLED) - def enableNativeUnion: Boolean = conf.getConf(NATIVE_UNION_ENABLED) + def enableNativeUnion: Boolean = getConf(NATIVE_UNION_ENABLED) - def enableColumnarExpand: Boolean = conf.getConf(COLUMNAR_EXPAND_ENABLED) + def enableColumnarExpand: Boolean = getConf(COLUMNAR_EXPAND_ENABLED) - def enableColumnarBroadcastExchange: Boolean = conf.getConf(COLUMNAR_BROADCAST_EXCHANGE_ENABLED) + def enableColumnarBroadcastExchange: Boolean = getConf(COLUMNAR_BROADCAST_EXCHANGE_ENABLED) - def enableColumnarBroadcastJoin: Boolean = conf.getConf(COLUMNAR_BROADCAST_JOIN_ENABLED) + def enableColumnarBroadcastJoin: Boolean = getConf(COLUMNAR_BROADCAST_JOIN_ENABLED) - def enableColumnarSample: Boolean = conf.getConf(COLUMNAR_SAMPLE_ENABLED) + def enableColumnarSample: Boolean = getConf(COLUMNAR_SAMPLE_ENABLED) - def enableColumnarArrowUDF: Boolean = conf.getConf(COLUMNAR_ARROW_UDF_ENABLED) + def enableColumnarArrowUDF: Boolean = getConf(COLUMNAR_ARROW_UDF_ENABLED) - def enableColumnarCoalesce: Boolean = conf.getConf(COLUMNAR_COALESCE_ENABLED) + def enableColumnarCoalesce: Boolean = getConf(COLUMNAR_COALESCE_ENABLED) - def columnarTableCacheEnabled: Boolean = conf.getConf(COLUMNAR_TABLE_CACHE_ENABLED) + def columnarTableCacheEnabled: Boolean = getConf(COLUMNAR_TABLE_CACHE_ENABLED) def enableRewriteDateTimestampComparison: Boolean = - conf.getConf(ENABLE_REWRITE_DATE_TIMESTAMP_COMPARISON) + getConf(ENABLE_REWRITE_DATE_TIMESTAMP_COMPARISON) def enableCollapseNestedGetJsonObject: Boolean = - conf.getConf(ENABLE_COLLAPSE_GET_JSON_OBJECT) + getConf(ENABLE_COLLAPSE_GET_JSON_OBJECT) def enableCHRewriteDateConversion: Boolean = - conf.getConf(ENABLE_CH_REWRITE_DATE_CONVERSION) + getConf(ENABLE_CH_REWRITE_DATE_CONVERSION) def enableCommonSubexpressionEliminate: Boolean = - conf.getConf(ENABLE_COMMON_SUBEXPRESSION_ELIMINATE) + getConf(ENABLE_COMMON_SUBEXPRESSION_ELIMINATE) def enableCountDistinctWithoutExpand: Boolean = - conf.getConf(ENABLE_COUNT_DISTINCT_WITHOUT_EXPAND) + getConf(ENABLE_COUNT_DISTINCT_WITHOUT_EXPAND) def enableExtendedColumnPruning: Boolean = - conf.getConf(ENABLE_EXTENDED_COLUMN_PRUNING) + getConf(ENABLE_EXTENDED_COLUMN_PRUNING) def veloxOrcScanEnabled: Boolean = - conf.getConf(VELOX_ORC_SCAN_ENABLED) + getConf(VELOX_ORC_SCAN_ENABLED) def forceOrcCharTypeScanFallbackEnabled: Boolean = - conf.getConf(VELOX_FORCE_ORC_CHAR_TYPE_SCAN_FALLBACK) - - def forceParquetTimestampTypeScanFallbackEnabled: Boolean = - conf.getConf(VELOX_FORCE_PARQUET_TIMESTAMP_TYPE_SCAN_FALLBACK) + getConf(VELOX_FORCE_ORC_CHAR_TYPE_SCAN_FALLBACK) def scanFileSchemeValidationEnabled: Boolean = - conf.getConf(VELOX_SCAN_FILE_SCHEME_VALIDATION_ENABLED) + getConf(VELOX_SCAN_FILE_SCHEME_VALIDATION_ENABLED) // Whether to use GlutenShuffleManager (experimental). def isUseGlutenShuffleManager: Boolean = @@ -161,44 +165,43 @@ class GlutenConfig(conf: SQLConf) extends Logging { .getConfString("spark.celeborn.client.spark.shuffle.writer", GLUTEN_HASH_SHUFFLE_WRITER) .toLowerCase(Locale.ROOT) - def enableColumnarShuffle: Boolean = conf.getConf(COLUMNAR_SHUFFLE_ENABLED) + def enableColumnarShuffle: Boolean = getConf(COLUMNAR_SHUFFLE_ENABLED) - def enablePreferColumnar: Boolean = conf.getConf(COLUMNAR_PREFER_ENABLED) + def enablePreferColumnar: Boolean = getConf(COLUMNAR_PREFER_ENABLED) - def enableOneRowRelationColumnar: Boolean = conf.getConf(COLUMNAR_ONE_ROW_RELATION_ENABLED) + def enableOneRowRelationColumnar: Boolean = getConf(COLUMNAR_ONE_ROW_RELATION_ENABLED) def physicalJoinOptimizationThrottle: Integer = - conf.getConf(COLUMNAR_PHYSICAL_JOIN_OPTIMIZATION_THROTTLE) + getConf(COLUMNAR_PHYSICAL_JOIN_OPTIMIZATION_THROTTLE) def enablePhysicalJoinOptimize: Boolean = - conf.getConf(COLUMNAR_PHYSICAL_JOIN_OPTIMIZATION_ENABLED) + getConf(COLUMNAR_PHYSICAL_JOIN_OPTIMIZATION_ENABLED) def logicalJoinOptimizationThrottle: Integer = - conf.getConf(COLUMNAR_LOGICAL_JOIN_OPTIMIZATION_THROTTLE) + getConf(COLUMNAR_LOGICAL_JOIN_OPTIMIZATION_THROTTLE) - def enableScanOnly: Boolean = conf.getConf(COLUMNAR_SCAN_ONLY_ENABLED) + def enableScanOnly: Boolean = getConf(COLUMNAR_SCAN_ONLY_ENABLED) - def tmpFile: Option[String] = conf.getConf(COLUMNAR_TEMP_DIR) + def tmpFile: Option[String] = getConf(COLUMNAR_TEMP_DIR) - @deprecated def broadcastCacheTimeout: Int = conf.getConf(COLUMNAR_BROADCAST_CACHE_TIMEOUT) + @deprecated def broadcastCacheTimeout: Int = getConf(COLUMNAR_BROADCAST_CACHE_TIMEOUT) def columnarShuffleSortPartitionsThreshold: Int = - conf.getConf(COLUMNAR_SHUFFLE_SORT_PARTITIONS_THRESHOLD) + getConf(COLUMNAR_SHUFFLE_SORT_PARTITIONS_THRESHOLD) def columnarShuffleSortColumnsThreshold: Int = - conf.getConf(COLUMNAR_SHUFFLE_SORT_COLUMNS_THRESHOLD) + getConf(COLUMNAR_SHUFFLE_SORT_COLUMNS_THRESHOLD) - def columnarShuffleReallocThreshold: Double = conf.getConf(COLUMNAR_SHUFFLE_REALLOC_THRESHOLD) + def columnarShuffleReallocThreshold: Double = getConf(COLUMNAR_SHUFFLE_REALLOC_THRESHOLD) - def columnarShuffleMergeThreshold: Double = conf.getConf(SHUFFLE_WRITER_MERGE_THRESHOLD) + def columnarShuffleMergeThreshold: Double = getConf(SHUFFLE_WRITER_MERGE_THRESHOLD) - def columnarShuffleCodec: Option[String] = conf.getConf(COLUMNAR_SHUFFLE_CODEC) + def columnarShuffleCodec: Option[String] = getConf(COLUMNAR_SHUFFLE_CODEC) def columnarShuffleCompressionMode: String = - conf.getConf(COLUMNAR_SHUFFLE_COMPRESSION_MODE) + getConf(COLUMNAR_SHUFFLE_COMPRESSION_MODE) - def columnarShuffleCodecBackend: Option[String] = conf - .getConf(COLUMNAR_SHUFFLE_CODEC_BACKEND) + def columnarShuffleCodecBackend: Option[String] = getConf(COLUMNAR_SHUFFLE_CODEC_BACKEND) .filter(Set(GLUTEN_QAT_BACKEND_NAME, GLUTEN_IAA_BACKEND_NAME).contains(_)) def columnarShuffleEnableQat: Boolean = @@ -208,54 +211,53 @@ class GlutenConfig(conf: SQLConf) extends Logging { columnarShuffleCodecBackend.contains(GlutenConfig.GLUTEN_IAA_BACKEND_NAME) def columnarShuffleCompressionThreshold: Int = - conf.getConf(COLUMNAR_SHUFFLE_COMPRESSION_THRESHOLD) + getConf(COLUMNAR_SHUFFLE_COMPRESSION_THRESHOLD) def columnarShuffleReaderBufferSize: Long = - conf.getConf(COLUMNAR_SHUFFLE_READER_BUFFER_SIZE) + getConf(COLUMNAR_SHUFFLE_READER_BUFFER_SIZE) - def maxBatchSize: Int = conf.getConf(COLUMNAR_MAX_BATCH_SIZE) + def maxBatchSize: Int = getConf(COLUMNAR_MAX_BATCH_SIZE) def columnarToRowMemThreshold: Long = - conf.getConf(GLUTEN_COLUMNAR_TO_ROW_MEM_THRESHOLD) + getConf(GLUTEN_COLUMNAR_TO_ROW_MEM_THRESHOLD) - def shuffleWriterBufferSize: Int = conf - .getConf(SHUFFLE_WRITER_BUFFER_SIZE) + def shuffleWriterBufferSize: Int = getConf(SHUFFLE_WRITER_BUFFER_SIZE) .getOrElse(maxBatchSize) - def enableColumnarLimit: Boolean = conf.getConf(COLUMNAR_LIMIT_ENABLED) + def enableColumnarLimit: Boolean = getConf(COLUMNAR_LIMIT_ENABLED) - def enableColumnarGenerate: Boolean = conf.getConf(COLUMNAR_GENERATE_ENABLED) + def enableColumnarGenerate: Boolean = getConf(COLUMNAR_GENERATE_ENABLED) def enableTakeOrderedAndProject: Boolean = - conf.getConf(COLUMNAR_TAKE_ORDERED_AND_PROJECT_ENABLED) + getConf(COLUMNAR_TAKE_ORDERED_AND_PROJECT_ENABLED) - def enableNativeBloomFilter: Boolean = conf.getConf(COLUMNAR_NATIVE_BLOOMFILTER_ENABLED) + def enableNativeBloomFilter: Boolean = getConf(COLUMNAR_NATIVE_BLOOMFILTER_ENABLED) def enableNativeHyperLogLogAggregateFunction: Boolean = - conf.getConf(COLUMNAR_NATIVE_HYPERLOGLOG_AGGREGATE_ENABLED) + getConf(COLUMNAR_NATIVE_HYPERLOGLOG_AGGREGATE_ENABLED) def columnarParquetWriteBlockSize: Long = - conf.getConf(COLUMNAR_PARQUET_WRITE_BLOCK_SIZE) + getConf(COLUMNAR_PARQUET_WRITE_BLOCK_SIZE) def columnarParquetWriteBlockRows: Long = - conf.getConf(COLUMNAR_PARQUET_WRITE_BLOCK_ROWS) + getConf(COLUMNAR_PARQUET_WRITE_BLOCK_ROWS) - def wholeStageFallbackThreshold: Int = conf.getConf(COLUMNAR_WHOLESTAGE_FALLBACK_THRESHOLD) + def wholeStageFallbackThreshold: Int = getConf(COLUMNAR_WHOLESTAGE_FALLBACK_THRESHOLD) - def queryFallbackThreshold: Int = conf.getConf(COLUMNAR_QUERY_FALLBACK_THRESHOLD) + def queryFallbackThreshold: Int = getConf(COLUMNAR_QUERY_FALLBACK_THRESHOLD) - def fallbackIgnoreRowToColumnar: Boolean = conf.getConf(COLUMNAR_FALLBACK_IGNORE_ROW_TO_COLUMNAR) + def fallbackIgnoreRowToColumnar: Boolean = getConf(COLUMNAR_FALLBACK_IGNORE_ROW_TO_COLUMNAR) - def fallbackExpressionsThreshold: Int = conf.getConf(COLUMNAR_FALLBACK_EXPRESSIONS_THRESHOLD) + def fallbackExpressionsThreshold: Int = getConf(COLUMNAR_FALLBACK_EXPRESSIONS_THRESHOLD) - def fallbackPreferColumnar: Boolean = conf.getConf(COLUMNAR_FALLBACK_PREFER_COLUMNAR) + def fallbackPreferColumnar: Boolean = getConf(COLUMNAR_FALLBACK_PREFER_COLUMNAR) def numaBindingInfo: GlutenNumaBindingInfo = { - val enableNumaBinding: Boolean = conf.getConf(COLUMNAR_NUMA_BINDING_ENABLED) + val enableNumaBinding: Boolean = getConf(COLUMNAR_NUMA_BINDING_ENABLED) if (!enableNumaBinding) { GlutenNumaBindingInfo(enableNumaBinding = false) } else { - val tmp = conf.getConf(COLUMNAR_NUMA_BINDING_CORE_RANGE) + val tmp = getConf(COLUMNAR_NUMA_BINDING_CORE_RANGE) if (tmp.isEmpty) { GlutenNumaBindingInfo(enableNumaBinding = false) } else { @@ -267,78 +269,78 @@ class GlutenConfig(conf: SQLConf) extends Logging { } } - def memoryIsolation: Boolean = conf.getConf(COLUMNAR_MEMORY_ISOLATION) + def memoryIsolation: Boolean = getConf(COLUMNAR_MEMORY_ISOLATION) - def memoryBacktraceAllocation: Boolean = conf.getConf(COLUMNAR_MEMORY_BACKTRACE_ALLOCATION) + def memoryBacktraceAllocation: Boolean = getConf(COLUMNAR_MEMORY_BACKTRACE_ALLOCATION) def numTaskSlotsPerExecutor: Int = { - val numSlots = conf.getConf(NUM_TASK_SLOTS_PER_EXECUTOR) + val numSlots = getConf(NUM_TASK_SLOTS_PER_EXECUTOR) assert(numSlots > 0, s"Number of task slot not found. This should not happen.") numSlots } - def offHeapMemorySize: Long = conf.getConf(COLUMNAR_OFFHEAP_SIZE_IN_BYTES) + def offHeapMemorySize: Long = getConf(COLUMNAR_OFFHEAP_SIZE_IN_BYTES) - def taskOffHeapMemorySize: Long = conf.getConf(COLUMNAR_TASK_OFFHEAP_SIZE_IN_BYTES) + def taskOffHeapMemorySize: Long = getConf(COLUMNAR_TASK_OFFHEAP_SIZE_IN_BYTES) - def memoryOverAcquiredRatio: Double = conf.getConf(COLUMNAR_MEMORY_OVER_ACQUIRED_RATIO) + def memoryOverAcquiredRatio: Double = getConf(COLUMNAR_MEMORY_OVER_ACQUIRED_RATIO) - def memoryReservationBlockSize: Long = conf.getConf(COLUMNAR_MEMORY_RESERVATION_BLOCK_SIZE) + def memoryReservationBlockSize: Long = getConf(COLUMNAR_MEMORY_RESERVATION_BLOCK_SIZE) def conservativeTaskOffHeapMemorySize: Long = - conf.getConf(COLUMNAR_CONSERVATIVE_TASK_OFFHEAP_SIZE_IN_BYTES) + getConf(COLUMNAR_CONSERVATIVE_TASK_OFFHEAP_SIZE_IN_BYTES) // Options used by RAS. - def enableRas: Boolean = conf.getConf(RAS_ENABLED) + def enableRas: Boolean = getConf(RAS_ENABLED) - def rasCostModel: String = conf.getConf(RAS_COST_MODEL) + def rasCostModel: String = getConf(RAS_COST_MODEL) - def rasRough2SizeBytesThreshold: Long = conf.getConf(RAS_ROUGH2_SIZEBYTES_THRESHOLD) + def rasRough2SizeBytesThreshold: Long = getConf(RAS_ROUGH2_SIZEBYTES_THRESHOLD) - def rasRough2R2cCost: Long = conf.getConf(RAS_ROUGH2_R2C_COST) + def rasRough2R2cCost: Long = getConf(RAS_ROUGH2_R2C_COST) - def rasRough2VanillaCost: Long = conf.getConf(RAS_ROUGH2_VANILLA_COST) + def rasRough2VanillaCost: Long = getConf(RAS_ROUGH2_VANILLA_COST) - def enableVeloxCache: Boolean = conf.getConf(COLUMNAR_VELOX_CACHE_ENABLED) + def enableVeloxCache: Boolean = getConf(COLUMNAR_VELOX_CACHE_ENABLED) - def veloxMemCacheSize: Long = conf.getConf(COLUMNAR_VELOX_MEM_CACHE_SIZE) + def veloxMemCacheSize: Long = getConf(COLUMNAR_VELOX_MEM_CACHE_SIZE) - def veloxSsdCachePath: String = conf.getConf(COLUMNAR_VELOX_SSD_CACHE_PATH) + def veloxSsdCachePath: String = getConf(COLUMNAR_VELOX_SSD_CACHE_PATH) - def veloxSsdCacheSize: Long = conf.getConf(COLUMNAR_VELOX_SSD_CACHE_SIZE) + def veloxSsdCacheSize: Long = getConf(COLUMNAR_VELOX_SSD_CACHE_SIZE) - def veloxSsdCacheShards: Integer = conf.getConf(COLUMNAR_VELOX_SSD_CACHE_SHARDS) + def veloxSsdCacheShards: Integer = getConf(COLUMNAR_VELOX_SSD_CACHE_SHARDS) - def veloxSsdCacheIOThreads: Integer = conf.getConf(COLUMNAR_VELOX_SSD_CACHE_IO_THREADS) + def veloxSsdCacheIOThreads: Integer = getConf(COLUMNAR_VELOX_SSD_CACHE_IO_THREADS) - def veloxSsdODirectEnabled: Boolean = conf.getConf(COLUMNAR_VELOX_SSD_ODIRECT_ENABLED) + def veloxSsdODirectEnabled: Boolean = getConf(COLUMNAR_VELOX_SSD_ODIRECT_ENABLED) def veloxConnectorIOThreads: Int = { - conf.getConf(COLUMNAR_VELOX_CONNECTOR_IO_THREADS).getOrElse(numTaskSlotsPerExecutor) + getConf(COLUMNAR_VELOX_CONNECTOR_IO_THREADS).getOrElse(numTaskSlotsPerExecutor) } - def veloxSplitPreloadPerDriver: Integer = conf.getConf(COLUMNAR_VELOX_SPLIT_PRELOAD_PER_DRIVER) + def veloxSplitPreloadPerDriver: Integer = getConf(COLUMNAR_VELOX_SPLIT_PRELOAD_PER_DRIVER) - def veloxSpillStrategy: String = conf.getConf(COLUMNAR_VELOX_SPILL_STRATEGY) + def veloxSpillStrategy: String = getConf(COLUMNAR_VELOX_SPILL_STRATEGY) - def veloxMaxSpillLevel: Int = conf.getConf(COLUMNAR_VELOX_MAX_SPILL_LEVEL) + def veloxMaxSpillLevel: Int = getConf(COLUMNAR_VELOX_MAX_SPILL_LEVEL) - def veloxMaxSpillFileSize: Long = conf.getConf(COLUMNAR_VELOX_MAX_SPILL_FILE_SIZE) + def veloxMaxSpillFileSize: Long = getConf(COLUMNAR_VELOX_MAX_SPILL_FILE_SIZE) - def veloxSpillFileSystem: String = conf.getConf(COLUMNAR_VELOX_SPILL_FILE_SYSTEM) + def veloxSpillFileSystem: String = getConf(COLUMNAR_VELOX_SPILL_FILE_SYSTEM) - def veloxMaxSpillRunRows: Long = conf.getConf(COLUMNAR_VELOX_MAX_SPILL_RUN_ROWS) + def veloxMaxSpillRunRows: Long = getConf(COLUMNAR_VELOX_MAX_SPILL_RUN_ROWS) - def veloxMaxSpillBytes: Long = conf.getConf(COLUMNAR_VELOX_MAX_SPILL_BYTES) + def veloxMaxSpillBytes: Long = getConf(COLUMNAR_VELOX_MAX_SPILL_BYTES) def veloxBloomFilterExpectedNumItems: Long = - conf.getConf(COLUMNAR_VELOX_BLOOM_FILTER_EXPECTED_NUM_ITEMS) + getConf(COLUMNAR_VELOX_BLOOM_FILTER_EXPECTED_NUM_ITEMS) - def veloxBloomFilterNumBits: Long = conf.getConf(COLUMNAR_VELOX_BLOOM_FILTER_NUM_BITS) + def veloxBloomFilterNumBits: Long = getConf(COLUMNAR_VELOX_BLOOM_FILTER_NUM_BITS) - def veloxBloomFilterMaxNumBits: Long = conf.getConf(COLUMNAR_VELOX_BLOOM_FILTER_MAX_NUM_BITS) + def veloxBloomFilterMaxNumBits: Long = getConf(COLUMNAR_VELOX_BLOOM_FILTER_MAX_NUM_BITS) - def castFromVarcharAddTrimNode: Boolean = conf.getConf(CAST_FROM_VARCHAR_ADD_TRIM_NODE) + def castFromVarcharAddTrimNode: Boolean = getConf(CAST_FROM_VARCHAR_ADD_TRIM_NODE) case class ResizeRange(min: Int, max: Int) { assert(max >= min) @@ -356,62 +358,61 @@ class GlutenConfig(conf: SQLConf) extends Logging { } def veloxResizeBatchesShuffleInput: Boolean = - conf.getConf(COLUMNAR_VELOX_RESIZE_BATCHES_SHUFFLE_INPUT) + getConf(COLUMNAR_VELOX_RESIZE_BATCHES_SHUFFLE_INPUT) def veloxResizeBatchesShuffleInputRange: ResizeRange = { - val standardSize = conf.getConf(COLUMNAR_MAX_BATCH_SIZE) + val standardSize = getConf(COLUMNAR_MAX_BATCH_SIZE) val defaultMinSize: Int = (0.25 * standardSize).toInt.max(1) - val minSize = conf - .getConf(COLUMNAR_VELOX_RESIZE_BATCHES_SHUFFLE_INPUT_MIN_SIZE) + val minSize = getConf(COLUMNAR_VELOX_RESIZE_BATCHES_SHUFFLE_INPUT_MIN_SIZE) .getOrElse(defaultMinSize) ResizeRange(minSize, Int.MaxValue) } def chColumnarShuffleSpillThreshold: Long = { - val threshold = conf.getConf(COLUMNAR_CH_SHUFFLE_SPILL_THRESHOLD) + val threshold = getConf(COLUMNAR_CH_SHUFFLE_SPILL_THRESHOLD) if (threshold == 0) { - (conf.getConf(COLUMNAR_TASK_OFFHEAP_SIZE_IN_BYTES) * 0.9).toLong + (getConf(COLUMNAR_TASK_OFFHEAP_SIZE_IN_BYTES) * 0.9).toLong } else { threshold } } - def chColumnarMaxSortBufferSize: Long = conf.getConf(COLUMNAR_CH_MAX_SORT_BUFFER_SIZE) + def chColumnarMaxSortBufferSize: Long = getConf(COLUMNAR_CH_MAX_SORT_BUFFER_SIZE) def chColumnarForceMemorySortShuffle: Boolean = - conf.getConf(COLUMNAR_CH_FORCE_MEMORY_SORT_SHUFFLE) + getConf(COLUMNAR_CH_FORCE_MEMORY_SORT_SHUFFLE) def cartesianProductTransformerEnabled: Boolean = - conf.getConf(CARTESIAN_PRODUCT_TRANSFORMER_ENABLED) + getConf(CARTESIAN_PRODUCT_TRANSFORMER_ENABLED) def broadcastNestedLoopJoinTransformerTransformerEnabled: Boolean = - conf.getConf(BROADCAST_NESTED_LOOP_JOIN_TRANSFORMER_ENABLED) + getConf(BROADCAST_NESTED_LOOP_JOIN_TRANSFORMER_ENABLED) - def transformPlanLogLevel: String = conf.getConf(TRANSFORM_PLAN_LOG_LEVEL) + def transformPlanLogLevel: String = getConf(TRANSFORM_PLAN_LOG_LEVEL) - def substraitPlanLogLevel: String = conf.getConf(SUBSTRAIT_PLAN_LOG_LEVEL) + def substraitPlanLogLevel: String = getConf(SUBSTRAIT_PLAN_LOG_LEVEL) - def validationLogLevel: String = conf.getConf(VALIDATION_LOG_LEVEL) + def validationLogLevel: String = getConf(VALIDATION_LOG_LEVEL) - def softAffinityLogLevel: String = conf.getConf(SOFT_AFFINITY_LOG_LEVEL) + def softAffinityLogLevel: String = getConf(SOFT_AFFINITY_LOG_LEVEL) // A comma-separated list of classes for the extended columnar pre rules - def extendedColumnarTransformRules: String = conf.getConf(EXTENDED_COLUMNAR_TRANSFORM_RULES) + def extendedColumnarTransformRules: String = getConf(EXTENDED_COLUMNAR_TRANSFORM_RULES) // A comma-separated list of classes for the extended columnar post rules - def extendedColumnarPostRules: String = conf.getConf(EXTENDED_COLUMNAR_POST_RULES) + def extendedColumnarPostRules: String = getConf(EXTENDED_COLUMNAR_POST_RULES) - def extendedExpressionTransformer: String = conf.getConf(EXTENDED_EXPRESSION_TRAN_CONF) + def extendedExpressionTransformer: String = getConf(EXTENDED_EXPRESSION_TRAN_CONF) def expressionBlacklist: Set[String] = { - val blacklist = conf.getConf(EXPRESSION_BLACK_LIST) + val blacklist = getConf(EXPRESSION_BLACK_LIST) val blacklistSet: Set[String] = if (blacklist.isDefined) { blacklist.get.toLowerCase(Locale.ROOT).trim.split(",").toSet } else { Set.empty } - if (conf.getConf(FALLBACK_REGEXP_EXPRESSIONS)) { + if (getConf(FALLBACK_REGEXP_EXPRESSIONS)) { val regexpList = "rlike,regexp_replace,regexp_extract,regexp_extract_all,split" regexpList.trim.split(",").toSet ++ blacklistSet } else { @@ -420,87 +421,93 @@ class GlutenConfig(conf: SQLConf) extends Logging { } def printStackOnValidationFailure: Boolean = - conf.getConf(VALIDATION_PRINT_FAILURE_STACK_) + getConf(VALIDATION_PRINT_FAILURE_STACK_) - def enableFallbackReport: Boolean = conf.getConf(FALLBACK_REPORTER_ENABLED) + def enableFallbackReport: Boolean = getConf(FALLBACK_REPORTER_ENABLED) def enableVeloxUserExceptionStacktrace: Boolean = - conf.getConf(COLUMNAR_VELOX_ENABLE_USER_EXCEPTION_STACKTRACE) + getConf(COLUMNAR_VELOX_ENABLE_USER_EXCEPTION_STACKTRACE) def memoryUseHugePages: Boolean = - conf.getConf(COLUMNAR_VELOX_MEMORY_USE_HUGE_PAGES) - - def debug: Boolean = conf.getConf(DEBUG_ENABLED) - def debugKeepJniWorkspace: Boolean = conf.getConf(DEBUG_KEEP_JNI_WORKSPACE) - def collectUtStats: Boolean = conf.getConf(UT_STATISTIC) - def benchmarkStageId: Int = conf.getConf(BENCHMARK_TASK_STAGEID) - def benchmarkPartitionId: String = conf.getConf(BENCHMARK_TASK_PARTITIONID) - def benchmarkTaskId: String = conf.getConf(BENCHMARK_TASK_TASK_ID) - def benchmarkSaveDir: String = conf.getConf(BENCHMARK_SAVE_DIR) - def textInputMaxBlockSize: Long = conf.getConf(TEXT_INPUT_ROW_MAX_BLOCK_SIZE) - def textIputEmptyAsDefault: Boolean = conf.getConf(TEXT_INPUT_EMPTY_AS_DEFAULT) + getConf(COLUMNAR_VELOX_MEMORY_USE_HUGE_PAGES) + + def debug: Boolean = getConf(DEBUG_ENABLED) + def debugKeepJniWorkspace: Boolean = getConf(DEBUG_KEEP_JNI_WORKSPACE) + def collectUtStats: Boolean = getConf(UT_STATISTIC) + def benchmarkStageId: Int = getConf(BENCHMARK_TASK_STAGEID) + def benchmarkPartitionId: String = getConf(BENCHMARK_TASK_PARTITIONID) + def benchmarkTaskId: String = getConf(BENCHMARK_TASK_TASK_ID) + def benchmarkSaveDir: String = getConf(BENCHMARK_SAVE_DIR) + def textInputMaxBlockSize: Long = getConf(TEXT_INPUT_ROW_MAX_BLOCK_SIZE) + def textIputEmptyAsDefault: Boolean = getConf(TEXT_INPUT_EMPTY_AS_DEFAULT) def enableParquetRowGroupMaxMinIndex: Boolean = - conf.getConf(ENABLE_PARQUET_ROW_GROUP_MAX_MIN_INDEX) + getConf(ENABLE_PARQUET_ROW_GROUP_MAX_MIN_INDEX) def enableVeloxFlushablePartialAggregation: Boolean = - conf.getConf(VELOX_FLUSHABLE_PARTIAL_AGGREGATION_ENABLED) + getConf(VELOX_FLUSHABLE_PARTIAL_AGGREGATION_ENABLED) def maxFlushableAggregationMemoryRatio: Double = - conf.getConf(MAX_PARTIAL_AGGREGATION_MEMORY_RATIO) + getConf(MAX_PARTIAL_AGGREGATION_MEMORY_RATIO) def maxExtendedFlushableAggregationMemoryRatio: Double = - conf.getConf(MAX_PARTIAL_AGGREGATION_MEMORY_RATIO) + getConf(MAX_PARTIAL_AGGREGATION_MEMORY_RATIO) def abandonFlushableAggregationMinPct: Int = - conf.getConf(ABANDON_PARTIAL_AGGREGATION_MIN_PCT) + getConf(ABANDON_PARTIAL_AGGREGATION_MIN_PCT) def abandonFlushableAggregationMinRows: Int = - conf.getConf(ABANDON_PARTIAL_AGGREGATION_MIN_ROWS) + getConf(ABANDON_PARTIAL_AGGREGATION_MIN_ROWS) // Please use `BackendsApiManager.getSettings.enableNativeWriteFiles()` instead - def enableNativeWriter: Option[Boolean] = conf.getConf(NATIVE_WRITER_ENABLED) + def enableNativeWriter: Option[Boolean] = getConf(NATIVE_WRITER_ENABLED) - def enableNativeArrowReader: Boolean = conf.getConf(NATIVE_ARROW_READER_ENABLED) + def enableNativeArrowReader: Boolean = getConf(NATIVE_ARROW_READER_ENABLED) def directorySizeGuess: Long = - conf.getConf(DIRECTORY_SIZE_GUESS) + getConf(DIRECTORY_SIZE_GUESS) def filePreloadThreshold: Long = - conf.getConf(FILE_PRELOAD_THRESHOLD) + getConf(FILE_PRELOAD_THRESHOLD) def prefetchRowGroups: Int = - conf.getConf(PREFETCH_ROW_GROUPS) + getConf(PREFETCH_ROW_GROUPS) def loadQuantum: Long = - conf.getConf(LOAD_QUANTUM) + getConf(LOAD_QUANTUM) def maxCoalescedDistance: String = - conf.getConf(MAX_COALESCED_DISTANCE_BYTES) + getConf(MAX_COALESCED_DISTANCE_BYTES) def maxCoalescedBytes: Long = - conf.getConf(MAX_COALESCED_BYTES) + getConf(MAX_COALESCED_BYTES) def cachePrefetchMinPct: Int = - conf.getConf(CACHE_PREFETCH_MINPCT) + getConf(CACHE_PREFETCH_MINPCT) - def enableColumnarProjectCollapse: Boolean = conf.getConf(ENABLE_COLUMNAR_PROJECT_COLLAPSE) + def enableColumnarProjectCollapse: Boolean = getConf(ENABLE_COLUMNAR_PROJECT_COLLAPSE) - def enableColumnarPartialProject: Boolean = conf.getConf(ENABLE_COLUMNAR_PARTIAL_PROJECT) + def enableColumnarPartialProject: Boolean = getConf(ENABLE_COLUMNAR_PARTIAL_PROJECT) - def awsSdkLogLevel: String = conf.getConf(AWS_SDK_LOG_LEVEL) + def awsSdkLogLevel: String = getConf(AWS_SDK_LOG_LEVEL) - def awsS3RetryMode: String = conf.getConf(AWS_S3_RETRY_MODE) + def awsS3RetryMode: String = getConf(AWS_S3_RETRY_MODE) - def awsConnectionTimeout: String = conf.getConf(AWS_S3_CONNECT_TIMEOUT) + def awsConnectionTimeout: String = getConf(AWS_S3_CONNECT_TIMEOUT) - def enableCastAvgAggregateFunction: Boolean = conf.getConf(COLUMNAR_NATIVE_CAST_AGGREGATE_ENABLED) + def enableCastAvgAggregateFunction: Boolean = getConf(COLUMNAR_NATIVE_CAST_AGGREGATE_ENABLED) def dynamicOffHeapSizingEnabled: Boolean = - conf.getConf(DYNAMIC_OFFHEAP_SIZING_ENABLED) + getConf(DYNAMIC_OFFHEAP_SIZING_ENABLED) - def enableHiveFileFormatWriter: Boolean = conf.getConf(NATIVE_HIVEFILEFORMAT_WRITER_ENABLED) + def enableHiveFileFormatWriter: Boolean = getConf(NATIVE_HIVEFILEFORMAT_WRITER_ENABLED) - def enableCelebornFallback: Boolean = conf.getConf(CELEBORN_FALLBACK_ENABLED) + def enableCelebornFallback: Boolean = getConf(CELEBORN_FALLBACK_ENABLED) - def enableHdfsViewfs: Boolean = conf.getConf(HDFS_VIEWFS_ENABLED) + def enableHdfsViewfs: Boolean = getConf(HDFS_VIEWFS_ENABLED) def enableBroadcastBuildRelationInOffheap: Boolean = - conf.getConf(VELOX_BROADCAST_BUILD_RELATION_USE_OFFHEAP) + getConf(VELOX_BROADCAST_BUILD_RELATION_USE_OFFHEAP) } object GlutenConfig { import SQLConf._ + def buildConf(key: String): ConfigBuilder = ConfigBuilder(key) + + def buildStaticConf(key: String): ConfigBuilder = { + ConfigBuilder(key).onCreate(_ => SQLConf.registerStaticConfigKey(key)) + } + val GLUTEN_ENABLED_BY_DEFAULT = true val GLUTEN_ENABLED_KEY = "spark.gluten.enabled" val GLUTEN_LIB_NAME = "spark.gluten.sql.columnar.libname" @@ -2184,13 +2191,6 @@ object GlutenConfig { .booleanConf .createWithDefault(true) - val VELOX_FORCE_PARQUET_TIMESTAMP_TYPE_SCAN_FALLBACK = - buildConf("spark.gluten.sql.parquet.timestampType.scan.fallback.enabled") - .internal() - .doc("Force fallback for parquet timestamp type scan.") - .booleanConf - .createWithDefault(false) - val VELOX_SCAN_FILE_SCHEME_VALIDATION_ENABLED = buildConf("spark.gluten.sql.scan.fileSchemeValidation.enabled") .internal() diff --git a/shims/common/src/main/scala/org/apache/spark/sql/internal/ConfigProvider.scala b/shims/common/src/main/scala/org/apache/spark/sql/internal/ConfigProvider.scala new file mode 100644 index 000000000000..8293a4a09bfb --- /dev/null +++ b/shims/common/src/main/scala/org/apache/spark/sql/internal/ConfigProvider.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.spark.sql.internal + +/** A source of configuration values. */ +trait ConfigProvider { + def get(key: String): Option[String] +} + +class SQLConfProvider(conf: SQLConf) extends ConfigProvider { + override def get(key: String): Option[String] = Option(conf.settings.get(key)) +} + +class MapProvider(conf: Map[String, String]) extends ConfigProvider { + override def get(key: String): Option[String] = conf.get(key) +} diff --git a/shims/common/src/main/scala/org/apache/spark/sql/internal/GlutenConfigUtil.scala b/shims/common/src/main/scala/org/apache/spark/sql/internal/GlutenConfigUtil.scala index 1a45572acdac..87b90938a025 100644 --- a/shims/common/src/main/scala/org/apache/spark/sql/internal/GlutenConfigUtil.scala +++ b/shims/common/src/main/scala/org/apache/spark/sql/internal/GlutenConfigUtil.scala @@ -16,15 +16,13 @@ */ package org.apache.spark.sql.internal -import org.apache.spark.internal.config.ConfigReader - -import scala.collection.JavaConverters._ +import org.apache.gluten.config._ object GlutenConfigUtil { - private def getConfString(reader: ConfigReader, key: String, value: String): String = { - Option(SQLConf.getConfigEntry(key)) + private def getConfString(configProvider: ConfigProvider, key: String, value: String): String = { + Option(ConfigEntry.findEntry(key)) .map { - _.readFrom(reader) match { + _.readFrom(configProvider) match { case o: Option[_] => o.map(_.toString).getOrElse(value) case null => value case v => v.toString @@ -34,11 +32,11 @@ object GlutenConfigUtil { } def parseConfig(conf: Map[String, String]): Map[String, String] = { - val reader = new ConfigReader(conf.filter(_._1.startsWith("spark.gluten.")).asJava) + val provider = new MapProvider(conf.filter(_._1.startsWith("spark.gluten."))) conf.map { case (k, v) => if (k.startsWith("spark.gluten.")) { - (k, getConfString(reader, k, v)) + (k, getConfString(provider, k, v)) } else { (k, v) } diff --git a/tools/gluten-it/common/src/main/java/org/apache/gluten/integration/BaseMixin.java b/tools/gluten-it/common/src/main/java/org/apache/gluten/integration/BaseMixin.java index 4ebf6106fd5f..ada777e87e49 100644 --- a/tools/gluten-it/common/src/main/java/org/apache/gluten/integration/BaseMixin.java +++ b/tools/gluten-it/common/src/main/java/org/apache/gluten/integration/BaseMixin.java @@ -21,6 +21,7 @@ import org.apache.gluten.integration.command.SparkRunModes; import org.apache.gluten.integration.ds.TpcdsSuite; import org.apache.gluten.integration.h.TpchSuite; +import org.apache.gluten.integration.metrics.MetricMapper; import org.apache.log4j.LogManager; import org.apache.log4j.Level; import org.apache.spark.SparkConf; @@ -83,24 +84,11 @@ public class BaseMixin { private Map extraSparkConf = Collections.emptyMap(); private SparkConf pickSparkConf(String preset) { - SparkConf conf; - switch (preset) { - case "vanilla": - conf = Constants.VANILLA_CONF(); - break; - case "velox": - conf = Constants.VELOX_CONF(); - break; - case "velox-with-celeborn": - conf = Constants.VELOX_WITH_CELEBORN_CONF(); - break; - case "velox-with-uniffle": - conf = Constants.VELOX_WITH_UNIFFLE_CONF(); - break; - default: - throw new IllegalArgumentException("Preset not found: " + preset); - } - return conf; + return Preset.get(preset).getConf(); + } + + private MetricMapper pickMetricMapper(String preset) { + return Preset.get(preset).getMetricMapper(); } public Integer runActions(Action[] actions) { @@ -129,25 +117,28 @@ public Integer runActions(Action[] actions) { mergeMapSafe(extraSparkConf, runModeEnumeration.extraSparkConf())).asScala().toMap( Predef.conforms()); + final MetricMapper baselineMetricMapper = pickMetricMapper(baselinePreset); + final MetricMapper testMetricMapper = pickMetricMapper(preset); + final Suite suite; switch (benchmarkType) { case "h": suite = new TpchSuite(runModeEnumeration.getSparkMasterUrl(), actions, testConf, baselineConf, extraSparkConfScala, level, errorOnMemLeak, dataDir, enableUi, enableHsUi, hsUiPort, disableAqe, disableBhj, - disableWscg, shufflePartitions, scanPartitions); + disableWscg, shufflePartitions, scanPartitions, baselineMetricMapper, testMetricMapper); break; case "ds": suite = new TpcdsSuite(runModeEnumeration.getSparkMasterUrl(), actions, testConf, baselineConf, extraSparkConfScala, level, errorOnMemLeak, dataDir, enableUi, enableHsUi, hsUiPort, disableAqe, disableBhj, - disableWscg, shufflePartitions, scanPartitions); + disableWscg, shufflePartitions, scanPartitions, baselineMetricMapper, testMetricMapper); break; case "clickbench": suite = new ClickBenchSuite(runModeEnumeration.getSparkMasterUrl(), actions, testConf, baselineConf, extraSparkConfScala, level, errorOnMemLeak, dataDir, enableUi, enableHsUi, hsUiPort, disableAqe, disableBhj, - disableWscg, shufflePartitions, scanPartitions); + disableWscg, shufflePartitions, scanPartitions, baselineMetricMapper, testMetricMapper); break; default: throw new IllegalArgumentException("TPC benchmark type not found: " + benchmarkType); diff --git a/tools/gluten-it/common/src/main/java/org/apache/gluten/integration/Preset.java b/tools/gluten-it/common/src/main/java/org/apache/gluten/integration/Preset.java new file mode 100644 index 000000000000..74eff7638721 --- /dev/null +++ b/tools/gluten-it/common/src/main/java/org/apache/gluten/integration/Preset.java @@ -0,0 +1,57 @@ +/* + * 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.integration; + +import org.apache.gluten.integration.metrics.MetricMapper; +import org.apache.spark.SparkConf; + +import java.util.HashMap; +import java.util.Map; + +public final class Preset { + private static final Map presets = new HashMap<>(); + + static { + presets.put("vanilla", new Preset(Constants.VANILLA_CONF(), Constants.VANILLA_METRIC_MAPPER())); + presets.put("velox", new Preset(Constants.VELOX_CONF(), Constants.VELOX_METRIC_MAPPER())); + presets.put("velox-with-celeborn", new Preset(Constants.VELOX_WITH_CELEBORN_CONF(), Constants.VELOX_METRIC_MAPPER())); + presets.put("velox-with-uniffle", new Preset(Constants.VELOX_WITH_UNIFFLE_CONF(), Constants.VELOX_METRIC_MAPPER())); + } + + public static Preset get(String name) { + if (!presets.containsKey(name)) { + throw new IllegalArgumentException("Non-existing preset name: " + name); + } + return presets.get(name); + } + + private final SparkConf conf; + private final MetricMapper metricMapper; + + public Preset(SparkConf conf, MetricMapper metricMapper) { + this.conf = conf; + this.metricMapper = metricMapper; + } + + public SparkConf getConf() { + return conf; + } + + public MetricMapper getMetricMapper() { + return metricMapper; + } +} diff --git a/tools/gluten-it/common/src/main/java/org/apache/gluten/integration/command/Parameterized.java b/tools/gluten-it/common/src/main/java/org/apache/gluten/integration/command/Parameterized.java index 225b492ef1ae..0b96ae3c7108 100644 --- a/tools/gluten-it/common/src/main/java/org/apache/gluten/integration/command/Parameterized.java +++ b/tools/gluten-it/common/src/main/java/org/apache/gluten/integration/command/Parameterized.java @@ -46,7 +46,7 @@ public class Parameterized implements Callable { @CommandLine.Option(names = {"--warmup-iterations"}, description = "Dry-run iterations before actually run the test", defaultValue = "0") private int warmupIterations; - @CommandLine.Option(names = {"-m", "--metric"}, description = "Specify a series of metrics to collect during execution") + @CommandLine.Option(names = {"-m", "--metric"}, description = "Specify a series of executor metrics to collect during execution") private String[] metrics = new String[0]; @CommandLine.Option(names = {"-d", "--dim"}, description = "Set a series of dimensions consisting of possible config options, example: -d=offheap:1g,spark.memory.offHeap.enabled=true,spark.memory.offHeap.size=1g") @@ -131,7 +131,7 @@ public Integer call() throws Exception { new org.apache.gluten.integration.action.Parameterized(dataGenMixin.getScale(), dataGenMixin.genPartitionedData(), queriesMixin.queries(), queriesMixin.explain(), queriesMixin.iterations(), warmupIterations, queriesMixin.noSessionReuse(), parsedDims, - excludedCombinations, metrics); + excludedCombinations, JavaConverters.asScalaBufferConverter(Arrays.asList(metrics)).asScala()); return mixin.runActions(ArrayUtils.addAll(dataGenMixin.makeActions(), parameterized)); } } diff --git a/tools/gluten-it/common/src/main/java/org/apache/gluten/integration/command/Queries.java b/tools/gluten-it/common/src/main/java/org/apache/gluten/integration/command/Queries.java index c19d66bdae75..456998cb2f07 100644 --- a/tools/gluten-it/common/src/main/java/org/apache/gluten/integration/command/Queries.java +++ b/tools/gluten-it/common/src/main/java/org/apache/gluten/integration/command/Queries.java @@ -18,8 +18,14 @@ import org.apache.gluten.integration.BaseMixin; import org.apache.commons.lang3.ArrayUtils; +import org.apache.gluten.integration.metrics.PlanMetric; import picocli.CommandLine; +import scala.collection.JavaConverters; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Set; import java.util.concurrent.Callable; @CommandLine.Command(name = "queries", mixinStandardHelpOptions = true, @@ -38,11 +44,18 @@ public class Queries implements Callable { @CommandLine.Option(names = {"--random-kill-tasks"}, description = "Every single task will get killed and retried after running for some time", defaultValue = "false") private boolean randomKillTasks; + @CommandLine.Option(names = {"--sql-metrics"}, description = "Collect SQL metrics from run queries and generate a simple report based on them. Available types: execution-time") + private Set collectSqlMetrics = Collections.emptySet(); + @Override public Integer call() throws Exception { + final List metricsReporters = new ArrayList<>(); + for (String type : collectSqlMetrics) { + metricsReporters.add(PlanMetric.newReporter(type)); + } org.apache.gluten.integration.action.Queries queries = new org.apache.gluten.integration.action.Queries(dataGenMixin.getScale(), dataGenMixin.genPartitionedData(), queriesMixin.queries(), - queriesMixin.explain(), queriesMixin.iterations(), randomKillTasks, queriesMixin.noSessionReuse()); + queriesMixin.explain(), queriesMixin.iterations(), randomKillTasks, queriesMixin.noSessionReuse(), JavaConverters.asScalaBufferConverter(metricsReporters).asScala()); return mixin.runActions(ArrayUtils.addAll(dataGenMixin.makeActions(), queries)); } } diff --git a/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/Constants.scala b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/Constants.scala index e680ce9d5dda..8804b2265295 100644 --- a/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/Constants.scala +++ b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/Constants.scala @@ -16,16 +16,12 @@ */ package org.apache.gluten.integration +import org.apache.gluten.integration.metrics.MetricMapper +import org.apache.gluten.integration.metrics.MetricMapper.SelfTimeMapper + import org.apache.spark.SparkConf import org.apache.spark.sql.TypeUtils -import org.apache.spark.sql.types.{ - DateType, - DecimalType, - DoubleType, - IntegerType, - LongType, - StringType -} +import org.apache.spark.sql.types._ import java.sql.Date @@ -40,16 +36,17 @@ object Constants { .set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager") .set("spark.sql.optimizer.runtime.bloomFilter.enabled", "true") .set("spark.sql.optimizer.runtime.bloomFilter.applicationSideScanSizeThreshold", "0") - .set("spark.gluten.sql.columnar.physicalJoinOptimizeEnable", "false") // q72 slow if false, q64 fails if true + .set( + "spark.gluten.sql.columnar.physicalJoinOptimizeEnable", + "false" + ) // q72 slow if false, q64 fails if true val VELOX_WITH_CELEBORN_CONF: SparkConf = new SparkConf(false) .set("spark.gluten.sql.columnar.forceShuffledHashJoin", "true") .set("spark.gluten.sql.columnar.shuffle.celeborn.fallback.enabled", "false") .set("spark.sql.parquet.enableVectorizedReader", "true") .set("spark.plugins", "org.apache.gluten.GlutenPlugin") - .set( - "spark.shuffle.manager", - "org.apache.spark.shuffle.gluten.celeborn.CelebornShuffleManager") + .set("spark.shuffle.manager", "org.apache.spark.shuffle.gluten.celeborn.CelebornShuffleManager") .set("spark.celeborn.shuffle.writer", "hash") .set("spark.celeborn.push.replicate.enabled", "false") .set("spark.celeborn.client.shuffle.compression.codec", "none") @@ -58,7 +55,10 @@ object Constants { .set("spark.dynamicAllocation.enabled", "false") .set("spark.sql.optimizer.runtime.bloomFilter.enabled", "true") .set("spark.sql.optimizer.runtime.bloomFilter.applicationSideScanSizeThreshold", "0") - .set("spark.gluten.sql.columnar.physicalJoinOptimizeEnable", "false") // q72 slow if false, q64 fails if true + .set( + "spark.gluten.sql.columnar.physicalJoinOptimizeEnable", + "false" + ) // q72 slow if false, q64 fails if true .set("spark.celeborn.push.data.timeout", "600s") .set("spark.celeborn.push.limit.inFlight.timeout", "1200s") @@ -78,6 +78,49 @@ object Constants { .set("spark.sql.optimizer.runtime.bloomFilter.applicationSideScanSizeThreshold", "0") .set("spark.gluten.sql.columnar.physicalJoinOptimizeEnable", "false") + val VANILLA_METRIC_MAPPER: MetricMapper = SelfTimeMapper( + Map( + "FileSourceScanExec" -> Set("metadataTime", "scanTime"), + "HashAggregateExec" -> Set("aggTime"), + "ProjectExec" -> Set(), // No available metrics provided by vanilla Spark. + "FilterExec" -> Set(), // No available metrics provided by vanilla Spark. + "WindowExec" -> Set(), // No available metrics provided by vanilla Spark. + "BroadcastExchangeExec" -> Set("broadcastTime", "buildTime", "collectTime"), + "BroadcastHashJoinExec" -> Set(), // No available metrics provided by vanilla Spark. + "ColumnarToRowExec" -> Set(), // No available metrics provided by vanilla Spark. + "ShuffleExchangeExec" -> Set("fetchWaitTime", "shuffleWriteTime"), + "ShuffledHashJoinExec" -> Set("buildTime"), + "WindowGroupLimitExec" -> Set() // No available metrics provided by vanilla Spark. + )) + + val VELOX_METRIC_MAPPER: MetricMapper = VANILLA_METRIC_MAPPER.and( + SelfTimeMapper( + Map( + "FileSourceScanExecTransformer" -> Set("scanTime", "pruningTime", "remainingFilterTime"), + "ProjectExecTransformer" -> Set("wallNanos"), + "FilterExecTransformer" -> Set("wallNanos"), + "SortExecTransformer" -> Set("wallNanos"), + "RegularHashAggregateExecTransformer" -> Set("aggWallNanos", "rowConstructionWallNanos"), + "FlushableHashAggregateExecTransformer" -> Set("aggWallNanos", "rowConstructionWallNanos"), + "VeloxColumnarToRowExec" -> Set("convertTime"), + "RowToVeloxColumnarExec" -> Set("convertTime"), + "VeloxResizeBatchesExec" -> Set("selfTime"), + "ColumnarShuffleExchangeExec" -> Set( + "splitTime", + "shuffleWallTime", + "fetchWaitTime", + "decompressTime", + "deserializeTime"), + "ShuffledHashJoinExecTransformer" -> Set("hashBuildWallNanos", "hashProbeWallNanos"), + "ColumnarBroadcastExchangeExec" -> Set("broadcastTime", "collectTime"), + "BroadcastHashJoinExecTransformer" -> Set("hashBuildWallNanos", "hashProbeWallNanos"), + "WindowExecTransformer" -> Set("wallNanos"), + "WindowGroupLimitExecTransformer" -> Set("wallNanos"), + "VeloxBroadcastNestedLoopJoinExecTransformer" -> Set("wallNanos"), + "ExpandExecTransformer" -> Set("wallNanos") + ) + )) + @deprecated val TYPE_MODIFIER_DATE_AS_DOUBLE: TypeModifier = new TypeModifier(TypeUtils.typeAccepts(_, DateType), DoubleType) { diff --git a/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/QueryRunner.scala b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/QueryRunner.scala index 9791242f18cd..35e86c499499 100644 --- a/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/QueryRunner.scala +++ b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/QueryRunner.scala @@ -16,9 +16,12 @@ */ package org.apache.gluten.integration +import org.apache.gluten.integration.metrics.MetricMapper + +import org.apache.spark.sql.{RunResult, SparkQueryRunner, SparkSession} + import com.google.common.base.Preconditions import org.apache.commons.lang3.exception.ExceptionUtils -import org.apache.spark.sql.{RunResult, SparkQueryRunner, SparkSession} import java.io.File @@ -39,11 +42,20 @@ class QueryRunner(val queryResourceFolder: String, val dataPath: String) { desc: String, caseId: String, explain: Boolean = false, - metrics: Array[String] = Array(), + sqlMetricMapper: MetricMapper = MetricMapper.dummy, + executorMetrics: Seq[String] = Nil, randomKillTasks: Boolean = false): QueryResult = { val path = "%s/%s.sql".format(queryResourceFolder, caseId) try { - val r = SparkQueryRunner.runQuery(spark, desc, path, explain, metrics, randomKillTasks) + val r = + SparkQueryRunner.runQuery( + spark, + desc, + path, + explain, + sqlMetricMapper, + executorMetrics, + randomKillTasks) println(s"Successfully ran query $caseId. Returned row count: ${r.rows.length}") Success(caseId, r) } catch { diff --git a/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/ShimUtils.scala b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/ShimUtils.scala index d2986bfa7c0c..90737394341f 100644 --- a/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/ShimUtils.scala +++ b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/ShimUtils.scala @@ -17,25 +17,25 @@ package org.apache.gluten.integration +import org.apache.spark.VersionUtils import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.encoders.{ExpressionEncoder, RowEncoder} import org.apache.spark.sql.types.StructType object ShimUtils { - def getExpressionEncoder(schema: StructType): ExpressionEncoder[Row] = { - try { + val sparkVersion = VersionUtils.majorMinorVersion() + if (VersionUtils.compareMajorMinorVersion(sparkVersion, (3, 5)) < 0) { RowEncoder.getClass .getMethod("apply", classOf[StructType]) .invoke(RowEncoder, schema) .asInstanceOf[ExpressionEncoder[Row]] - } catch { - case _: Exception => - // to be compatible with Spark 3.5 and later - ExpressionEncoder.getClass - .getMethod("apply", classOf[StructType]) - .invoke(ExpressionEncoder, schema) - .asInstanceOf[ExpressionEncoder[Row]] + } else { + // to be compatible with Spark 3.5 and later + ExpressionEncoder.getClass + .getMethod("apply", classOf[StructType]) + .invoke(ExpressionEncoder, schema) + .asInstanceOf[ExpressionEncoder[Row]] } } } diff --git a/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/Suite.scala b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/Suite.scala index 86847ed09a5a..2e5d781bb5d5 100644 --- a/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/Suite.scala +++ b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/Suite.scala @@ -17,6 +17,7 @@ package org.apache.gluten.integration import org.apache.gluten.integration.action.Action +import org.apache.gluten.integration.metrics.MetricMapper import org.apache.spark.SparkConf import org.apache.spark.deploy.history.HistoryServerHelper @@ -44,7 +45,9 @@ abstract class Suite( private val disableBhj: Boolean, private val disableWscg: Boolean, private val shufflePartitions: Int, - private val scanPartitions: Int) { + private val scanPartitions: Int, + private val baselineMetricMapper: MetricMapper, + private val testMetricMapper: MetricMapper) { resetLogLevel() @@ -174,6 +177,14 @@ abstract class Suite( testConf.clone() } + private[integration] def getBaselineMetricMapper(): MetricMapper = { + baselineMetricMapper + } + + private[integration] def getTestMetricMapper(): MetricMapper = { + testMetricMapper + } + protected def historyWritePath(): String private[integration] def dataWritePath(scale: Double, genPartitionedData: Boolean): String @@ -185,7 +196,6 @@ abstract class Suite( private[integration] def allQueryIds(): Array[String] private[integration] def desc(): String - } object Suite {} diff --git a/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/action/Parameterized.scala b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/action/Parameterized.scala index c9ebb9754394..95a2dd6313da 100644 --- a/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/action/Parameterized.scala +++ b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/action/Parameterized.scala @@ -40,7 +40,7 @@ class Parameterized( noSessionReuse: Boolean, configDimensions: Seq[Parameterized.Dim], excludedCombinations: Seq[Set[Parameterized.DimKv]], - metrics: Array[String]) + metrics: Seq[String]) extends Action { import Parameterized._ @@ -301,7 +301,7 @@ object Parameterized { coords.foreach(coord => inc .next() - .write(coord.queryResult.asSuccessOption().map(_.runResult.metrics(metricName))))) + .write(coord.queryResult.asSuccessOption().map(_.runResult.executorMetrics(metricName))))) coords.foreach(coord => inc .next() @@ -346,9 +346,9 @@ object Parameterized { coordinate: Coordinate, desc: String, explain: Boolean, - metrics: Array[String]): TestResultLine.Coord = { + metrics: Seq[String]): TestResultLine.Coord = { val testDesc = "Query %s [%s] %s".format(desc, id, coordinate) - val result = runner.runQuery(spark, testDesc, id, explain, metrics) + val result = runner.runQuery(spark, testDesc, id, explain, executorMetrics = metrics) TestResultLine.Coord(coordinate, result) } @@ -358,6 +358,6 @@ object Parameterized { id: String, coordinate: Coordinate, desc: String): Unit = { - runQuery(runner, session, id, coordinate, desc, explain = false, Array.empty) + runQuery(runner, session, id, coordinate, desc, explain = false, Nil) } } diff --git a/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/action/Queries.scala b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/action/Queries.scala index b8a42f393932..bfcb1aafd177 100644 --- a/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/action/Queries.scala +++ b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/action/Queries.scala @@ -16,12 +16,14 @@ */ package org.apache.gluten.integration.action +import org.apache.gluten.integration.{QueryRunner, Suite, TableCreator} import org.apache.gluten.integration.QueryRunner.QueryResult import org.apache.gluten.integration.action.Actions.QuerySelector import org.apache.gluten.integration.action.TableRender.RowParser.FieldAppender.RowAppender +import org.apache.gluten.integration.metrics.{MetricMapper, PlanMetric} import org.apache.gluten.integration.stat.RamStat -import org.apache.gluten.integration.{QueryRunner, Suite, TableCreator} -import org.apache.spark.sql.{SparkSession} + +import org.apache.spark.sql.SparkSession case class Queries( scale: Double, @@ -30,8 +32,9 @@ case class Queries( explain: Boolean, iterations: Int, randomKillTasks: Boolean, - noSessionReuse: Boolean) - extends Action { + noSessionReuse: Boolean, + metricsReporters: Seq[PlanMetric.Reporter]) + extends Action { import Queries._ override def execute(suite: Suite): Boolean = { @@ -52,6 +55,7 @@ case class Queries( queryId, suite.desc(), explain, + suite.getTestMetricMapper(), randomKillTasks) } finally { if (noSessionReuse) { @@ -64,6 +68,7 @@ case class Queries( val passedCount = results.count(l => l.queryResult.succeeded()) val count = results.count(_ => true) + val succeeded = results.filter(_.queryResult.succeeded()) // RAM stats println("Performing GC to collect RAM statistics... ") @@ -73,14 +78,22 @@ case class Queries( "RAM statistics: JVM Heap size: %d KiB (total %d KiB), Process RSS: %d KiB\n", RamStat.getJvmHeapUsed(), RamStat.getJvmHeapTotal(), - RamStat.getProcessRamUsed()) - + RamStat.getProcessRamUsed() + ) println("") + + val sqlMetrics = succeeded.flatMap(_.queryResult.asSuccess().runResult.sqlMetrics) + metricsReporters.foreach { + r => + val report = r.toString(sqlMetrics) + println(report) + println("") + } + println("Test report: ") println("") printf("Summary: %d out of %d queries passed. \n", passedCount, count) println("") - val succeeded = results.filter(_.queryResult.succeeded()) val all = succeeded.map(_.queryResult).asSuccesses().agg("all").map(s => TestResultLine(s)) Queries.printResults(succeeded ++ all) println("") @@ -133,9 +146,7 @@ object Queries { "Plan Time (Millis)", "Query Time (Millis)") - results.foreach { line => - render.appendRow(line) - } + results.foreach(line => render.appendRow(line)) render.print(System.out) } @@ -147,11 +158,18 @@ object Queries { id: String, desc: String, explain: Boolean, + metricMapper: MetricMapper, randomKillTasks: Boolean): TestResultLine = { println(s"Running query: $id...") val testDesc = "Query %s [%s]".format(desc, id) val result = - runner.runQuery(session, testDesc, id, explain = explain, randomKillTasks = randomKillTasks) + runner.runQuery( + session, + testDesc, + id, + explain = explain, + sqlMetricMapper = metricMapper, + randomKillTasks = randomKillTasks) TestResultLine(result) } } diff --git a/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/action/TableRender.scala b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/action/TableRender.scala index 2b1cca61e3f4..c9cbc330dca9 100644 --- a/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/action/TableRender.scala +++ b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/action/TableRender.scala @@ -213,8 +213,16 @@ object TableRender { } object RowParser { + implicit object StringSeqParser extends RowParser[Seq[String]] { + override def parse(rowFactory: RowAppender, row: Seq[String]): Unit = { + val inc = rowFactory.incremental() + row.foreach(ceil => inc.next().write(ceil)) + } + } + trait FieldAppender { def child(name: String): FieldAppender + def write(value: Any): Unit } diff --git a/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/action/package.scala b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/action/package.scala index a84915ebe1a9..3324ff446c8d 100644 --- a/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/action/package.scala +++ b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/action/package.scala @@ -40,7 +40,8 @@ package object action { c1.runResult.rows ++ c2.runResult.rows, c1.runResult.planningTimeMillis + c2.runResult.planningTimeMillis, c1.runResult.executionTimeMillis + c2.runResult.executionTimeMillis, - (c1.runResult.metrics, c2.runResult.metrics).sumUp)) + c1.runResult.sqlMetrics ++ c2.runResult.sqlMetrics, + (c1.runResult.executorMetrics, c2.runResult.executorMetrics).sumUp)) } } } diff --git a/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/clickbench/ClickBenchSuite.scala b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/clickbench/ClickBenchSuite.scala index da0c6b8ce049..852c934564ae 100644 --- a/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/clickbench/ClickBenchSuite.scala +++ b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/clickbench/ClickBenchSuite.scala @@ -16,18 +16,20 @@ */ package org.apache.gluten.integration.clickbench -import org.apache.gluten.integration.action.Action import org.apache.gluten.integration.{DataGen, Suite, TableCreator} -import org.apache.log4j.Level +import org.apache.gluten.integration.action.Action +import org.apache.gluten.integration.metrics.MetricMapper + import org.apache.spark.SparkConf +import org.apache.log4j.Level + import java.io.File /** * ClickBench: a Benchmark For Analytical Databases * - * See the project: https://github.com/ClickHouse/ClickBench - * Site: https://benchmark.clickhouse.com/ + * See the project: https://github.com/ClickHouse/ClickBench Site: https://benchmark.clickhouse.com/ */ class ClickBenchSuite( val masterUrl: String, @@ -45,23 +47,28 @@ class ClickBenchSuite( val disableBhj: Boolean, val disableWscg: Boolean, val shufflePartitions: Int, - val scanPartitions: Int) - extends Suite( - masterUrl, - actions, - testConf, - baselineConf, - extraSparkConf, - logLevel, - errorOnMemLeak, - enableUi, - enableHsUi, - hsUiPort, - disableAqe, - disableBhj, - disableWscg, - shufflePartitions, - scanPartitions) { + val scanPartitions: Int, + val baselineMetricMapper: MetricMapper, + val testMetricMapper: MetricMapper) + extends Suite( + masterUrl, + actions, + testConf, + baselineConf, + extraSparkConf, + logLevel, + errorOnMemLeak, + enableUi, + enableHsUi, + hsUiPort, + disableAqe, + disableBhj, + disableWscg, + shufflePartitions, + scanPartitions, + baselineMetricMapper, + testMetricMapper + ) { import ClickBenchSuite._ override protected def historyWritePath(): String = HISTORY_WRITE_PATH @@ -94,7 +101,7 @@ private object ClickBenchSuite { private val ALL_QUERY_IDS = (1 to 43).map(i => s"q$i").toArray private def checkDataGenArgs(scale: Double, genPartitionedData: Boolean): Unit = { - assert(scale == 1.0D, "ClickBench suite doesn't support scale factor other than 1") + assert(scale == 1.0d, "ClickBench suite doesn't support scale factor other than 1") assert(!genPartitionedData, "ClickBench suite doesn't support generating partitioned data") } } diff --git a/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/ds/TpcdsSuite.scala b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/ds/TpcdsSuite.scala index 9e6fa1edb643..52a30a2282b6 100644 --- a/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/ds/TpcdsSuite.scala +++ b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/ds/TpcdsSuite.scala @@ -16,12 +16,15 @@ */ package org.apache.gluten.integration.ds +import org.apache.gluten.integration.{DataGen, Suite, TableCreator, TypeModifier} import org.apache.gluten.integration.action.Action import org.apache.gluten.integration.ds.TpcdsSuite.{ALL_QUERY_IDS, HISTORY_WRITE_PATH, TPCDS_WRITE_RELATIVE_PATH} -import org.apache.gluten.integration.{DataGen, Suite, TableCreator, TypeModifier} -import org.apache.log4j.Level +import org.apache.gluten.integration.metrics.MetricMapper + import org.apache.spark.SparkConf +import org.apache.log4j.Level + import java.io.File class TpcdsSuite( @@ -40,23 +43,28 @@ class TpcdsSuite( val disableBhj: Boolean, val disableWscg: Boolean, val shufflePartitions: Int, - val scanPartitions: Int) - extends Suite( - masterUrl, - actions, - testConf, - baselineConf, - extraSparkConf, - logLevel, - errorOnMemLeak, - enableUi, - enableHsUi, - hsUiPort, - disableAqe, - disableBhj, - disableWscg, - shufflePartitions, - scanPartitions) { + val scanPartitions: Int, + val baselineMetricMapper: MetricMapper, + val testMetricMapper: MetricMapper) + extends Suite( + masterUrl, + actions, + testConf, + baselineConf, + extraSparkConf, + logLevel, + errorOnMemLeak, + enableUi, + enableHsUi, + hsUiPort, + disableAqe, + disableBhj, + disableWscg, + shufflePartitions, + scanPartitions, + baselineMetricMapper, + testMetricMapper + ) { override protected def historyWritePath(): String = HISTORY_WRITE_PATH @@ -196,6 +204,7 @@ object TpcdsSuite { "q96", "q97", "q98", - "q99") + "q99" + ) private val HISTORY_WRITE_PATH = "/tmp/tpcds-history" } diff --git a/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/h/TpchSuite.scala b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/h/TpchSuite.scala index fc3ad1310649..e48d52393d79 100644 --- a/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/h/TpchSuite.scala +++ b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/h/TpchSuite.scala @@ -18,6 +18,7 @@ package org.apache.gluten.integration.h import org.apache.gluten.integration.action.Action import org.apache.gluten.integration.h.TpchSuite.{HISTORY_WRITE_PATH, TPCH_WRITE_RELATIVE_PATH} +import org.apache.gluten.integration.metrics.MetricMapper import org.apache.gluten.integration.{DataGen, Suite, TableCreator, TypeModifier} import org.apache.log4j.Level import org.apache.spark.SparkConf @@ -25,22 +26,24 @@ import org.apache.spark.SparkConf import java.io.File class TpchSuite( - val masterUrl: String, - val actions: Array[Action], - val testConf: SparkConf, - val baselineConf: SparkConf, - val extraSparkConf: Map[String, String], - val logLevel: Level, - val errorOnMemLeak: Boolean, - val dataDir: String, - val enableUi: Boolean, - val enableHsUi: Boolean, - val hsUiPort: Int, - val disableAqe: Boolean, - val disableBhj: Boolean, - val disableWscg: Boolean, - val shufflePartitions: Int, - val scanPartitions: Int) + val masterUrl: String, + val actions: Array[Action], + val testConf: SparkConf, + val baselineConf: SparkConf, + val extraSparkConf: Map[String, String], + val logLevel: Level, + val errorOnMemLeak: Boolean, + val dataDir: String, + val enableUi: Boolean, + val enableHsUi: Boolean, + val hsUiPort: Int, + val disableAqe: Boolean, + val disableBhj: Boolean, + val disableWscg: Boolean, + val shufflePartitions: Int, + val scanPartitions: Int, + val baselineMetricMapper: MetricMapper, + val testMetricMapper: MetricMapper) extends Suite( masterUrl, actions, @@ -56,7 +59,9 @@ class TpchSuite( disableBhj, disableWscg, shufflePartitions, - scanPartitions) { + scanPartitions, + baselineMetricMapper, + testMetricMapper) { override protected def historyWritePath(): String = HISTORY_WRITE_PATH diff --git a/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/metrics/MetricMapper.scala b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/metrics/MetricMapper.scala new file mode 100644 index 000000000000..60be476fcffe --- /dev/null +++ b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/metrics/MetricMapper.scala @@ -0,0 +1,64 @@ +/* + * 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.integration.metrics + +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.metric.SQLMetric + +trait MetricMapper { + def map(node: SparkPlan, key: String, metric: SQLMetric): Seq[MetricTag[_]] +} + +object MetricMapper { + val dummy: MetricMapper = (node: SparkPlan, key: String, metric: SQLMetric) => Nil + + case class SelfTimeMapper(selfTimeKeys: Map[String, Set[String]]) + extends MetricMapper { + override def map(node: SparkPlan, key: String, metric: SQLMetric): Seq[MetricTag[_]] = { + val className = node.getClass.getSimpleName + if (selfTimeKeys.contains(className)) { + if (selfTimeKeys(className).contains(key)) { + return Seq(MetricTag.IsSelfTime()) + } + } + Nil + } + } + + implicit class TypedMetricMapperOps(mapper: MetricMapper) { + private def unwrap( + mapper: MetricMapper): Seq[MetricMapper] = + mapper match { + case c: ChainedTypeMetricMapper => c.mappers + case other => Seq(other) + } + + def and( + other: MetricMapper): MetricMapper = { + new ChainedTypeMetricMapper(unwrap(mapper) ++ unwrap(other)) + } + } + + private class ChainedTypeMetricMapper(val mappers: Seq[MetricMapper]) + extends MetricMapper { + assert(!mappers.exists(_.isInstanceOf[ChainedTypeMetricMapper])) + override def map(node: SparkPlan, key: String, metric: SQLMetric): Seq[MetricTag[_]] = { + mappers.flatMap(m => m.map(node, key, metric)) + } + } +} diff --git a/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/metrics/MetricTag.scala b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/metrics/MetricTag.scala new file mode 100644 index 000000000000..17dee6952df3 --- /dev/null +++ b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/metrics/MetricTag.scala @@ -0,0 +1,39 @@ +/* + * 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.integration.metrics + +import scala.reflect.{ClassTag, classTag} + +trait MetricTag[T] { + import MetricTag._ + final def name(): String = nameOf(ClassTag(this.getClass)) + def value(): T +} + +object MetricTag { + def nameOf[T <: MetricTag[_]: ClassTag]: String = { + val clazz = classTag[T].runtimeClass + assert(classOf[MetricTag[_]].isAssignableFrom(clazz)) + clazz.getSimpleName + } + case class IsSelfTime() extends MetricTag[Nothing] { + override def value(): Nothing = { + throw new UnsupportedOperationException() + } + } +} diff --git a/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/metrics/PlanMetric.scala b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/metrics/PlanMetric.scala new file mode 100644 index 000000000000..7066001e0d6f --- /dev/null +++ b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/metrics/PlanMetric.scala @@ -0,0 +1,97 @@ +/* + * 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.integration.metrics + +import org.apache.gluten.integration.action.TableRender +import org.apache.gluten.integration.action.TableRender.Field.Leaf + +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.metric.SQLMetric + +import org.apache.commons.io.output.ByteArrayOutputStream + +import java.io.File +import java.nio.charset.Charset + +import scala.reflect.ClassTag + +case class PlanMetric( + queryPath: String, + plan: SparkPlan, + key: String, + metric: SQLMetric, + tags: Map[String, Seq[MetricTag[_]]]) { + + def containsTags[T <: MetricTag[_]: ClassTag]: Boolean = { + val name = MetricTag.nameOf[T] + tags.contains(name) + } + def getTags[T <: MetricTag[_]: ClassTag]: Seq[T] = { + require(containsTags[T]) + val name = MetricTag.nameOf[T] + tags(name).asInstanceOf[Seq[T]] + } +} + +object PlanMetric { + def newReporter(`type`: String): Reporter = `type` match { + case "execution-time" => new SelfTimeReporter(10) + case other => throw new IllegalArgumentException(s"Metric reporter type $other not defined") + } + + sealed trait Reporter { + def toString(metrics: Seq[PlanMetric]): String + } + + class SelfTimeReporter(topN: Int) extends Reporter { + private def toNanoTime(m: SQLMetric): Long = m.metricType match { + case "nsTiming" => m.value + case "timing" => m.value * 1000000 + } + + override def toString(metrics: Seq[PlanMetric]): String = { + val sb = new StringBuilder() + val selfTimes = metrics + .filter(_.containsTags[MetricTag.IsSelfTime]) + val sorted = selfTimes.sortBy(m => toNanoTime(m.metric))(Ordering.Long.reverse) + sb.append(s"Top $topN plan nodes that took longest time to execute: ") + sb.append(System.lineSeparator()) + sb.append(System.lineSeparator()) + val tr: TableRender[Seq[String]] = + TableRender.create( + Leaf("Query"), + Leaf("Node ID"), + Leaf("Node Name"), + Leaf("Execution Time (ns)")) + for (i <- 0 until (topN.min(sorted.size))) { + val m = sorted(i) + val f = new File(m.queryPath).toPath.getFileName.toString + tr.appendRow( + Seq( + f, + m.plan.id.toString, + m.plan.nodeName, + s"[${m.metric.name.getOrElse("")}] ${toNanoTime(m.metric).toString}")) + } + val out = new ByteArrayOutputStream() + tr.print(out) + sb.append(out.toString(Charset.defaultCharset)) + sb.toString() + } + } +} diff --git a/tools/gluten-it/common/src/main/scala/org/apache/spark/VersionUtils.scala b/tools/gluten-it/common/src/main/scala/org/apache/spark/VersionUtils.scala new file mode 100644 index 000000000000..7756c5e1d7e2 --- /dev/null +++ b/tools/gluten-it/common/src/main/scala/org/apache/spark/VersionUtils.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.spark + +object VersionUtils { + def majorMinorVersion(): (Int, Int) = { + org.apache.spark.util.VersionUtils.majorMinorVersion(org.apache.spark.SPARK_VERSION) + } + + // Returns X. X < 0 if one < other, x == 0 if one == other, x > 0 if one > other. + def compareMajorMinorVersion(one: (Int, Int), other: (Int, Int)): Int = { + val base = 1000 + assert(one._2 < base && other._2 < base) + one._1 * base + one._2 - (other._1 * base + other._2) + } +} diff --git a/tools/gluten-it/common/src/main/scala/org/apache/spark/deploy/history/GlutenItHistoryServerPlugin.scala b/tools/gluten-it/common/src/main/scala/org/apache/spark/deploy/history/GlutenItHistoryServerPlugin.scala index 4720d3e4a8c0..1af31e13c72b 100644 --- a/tools/gluten-it/common/src/main/scala/org/apache/spark/deploy/history/GlutenItHistoryServerPlugin.scala +++ b/tools/gluten-it/common/src/main/scala/org/apache/spark/deploy/history/GlutenItHistoryServerPlugin.scala @@ -117,7 +117,7 @@ class GlutenItHistoryServerPlugin extends AppHistoryServerPlugin { Seq() } - override def setupUI(ui: _root_.org.apache.spark.ui.SparkUI): Unit = { + override def setupUI(ui: org.apache.spark.ui.SparkUI): Unit = { // no-op } } diff --git a/tools/gluten-it/common/src/main/scala/org/apache/spark/sql/SparkQueryRunner.scala b/tools/gluten-it/common/src/main/scala/org/apache/spark/sql/SparkQueryRunner.scala index b68f74c1d5ed..c529e025de55 100644 --- a/tools/gluten-it/common/src/main/scala/org/apache/spark/sql/SparkQueryRunner.scala +++ b/tools/gluten-it/common/src/main/scala/org/apache/spark/sql/SparkQueryRunner.scala @@ -16,18 +16,22 @@ */ package org.apache.spark.sql +import org.apache.gluten.integration.metrics.{MetricMapper, MetricTag, PlanMetric} import org.apache.spark.{SparkContext, Success, TaskKilled} import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.scheduler.{SparkListener, SparkListenerExecutorMetricsUpdate, SparkListenerTaskEnd, SparkListenerTaskStart} import org.apache.spark.sql.KillTaskListener.INIT_WAIT_TIME_MS import org.apache.spark.sql.catalyst.QueryPlanningTracker - +import org.apache.spark.sql.execution.{QueryExecution, SparkPlan} +import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, QueryStageExec} +import org.apache.spark.sql.execution.exchange.ReusedExchangeExec import com.google.common.base.Preconditions import org.apache.commons.lang3.RandomUtils import java.io.ByteArrayOutputStream import java.nio.charset.StandardCharsets import java.util.concurrent.atomic.AtomicInteger +import scala.collection.mutable object SparkQueryRunner { private val availableExecutorMetrics: Set[String] = Set( @@ -54,22 +58,24 @@ object SparkQueryRunner { desc: String, queryPath: String, explain: Boolean, - metrics: Array[String], + metricMapper: MetricMapper, + executorMetrics: Seq[String], randomKillTasks: Boolean): RunResult = { - val unrecognizableMetrics = metrics.filter(!availableExecutorMetrics.contains(_)) + val unrecognizableMetrics = executorMetrics.filter(!availableExecutorMetrics.contains(_)) if (unrecognizableMetrics.nonEmpty) { throw new IllegalArgumentException( "Unrecognizable metric names: " + unrecognizableMetrics.mkString("Array(", ", ", ")")) } + val sc = spark.sparkContext sc.setJobDescription(desc) - // metrics listener + // Executor metrics listener. val em = new ExecutorMetrics() val metricsListener = new MetricsListener(em) sc.addSparkListener(metricsListener) - // kill task listener + // kill task listener. val killTaskListener: Option[KillTaskListener] = if (randomKillTasks) { Some(new KillTaskListener(sc)) } else { @@ -86,6 +92,7 @@ object SparkQueryRunner { val rows = QueryPlanningTracker.withTracker(tracker) { df.collect() } + val totalMillis = (System.nanoTime() - prev) / 1000000L if (explain) { df.explain(extended = true) } @@ -95,9 +102,15 @@ object SparkQueryRunner { val otherRulesMillis = tracker.rules.map(_._2.totalTimeNs).sum / 1000000L val planMillis = sparkRulesMillis + otherRulesMillis - val totalMillis = (System.nanoTime() - prev) / 1000000L - val collectedMetrics = metrics.map(name => (name, em.getMetricValue(name))).toMap - RunResult(rows, planMillis, totalMillis - planMillis, collectedMetrics) + val collectedExecutorMetrics = + executorMetrics.map(name => (name, em.getMetricValue(name))).toMap + val collectedSQLMetrics = collectSQLMetrics(queryPath, metricMapper, df.queryExecution) + RunResult( + rows, + planMillis, + totalMillis - planMillis, + collectedSQLMetrics, + collectedExecutorMetrics) } finally { sc.removeSparkListener(metricsListener) killTaskListener.foreach( @@ -110,6 +123,44 @@ object SparkQueryRunner { } } + private def collectAllNodes(plan: SparkPlan, nodes: mutable.LinkedHashMap[Int, SparkPlan]): Unit = + plan match { + case a: AdaptiveSparkPlanExec => + nodes += a.id -> a + collectAllNodes(a.executedPlan, nodes) + case q: QueryStageExec => + nodes += q.id -> q + collectAllNodes(q.plan, nodes) + case r: ReusedExchangeExec => + nodes += r.id -> r + collectAllNodes(r.child, nodes) + case other => + nodes += other.id -> other + other.children.foreach(c => collectAllNodes(c, nodes)) + } + + private def collectSQLMetrics(queryPath: String, mapper: MetricMapper, qe: QueryExecution): Seq[PlanMetric] = { + val nodes = mutable.LinkedHashMap[Int, SparkPlan]() + collectAllNodes(qe.executedPlan, nodes) + val all = nodes.flatMap { + case (_, p) => + p.metrics.map { + case keyValue @ (k, m) => + val tags = mapper.map(p, k, m) + val tagMapMutable = mutable.Map[String, mutable.Buffer[MetricTag[_]]]() + tags.foreach { + tag: MetricTag[_] => + val buffer = + tagMapMutable.getOrElseUpdate(tag.name(), mutable.ListBuffer[MetricTag[_]]()) + buffer += tag + } + val tagMap = tagMapMutable.map { case (k, v) => (k, v.toSeq) }.toMap + PlanMetric(queryPath, p, k, m, tagMap) + } + } + all.toSeq + } + private def resourceToString(resource: String): String = { val inStream = SparkQueryRunner.getClass.getResourceAsStream(resource) Preconditions.checkNotNull(inStream) @@ -135,7 +186,8 @@ case class RunResult( rows: Seq[Row], planningTimeMillis: Long, executionTimeMillis: Long, - metrics: Map[String, Long]) + sqlMetrics: Seq[PlanMetric], + executorMetrics: Map[String, Long]) class MetricsListener(em: ExecutorMetrics) extends SparkListener { override def onExecutorMetricsUpdate( diff --git a/tools/gluten-it/common/src/test/java/org/apache/gluten/integration/action/TableRenderTest.scala b/tools/gluten-it/common/src/test/java/org/apache/gluten/integration/action/TableRenderTest.scala index 1efc72148928..79e7d3d70272 100644 --- a/tools/gluten-it/common/src/test/java/org/apache/gluten/integration/action/TableRenderTest.scala +++ b/tools/gluten-it/common/src/test/java/org/apache/gluten/integration/action/TableRenderTest.scala @@ -26,12 +26,7 @@ object TableRenderTest { def case0(): Unit = { val render: TableRender[Seq[String]] = TableRender.create( Branch("ABC", List(Branch("AB", List(Leaf("A"), Leaf("B"))), Leaf("C"))), - Branch("DE", List(Leaf("D"), Leaf("E"))))(new RowParser[Seq[String]] { - override def parse(rowFactory: FieldAppender.RowAppender, row: Seq[String]): Unit = { - val inc = rowFactory.incremental() - row.foreach(ceil => inc.next().write(ceil)) - } - }) + Branch("DE", List(Leaf("D"), Leaf("E")))) render.print(Console.out) Console.out.println() } @@ -39,12 +34,7 @@ object TableRenderTest { def case1(): Unit = { val render: TableRender[Seq[String]] = TableRender.create( Branch("ABC", List(Branch("AB", List(Leaf("A"), Leaf("B"))), Leaf("C"))), - Branch("DE", List(Leaf("D"), Leaf("E"))))(new RowParser[Seq[String]] { - override def parse(rowFactory: FieldAppender.RowAppender, row: Seq[String]): Unit = { - val inc = rowFactory.incremental() - row.foreach(ceil => inc.next().write(ceil)) - } - }) + Branch("DE", List(Leaf("D"), Leaf("E")))) render.appendRow(List("aaaa", "b", "cccccc", "d", "eeeee")) render.print(Console.out) @@ -54,12 +44,7 @@ object TableRenderTest { def case2(): Unit = { val render: TableRender[Seq[String]] = TableRender.create( Branch("ABC", List(Branch("AAAAAAAAABBBBBB", List(Leaf("A"), Leaf("B"))), Leaf("C"))), - Branch("DE", List(Leaf("D"), Leaf("E"))))(new RowParser[Seq[String]] { - override def parse(rowFactory: FieldAppender.RowAppender, row: Seq[String]): Unit = { - val inc = rowFactory.incremental() - row.foreach(ceil => inc.next().write(ceil)) - } - }) + Branch("DE", List(Leaf("D"), Leaf("E")))) render.appendRow(List("aaaa", "b", "cccccc", "d", "eeeee")) render.print(Console.out) @@ -69,12 +54,7 @@ object TableRenderTest { def case3(): Unit = { val render: TableRender[Seq[String]] = TableRender.create( Branch("ABC", List(Branch("AB", List(Leaf("A"), Leaf("B"))), Leaf("CCCCCCCCCCCCC"))), - Branch("DE", List(Leaf("D"), Leaf("E"))))(new RowParser[Seq[String]] { - override def parse(rowFactory: FieldAppender.RowAppender, row: Seq[String]): Unit = { - val inc = rowFactory.incremental() - row.foreach(ceil => inc.next().write(ceil)) - } - }) + Branch("DE", List(Leaf("D"), Leaf("E")))) render.appendRow(List("aaaa", "b", "cccccc", "d", "eeeee")) render.appendRow(List("aaaaaaaaaaaaa", "b", "cccccc", "ddddddddddd", "eeeee")) @@ -87,12 +67,7 @@ object TableRenderTest { Branch( "ABBBBBBBBBBBBBBBBBBBBBBBBBBBBC", List(Branch("AB", List(Leaf("A"), Leaf("B"))), Leaf("C"))), - Branch("DE", List(Leaf("D"), Leaf("E"))))(new RowParser[Seq[String]] { - override def parse(rowFactory: FieldAppender.RowAppender, row: Seq[String]): Unit = { - val inc = rowFactory.incremental() - row.foreach(ceil => inc.next().write(ceil)) - } - }) + Branch("DE", List(Leaf("D"), Leaf("E")))) render.appendRow(List("aaaa", "b", "cccccc", "d", "eeeee")) render.print(Console.out) @@ -104,12 +79,7 @@ object TableRenderTest { val render: TableRender[Seq[String]] = TableRender.create( Leaf("Query ID"), Branch("Succeeded", leafs), - Branch("Row Count", leafs))(new RowParser[Seq[String]] { - override def parse(rowFactory: FieldAppender.RowAppender, row: Seq[String]): Unit = { - val inc = rowFactory.incremental() - row.foreach(ceil => inc.next().write(ceil)) - } - }) + Branch("Row Count", leafs)) render.appendRow( List("q1", "true", "true", "true && true && true && true", "true", "1", "1", "1", "1")) diff --git a/tools/workload/benchmark_velox/initialize.ipynb b/tools/workload/benchmark_velox/initialize.ipynb index 0f84da52cc52..e2b6fab7ba45 100644 --- a/tools/workload/benchmark_velox/initialize.ipynb +++ b/tools/workload/benchmark_velox/initialize.ipynb @@ -329,7 +329,7 @@ "\n", "nohup jupyter notebook --ip=0.0.0.0 --port=8888 &\n", "\n", - "cp ~/gluten/tools/workload/benchmark_velox ~/ipython/\n", + "find ~/gluten/tools/workload/benchmark_velox/ -maxdepth 1 -type f -exec cp {} ~/ipython \\;\n", "```" ] }, @@ -2232,6 +2232,18 @@ " !git clone https://github.com/apache/incubator-gluten.git gluten" ] }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "# Build Arrow for the first time build.\n", + "!sed -i 's/--build_arrow=OFF/--build_arrow=ON/' ~/gluten/dev/package-vcpkg.sh" + ] + }, { "cell_type": "code", "execution_count": null, diff --git a/tools/workload/benchmark_velox/native_sql_initialize.ipynb b/tools/workload/benchmark_velox/native_sql_initialize.ipynb index c980c00aaf32..f606260c95f6 100644 --- a/tools/workload/benchmark_velox/native_sql_initialize.ipynb +++ b/tools/workload/benchmark_velox/native_sql_initialize.ipynb @@ -895,12 +895,12 @@ " \n", " self.stopped = True\n", "\n", - " def run_perf_analysis(self, disk_dev, nic_dev, proxy='', emails=[], pr=''):\n", + " def run_perf_analysis(self, server_gluten_home, disk_dev, nic_dev, proxy='', emails=[], pr=''):\n", " if not self.server:\n", " return\n", "\n", - " run_script=f'{gluten_home}/tools/workload/benchmark_velox/analysis/run_perf_analysis.sh'\n", - " \n", + " run_script=f'{server_gluten_home}/tools/workload/benchmark_velox/analysis/run_perf_analysis.sh'\n", + "\n", " disk=','.join(disk_dev)\n", " nic=','.join(nic_dev)\n", "\n", @@ -1293,7 +1293,7 @@ " .set('spark.shuffle.manager','org.apache.spark.shuffle.sort.ColumnarShuffleManager')\\\n", " .set('spark.gluten.sql.columnar.backend.lib','velox')\\\n", " .set('spark.gluten.sql.columnar.maxBatchSize',4096)\\\n", - " .set('spark.gluten.sql.columnar.forceshuffledhashjoin',True)\\\n", + " .set('spark.gluten.sql.columnar.forceShuffledHashJoin',True)\\\n", " .set('spark.executorEnv.LD_PRELOAD', findjemalloc())\\\n", " .set('spark.gluten.sql.columnar.coalesce.batches', 'true')\n", " \n", diff --git a/tools/workload/benchmark_velox/params.yaml.template b/tools/workload/benchmark_velox/params.yaml.template index cde6dbec0d4c..285afb11cb7c 100644 --- a/tools/workload/benchmark_velox/params.yaml.template +++ b/tools/workload/benchmark_velox/params.yaml.template @@ -62,6 +62,9 @@ app_name: '' # Hostname or IP to server for perf analysis. Able to connect via ssh. server: '' +# Gluten home on server. +server_gluten_home: /home/sparkuser/gluten + # Specify the directory on perf analysis server. Usually a codename for this run. base_dir: test @@ -80,4 +83,4 @@ emails: - # Pull request number. -pr: '' \ No newline at end of file +pr: '' diff --git a/tools/workload/benchmark_velox/tpc_workload.ipynb b/tools/workload/benchmark_velox/tpc_workload.ipynb index 61cd1af3c8a9..3f6a24e9b1c8 100644 --- a/tools/workload/benchmark_velox/tpc_workload.ipynb +++ b/tools/workload/benchmark_velox/tpc_workload.ipynb @@ -77,6 +77,9 @@ "# Hostname or IP to server for perf analysis. Able to connect via ssh.\n", "server=''\n", "\n", + "# Gluten home on server.\n", + "server_gluten_home='/home/sparkuser/gluten'\n", + "\n", "# Specify the directory on perf analysis server. Usually a codename for this run.\n", "base_dir=''\n", "\n", @@ -272,7 +275,7 @@ "metadata": {}, "outputs": [], "source": [ - "test_tpc.run_query('q1', explain=False, print_result=False, load_table=True)" + "test_tpc.power_run(explain=False, print_result=False, load_table=True)" ] }, { @@ -291,7 +294,7 @@ "outputs": [], "source": [ "if analyze_perf:\n", - " test_tpc.run_perf_analysis(disk_dev, nic_dev, proxy, emails, pr)" + " test_tpc.run_perf_analysis(server_gluten_home, disk_dev, nic_dev, proxy, emails, pr)" ] }, {