From 51d0a37faafc0457d594d180f7787e8c0da1f518 Mon Sep 17 00:00:00 2001 From: Preetesh2110 <110464118+Preetesh2110@users.noreply.github.com> Date: Sat, 10 Aug 2024 02:16:54 +0000 Subject: [PATCH] [VL] Add Scala 2.13 support (#6326) * [VL] Add Scala 2.13 support * Fix scalaStyle issues * Fix Scala Style issues * Add Spark 3.5.1 and Scala 2.13 test in workflow * Add run-spark-test-spark35-scala213 job * Add Spark 3.5.1 and Scala 2.13 test in workflow * Fix tests failures * Fix tests failures * ScalaStyle fix * Fix SoftAffinitySuite * Fix ArrowUtil error * Fix backend-velox scala issues * Fix ColumnarArrowEvalPythonExec issues * Fix ColumnarArrowEvalPythonExec issues * Fix TestOperator.scala for style issues * Fix TestOperator.scala for style issues * Fix issues in DeltaRewriteTransformerRules.scala * DeltaRewriteTransformerRules fix * Fix style issues * Fix issues * Fix issues * Fix issues * Fix issues * Fix issues * Fix issues * Fix issues --------- Co-authored-by: Hongze Zhang --- .github/workflows/velox_docker.yml | 68 +++++++++++++++++++ .scalafmt.conf | 2 +- .../CHHashAggregateExecTransformer.scala | 5 +- .../parquet/GlutenParquetFilterSuite.scala | 6 +- .../benchmarks/CHStorageJoinBenchmark.scala | 2 +- .../execution/GenerateExecTransformer.scala | 2 +- .../HashAggregateExecTransformer.scala | 3 +- .../python/ColumnarArrowEvalPythonExec.scala | 10 +-- .../spark/sql/expression/UDFResolver.scala | 8 +-- .../gluten/execution/TestOperator.scala | 8 ++- .../softaffinity/SoftAffinitySuite.scala | 18 ++++- .../apache/gluten/utils/ArrowAbiUtil.scala | 4 +- .../spark/sql/utils/SparkArrowUtil.scala | 4 +- .../DeltaRewriteTransformerRules.scala | 8 +-- .../gluten/ras/path/PathFinderSuite.scala | 14 ++-- .../ras/specific/CyclicSearchSpaceSuite.scala | 9 +-- .../sql/GlutenColumnExpressionSuite.scala | 6 +- .../spark/sql/GlutenSQLQueryTestSuite.scala | 27 ++++---- .../sql/GlutenColumnExpressionSuite.scala | 6 +- .../spark/sql/GlutenSQLQueryTestSuite.scala | 27 ++++---- .../sql/GlutenColumnExpressionSuite.scala | 6 +- .../spark/sql/GlutenSQLQueryTestSuite.scala | 27 ++++---- .../GlutenV1WriteCommandSuite.scala | 12 ++-- .../sql/GlutenColumnExpressionSuite.scala | 6 +- .../spark/sql/GlutenSQLQueryTestSuite.scala | 27 ++++---- .../GlutenV1WriteCommandSuite.scala | 12 ++-- pom.xml | 3 +- shims/spark32/pom.xml | 4 +- shims/spark33/pom.xml | 4 +- shims/spark34/pom.xml | 4 +- 30 files changed, 217 insertions(+), 125 deletions(-) diff --git a/.github/workflows/velox_docker.yml b/.github/workflows/velox_docker.yml index f3066b7baae2..866ea48f18a3 100644 --- a/.github/workflows/velox_docker.yml +++ b/.github/workflows/velox_docker.yml @@ -1083,6 +1083,74 @@ jobs: name: golden-files-spark35 path: /tmp/tpch-approved-plan/** + run-spark-test-spark35-scala213: + needs: build-native-lib-centos-8 + runs-on: ubuntu-20.04 + container: ghcr.io/facebookincubator/velox-dev:centos8 + env: + CCACHE_DIR: "${{ github.workspace }}/.ccache" + steps: + - uses: actions/checkout@v2 + - name: Download All Artifacts + uses: actions/download-artifact@v2 + with: + name: velox-native-lib-centos-8-${{github.sha}} + path: ./cpp/build/releases + - name: Download UDF Example Lib + uses: actions/download-artifact@v2 + with: + name: udf-example-lib-centos-8-${{github.sha}} + path: ./cpp/build/velox/udf/examples/ + - name: Download Arrow Jars + uses: actions/download-artifact@v2 + with: + name: arrow-jars-centos-8-${{github.sha}} + path: /root/.m2/repository/org/apache/arrow/ + - name: Update mirror list + run: | + sed -i -e "s|mirrorlist=|#mirrorlist=|g" /etc/yum.repos.d/CentOS-* || true + sed -i -e "s|#baseurl=http://mirror.centos.org|baseurl=http://vault.centos.org|g" /etc/yum.repos.d/CentOS-* || true + - name: Setup build dependency + run: | + yum install sudo patch java-1.8.0-openjdk-devel wget -y + wget https://downloads.apache.org/maven/maven-3/3.8.8/binaries/apache-maven-3.8.8-bin.tar.gz + tar -xvf apache-maven-3.8.8-bin.tar.gz + mv apache-maven-3.8.8 /usr/lib/maven + echo "PATH=${PATH}:/usr/lib/maven/bin" >> $GITHUB_ENV + - name: Get Ccache + uses: actions/cache/restore@v3 + with: + path: '${{ env.CCACHE_DIR }}' + key: ccache-centos-release-default + - name: Ensure Cache Dirs Exists + working-directory: ${{ github.workspace }} + run: | + mkdir -p '${{ env.CCACHE_DIR }}' + - name: Prepare spark.test.home for Spark 3.5.1 (other tests) + run: | + cd $GITHUB_WORKSPACE/ && \ + wget https://archive.apache.org/dist/spark/spark-3.5.1/spark-3.5.1-bin-hadoop3.tgz && \ + tar --strip-components=1 -xf spark-3.5.1-bin-hadoop3.tgz spark-3.5.1-bin-hadoop3/jars/ && \ + rm -rf spark-3.5.1-bin-hadoop3.tgz && \ + mkdir -p $GITHUB_WORKSPACE//shims/spark35/spark_home/assembly/target/scala-2.13 && \ + mv jars $GITHUB_WORKSPACE//shims/spark35/spark_home/assembly/target/scala-2.13 && \ + cd $GITHUB_WORKSPACE// && \ + wget https://github.com/apache/spark/archive/refs/tags/v3.5.1.tar.gz && \ + tar --strip-components=1 -xf v3.5.1.tar.gz spark-3.5.1/sql/core/src/test/resources/ && \ + mkdir -p shims/spark35/spark_home/ && \ + mv sql shims/spark35/spark_home/ && \ + dnf module -y install python39 && \ + alternatives --set python3 /usr/bin/python3.9 && \ + pip3 install setuptools && \ + pip3 install pyspark==3.5.1 cython && \ + pip3 install pandas pyarrow + - name: Build and Run unit test for Spark 3.5.1 with scala-2.13 (other tests) + run: | + cd $GITHUB_WORKSPACE/ + export SPARK_SCALA_VERSION=2.13 + $MVN_CMD clean install -Pspark-3.5 -Pscala-2.13 -Pbackends-velox -Pceleborn -Piceberg -Pdelta -Pspark-ut -DargLine="-Dspark.test.home=$GITHUB_WORKSPACE//shims/spark35/spark_home/" -DtagsToExclude=org.apache.spark.tags.ExtendedSQLTest,org.apache.gluten.tags.UDFTest,org.apache.gluten.tags.SkipTestTags && \ + $MVN_CMD test -Pspark-3.5 -Pscala-2.13 -Pbackends-velox -Piceberg -Pdelta -DtagsToExclude=None -DtagsToInclude=org.apache.gluten.tags.UDFTest + run-spark-test-spark35-slow: needs: build-native-lib-centos-8 runs-on: ubuntu-20.04 diff --git a/.scalafmt.conf b/.scalafmt.conf index e65c0217fc58..937ab11383e3 100644 --- a/.scalafmt.conf +++ b/.scalafmt.conf @@ -1,7 +1,7 @@ runner.dialect = scala212 # Version is required to make sure IntelliJ picks the right version -version = 3.5.9 +version = 3.8.3 preset = default # Max column diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHHashAggregateExecTransformer.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHHashAggregateExecTransformer.scala index 7e688814381b..6c1fee39c423 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHHashAggregateExecTransformer.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHHashAggregateExecTransformer.scala @@ -370,8 +370,9 @@ case class CHHashAggregateExecTransformer( // Use approxPercentile.nullable as the nullable of the struct type // to make sure it returns null when input is empty fields = fields :+ (approxPercentile.child.dataType, approxPercentile.nullable) - fields = fields :+ (approxPercentile.percentageExpression.dataType, - approxPercentile.percentageExpression.nullable) + fields = fields :+ ( + approxPercentile.percentageExpression.dataType, + approxPercentile.percentageExpression.nullable) (makeStructType(fields), attr.nullable) case _ => (makeStructTypeSingleOne(attr.dataType, attr.nullable), attr.nullable) diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/parquet/GlutenParquetFilterSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/parquet/GlutenParquetFilterSuite.scala index b4e4cea9173b..1e6509c00884 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/parquet/GlutenParquetFilterSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/parquet/GlutenParquetFilterSuite.scala @@ -391,13 +391,13 @@ class GlutenParquetFilterSuite 'p_size.int >= 1, 'p_partkey.long.isNotNull, ('p_brand.string === "Brand#12" && - ('p_container.string in ("SM CASE", "SM BOX", "SM PACK", "SM PKG")) && + ('p_container.string.in("SM CASE", "SM BOX", "SM PACK", "SM PKG")) && 'p_size.int <= 5) || ('p_brand.string === "Brand#23" && - ('p_container.string in ("MED BAG", "MED BOX", "MED PKG", "MED PACK")) && + ('p_container.string.in("MED BAG", "MED BOX", "MED PKG", "MED PACK")) && 'p_size.int <= 10) || ('p_brand.string === "Brand#34" && - ('p_container.string in ("LG CASE", "LG BOX", "LG PACK", "LG PKG")) && + ('p_container.string.in("LG CASE", "LG BOX", "LG PACK", "LG PKG")) && 'p_size.int <= 15) ) ), diff --git a/backends-clickhouse/src/test/scala/org/apache/spark/sql/execution/benchmarks/CHStorageJoinBenchmark.scala b/backends-clickhouse/src/test/scala/org/apache/spark/sql/execution/benchmarks/CHStorageJoinBenchmark.scala index 194eccc50878..f8cd4bf57cc3 100644 --- a/backends-clickhouse/src/test/scala/org/apache/spark/sql/execution/benchmarks/CHStorageJoinBenchmark.scala +++ b/backends-clickhouse/src/test/scala/org/apache/spark/sql/execution/benchmarks/CHStorageJoinBenchmark.scala @@ -97,7 +97,7 @@ object CHStorageJoinBenchmark extends SqlBasedBenchmark with CHSqlBasedBenchmark _numRows += batch.numRows } Iterator((_numRows, blockNativeWriter.collectAsByteArray())) - // Iterator((_numRows, new Array[Byte](0))) + // Iterator((_numRows, new Array[Byte](0))) } .collect val count0 = countsAndBytes.map(_._1).sum diff --git a/backends-velox/src/main/scala/org/apache/gluten/execution/GenerateExecTransformer.scala b/backends-velox/src/main/scala/org/apache/gluten/execution/GenerateExecTransformer.scala index c7b81d55fa06..be76ba54ed72 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/execution/GenerateExecTransformer.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/execution/GenerateExecTransformer.scala @@ -228,7 +228,7 @@ object PullOutGenerateProjectHelper extends PullOutProjectHelper { } } - newProjections += Alias(CreateArray(fieldArray), generatePreAliasName)() + newProjections += Alias(CreateArray(fieldArray.toSeq), generatePreAliasName)() } // Plug in a Project between Generate and its child. diff --git a/backends-velox/src/main/scala/org/apache/gluten/execution/HashAggregateExecTransformer.scala b/backends-velox/src/main/scala/org/apache/gluten/execution/HashAggregateExecTransformer.scala index 4f33ae7c718c..9c5b68e7bff1 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/execution/HashAggregateExecTransformer.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/execution/HashAggregateExecTransformer.scala @@ -396,7 +396,8 @@ abstract class HashAggregateExecTransformer( childNodes.add(expressionNode) } } - exprNodes.add(getRowConstructNode(args, childNodes, newInputAttributes, aggFunc)) + exprNodes.add( + getRowConstructNode(args, childNodes, newInputAttributes.toSeq, aggFunc)) case other => throw new GlutenNotSupportException(s"$other is not supported.") } diff --git a/backends-velox/src/main/scala/org/apache/spark/api/python/ColumnarArrowEvalPythonExec.scala b/backends-velox/src/main/scala/org/apache/spark/api/python/ColumnarArrowEvalPythonExec.scala index 88280ff2edde..0e01c9d5d82f 100644 --- a/backends-velox/src/main/scala/org/apache/spark/api/python/ColumnarArrowEvalPythonExec.scala +++ b/backends-velox/src/main/scala/org/apache/spark/api/python/ColumnarArrowEvalPythonExec.scala @@ -44,7 +44,7 @@ import java.io.{DataInputStream, DataOutputStream} import java.net.Socket import java.util.concurrent.atomic.AtomicBoolean -import scala.collection.{mutable, Seq} +import scala.collection.mutable import scala.collection.mutable.ArrayBuffer class ColumnarArrowPythonRunner( @@ -54,7 +54,7 @@ class ColumnarArrowPythonRunner( schema: StructType, timeZoneId: String, conf: Map[String, String]) - extends BasePythonRunnerShim(funcs, evalType, argOffsets) { + extends BasePythonRunnerShim(funcs.toSeq, evalType, argOffsets) { override val simplifiedTraceback: Boolean = SQLConf.get.pysparkSimplifiedTraceback @@ -239,7 +239,7 @@ case class ColumnarArrowEvalPythonExec( val arrowSafeTypeCheck = Seq( SQLConf.PANDAS_ARROW_SAFE_TYPE_CONVERSION.key -> conf.arrowSafeTypeConversion.toString) - Map(timeZoneConf ++ pandasColsByName ++ arrowSafeTypeCheck: _*) + Map(timeZoneConf.toSeq ++ pandasColsByName.toSeq ++ arrowSafeTypeCheck: _*) } private val pythonRunnerConf = getPythonRunnerConfMap(conf) @@ -280,7 +280,7 @@ case class ColumnarArrowEvalPythonExec( case children => // There should not be any other UDFs, or the children can't be evaluated directly. assert(children.forall(_.find(_.isInstanceOf[PythonUDF]).isEmpty)) - (ChainedPythonFunctions(Seq(udf.func)), udf.children) + (ChainedPythonFunctions(Seq(udf.func).toSeq), udf.children) } } @@ -410,7 +410,7 @@ object PullOutArrowEvalPythonPreProjectHelper extends PullOutProjectHelper { val (chained, children) = collectFunctions(u) (ChainedPythonFunctions(chained.funcs ++ Seq(udf.func)), children) case children => - (ChainedPythonFunctions(Seq(udf.func)), udf.children) + (ChainedPythonFunctions(Seq(udf.func).toSeq), udf.children) } } diff --git a/backends-velox/src/main/scala/org/apache/spark/sql/expression/UDFResolver.scala b/backends-velox/src/main/scala/org/apache/spark/sql/expression/UDFResolver.scala index a2b6d5259a11..ab83c55ee306 100644 --- a/backends-velox/src/main/scala/org/apache/spark/sql/expression/UDFResolver.scala +++ b/backends-velox/src/main/scala/org/apache/spark/sql/expression/UDFResolver.scala @@ -120,12 +120,12 @@ case class UDFExpression( object UDFResolver extends Logging { private val UDFNames = mutable.HashSet[String]() // (udf_name, arg1, arg2, ...) => return type - private val UDFMap = mutable.HashMap[String, mutable.MutableList[UDFSignature]]() + private val UDFMap = mutable.HashMap[String, mutable.ListBuffer[UDFSignature]]() private val UDAFNames = mutable.HashSet[String]() // (udaf_name, arg1, arg2, ...) => return type, intermediate attributes private val UDAFMap = - mutable.HashMap[String, mutable.MutableList[UDAFSignature]]() + mutable.HashMap[String, mutable.ListBuffer[UDAFSignature]]() private val LIB_EXTENSION = ".so" @@ -152,7 +152,7 @@ object UDFResolver extends Logging { allowTypeConversion: Boolean): Unit = { assert(argTypes.dataType.isInstanceOf[StructType]) val v = - UDFMap.getOrElseUpdate(name, mutable.MutableList[UDFSignature]()) + UDFMap.getOrElseUpdate(name, mutable.ListBuffer[UDFSignature]()) v += UDFSignature( returnType, argTypes.dataType.asInstanceOf[StructType].fields.map(_.dataType), @@ -200,7 +200,7 @@ object UDFResolver extends Logging { } val v = - UDAFMap.getOrElseUpdate(name, mutable.MutableList[UDAFSignature]()) + UDAFMap.getOrElseUpdate(name, mutable.ListBuffer[UDAFSignature]()) v += UDAFSignature( returnType, argTypes.dataType.asInstanceOf[StructType].fields.map(_.dataType), diff --git a/backends-velox/src/test/scala/org/apache/gluten/execution/TestOperator.scala b/backends-velox/src/test/scala/org/apache/gluten/execution/TestOperator.scala index 2a2671926994..a0ea7d7267b4 100644 --- a/backends-velox/src/test/scala/org/apache/gluten/execution/TestOperator.scala +++ b/backends-velox/src/test/scala/org/apache/gluten/execution/TestOperator.scala @@ -1617,9 +1617,11 @@ class TestOperator extends VeloxWholeStageTransformerSuite with AdaptiveSparkPla path => (0 to 3).toDF("x").write.parquet(path.getCanonicalPath) spark.read.parquet(path.getCanonicalPath).createOrReplaceTempView("view") - runQueryAndCompare( - "SELECT x FROM view WHERE cast(x as timestamp) " + - "IN ('1970-01-01 08:00:00.001','1970-01-01 08:00:00.2')")(_) + runQueryAndCompare(s""" + |SELECT x FROM view + |WHERE cast(x as timestamp) + |IN ('1970-01-01 08:00:00.001','1970-01-01 08:00:00.2') + |""".stripMargin)(_ => ()) } } diff --git a/gluten-core/src/test/scala/org/apache/spark/softaffinity/SoftAffinitySuite.scala b/gluten-core/src/test/scala/org/apache/spark/softaffinity/SoftAffinitySuite.scala index c6c4fcc5fa1f..ea3e50e81282 100644 --- a/gluten-core/src/test/scala/org/apache/spark/softaffinity/SoftAffinitySuite.scala +++ b/gluten-core/src/test/scala/org/apache/spark/softaffinity/SoftAffinitySuite.scala @@ -39,6 +39,8 @@ class SoftAffinitySuite extends QueryTest with SharedSparkSession with Predicate .set(GlutenConfig.GLUTEN_SOFT_AFFINITY_REPLICATIONS_NUM, "2") .set(GlutenConfig.GLUTEN_SOFT_AFFINITY_MIN_TARGET_HOSTS, "2") + val scalaVersion = scala.util.Properties.versionNumberString + def generateNativePartition1(): Unit = { val partition = FilePartition( 0, @@ -97,7 +99,13 @@ class SoftAffinitySuite extends QueryTest with SharedSparkSession with Predicate val nativePartition = GlutenPartition(0, PlanBuilder.EMPTY_PLAN, locations = locations) - assertResult(Set("host-1", "host-4", "host-5")) { + val affinityResultSet = if (scalaVersion.startsWith("2.12")) { + Set("host-1", "host-4", "host-5") + } else if (scalaVersion.startsWith("2.13")) { + Set("host-5", "host-4", "host-2") + } + + assertResult(affinityResultSet) { nativePartition.preferredLocations().toSet } } @@ -184,7 +192,13 @@ class SoftAffinitySuite extends QueryTest with SharedSparkSession with Predicate val nativePartition = GlutenPartition(0, PlanBuilder.EMPTY_PLAN, locations = locations) - assertResult(Set("host-1", "host-5", "host-6")) { + val affinityResultSet = if (scalaVersion.startsWith("2.12")) { + Set("host-1", "host-5", "host-6") + } else if (scalaVersion.startsWith("2.13")) { + Set("host-6", "host-5", "host-2") + } + + assertResult(affinityResultSet) { nativePartition.preferredLocations().toSet } } diff --git a/gluten-data/src/main/scala/org/apache/gluten/utils/ArrowAbiUtil.scala b/gluten-data/src/main/scala/org/apache/gluten/utils/ArrowAbiUtil.scala index 442ae74bac98..8c6161e0c44c 100644 --- a/gluten-data/src/main/scala/org/apache/gluten/utils/ArrowAbiUtil.scala +++ b/gluten-data/src/main/scala/org/apache/gluten/utils/ArrowAbiUtil.scala @@ -119,7 +119,7 @@ object ArrowAbiUtil { } } - def exportField(allocator: BufferAllocator, field: Field, out: ArrowSchema) { + def exportField(allocator: BufferAllocator, field: Field, out: ArrowSchema): Unit = { val dictProvider = new CDataDictionaryProvider try { Data.exportField(allocator, field, dictProvider, out) @@ -128,7 +128,7 @@ object ArrowAbiUtil { } } - def exportSchema(allocator: BufferAllocator, schema: Schema, out: ArrowSchema) { + def exportSchema(allocator: BufferAllocator, schema: Schema, out: ArrowSchema): Unit = { val dictProvider = new CDataDictionaryProvider try { Data.exportSchema(allocator, schema, dictProvider, out) diff --git a/gluten-data/src/main/scala/org/apache/spark/sql/utils/SparkArrowUtil.scala b/gluten-data/src/main/scala/org/apache/spark/sql/utils/SparkArrowUtil.scala index 014956d84e9c..ec6ac35af3e7 100644 --- a/gluten-data/src/main/scala/org/apache/spark/sql/utils/SparkArrowUtil.scala +++ b/gluten-data/src/main/scala/org/apache/spark/sql/utils/SparkArrowUtil.scala @@ -134,7 +134,7 @@ object SparkArrowUtil { val dt = fromArrowField(child) StructField(child.getName, dt, child.isNullable) } - StructType(fields) + StructType(fields.toSeq) case arrowType => fromArrowType(arrowType) } } @@ -147,7 +147,7 @@ object SparkArrowUtil { } def fromArrowSchema(schema: Schema): StructType = { - StructType(schema.getFields.asScala.map { + StructType(schema.getFields.asScala.toSeq.map { field => val dt = fromArrowField(field) StructField(field.getName, dt, field.isNullable) diff --git a/gluten-delta/src/main/scala/org/apache/gluten/extension/DeltaRewriteTransformerRules.scala b/gluten-delta/src/main/scala/org/apache/gluten/extension/DeltaRewriteTransformerRules.scala index 76eb53dbd022..fed837d308be 100644 --- a/gluten-delta/src/main/scala/org/apache/gluten/extension/DeltaRewriteTransformerRules.scala +++ b/gluten-delta/src/main/scala/org/apache/gluten/extension/DeltaRewriteTransformerRules.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.delta.{DeltaColumnMapping, DeltaParquetFileFormat, N import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.datasources.FileFormat -import scala.collection._ +import scala.collection.mutable.ListBuffer class DeltaRewriteTransformerRules extends RewriteTransformerRules { override def rules: Seq[Rule[SparkPlan]] = columnMappingRule :: Nil @@ -87,8 +87,8 @@ object DeltaRewriteTransformerRules { )(SparkSession.active) // transform output's name into physical name so Reader can read data correctly // should keep the columns order the same as the origin output - val originColumnNames = mutable.ListBuffer.empty[String] - val transformedAttrs = mutable.ListBuffer.empty[Attribute] + val originColumnNames = ListBuffer.empty[String] + val transformedAttrs = ListBuffer.empty[Attribute] def mapAttribute(attr: Attribute) = { val newAttr = if (!plan.isMetadataColumn(attr)) { DeltaColumnMapping @@ -142,7 +142,7 @@ object DeltaRewriteTransformerRules { val expr = (transformedAttrs, originColumnNames).zipped.map { (attr, columnName) => Alias(attr, columnName)(exprId = attr.exprId) } - val projectExecTransformer = ProjectExecTransformer(expr, scanExecTransformer) + val projectExecTransformer = ProjectExecTransformer(expr.toSeq, scanExecTransformer) projectExecTransformer case _ => plan } diff --git a/gluten-ras/common/src/test/scala/org/apache/gluten/ras/path/PathFinderSuite.scala b/gluten-ras/common/src/test/scala/org/apache/gluten/ras/path/PathFinderSuite.scala index b5ea3fc3cf6e..4b3a675cd843 100644 --- a/gluten-ras/common/src/test/scala/org/apache/gluten/ras/path/PathFinderSuite.scala +++ b/gluten-ras/common/src/test/scala/org/apache/gluten/ras/path/PathFinderSuite.scala @@ -262,18 +262,18 @@ class PathFinderSuite extends AnyFunSuite { assert(path.plan() == Binary(n1, Group(1), Group(2))) assert( - path.dive(state, 1).map(_.plan()) == List( + path.dive(state, 1).map(_.plan()).toList == List( Binary(n1, Unary(n2, Group(3)), Unary(n3, Group(4))))) assert( - path.dive(state, 2).map(_.plan()) == List( + path.dive(state, 2).map(_.plan()).toList == List( Binary(n1, Unary(n2, Leaf(n4, 1)), Unary(n3, Leaf(n5, 1))), Binary(n1, Unary(n2, Leaf(n4, 1)), Unary(n3, Leaf(n6, 1))))) assert( - path.dive(state, 3).map(_.plan()) == List( + path.dive(state, 3).map(_.plan()).toList == List( Binary(n1, Unary(n2, Leaf(n4, 1)), Unary(n3, Leaf(n5, 1))), Binary(n1, Unary(n2, Leaf(n4, 1)), Unary(n3, Leaf(n6, 1))))) assert( - path.dive(state, RasPath.INF_DEPTH).map(_.plan()) == List( + path.dive(state, RasPath.INF_DEPTH).map(_.plan()).toList == List( Binary(n1, Unary(n2, Leaf(n4, 1)), Unary(n3, Leaf(n5, 1))), Binary(n1, Unary(n2, Leaf(n4, 1)), Unary(n3, Leaf(n6, 1))))) } @@ -338,13 +338,13 @@ class PathFinderSuite extends AnyFunSuite { path.dive(state, 1).map(_.plan()).toSeq == List( Binary(n1, Binary(n2, Group(3), Group(4)), Leaf(n3, 1)))) assert( - path.dive(state, 2).map(_.plan()) == List( + path.dive(state, 2).map(_.plan()).toList == List( Binary(n1, Binary(n2, Leaf(n4, 1), Leaf(n5, 1)), Leaf(n3, 1)))) assert( - path.dive(state, 3).map(_.plan()) == List( + path.dive(state, 3).map(_.plan()).toList == List( Binary(n1, Binary(n2, Leaf(n4, 1), Leaf(n5, 1)), Leaf(n3, 1)))) assert( - path.dive(state, RasPath.INF_DEPTH).map(_.plan()) == List( + path.dive(state, RasPath.INF_DEPTH).map(_.plan()).toList == List( Binary(n1, Binary(n2, Leaf(n4, 1), Leaf(n5, 1)), Leaf(n3, 1)))) } } diff --git a/gluten-ras/common/src/test/scala/org/apache/gluten/ras/specific/CyclicSearchSpaceSuite.scala b/gluten-ras/common/src/test/scala/org/apache/gluten/ras/specific/CyclicSearchSpaceSuite.scala index d27292fb5361..077921b697bf 100644 --- a/gluten-ras/common/src/test/scala/org/apache/gluten/ras/specific/CyclicSearchSpaceSuite.scala +++ b/gluten-ras/common/src/test/scala/org/apache/gluten/ras/specific/CyclicSearchSpaceSuite.scala @@ -65,11 +65,12 @@ abstract class CyclicSearchSpaceSuite extends AnyFunSuite { PathFinder.builder(ras, mockState).depth(depth).build().find(can) } - assert(find(node1, 1).map(p => p.plan()) == List(Unary("node1", Group(0)))) - assert(find(node1, 2).map(p => p.plan()) == List(Unary("node1", Leaf("node2", 1)))) - assert(find(node1, 3).map(p => p.plan()) == List(Unary("node1", Leaf("node2", 1)))) + assert(find(node1, 1).map(p => p.plan()).toList == List(Unary("node1", Group(0)))) + assert(find(node1, 2).map(p => p.plan()).toList == List(Unary("node1", Leaf("node2", 1)))) + assert(find(node1, 3).map(p => p.plan()).toList == List(Unary("node1", Leaf("node2", 1)))) assert( - find(node1, RasPath.INF_DEPTH).map(p => p.plan()) == List(Unary("node1", Leaf("node2", 1)))) + find(node1, RasPath.INF_DEPTH).map(p => p.plan()).toList == List( + Unary("node1", Leaf("node2", 1)))) } test("Cyclic - find best, simple self cycle") { diff --git a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/GlutenColumnExpressionSuite.scala b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/GlutenColumnExpressionSuite.scala index 437cef29215c..dbc70bf74598 100644 --- a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/GlutenColumnExpressionSuite.scala +++ b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/GlutenColumnExpressionSuite.scala @@ -34,7 +34,7 @@ class GlutenColumnExpressionSuite extends ColumnExpressionSuite with GlutenSQLTe strDf.select(raise_error($"a")).collect() } assert(e2.getCause.isInstanceOf[RuntimeException]) - assert(e2.getCause.getMessage contains "hello") + assert(e2.getCause.getMessage.contains("hello")) } testGluten("assert_true") { @@ -58,7 +58,7 @@ class GlutenColumnExpressionSuite extends ColumnExpressionSuite with GlutenSQLTe nullDf.select(assert_true($"cond", $"n")).collect() } assert(e2.getCause.isInstanceOf[RuntimeException]) - assert(e2.getCause.getMessage contains "first row") + assert(e2.getCause.getMessage.contains("first row")) // assert_true(condition) val intDf = Seq((0, 1)).toDF("a", "b") @@ -67,7 +67,7 @@ class GlutenColumnExpressionSuite extends ColumnExpressionSuite with GlutenSQLTe intDf.select(assert_true($"a" > $"b")).collect() } assert(e3.getCause.isInstanceOf[RuntimeException]) - assert(e3.getCause.getMessage contains "'('a > 'b)' is not true!") + assert(e3.getCause.getMessage.contains("'('a > 'b)' is not true!")) } testGluten( diff --git a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala index 4fbd89bda05c..fec513349d5f 100644 --- a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala +++ b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala @@ -71,20 +71,19 @@ import scala.util.control.NonFatal * The format for input files is simple: * 1. A list of SQL queries separated by semicolons by default. If the semicolon cannot * effectively separate the SQL queries in the test file(e.g. bracketed comments), please use - * --QUERY-DELIMITER-START and --QUERY-DELIMITER-END. Lines starting with - * --QUERY-DELIMITER-START and --QUERY-DELIMITER-END represent the beginning and end of a query, - * respectively. Code that is not surrounded by lines that begin with --QUERY-DELIMITER-START and - * --QUERY-DELIMITER-END is still separated by semicolons. 2. Lines starting with -- are treated as - * comments and ignored. 3. Lines starting with --SET are used to specify the configs when running - * this testing file. You can set multiple configs in one --SET, using comma to separate them. Or - * you can use multiple - * --SET statements. 4. Lines starting with --IMPORT are used to load queries from another test - * file. 5. Lines starting with --CONFIG_DIM are used to specify config dimensions of this testing - * file. The dimension name is decided by the string after --CONFIG_DIM. For example, --CONFIG_DIM1 - * belongs to dimension 1. One dimension can have multiple lines, each line representing one config - * set (one or more configs, separated by comma). Spark will run this testing file many times, each - * time picks one config set from each dimension, until all the combinations are tried. For example, - * if dimension 1 has 2 lines, dimension 2 has 3 lines, this testing file will be run 6 times + * --QUERY-DELIMITER-START and --QUERY-DELIMITER-END. Lines starting with --QUERY-DELIMITER-START + * and --QUERY-DELIMITER-END represent the beginning and end of a query, respectively. Code that is + * not surrounded by lines that begin with --QUERY-DELIMITER-START and --QUERY-DELIMITER-END is + * still separated by semicolons. 2. Lines starting with -- are treated as comments and ignored. 3. + * Lines starting with --SET are used to specify the configs when running this testing file. You can + * set multiple configs in one --SET, using comma to separate them. Or you can use multiple --SET + * statements. 4. Lines starting with --IMPORT are used to load queries from another test file. 5. + * Lines starting with --CONFIG_DIM are used to specify config dimensions of this testing file. The + * dimension name is decided by the string after --CONFIG_DIM. For example, --CONFIG_DIM1 belongs to + * dimension 1. One dimension can have multiple lines, each line representing one config set (one or + * more configs, separated by comma). Spark will run this testing file many times, each time picks + * one config set from each dimension, until all the combinations are tried. For example, if + * dimension 1 has 2 lines, dimension 2 has 3 lines, this testing file will be run 6 times * (cartesian product). * * For example: diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/GlutenColumnExpressionSuite.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/GlutenColumnExpressionSuite.scala index 437cef29215c..dbc70bf74598 100644 --- a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/GlutenColumnExpressionSuite.scala +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/GlutenColumnExpressionSuite.scala @@ -34,7 +34,7 @@ class GlutenColumnExpressionSuite extends ColumnExpressionSuite with GlutenSQLTe strDf.select(raise_error($"a")).collect() } assert(e2.getCause.isInstanceOf[RuntimeException]) - assert(e2.getCause.getMessage contains "hello") + assert(e2.getCause.getMessage.contains("hello")) } testGluten("assert_true") { @@ -58,7 +58,7 @@ class GlutenColumnExpressionSuite extends ColumnExpressionSuite with GlutenSQLTe nullDf.select(assert_true($"cond", $"n")).collect() } assert(e2.getCause.isInstanceOf[RuntimeException]) - assert(e2.getCause.getMessage contains "first row") + assert(e2.getCause.getMessage.contains("first row")) // assert_true(condition) val intDf = Seq((0, 1)).toDF("a", "b") @@ -67,7 +67,7 @@ class GlutenColumnExpressionSuite extends ColumnExpressionSuite with GlutenSQLTe intDf.select(assert_true($"a" > $"b")).collect() } assert(e3.getCause.isInstanceOf[RuntimeException]) - assert(e3.getCause.getMessage contains "'('a > 'b)' is not true!") + assert(e3.getCause.getMessage.contains("'('a > 'b)' is not true!")) } testGluten( diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala index 6e2a9efa87c0..39fe8cf57a18 100644 --- a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala @@ -71,20 +71,19 @@ import scala.util.control.NonFatal * The format for input files is simple: * 1. A list of SQL queries separated by semicolons by default. If the semicolon cannot * effectively separate the SQL queries in the test file(e.g. bracketed comments), please use - * --QUERY-DELIMITER-START and --QUERY-DELIMITER-END. Lines starting with - * --QUERY-DELIMITER-START and --QUERY-DELIMITER-END represent the beginning and end of a query, - * respectively. Code that is not surrounded by lines that begin with --QUERY-DELIMITER-START and - * --QUERY-DELIMITER-END is still separated by semicolons. 2. Lines starting with -- are treated as - * comments and ignored. 3. Lines starting with --SET are used to specify the configs when running - * this testing file. You can set multiple configs in one --SET, using comma to separate them. Or - * you can use multiple - * --SET statements. 4. Lines starting with --IMPORT are used to load queries from another test - * file. 5. Lines starting with --CONFIG_DIM are used to specify config dimensions of this testing - * file. The dimension name is decided by the string after --CONFIG_DIM. For example, --CONFIG_DIM1 - * belongs to dimension 1. One dimension can have multiple lines, each line representing one config - * set (one or more configs, separated by comma). Spark will run this testing file many times, each - * time picks one config set from each dimension, until all the combinations are tried. For example, - * if dimension 1 has 2 lines, dimension 2 has 3 lines, this testing file will be run 6 times + * --QUERY-DELIMITER-START and --QUERY-DELIMITER-END. Lines starting with --QUERY-DELIMITER-START + * and --QUERY-DELIMITER-END represent the beginning and end of a query, respectively. Code that is + * not surrounded by lines that begin with --QUERY-DELIMITER-START and --QUERY-DELIMITER-END is + * still separated by semicolons. 2. Lines starting with -- are treated as comments and ignored. 3. + * Lines starting with --SET are used to specify the configs when running this testing file. You can + * set multiple configs in one --SET, using comma to separate them. Or you can use multiple --SET + * statements. 4. Lines starting with --IMPORT are used to load queries from another test file. 5. + * Lines starting with --CONFIG_DIM are used to specify config dimensions of this testing file. The + * dimension name is decided by the string after --CONFIG_DIM. For example, --CONFIG_DIM1 belongs to + * dimension 1. One dimension can have multiple lines, each line representing one config set (one or + * more configs, separated by comma). Spark will run this testing file many times, each time picks + * one config set from each dimension, until all the combinations are tried. For example, if + * dimension 1 has 2 lines, dimension 2 has 3 lines, this testing file will be run 6 times * (cartesian product). * * For example: diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenColumnExpressionSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenColumnExpressionSuite.scala index 437cef29215c..dbc70bf74598 100644 --- a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenColumnExpressionSuite.scala +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenColumnExpressionSuite.scala @@ -34,7 +34,7 @@ class GlutenColumnExpressionSuite extends ColumnExpressionSuite with GlutenSQLTe strDf.select(raise_error($"a")).collect() } assert(e2.getCause.isInstanceOf[RuntimeException]) - assert(e2.getCause.getMessage contains "hello") + assert(e2.getCause.getMessage.contains("hello")) } testGluten("assert_true") { @@ -58,7 +58,7 @@ class GlutenColumnExpressionSuite extends ColumnExpressionSuite with GlutenSQLTe nullDf.select(assert_true($"cond", $"n")).collect() } assert(e2.getCause.isInstanceOf[RuntimeException]) - assert(e2.getCause.getMessage contains "first row") + assert(e2.getCause.getMessage.contains("first row")) // assert_true(condition) val intDf = Seq((0, 1)).toDF("a", "b") @@ -67,7 +67,7 @@ class GlutenColumnExpressionSuite extends ColumnExpressionSuite with GlutenSQLTe intDf.select(assert_true($"a" > $"b")).collect() } assert(e3.getCause.isInstanceOf[RuntimeException]) - assert(e3.getCause.getMessage contains "'('a > 'b)' is not true!") + assert(e3.getCause.getMessage.contains("'('a > 'b)' is not true!")) } testGluten( diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala index 8a291990ea31..65578eb52bca 100644 --- a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala @@ -74,20 +74,19 @@ import scala.util.control.NonFatal * The format for input files is simple: * 1. A list of SQL queries separated by semicolons by default. If the semicolon cannot * effectively separate the SQL queries in the test file(e.g. bracketed comments), please use - * --QUERY-DELIMITER-START and --QUERY-DELIMITER-END. Lines starting with - * --QUERY-DELIMITER-START and --QUERY-DELIMITER-END represent the beginning and end of a query, - * respectively. Code that is not surrounded by lines that begin with --QUERY-DELIMITER-START and - * --QUERY-DELIMITER-END is still separated by semicolons. 2. Lines starting with -- are treated as - * comments and ignored. 3. Lines starting with --SET are used to specify the configs when running - * this testing file. You can set multiple configs in one --SET, using comma to separate them. Or - * you can use multiple - * --SET statements. 4. Lines starting with --IMPORT are used to load queries from another test - * file. 5. Lines starting with --CONFIG_DIM are used to specify config dimensions of this testing - * file. The dimension name is decided by the string after --CONFIG_DIM. For example, --CONFIG_DIM1 - * belongs to dimension 1. One dimension can have multiple lines, each line representing one config - * set (one or more configs, separated by comma). Spark will run this testing file many times, each - * time picks one config set from each dimension, until all the combinations are tried. For example, - * if dimension 1 has 2 lines, dimension 2 has 3 lines, this testing file will be run 6 times + * --QUERY-DELIMITER-START and --QUERY-DELIMITER-END. Lines starting with --QUERY-DELIMITER-START + * and --QUERY-DELIMITER-END represent the beginning and end of a query, respectively. Code that is + * not surrounded by lines that begin with --QUERY-DELIMITER-START and --QUERY-DELIMITER-END is + * still separated by semicolons. 2. Lines starting with -- are treated as comments and ignored. 3. + * Lines starting with --SET are used to specify the configs when running this testing file. You can + * set multiple configs in one --SET, using comma to separate them. Or you can use multiple --SET + * statements. 4. Lines starting with --IMPORT are used to load queries from another test file. 5. + * Lines starting with --CONFIG_DIM are used to specify config dimensions of this testing file. The + * dimension name is decided by the string after --CONFIG_DIM. For example, --CONFIG_DIM1 belongs to + * dimension 1. One dimension can have multiple lines, each line representing one config set (one or + * more configs, separated by comma). Spark will run this testing file many times, each time picks + * one config set from each dimension, until all the combinations are tried. For example, if + * dimension 1 has 2 lines, dimension 2 has 3 lines, this testing file will be run 6 times * (cartesian product). * * For example: diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenV1WriteCommandSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenV1WriteCommandSuite.scala index 726ace3a15f1..eb6794bba812 100644 --- a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenV1WriteCommandSuite.scala +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenV1WriteCommandSuite.scala @@ -151,7 +151,8 @@ class GlutenV1WriteCommandSuite ), false, _, - _) => + _ + ) => true case SortExecTransformer( Seq( @@ -168,7 +169,8 @@ class GlutenV1WriteCommandSuite ), false, _, - _) => + _ + ) => true case _ => false }, @@ -233,7 +235,8 @@ class GlutenV1WriteCommandSuite ), false, _, - _) => + _ + ) => true case SortExecTransformer( Seq( @@ -250,7 +253,8 @@ class GlutenV1WriteCommandSuite ), false, _, - _) => + _ + ) => true case _ => false }, diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/GlutenColumnExpressionSuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/GlutenColumnExpressionSuite.scala index 437cef29215c..dbc70bf74598 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/GlutenColumnExpressionSuite.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/GlutenColumnExpressionSuite.scala @@ -34,7 +34,7 @@ class GlutenColumnExpressionSuite extends ColumnExpressionSuite with GlutenSQLTe strDf.select(raise_error($"a")).collect() } assert(e2.getCause.isInstanceOf[RuntimeException]) - assert(e2.getCause.getMessage contains "hello") + assert(e2.getCause.getMessage.contains("hello")) } testGluten("assert_true") { @@ -58,7 +58,7 @@ class GlutenColumnExpressionSuite extends ColumnExpressionSuite with GlutenSQLTe nullDf.select(assert_true($"cond", $"n")).collect() } assert(e2.getCause.isInstanceOf[RuntimeException]) - assert(e2.getCause.getMessage contains "first row") + assert(e2.getCause.getMessage.contains("first row")) // assert_true(condition) val intDf = Seq((0, 1)).toDF("a", "b") @@ -67,7 +67,7 @@ class GlutenColumnExpressionSuite extends ColumnExpressionSuite with GlutenSQLTe intDf.select(assert_true($"a" > $"b")).collect() } assert(e3.getCause.isInstanceOf[RuntimeException]) - assert(e3.getCause.getMessage contains "'('a > 'b)' is not true!") + assert(e3.getCause.getMessage.contains("'('a > 'b)' is not true!")) } testGluten( diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala index 8a6f5f32f891..276bc95be586 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala @@ -74,20 +74,19 @@ import scala.util.control.NonFatal * The format for input files is simple: * 1. A list of SQL queries separated by semicolons by default. If the semicolon cannot * effectively separate the SQL queries in the test file(e.g. bracketed comments), please use - * --QUERY-DELIMITER-START and --QUERY-DELIMITER-END. Lines starting with - * --QUERY-DELIMITER-START and --QUERY-DELIMITER-END represent the beginning and end of a query, - * respectively. Code that is not surrounded by lines that begin with --QUERY-DELIMITER-START and - * --QUERY-DELIMITER-END is still separated by semicolons. 2. Lines starting with -- are treated as - * comments and ignored. 3. Lines starting with --SET are used to specify the configs when running - * this testing file. You can set multiple configs in one --SET, using comma to separate them. Or - * you can use multiple - * --SET statements. 4. Lines starting with --IMPORT are used to load queries from another test - * file. 5. Lines starting with --CONFIG_DIM are used to specify config dimensions of this testing - * file. The dimension name is decided by the string after --CONFIG_DIM. For example, --CONFIG_DIM1 - * belongs to dimension 1. One dimension can have multiple lines, each line representing one config - * set (one or more configs, separated by comma). Spark will run this testing file many times, each - * time picks one config set from each dimension, until all the combinations are tried. For example, - * if dimension 1 has 2 lines, dimension 2 has 3 lines, this testing file will be run 6 times + * --QUERY-DELIMITER-START and --QUERY-DELIMITER-END. Lines starting with --QUERY-DELIMITER-START + * and --QUERY-DELIMITER-END represent the beginning and end of a query, respectively. Code that is + * not surrounded by lines that begin with --QUERY-DELIMITER-START and --QUERY-DELIMITER-END is + * still separated by semicolons. 2. Lines starting with -- are treated as comments and ignored. 3. + * Lines starting with --SET are used to specify the configs when running this testing file. You can + * set multiple configs in one --SET, using comma to separate them. Or you can use multiple --SET + * statements. 4. Lines starting with --IMPORT are used to load queries from another test file. 5. + * Lines starting with --CONFIG_DIM are used to specify config dimensions of this testing file. The + * dimension name is decided by the string after --CONFIG_DIM. For example, --CONFIG_DIM1 belongs to + * dimension 1. One dimension can have multiple lines, each line representing one config set (one or + * more configs, separated by comma). Spark will run this testing file many times, each time picks + * one config set from each dimension, until all the combinations are tried. For example, if + * dimension 1 has 2 lines, dimension 2 has 3 lines, this testing file will be run 6 times * (cartesian product). * * For example: diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenV1WriteCommandSuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenV1WriteCommandSuite.scala index fcaf75a4d5c1..5fc887d8d410 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenV1WriteCommandSuite.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenV1WriteCommandSuite.scala @@ -152,7 +152,8 @@ class GlutenV1WriteCommandSuite ), false, _, - _) => + _ + ) => true case SortExecTransformer( Seq( @@ -169,7 +170,8 @@ class GlutenV1WriteCommandSuite ), false, _, - _) => + _ + ) => true case _ => false }, @@ -233,7 +235,8 @@ class GlutenV1WriteCommandSuite ), false, _, - _) => + _ + ) => true case SortExecTransformer( Seq( @@ -250,7 +253,8 @@ class GlutenV1WriteCommandSuite ), false, _, - _) => + _ + ) => true case _ => false }, diff --git a/pom.xml b/pom.xml index fd44d06b759c..4fb327f9f7dd 100644 --- a/pom.xml +++ b/pom.xml @@ -97,7 +97,7 @@ 32.0.1-jre 2.27.2 - 3.5.9 + 3.8.3 package /* @@ -147,6 +147,7 @@ 2.13.8 2.13 + 3.8.3 diff --git a/shims/spark32/pom.xml b/shims/spark32/pom.xml index 802b758c8531..7e9dcb226dbb 100644 --- a/shims/spark32/pom.xml +++ b/shims/spark32/pom.xml @@ -43,13 +43,13 @@ org.apache.spark - spark-catalyst_2.12 + spark-catalyst_${scala.binary.version} provided true org.apache.spark - spark-core_2.12 + spark-core_${scala.binary.version} provided true diff --git a/shims/spark33/pom.xml b/shims/spark33/pom.xml index f552eddf3fb0..e17a639baa35 100644 --- a/shims/spark33/pom.xml +++ b/shims/spark33/pom.xml @@ -43,13 +43,13 @@ org.apache.spark - spark-catalyst_2.12 + spark-catalyst_${scala.binary.version} provided true org.apache.spark - spark-core_2.12 + spark-core_${scala.binary.version} provided true diff --git a/shims/spark34/pom.xml b/shims/spark34/pom.xml index 5d6a5dd1960f..bd1de9fe8561 100644 --- a/shims/spark34/pom.xml +++ b/shims/spark34/pom.xml @@ -43,13 +43,13 @@ org.apache.spark - spark-catalyst_2.12 + spark-catalyst_${scala.binary.version} provided true org.apache.spark - spark-core_2.12 + spark-core_${scala.binary.version} provided true