From 30018528733d52379f0cf42f11be516799daab1a Mon Sep 17 00:00:00 2001 From: Raza Jafri Date: Tue, 28 May 2024 08:30:57 -0700 Subject: [PATCH 01/68] Add Support for Multiple Filtering Keys for Subquery Broadcast (#10858) * Add support for multiple filtering keys for subquery broadcast * Signing off Signed-off-by: Raza Jafri * Fixed test compilation --------- Signed-off-by: Raza Jafri --- .../execution/GpuSubqueryBroadcastExec.scala | 63 +++++++++---------- .../execution/GpuSubqueryBroadcastMeta.scala | 56 +++++++++++++++++ .../execution/GpuSubqueryBroadcastMeta.scala | 35 +++++++++++ .../spark/rapids/DynamicPruningSuite.scala | 2 +- 4 files changed, 123 insertions(+), 33 deletions(-) create mode 100644 sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastMeta.scala create mode 100644 sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastMeta.scala diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastExec.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastExec.scala index 1861a9f2515..72ed0e79504 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastExec.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2023, NVIDIA CORPORATION. + * Copyright (c) 2021-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -41,14 +41,14 @@ import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.ThreadUtils -class GpuSubqueryBroadcastMeta( +abstract class GpuSubqueryBroadcastMetaBase( s: SubqueryBroadcastExec, conf: RapidsConf, p: Option[RapidsMeta[_, _, _]], r: DataFromReplacementRule) extends SparkPlanMeta[SubqueryBroadcastExec](s, conf, p, r) { - private var broadcastBuilder: () => SparkPlan = _ + protected var broadcastBuilder: () => SparkPlan = _ override val childExprs: Seq[BaseExprMeta[_]] = Nil @@ -140,13 +140,8 @@ class GpuSubqueryBroadcastMeta( */ override def convertToCpu(): SparkPlan = s - override def convertToGpu(): GpuExec = { - GpuSubqueryBroadcastExec(s.name, s.index, s.buildKeys, broadcastBuilder())( - getBroadcastModeKeyExprs) - } - /** Extract the broadcast mode key expressions if there are any. */ - private def getBroadcastModeKeyExprs: Option[Seq[Expression]] = { + protected def getBroadcastModeKeyExprs: Option[Seq[Expression]] = { val broadcastMode = s.child match { case b: BroadcastExchangeExec => b.mode @@ -170,7 +165,7 @@ class GpuSubqueryBroadcastMeta( case class GpuSubqueryBroadcastExec( name: String, - index: Int, + indices: Seq[Int], buildKeys: Seq[Expression], child: SparkPlan)(modeKeys: Option[Seq[Expression]]) extends ShimBaseSubqueryExec with GpuExec with ShimUnaryExecNode { @@ -182,16 +177,18 @@ case class GpuSubqueryBroadcastExec( // correctly report the output length, so that `InSubqueryExec` can know it's the single-column // execution mode, not multi-column. override def output: Seq[Attribute] = { - val key = buildKeys(index) - val name = key match { - case n: NamedExpression => - n.name - case cast: Cast if cast.child.isInstanceOf[NamedExpression] => - cast.child.asInstanceOf[NamedExpression].name - case _ => - "key" + indices.map { index => + val key = buildKeys(index) + val name = key match { + case n: NamedExpression => + n.name + case cast: Cast if cast.child.isInstanceOf[NamedExpression] => + cast.child.asInstanceOf[NamedExpression].name + case _ => + "key" + } + AttributeReference(name, key.dataType, key.nullable)() } - Seq(AttributeReference(name, key.dataType, key.nullable)()) } override lazy val additionalMetrics: Map[String, GpuMetric] = Map( @@ -200,7 +197,7 @@ case class GpuSubqueryBroadcastExec( override def doCanonicalize(): SparkPlan = { val keys = buildKeys.map(k => QueryPlan.normalizeExpressions(k, child.output)) - GpuSubqueryBroadcastExec("dpp", index, keys, child.canonicalized)(modeKeys) + GpuSubqueryBroadcastExec("dpp", indices, keys, child.canonicalized)(modeKeys) } @transient @@ -235,28 +232,30 @@ case class GpuSubqueryBroadcastExec( // are being extracted. The CPU already has the key projections applied in the broadcast // data and thus does not have similar logic here. val broadcastModeProject = modeKeys.map { keyExprs => - val keyExpr = if (GpuHashJoin.canRewriteAsLongType(buildKeys)) { + val exprs = if (GpuHashJoin.canRewriteAsLongType(buildKeys)) { // in this case, there is only 1 key expression since it's a packed version that encompasses // multiple integral values into a single long using bit logic. In CPU Spark, the broadcast // would create a LongHashedRelation instead of a standard HashedRelation. - keyExprs.head + indices.map { _ => keyExprs.head } } else { - keyExprs(index) + indices.map { idx => keyExprs(idx) } } - UnsafeProjection.create(keyExpr) + UnsafeProjection.create(exprs) } - // Use the single output of the broadcast mode projection if it exists - val rowProjectIndex = if (broadcastModeProject.isDefined) 0 else index - val rowExpr = if (GpuHashJoin.canRewriteAsLongType(buildKeys)) { + val rowExprs = if (GpuHashJoin.canRewriteAsLongType(buildKeys)) { // Since this is the expected output for a LongHashedRelation, we can extract the key from the - // long packed key using bit logic, using this method available in HashJoin to give us the - // correct key expression. - HashJoin.extractKeyExprAt(buildKeys, index) + // long packed key using bit logic, using this method available in HashJoin to give us the + // correct key expression. + indices.map { idx => HashJoin.extractKeyExprAt(buildKeys, idx) } } else { - BoundReference(rowProjectIndex, buildKeys(index).dataType, buildKeys(index).nullable) + indices.map { idx => + // Use the single output of the broadcast mode projection if it exists + val rowProjectIndex = if (broadcastModeProject.isDefined) 0 else idx + BoundReference(rowProjectIndex, buildKeys(idx).dataType, buildKeys(idx).nullable) + } } - val rowProject = UnsafeProjection.create(rowExpr) + val rowProject = UnsafeProjection.create(rowExprs) // Deserializes the batch on the host. Then, transforms it to rows and performs row-wise // projection. We should NOT run any device operation on the driver node. diff --git a/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastMeta.scala b/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastMeta.scala new file mode 100644 index 00000000000..9bcfa33ab87 --- /dev/null +++ b/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastMeta.scala @@ -0,0 +1,56 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed 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. + */ +/*** spark-rapids-shim-json-lines +{"spark": "311"} +{"spark": "312"} +{"spark": "313"} +{"spark": "320"} +{"spark": "321"} +{"spark": "321cdh"} +{"spark": "322"} +{"spark": "323"} +{"spark": "324"} +{"spark": "330"} +{"spark": "330cdh"} +{"spark": "331"} +{"spark": "332"} +{"spark": "332cdh"} +{"spark": "333"} +{"spark": "334"} +{"spark": "340"} +{"spark": "341"} +{"spark": "342"} +{"spark": "343"} +{"spark": "350"} +{"spark": "351"} +spark-rapids-shim-json-lines ***/ +package org.apache.spark.sql.rapids.execution + +import com.nvidia.spark.rapids.{DataFromReplacementRule, GpuExec, RapidsConf, RapidsMeta} + +import org.apache.spark.sql.execution.SubqueryBroadcastExec + +class GpuSubqueryBroadcastMeta( + s: SubqueryBroadcastExec, + conf: RapidsConf, + p: Option[RapidsMeta[_, _, _]], + r: DataFromReplacementRule) extends + GpuSubqueryBroadcastMetaBase(s, conf, p, r) { + override def convertToGpu(): GpuExec = { + GpuSubqueryBroadcastExec(s.name, Seq(s.index), s.buildKeys, broadcastBuilder())( + getBroadcastModeKeyExprs) + } +} diff --git a/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastMeta.scala b/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastMeta.scala new file mode 100644 index 00000000000..c16564f523e --- /dev/null +++ b/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastMeta.scala @@ -0,0 +1,35 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed 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. + */ +/*** spark-rapids-shim-json-lines +{"spark": "400"} +spark-rapids-shim-json-lines ***/ +package org.apache.spark.sql.rapids.execution + +import com.nvidia.spark.rapids.{DataFromReplacementRule, GpuExec, RapidsConf, RapidsMeta} + +import org.apache.spark.sql.execution.SubqueryBroadcastExec + +class GpuSubqueryBroadcastMeta( + s: SubqueryBroadcastExec, + conf: RapidsConf, + p: Option[RapidsMeta[_, _, _]], + r: DataFromReplacementRule) extends + GpuSubqueryBroadcastMetaBase(s, conf, p, r) { + override def convertToGpu(): GpuExec = { + GpuSubqueryBroadcastExec(s.name, s.indices, s.buildKeys, broadcastBuilder())( + getBroadcastModeKeyExprs) + } +} diff --git a/tests/src/test/spark321/scala/com/nvidia/spark/rapids/DynamicPruningSuite.scala b/tests/src/test/spark321/scala/com/nvidia/spark/rapids/DynamicPruningSuite.scala index 2d4156d1b3b..722e5bb215b 100644 --- a/tests/src/test/spark321/scala/com/nvidia/spark/rapids/DynamicPruningSuite.scala +++ b/tests/src/test/spark321/scala/com/nvidia/spark/rapids/DynamicPruningSuite.scala @@ -66,7 +66,7 @@ class DynamicPruningSuite // NOTE: We remove the AdaptiveSparkPlanExec since we can't re-run the new plan // under AQE because that fundamentally requires some rewrite and stage // ordering which we can't do for this test. - case GpuSubqueryBroadcastExec(name, index, buildKeys, child) => + case GpuSubqueryBroadcastExec(name, Seq(index), buildKeys, child) => val newChild = child match { case a @ AdaptiveSparkPlanExec(_, _, _, _, _) => (new GpuTransitionOverrides()).apply(ColumnarToRowExec(a.executedPlan)) From d1e986d6b7ab5cdb44b68a601f566c17ce3c9bcf Mon Sep 17 00:00:00 2001 From: Haoyang Li Date: Wed, 29 May 2024 00:05:05 +0800 Subject: [PATCH 02/68] Fix NoSuchElementException when rlike with empty pattern (#10914) Signed-off-by: Haoyang Li --- docs/compatibility.md | 1 + integration_tests/src/main/python/regexp_test.py | 9 +++++++++ .../main/scala/com/nvidia/spark/rapids/RegexParser.scala | 8 ++++---- 3 files changed, 14 insertions(+), 4 deletions(-) diff --git a/docs/compatibility.md b/docs/compatibility.md index 574465b4496..f9af6764498 100644 --- a/docs/compatibility.md +++ b/docs/compatibility.md @@ -495,6 +495,7 @@ The following regular expression patterns are not yet supported on the GPU and w - Character classes that use union, intersection, or subtraction semantics, such as `[a-d[m-p]]`, `[a-z&&[def]]`, or `[a-z&&[^bc]]` - Empty groups: `()` +- Empty pattern: `""` Work is ongoing to increase the range of regular expressions that can run on the GPU. diff --git a/integration_tests/src/main/python/regexp_test.py b/integration_tests/src/main/python/regexp_test.py index 2372d700d72..89929eb6762 100644 --- a/integration_tests/src/main/python/regexp_test.py +++ b/integration_tests/src/main/python/regexp_test.py @@ -790,6 +790,15 @@ def test_rlike_fallback_empty_group(): 'RLike', conf=_regexp_conf) +@allow_non_gpu('ProjectExec', 'RLike') +def test_rlike_fallback_empty_pattern(): + gen = mk_str_gen('[abcd]{1,3}') + assert_gpu_fallback_collect( + lambda spark: unary_op_df(spark, gen).selectExpr( + 'a rlike ""'), + 'RLike', + conf=_regexp_conf) + def test_rlike_escape(): gen = mk_str_gen('[ab]{0,2};?[\\-\\+]{0,2}/?') assert_gpu_and_cpu_are_equal_collect( diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala index 99d8d124a52..45d5e07dd73 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala @@ -2045,10 +2045,10 @@ object RegexRewrite { private def getPrefixRangePattern(astLs: collection.Seq[RegexAST]): Option[(String, Int, Int, Int)] = { val haveLiteralPrefix = isliteralString(astLs.dropRight(1)) - val endsWithRange = astLs.last match { - case RegexRepetition( - RegexCharacterClass(false,ListBuffer(RegexCharacterRange(a,b))), - quantifier) => { + val endsWithRange = astLs.lastOption match { + case Some(RegexRepetition( + RegexCharacterClass(false, ListBuffer(RegexCharacterRange(a,b))), + quantifier)) => { val (start, end) = (a, b) match { case (RegexChar(start), RegexChar(end)) => (start, end) case _ => return None From 02f45950011da0906f030e840874314de12fe188 Mon Sep 17 00:00:00 2001 From: Zach Puller Date: Tue, 28 May 2024 13:29:47 -0700 Subject: [PATCH 03/68] append zpuller to authorized user of blossom-ci (#10929) Signed-off-by: Zach Puller --- .github/workflows/blossom-ci.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.github/workflows/blossom-ci.yml b/.github/workflows/blossom-ci.yml index 848be4e0a56..b3cbbb6ad14 100644 --- a/.github/workflows/blossom-ci.yml +++ b/.github/workflows/blossom-ci.yml @@ -70,6 +70,7 @@ jobs: parthosa,\ liurenjie1024,\ binmahone,\ + zpuller,\ ', format('{0},', github.actor)) && github.event.comment.body == 'build' steps: - name: Check if comment is issued by authorized person From 91bdb921780788969f5d5c441afe726f6805fb85 Mon Sep 17 00:00:00 2001 From: Alessandro Bellina Date: Tue, 28 May 2024 16:36:59 -0500 Subject: [PATCH 04/68] Disable default cuDF pinned pool (#10868) * Disabling the cuDF default pinned pool for 24.06 Signed-off-by: Alessandro Bellina * Add a warning in case we can't configure the cuDF default pool --------- Signed-off-by: Alessandro Bellina --- .../scala/com/nvidia/spark/rapids/GpuDeviceManager.scala | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuDeviceManager.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuDeviceManager.scala index 94390100d2e..2cfce60c4a5 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuDeviceManager.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuDeviceManager.scala @@ -447,6 +447,12 @@ object GpuDeviceManager extends Logging { } else { (conf.pinnedPoolSize, -1L) } + // disable the cuDF provided default pinned pool for now + if (!PinnedMemoryPool.configureDefaultCudfPinnedPoolSize(0L)) { + // This is OK in tests because they don't unload/reload our shared + // library, and in prod it would be nice to know about it. + logWarning("The default cuDF host pool was already configured") + } if (!PinnedMemoryPool.isInitialized && pinnedSize > 0) { logInfo(s"Initializing pinned memory pool (${pinnedSize / 1024 / 1024.0} MiB)") PinnedMemoryPool.initialize(pinnedSize, gpuId, setCuioDefaultResource) From d2a6e38a974eb423e7f7a2b88f674527f1cf4339 Mon Sep 17 00:00:00 2001 From: Haoyang Li Date: Wed, 29 May 2024 06:55:55 +0800 Subject: [PATCH 05/68] Fix type convert error in format_number scalar input (#10900) Signed-off-by: Haoyang Li --- .../scala/org/apache/spark/sql/rapids/stringFunctions.scala | 2 +- .../org/apache/spark/sql/rapids/utils/RapidsTestSettings.scala | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringFunctions.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringFunctions.scala index 0aadc2c4e66..b875c84edbf 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringFunctions.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringFunctions.scala @@ -2384,7 +2384,7 @@ case class GpuFormatNumber(x: Expression, d: Expression) } override def doColumnar(numRows: Int, lhs: GpuScalar, rhs: GpuScalar): ColumnVector = { - withResource(GpuColumnVector.from(lhs, numRows, dataType)) { col => + withResource(GpuColumnVector.from(lhs, numRows, lhs.dataType)) { col => doColumnar(col, rhs) } } diff --git a/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/utils/RapidsTestSettings.scala b/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/utils/RapidsTestSettings.scala index d0d67c41f0e..7d269cc07e4 100644 --- a/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/utils/RapidsTestSettings.scala +++ b/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/utils/RapidsTestSettings.scala @@ -82,7 +82,6 @@ class RapidsTestSettings extends BackendTestSettings { enableSuite[RapidsStringExpressionsSuite] .exclude("concat", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10775")) .exclude("string substring_index function", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10775")) - .exclude("format_number / FormatNumber", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10775")) .exclude("SPARK-22498: Concat should not generate codes beyond 64KB", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10775")) .exclude("SPARK-22549: ConcatWs should not generate codes beyond 64KB", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10775")) .exclude("SPARK-22550: Elt should not generate codes beyond 64KB", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10775")) From 7631e828febb332008b9b589b686f46c47228730 Mon Sep 17 00:00:00 2001 From: Haoyang Li Date: Wed, 29 May 2024 06:56:45 +0800 Subject: [PATCH 06/68] Use upper case for LEGACY_TIME_PARSER_POLICY to fix a spark UT (#10903) Signed-off-by: Haoyang Li --- .../src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala | 2 +- .../org/apache/spark/sql/rapids/utils/RapidsTestSettings.scala | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala index c91729ebc5f..55b883a479b 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala @@ -4349,7 +4349,7 @@ object GpuOverrides extends Logging { def getTimeParserPolicy: TimeParserPolicy = { val policy = SQLConf.get.getConfString(SQLConf.LEGACY_TIME_PARSER_POLICY.key, "EXCEPTION") - policy match { + policy.toUpperCase match { case "LEGACY" => LegacyTimeParserPolicy case "EXCEPTION" => ExceptionTimeParserPolicy case "CORRECTED" => CorrectedTimeParserPolicy diff --git a/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/utils/RapidsTestSettings.scala b/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/utils/RapidsTestSettings.scala index 7d269cc07e4..3ccd478d368 100644 --- a/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/utils/RapidsTestSettings.scala +++ b/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/utils/RapidsTestSettings.scala @@ -76,7 +76,6 @@ class RapidsTestSettings extends BackendTestSettings { .exclude("SPARK-31716: inferring should handle malformed input", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10773")) .exclude("SPARK-24190: restrictions for JSONOptions in read", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10773")) .exclude("exception mode for parsing date/timestamp string", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10773")) - .exclude("SPARK-37360: Timestamp type inference for a mix of TIMESTAMP_NTZ and TIMESTAMP_LTZ", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10773")) enableSuite[RapidsMathFunctionsSuite] enableSuite[RapidsRegexpExpressionsSuite] enableSuite[RapidsStringExpressionsSuite] From def596cb9b800d9ce8f422fdc5c1f85adf0637ae Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Tue, 28 May 2024 20:57:24 -0500 Subject: [PATCH 07/68] Add support for self-contained profiling (#10870) * Add support for self-contained profiling Signed-off-by: Jason Lowe * Use Scala regex, add executor-side logging on profile startup/shutdown * Use reflection to handle potentially missing Hadoop CallerContext * scala 2.13 fix --------- Signed-off-by: Jason Lowe --- .../com/nvidia/spark/rapids/Plugin.scala | 5 + .../spark/rapids/RangeConfMatcher.scala | 87 ++++ .../com/nvidia/spark/rapids/RapidsConf.scala | 83 ++++ .../com/nvidia/spark/rapids/profiler.scala | 404 ++++++++++++++++++ .../sql/rapids/execution/TrampolineUtil.scala | 14 +- .../spark/rapids/RangeConfMatcherSuite.scala | 82 ++++ 6 files changed, 671 insertions(+), 4 deletions(-) create mode 100644 sql-plugin/src/main/scala/com/nvidia/spark/rapids/RangeConfMatcher.scala create mode 100644 sql-plugin/src/main/scala/com/nvidia/spark/rapids/profiler.scala create mode 100644 sql-plugin/src/test/scala/com/nvidia/spark/rapids/RangeConfMatcherSuite.scala diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/Plugin.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/Plugin.scala index bd5d76e2337..74204e2d698 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/Plugin.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/Plugin.scala @@ -446,6 +446,7 @@ class RapidsDriverPlugin extends DriverPlugin with Logging { } rapidsShuffleHeartbeatManager.executorHeartbeat(id) case m: GpuCoreDumpMsg => GpuCoreDumpHandler.handleMsg(m) + case m: ProfileMsg => ProfilerOnDriver.handleMsg(m) case m => throw new IllegalStateException(s"Unknown message $m") } } @@ -458,6 +459,7 @@ class RapidsDriverPlugin extends DriverPlugin with Logging { RapidsPluginUtils.detectMultipleJars(conf) RapidsPluginUtils.logPluginMode(conf) GpuCoreDumpHandler.driverInit(sc, conf) + ProfilerOnDriver.init(sc, conf) if (GpuShuffleEnv.isRapidsShuffleAvailable(conf)) { GpuShuffleEnv.initShuffleManager() @@ -507,6 +509,7 @@ class RapidsExecutorPlugin extends ExecutorPlugin with Logging { val sparkConf = pluginContext.conf() val numCores = RapidsPluginUtils.estimateCoresOnExec(sparkConf) val conf = new RapidsConf(extraConf.asScala.toMap) + ProfilerOnExecutor.init(pluginContext, conf) // Checks if the current GPU architecture is supported by the // spark-rapids-jni and cuDF libraries. @@ -656,6 +659,7 @@ class RapidsExecutorPlugin extends ExecutorPlugin with Logging { GpuSemaphore.shutdown() PythonWorkerSemaphore.shutdown() GpuDeviceManager.shutdown() + ProfilerOnExecutor.shutdown() Option(rapidsShuffleHeartbeatEndpoint).foreach(_.close()) extraExecutorPlugins.foreach(_.shutdown()) FileCache.shutdown() @@ -692,6 +696,7 @@ class RapidsExecutorPlugin extends ExecutorPlugin with Logging { override def onTaskStart(): Unit = { startTaskNvtx(TaskContext.get) extraExecutorPlugins.foreach(_.onTaskStart()) + ProfilerOnExecutor.onTaskStart() } override def onTaskSucceeded(): Unit = { diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RangeConfMatcher.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RangeConfMatcher.scala new file mode 100644 index 00000000000..951e8f71990 --- /dev/null +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RangeConfMatcher.scala @@ -0,0 +1,87 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed 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 com.nvidia.spark.rapids + +import scala.util.Try + +/** + * Determines if a value is in a comma-separated list of values and/or + * hyphenated ranges provided by the user for a configuration setting. + */ +class RangeConfMatcher(configKey: String, configValue: Option[String]) { + def this(conf: RapidsConf, entry: ConfEntry[String]) = { + this(entry.key, Some(conf.get(entry))) + } + + def this(conf: RapidsConf, entry: OptionalConfEntry[String]) = { + this(entry.key, conf.get(entry)) + } + + private val (stringSet, intRanges) = { + configValue.map { cv => + val parts = cv.split(',') + val (rangeParts, singleParts) = parts.partition(_.contains('-')) + val ranges = try { + rangeParts.map(RangeConfMatcher.parseRange) + } catch { + case e: IllegalArgumentException => + throw new IllegalArgumentException(s"Invalid range settings for $configKey: $cv", e) + } + (singleParts.map(_.trim).toSet, ranges) + }.getOrElse((Set.empty[String], Array.empty[(Int, Int)])) + } + + val isEmpty: Boolean = stringSet.isEmpty && intRanges.isEmpty + val nonEmpty: Boolean = !isEmpty + + def size: Int = { + stringSet.size + intRanges.map { + case (start, end) => end - start + 1 + }.sum + } + + /** Returns true if the string value is in the configured values or ranges. */ + def contains(v: String): Boolean = { + stringSet.contains(v) || (intRanges.nonEmpty && Try(v.toInt).map(checkRanges).getOrElse(false)) + } + + /** Returns true if the integer value is in the configured values or ranges. */ + def contains(v: Int): Boolean = { + checkRanges(v) || stringSet.contains(v.toString) + } + + private def checkRanges(v: Int): Boolean = { + intRanges.exists { + case (start, end) => start <= v && v <= end + } + } +} + +object RangeConfMatcher { + def parseRange(rangeStr: String): (Int,Int) = { + val rangePair = rangeStr.split('-') + if (rangePair.length != 2) { + throw new IllegalArgumentException(s"Invalid range: $rangeStr") + } + val start = rangePair.head.trim.toInt + val end = rangePair.last.trim.toInt + if (end < start) { + throw new IllegalArgumentException(s"Invalid range: $rangeStr") + } + (start, end) + } +} \ No newline at end of file diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala index f7ff38cb193..1f9380ddbd0 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala @@ -708,6 +708,71 @@ val GPU_COREDUMP_PIPE_PATTERN = conf("spark.rapids.gpu.coreDump.pipePattern") .checkValues(Set("DEBUG", "MODERATE", "ESSENTIAL")) .createWithDefault("MODERATE") + val PROFILE_PATH = conf("spark.rapids.profile.pathPrefix") + .doc("Enables profiling and specifies a URI path to use when writing profile data") + .internal() + .stringConf + .createOptional + + val PROFILE_EXECUTORS = conf("spark.rapids.profile.executors") + .doc("Comma-separated list of executors IDs and hyphenated ranges of executor IDs to " + + "profile when profiling is enabled") + .internal() + .stringConf + .createWithDefault("0") + + val PROFILE_TIME_RANGES_SECONDS = conf("spark.rapids.profile.timeRangesInSeconds") + .doc("Comma-separated list of start-end ranges of time, in seconds, since executor startup " + + "to start and stop profiling. For example, a value of 10-30,100-110 will have the profiler " + + "wait for 10 seconds after executor startup then profile for 20 seconds, then wait for " + + "70 seconds then profile again for the next 10 seconds") + .internal() + .stringConf + .createOptional + + val PROFILE_JOBS = conf("spark.rapids.profile.jobs") + .doc("Comma-separated list of job IDs and hyphenated ranges of job IDs to " + + "profile when profiling is enabled") + .internal() + .stringConf + .createOptional + + val PROFILE_STAGES = conf("spark.rapids.profile.stages") + .doc("Comma-separated list of stage IDs and hyphenated ranges of stage IDs to " + + "profile when profiling is enabled") + .internal() + .stringConf + .createOptional + + val PROFILE_DRIVER_POLL_MILLIS = conf("spark.rapids.profile.driverPollMillis") + .doc("Interval in milliseconds the executors will poll for job and stage completion when " + + "stage-level profiling is used.") + .internal() + .integerConf + .createWithDefault(1000) + + val PROFILE_COMPRESSION = conf("spark.rapids.profile.compression") + .doc("Specifies the compression codec to use when writing profile data, one of " + + "zstd or none") + .internal() + .stringConf + .transform(_.toLowerCase(java.util.Locale.ROOT)) + .checkValues(Set("zstd", "none")) + .createWithDefault("zstd") + + val PROFILE_FLUSH_PERIOD_MILLIS = conf("spark.rapids.profile.flushPeriodMillis") + .doc("Specifies the time period in milliseconds to flush profile records. " + + "A value <= 0 will disable time period flushing.") + .internal() + .integerConf + .createWithDefault(0) + + val PROFILE_WRITE_BUFFER_SIZE = conf("spark.rapids.profile.writeBufferSize") + .doc("Buffer size to use when writing profile records.") + .internal() + .bytesConf(ByteUnit.BYTE) + .createWithDefault(1024 * 1024) + // ENABLE/DISABLE PROCESSING val SQL_ENABLED = conf("spark.rapids.sql.enabled") @@ -2397,6 +2462,24 @@ class RapidsConf(conf: Map[String, String]) extends Logging { lazy val metricsLevel: String = get(METRICS_LEVEL) + lazy val profilePath: Option[String] = get(PROFILE_PATH) + + lazy val profileExecutors: String = get(PROFILE_EXECUTORS) + + lazy val profileTimeRangesSeconds: Option[String] = get(PROFILE_TIME_RANGES_SECONDS) + + lazy val profileJobs: Option[String] = get(PROFILE_JOBS) + + lazy val profileStages: Option[String] = get(PROFILE_STAGES) + + lazy val profileDriverPollMillis: Int = get(PROFILE_DRIVER_POLL_MILLIS) + + lazy val profileCompression: String = get(PROFILE_COMPRESSION) + + lazy val profileFlushPeriodMillis: Int = get(PROFILE_FLUSH_PERIOD_MILLIS) + + lazy val profileWriteBufferSize: Long = get(PROFILE_WRITE_BUFFER_SIZE) + lazy val isSqlEnabled: Boolean = get(SQL_ENABLED) lazy val isSqlExecuteOnGPU: Boolean = get(SQL_MODE).equals("executeongpu") diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/profiler.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/profiler.scala new file mode 100644 index 00000000000..e6e2bcc9f7d --- /dev/null +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/profiler.scala @@ -0,0 +1,404 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed 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 com.nvidia.spark.rapids + +import java.lang.reflect.Method +import java.nio.ByteBuffer +import java.nio.channels.{Channels, WritableByteChannel} +import java.util.concurrent.{ConcurrentHashMap, Future, ScheduledExecutorService, TimeUnit} + +import scala.collection.mutable + +import com.nvidia.spark.rapids.jni.Profiler +import org.apache.hadoop.fs.Path + +import org.apache.spark.{SparkContext, TaskContext} +import org.apache.spark.api.plugin.PluginContext +import org.apache.spark.internal.Logging +import org.apache.spark.io.CompressionCodec +import org.apache.spark.scheduler.{SparkListener, SparkListenerJobEnd, SparkListenerStageCompleted} +import org.apache.spark.sql.rapids.execution.TrampolineUtil +import org.apache.spark.util.SerializableConfiguration + +object ProfilerOnExecutor extends Logging { + private val jobPattern = raw"SPARK_.*_JId_([0-9]+).*".r + private var writer: Option[ProfileWriter] = None + private var timeRanges: Option[Seq[(Long, Long)]] = None + private var jobRanges: RangeConfMatcher = null + private var stageRanges: RangeConfMatcher = null + // NOTE: Active sets are updated asynchronously, synchronize on ProfilerOnExecutor to access + private val activeJobs = mutable.HashSet[Int]() + private val activeStages = mutable.HashSet[Int]() + private var timer: Option[ScheduledExecutorService] = None + private var timerFuture: Option[Future[_]] = None + private var driverPollMillis = 0 + private val startTimestamp = System.nanoTime() + private var isProfileActive = false + private var currentContextMethod: Method = null + private var getContextMethod: Method = null + + def init(pluginCtx: PluginContext, conf: RapidsConf): Unit = { + require(writer.isEmpty, "Already initialized") + timeRanges = conf.profileTimeRangesSeconds.map(parseTimeRanges) + jobRanges = new RangeConfMatcher(conf, RapidsConf.PROFILE_JOBS) + stageRanges = new RangeConfMatcher(conf, RapidsConf.PROFILE_STAGES) + driverPollMillis = conf.profileDriverPollMillis + if (timeRanges.isDefined && (stageRanges.nonEmpty || jobRanges.nonEmpty)) { + throw new UnsupportedOperationException( + "Profiling with time ranges and stage or job ranges simultaneously is not supported") + } + if (jobRanges.nonEmpty) { + // Hadoop's CallerContext is used to identify the job ID of a task on the executor. + val callerContextClass = TrampolineUtil.classForName("org.apache.hadoop.ipc.CallerContext") + currentContextMethod = callerContextClass.getMethod("getCurrent") + getContextMethod = callerContextClass.getMethod("getContext") + } + writer = conf.profilePath.flatMap { pathPrefix => + val executorId = pluginCtx.executorID() + if (shouldProfile(executorId, conf)) { + logInfo("Initializing profiler") + if (jobRanges.nonEmpty) { + // Need caller context enabled to get the job ID of a task on the executor + TrampolineUtil.getSparkHadoopUtilConf.setBoolean("hadoop.caller.context.enabled", true) + } + val codec = conf.profileCompression match { + case "none" => None + case c => Some(TrampolineUtil.createCodec(pluginCtx.conf(), c)) + } + val w = new ProfileWriter(pluginCtx, pathPrefix, codec) + Profiler.init(w, conf.profileWriteBufferSize, conf.profileFlushPeriodMillis) + Some(w) + } else { + None + } + } + writer.foreach { _ => + updateAndSchedule() + } + } + + def onTaskStart(): Unit = { + if (jobRanges.nonEmpty) { + val callerCtx = currentContextMethod.invoke(null) + if (callerCtx != null) { + getContextMethod.invoke(callerCtx).asInstanceOf[String] match { + case jobPattern(jid) => + val jobId = jid.toInt + if (jobRanges.contains(jobId)) { + synchronized { + activeJobs.add(jobId) + enable() + startPollingDriver() + } + } + case _ => + } + } + } + if (stageRanges.nonEmpty) { + val taskCtx = TaskContext.get + val stageId = taskCtx.stageId + if (stageRanges.contains(stageId)) { + synchronized { + activeStages.add(taskCtx.stageId) + enable() + startPollingDriver() + } + } + } + } + + def shutdown(): Unit = { + writer.foreach { w => + timerFuture.foreach(_.cancel(false)) + timerFuture = None + Profiler.shutdown() + w.close() + } + writer = None + } + + private def enable(): Unit = { + writer.foreach { w => + if (!isProfileActive) { + Profiler.start() + isProfileActive = true + w.pluginCtx.send(ProfileStatusMsg(w.executorId, "profile started")) + } + } + } + + private def disable(): Unit = { + writer.foreach { w => + if (isProfileActive) { + Profiler.stop() + isProfileActive = false + w.pluginCtx.send(ProfileStatusMsg(w.executorId, "profile stopped")) + } + } + } + + private def shouldProfile(executorId: String, conf: RapidsConf): Boolean = { + val matcher = new RangeConfMatcher(conf, RapidsConf.PROFILE_EXECUTORS) + matcher.contains(executorId) + } + + private def parseTimeRanges(confVal: String): Seq[(Long, Long)] = { + val ranges = try { + confVal.split(',').map(RangeConfMatcher.parseRange).map { + case (start, end) => + // convert relative time in seconds to absolute time in nanoseconds + (startTimestamp + TimeUnit.SECONDS.toNanos(start), + startTimestamp + TimeUnit.SECONDS.toNanos(end)) + } + } catch { + case e: IllegalArgumentException => + throw new IllegalArgumentException( + s"Invalid range settings for ${RapidsConf.PROFILE_TIME_RANGES_SECONDS}: $confVal", e) + } + ranges.sorted.toIndexedSeq + } + + private def updateAndSchedule(): Unit = { + if (timeRanges.isDefined) { + if (timer.isEmpty) { + timer = Some(TrampolineUtil.newDaemonSingleThreadScheduledExecutor("profiler timer")) + } + val now = System.nanoTime() + // skip time ranges that have already passed + val currentRanges = timeRanges.get.dropWhile { + case (_, end) => end <= now + } + timeRanges = Some(currentRanges) + if (currentRanges.isEmpty) { + logWarning("No further time ranges to profile, shutting down") + shutdown() + } else { + currentRanges.headOption.foreach { + case (start, end) => + val delay = if (start <= now) { + enable() + end - now + } else { + disable() + start - now + } + timerFuture = Some(timer.get.schedule(new Runnable { + override def run(): Unit = try { + updateAndSchedule() + } catch { + case e: Exception => + logError(s"Error in profiler timer task", e) + } + }, delay, TimeUnit.NANOSECONDS)) + } + } + } else if (jobRanges.nonEmpty || stageRanges.nonEmpty) { + // nothing to do yet, profiling will start when tasks for targeted job/stage are seen + } else { + enable() + } + } + + private def startPollingDriver(): Unit = { + if (timerFuture.isEmpty) { + if (timer.isEmpty) { + timer = Some(TrampolineUtil.newDaemonSingleThreadScheduledExecutor("profiler timer")) + } + timerFuture = Some(timer.get.scheduleWithFixedDelay(() => try { + updateActiveFromDriver() + } catch { + case e: Exception => + logError("Profiler timer task error: ", e) + }, driverPollMillis, driverPollMillis, TimeUnit.MILLISECONDS)) + } + } + + private def stopPollingDriver(): Unit = { + timerFuture.foreach(_.cancel(false)) + timerFuture = None + } + + private def updateActiveFromDriver(): Unit = { + writer.foreach { w => + val (jobs, stages) = synchronized { + (activeJobs.toArray, activeStages.toArray) + } + val (completedJobs, completedStages, allDone) = + w.pluginCtx.ask(ProfileJobStageQueryMsg(jobs, stages)) + .asInstanceOf[(Array[Int], Array[Int], Boolean)] + if (completedJobs.nonEmpty || completedStages.nonEmpty) { + synchronized { + completedJobs.foreach(activeJobs.remove) + completedStages.foreach(activeStages.remove) + if (activeJobs.isEmpty && activeStages.isEmpty) { + disable() + stopPollingDriver() + } + } + } + if (allDone) { + logWarning("No further jobs or stages to profile, shutting down") + shutdown() + } + } + } +} + +class ProfileWriter( + val pluginCtx: PluginContext, + profilePathPrefix: String, + codec: Option[CompressionCodec]) extends Profiler.DataWriter with Logging { + val executorId: String = pluginCtx.executorID() + private val outPath = getOutputPath(profilePathPrefix, codec) + private val out = openOutput(codec) + private var isClosed = false + + override def write(data: ByteBuffer): Unit = { + if (!isClosed) { + while (data.hasRemaining) { + out.write(data) + } + } + } + + override def close(): Unit = { + if (!isClosed) { + isClosed = true + out.close() + logWarning(s"Profiling completed, output written to $outPath") + pluginCtx.send(ProfileEndMsg(executorId, outPath.toString)) + } + } + + private def getAppId: String = { + val appId = pluginCtx.conf.get("spark.app.id", "") + if (appId.isEmpty) { + java.lang.management.ManagementFactory.getRuntimeMXBean.getName + } else { + appId + } + } + + private def getOutputPath(prefix: String, codec: Option[CompressionCodec]): Path = { + val parentDir = new Path(prefix) + val suffix = codec.map(c => "." + TrampolineUtil.getCodecShortName(c.getClass.getName)) + .getOrElse("") + new Path(parentDir, s"rapids-profile-$getAppId-$executorId.bin$suffix") + } + + private def openOutput(codec: Option[CompressionCodec]): WritableByteChannel = { + logWarning(s"Profiler initialized, output will be written to $outPath") + val hadoopConf = pluginCtx.ask(ProfileInitMsg(executorId, outPath.toString)) + .asInstanceOf[SerializableConfiguration].value + val fs = outPath.getFileSystem(hadoopConf) + val fsStream = fs.create(outPath, false) + val outStream = codec.map(_.compressedOutputStream(fsStream)).getOrElse(fsStream) + Channels.newChannel(outStream) + } +} + +object ProfilerOnDriver extends Logging { + private var hadoopConf: SerializableConfiguration = null + private var jobRanges: RangeConfMatcher = null + private var numJobsToProfile: Long = 0L + private var stageRanges: RangeConfMatcher = null + private var numStagesToProfile: Long = 0L + private val completedJobs = new ConcurrentHashMap[Int, Unit]() + private val completedStages = new ConcurrentHashMap[Int, Unit]() + private var isJobsStageProfilingComplete = false + + def init(sc: SparkContext, conf: RapidsConf): Unit = { + // if no profile path, profiling is disabled and nothing to do + conf.profilePath.foreach { _ => + hadoopConf = new SerializableConfiguration(sc.hadoopConfiguration) + jobRanges = new RangeConfMatcher(conf, RapidsConf.PROFILE_JOBS) + stageRanges = new RangeConfMatcher(conf, RapidsConf.PROFILE_STAGES) + if (jobRanges.nonEmpty || stageRanges.nonEmpty) { + numJobsToProfile = jobRanges.size + numStagesToProfile = stageRanges.size + if (jobRanges.nonEmpty) { + // Need caller context enabled to get the job ID of a task on the executor + try { + TrampolineUtil.classForName("org.apache.hadoop.ipc.CallerContext") + } catch { + case _: ClassNotFoundException => + throw new UnsupportedOperationException(s"${RapidsConf.PROFILE_JOBS} requires " + + "Hadoop CallerContext which is unavailable.") + } + sc.getConf.set("hadoop.caller.context.enabled", "true") + } + sc.addSparkListener(Listener) + } + } + } + + def handleMsg(m: ProfileMsg): AnyRef = m match { + case ProfileInitMsg(executorId, path) => + logWarning(s"Profiling: Executor $executorId initialized profiler, writing to $path") + if (hadoopConf == null) { + throw new IllegalStateException("Hadoop configuration not set") + } + hadoopConf + case ProfileStatusMsg(executorId, msg) => + logWarning(s"Profiling: Executor $executorId: $msg") + null + case ProfileJobStageQueryMsg(activeJobs, activeStages) => + val filteredJobs = activeJobs.filter(j => completedJobs.containsKey(j)) + val filteredStages = activeStages.filter(s => completedStages.containsKey(s)) + (filteredJobs, filteredStages, isJobsStageProfilingComplete) + case ProfileEndMsg(executorId, path) => + logWarning(s"Profiling: Executor $executorId ended profiling, profile written to $path") + null + case _ => + throw new IllegalStateException(s"Unexpected profile msg: $m") + } + + private object Listener extends SparkListener { + override def onJobEnd(jobEnd: SparkListenerJobEnd): Unit = { + val jobId = jobEnd.jobId + if (jobRanges.contains(jobId)) { + completedJobs.putIfAbsent(jobId, ()) + isJobsStageProfilingComplete = completedJobs.size == numJobsToProfile && + completedStages.size == numStagesToProfile + } + } + + override def onStageCompleted(stageCompleted: SparkListenerStageCompleted): Unit = { + val stageId = stageCompleted.stageInfo.stageId + if (stageRanges.contains(stageId)) { + completedStages.putIfAbsent(stageId, ()) + isJobsStageProfilingComplete = completedJobs.size == numJobsToProfile && + completedStages.size == numStagesToProfile + } + } + } +} + +trait ProfileMsg + +case class ProfileInitMsg(executorId: String, path: String) extends ProfileMsg +case class ProfileStatusMsg(executorId: String, msg: String) extends ProfileMsg +case class ProfileEndMsg(executorId: String, path: String) extends ProfileMsg + +// Reply is a tuple of: +// - array of jobs that have completed +// - array of stages that have completed +// - boolean if there are no further jobs/stages to profile +case class ProfileJobStageQueryMsg( + activeJobs: Array[Int], + activeStages: Array[Int]) extends ProfileMsg diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/TrampolineUtil.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/TrampolineUtil.scala index eb16edc0b51..5ffe08348f1 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/TrampolineUtil.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/TrampolineUtil.scala @@ -16,8 +16,9 @@ package org.apache.spark.sql.rapids.execution -import java.util.concurrent.ThreadPoolExecutor +import java.util.concurrent.{ScheduledExecutorService, ThreadPoolExecutor} +import org.apache.hadoop.conf.Configuration import org.json4s.JsonAST import org.apache.spark.{SparkConf, SparkContext, SparkEnv, SparkMasterRegex, SparkUpgradeException, TaskContext} @@ -40,7 +41,7 @@ import org.apache.spark.sql.rapids.shims.DataTypeUtilsShim import org.apache.spark.sql.rapids.shims.SparkUpgradeExceptionShims import org.apache.spark.sql.types.{DataType, StructType} import org.apache.spark.storage.BlockManagerId -import org.apache.spark.util.{ShutdownHookManager, Utils} +import org.apache.spark.util.{ShutdownHookManager, ThreadUtils, Utils} object TrampolineUtil { def doExecuteBroadcast[T](child: SparkPlan): Broadcast[T] = child.doExecuteBroadcast() @@ -228,11 +229,16 @@ object TrampolineUtil { // We want to utilize the ThreadUtils class' ThreadPoolExecutor creation // which gives us important Hadoop config variables that are needed for the // Unity Catalog authentication - org.apache.spark.util.ThreadUtils.newDaemonCachedThreadPool(prefix, maxThreadNumber, - keepAliveSeconds) + ThreadUtils.newDaemonCachedThreadPool(prefix, maxThreadNumber, keepAliveSeconds) + } + + def newDaemonSingleThreadScheduledExecutor(threadName: String): ScheduledExecutorService = { + ThreadUtils.newDaemonSingleThreadScheduledExecutor(threadName) } def postEvent(sc: SparkContext, sparkEvent: SparkListenerEvent): Unit = { sc.listenerBus.post(sparkEvent) } + + def getSparkHadoopUtilConf: Configuration = SparkHadoopUtil.get.conf } diff --git a/sql-plugin/src/test/scala/com/nvidia/spark/rapids/RangeConfMatcherSuite.scala b/sql-plugin/src/test/scala/com/nvidia/spark/rapids/RangeConfMatcherSuite.scala new file mode 100644 index 00000000000..518bd3012f2 --- /dev/null +++ b/sql-plugin/src/test/scala/com/nvidia/spark/rapids/RangeConfMatcherSuite.scala @@ -0,0 +1,82 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed 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 com.nvidia.spark.rapids + +import org.scalatest.funsuite.AnyFunSuite + +class RangeConfMatcherSuite extends AnyFunSuite { + + test("empty") { + val conf = new RapidsConf(Map(RapidsConf.PROFILE_EXECUTORS.key -> "")) + val matcher = new RangeConfMatcher(conf, RapidsConf.PROFILE_EXECUTORS) + assert(!matcher.contains("x")) + assert(!matcher.contains(0)) + } + + test("bad ranges") { + Seq("-", "-4", "4-", "4-3", "d-4", "4-d", "23a-24b", "3-5,8,x-y").foreach { v => + val conf = new RapidsConf(Map(RapidsConf.PROFILE_EXECUTORS.key -> v)) + assertThrows[IllegalArgumentException] { + new RangeConfMatcher(conf, RapidsConf.PROFILE_EXECUTORS) + } + } + } + + test("singles") { + Seq("driver", "0,driver", "0, driver", "driver, 0", "1, driver, x").foreach { v => + val conf = new RapidsConf(Map(RapidsConf.PROFILE_EXECUTORS.key -> v)) + val matcher = new RangeConfMatcher(conf, RapidsConf.PROFILE_EXECUTORS) + assert(matcher.contains("driver")) + assert(!matcher.contains("driverx")) + assert(!matcher.contains("xdriver")) + assert(!matcher.contains("drive")) + assert(!matcher.contains("drive")) + } + } + + test("range only") { + Seq("7-7", "3-7", "2-30", "2-3,5-7,8-10", "2-3, 5-7, 8-10", + " 2 - 3, 5 - 7, 8 - 10").foreach { v => + val conf = new RapidsConf(Map(RapidsConf.PROFILE_EXECUTORS.key -> v)) + val matcher = new RangeConfMatcher(conf, RapidsConf.PROFILE_EXECUTORS) + assert(matcher.contains("7")) + assert(matcher.contains(7)) + assert(!matcher.contains("0")) + assert(!matcher.contains(0)) + assert(!matcher.contains("70")) + assert(!matcher.contains(70)) + } + } + + test("singles range mix") { + Seq("driver,7-10", "driver, 7 - 10", "driver, 7-10", "3-5,7,1-3,driver").foreach { v => + val conf = new RapidsConf(Map(RapidsConf.PROFILE_EXECUTORS.key -> v)) + val matcher = new RangeConfMatcher(conf, RapidsConf.PROFILE_EXECUTORS) + assert(matcher.contains("driver")) + assert(!matcher.contains("driverx")) + assert(!matcher.contains("xdriver")) + assert(!matcher.contains("drive")) + assert(!matcher.contains("drive")) + assert(matcher.contains("7")) + assert(matcher.contains(7)) + assert(!matcher.contains("0")) + assert(!matcher.contains(0)) + assert(!matcher.contains("70")) + assert(!matcher.contains(70)) + } + } +} From bb05b179acc7c4a17bbf794d16ce3f0273e0f9dc Mon Sep 17 00:00:00 2001 From: Raza Jafri Date: Tue, 28 May 2024 21:57:28 -0700 Subject: [PATCH 08/68] Revert "Add Support for Multiple Filtering Keys for Subquery Broadcast " (#10934) * Revert "Add Support for Multiple Filtering Keys for Subquery Broadcast (#10858)" This reverts commit 30018528733d52379f0cf42f11be516799daab1a. * Signing off Signed-off-by: Raza Jafri --------- Signed-off-by: Raza Jafri --- .../execution/GpuSubqueryBroadcastExec.scala | 63 ++++++++++--------- .../execution/GpuSubqueryBroadcastMeta.scala | 56 ----------------- .../execution/GpuSubqueryBroadcastMeta.scala | 35 ----------- .../spark/rapids/DynamicPruningSuite.scala | 2 +- 4 files changed, 33 insertions(+), 123 deletions(-) delete mode 100644 sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastMeta.scala delete mode 100644 sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastMeta.scala diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastExec.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastExec.scala index 72ed0e79504..1861a9f2515 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastExec.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2024, NVIDIA CORPORATION. + * Copyright (c) 2021-2023, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -41,14 +41,14 @@ import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.ThreadUtils -abstract class GpuSubqueryBroadcastMetaBase( +class GpuSubqueryBroadcastMeta( s: SubqueryBroadcastExec, conf: RapidsConf, p: Option[RapidsMeta[_, _, _]], r: DataFromReplacementRule) extends SparkPlanMeta[SubqueryBroadcastExec](s, conf, p, r) { - protected var broadcastBuilder: () => SparkPlan = _ + private var broadcastBuilder: () => SparkPlan = _ override val childExprs: Seq[BaseExprMeta[_]] = Nil @@ -140,8 +140,13 @@ abstract class GpuSubqueryBroadcastMetaBase( */ override def convertToCpu(): SparkPlan = s + override def convertToGpu(): GpuExec = { + GpuSubqueryBroadcastExec(s.name, s.index, s.buildKeys, broadcastBuilder())( + getBroadcastModeKeyExprs) + } + /** Extract the broadcast mode key expressions if there are any. */ - protected def getBroadcastModeKeyExprs: Option[Seq[Expression]] = { + private def getBroadcastModeKeyExprs: Option[Seq[Expression]] = { val broadcastMode = s.child match { case b: BroadcastExchangeExec => b.mode @@ -165,7 +170,7 @@ abstract class GpuSubqueryBroadcastMetaBase( case class GpuSubqueryBroadcastExec( name: String, - indices: Seq[Int], + index: Int, buildKeys: Seq[Expression], child: SparkPlan)(modeKeys: Option[Seq[Expression]]) extends ShimBaseSubqueryExec with GpuExec with ShimUnaryExecNode { @@ -177,18 +182,16 @@ case class GpuSubqueryBroadcastExec( // correctly report the output length, so that `InSubqueryExec` can know it's the single-column // execution mode, not multi-column. override def output: Seq[Attribute] = { - indices.map { index => - val key = buildKeys(index) - val name = key match { - case n: NamedExpression => - n.name - case cast: Cast if cast.child.isInstanceOf[NamedExpression] => - cast.child.asInstanceOf[NamedExpression].name - case _ => - "key" - } - AttributeReference(name, key.dataType, key.nullable)() + val key = buildKeys(index) + val name = key match { + case n: NamedExpression => + n.name + case cast: Cast if cast.child.isInstanceOf[NamedExpression] => + cast.child.asInstanceOf[NamedExpression].name + case _ => + "key" } + Seq(AttributeReference(name, key.dataType, key.nullable)()) } override lazy val additionalMetrics: Map[String, GpuMetric] = Map( @@ -197,7 +200,7 @@ case class GpuSubqueryBroadcastExec( override def doCanonicalize(): SparkPlan = { val keys = buildKeys.map(k => QueryPlan.normalizeExpressions(k, child.output)) - GpuSubqueryBroadcastExec("dpp", indices, keys, child.canonicalized)(modeKeys) + GpuSubqueryBroadcastExec("dpp", index, keys, child.canonicalized)(modeKeys) } @transient @@ -232,30 +235,28 @@ case class GpuSubqueryBroadcastExec( // are being extracted. The CPU already has the key projections applied in the broadcast // data and thus does not have similar logic here. val broadcastModeProject = modeKeys.map { keyExprs => - val exprs = if (GpuHashJoin.canRewriteAsLongType(buildKeys)) { + val keyExpr = if (GpuHashJoin.canRewriteAsLongType(buildKeys)) { // in this case, there is only 1 key expression since it's a packed version that encompasses // multiple integral values into a single long using bit logic. In CPU Spark, the broadcast // would create a LongHashedRelation instead of a standard HashedRelation. - indices.map { _ => keyExprs.head } + keyExprs.head } else { - indices.map { idx => keyExprs(idx) } + keyExprs(index) } - UnsafeProjection.create(exprs) + UnsafeProjection.create(keyExpr) } - val rowExprs = if (GpuHashJoin.canRewriteAsLongType(buildKeys)) { + // Use the single output of the broadcast mode projection if it exists + val rowProjectIndex = if (broadcastModeProject.isDefined) 0 else index + val rowExpr = if (GpuHashJoin.canRewriteAsLongType(buildKeys)) { // Since this is the expected output for a LongHashedRelation, we can extract the key from the - // long packed key using bit logic, using this method available in HashJoin to give us the - // correct key expression. - indices.map { idx => HashJoin.extractKeyExprAt(buildKeys, idx) } + // long packed key using bit logic, using this method available in HashJoin to give us the + // correct key expression. + HashJoin.extractKeyExprAt(buildKeys, index) } else { - indices.map { idx => - // Use the single output of the broadcast mode projection if it exists - val rowProjectIndex = if (broadcastModeProject.isDefined) 0 else idx - BoundReference(rowProjectIndex, buildKeys(idx).dataType, buildKeys(idx).nullable) - } + BoundReference(rowProjectIndex, buildKeys(index).dataType, buildKeys(index).nullable) } - val rowProject = UnsafeProjection.create(rowExprs) + val rowProject = UnsafeProjection.create(rowExpr) // Deserializes the batch on the host. Then, transforms it to rows and performs row-wise // projection. We should NOT run any device operation on the driver node. diff --git a/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastMeta.scala b/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastMeta.scala deleted file mode 100644 index 9bcfa33ab87..00000000000 --- a/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastMeta.scala +++ /dev/null @@ -1,56 +0,0 @@ -/* - * Copyright (c) 2024, NVIDIA CORPORATION. - * - * Licensed 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. - */ -/*** spark-rapids-shim-json-lines -{"spark": "311"} -{"spark": "312"} -{"spark": "313"} -{"spark": "320"} -{"spark": "321"} -{"spark": "321cdh"} -{"spark": "322"} -{"spark": "323"} -{"spark": "324"} -{"spark": "330"} -{"spark": "330cdh"} -{"spark": "331"} -{"spark": "332"} -{"spark": "332cdh"} -{"spark": "333"} -{"spark": "334"} -{"spark": "340"} -{"spark": "341"} -{"spark": "342"} -{"spark": "343"} -{"spark": "350"} -{"spark": "351"} -spark-rapids-shim-json-lines ***/ -package org.apache.spark.sql.rapids.execution - -import com.nvidia.spark.rapids.{DataFromReplacementRule, GpuExec, RapidsConf, RapidsMeta} - -import org.apache.spark.sql.execution.SubqueryBroadcastExec - -class GpuSubqueryBroadcastMeta( - s: SubqueryBroadcastExec, - conf: RapidsConf, - p: Option[RapidsMeta[_, _, _]], - r: DataFromReplacementRule) extends - GpuSubqueryBroadcastMetaBase(s, conf, p, r) { - override def convertToGpu(): GpuExec = { - GpuSubqueryBroadcastExec(s.name, Seq(s.index), s.buildKeys, broadcastBuilder())( - getBroadcastModeKeyExprs) - } -} diff --git a/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastMeta.scala b/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastMeta.scala deleted file mode 100644 index c16564f523e..00000000000 --- a/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastMeta.scala +++ /dev/null @@ -1,35 +0,0 @@ -/* - * Copyright (c) 2024, NVIDIA CORPORATION. - * - * Licensed 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. - */ -/*** spark-rapids-shim-json-lines -{"spark": "400"} -spark-rapids-shim-json-lines ***/ -package org.apache.spark.sql.rapids.execution - -import com.nvidia.spark.rapids.{DataFromReplacementRule, GpuExec, RapidsConf, RapidsMeta} - -import org.apache.spark.sql.execution.SubqueryBroadcastExec - -class GpuSubqueryBroadcastMeta( - s: SubqueryBroadcastExec, - conf: RapidsConf, - p: Option[RapidsMeta[_, _, _]], - r: DataFromReplacementRule) extends - GpuSubqueryBroadcastMetaBase(s, conf, p, r) { - override def convertToGpu(): GpuExec = { - GpuSubqueryBroadcastExec(s.name, s.indices, s.buildKeys, broadcastBuilder())( - getBroadcastModeKeyExprs) - } -} diff --git a/tests/src/test/spark321/scala/com/nvidia/spark/rapids/DynamicPruningSuite.scala b/tests/src/test/spark321/scala/com/nvidia/spark/rapids/DynamicPruningSuite.scala index 722e5bb215b..2d4156d1b3b 100644 --- a/tests/src/test/spark321/scala/com/nvidia/spark/rapids/DynamicPruningSuite.scala +++ b/tests/src/test/spark321/scala/com/nvidia/spark/rapids/DynamicPruningSuite.scala @@ -66,7 +66,7 @@ class DynamicPruningSuite // NOTE: We remove the AdaptiveSparkPlanExec since we can't re-run the new plan // under AQE because that fundamentally requires some rewrite and stage // ordering which we can't do for this test. - case GpuSubqueryBroadcastExec(name, Seq(index), buildKeys, child) => + case GpuSubqueryBroadcastExec(name, index, buildKeys, child) => val newChild = child match { case a @ AdaptiveSparkPlanExec(_, _, _, _, _) => (new GpuTransitionOverrides()).apply(ColumnarToRowExec(a.executedPlan)) From f0b13ed451086bda4b786187c800fc3afd9ea9f5 Mon Sep 17 00:00:00 2001 From: Raza Jafri Date: Wed, 29 May 2024 08:20:48 -0700 Subject: [PATCH 09/68] Fixed Databricks build [databricks] (#10933) * Fixed Databricks build * Signing off Signed-off-by: Raza Jafri * Removed unused import --------- Signed-off-by: Raza Jafri --- .../nvidia/spark/rapids/shims/Spark321PlusDBShims.scala | 4 ++-- .../execution/{shims => }/GpuSubqueryBroadcastMeta.scala | 7 +++---- 2 files changed, 5 insertions(+), 6 deletions(-) rename sql-plugin/src/main/spark330db/scala/org/apache/spark/rapids/execution/{shims => }/GpuSubqueryBroadcastMeta.scala (96%) diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/Spark321PlusDBShims.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/Spark321PlusDBShims.scala index 5e5bbed2942..fb60f15db4d 100644 --- a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/Spark321PlusDBShims.scala +++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/Spark321PlusDBShims.scala @@ -41,8 +41,8 @@ import org.apache.spark.sql.execution.exchange._ import org.apache.spark.sql.execution.python._ import org.apache.spark.sql.execution.window._ import org.apache.spark.sql.rapids.GpuSubstring -import org.apache.spark.sql.rapids.execution._ -import org.apache.spark.sql.rapids.execution.shims.{GpuSubqueryBroadcastMeta,ReuseGpuBroadcastExchangeAndSubquery} +import org.apache.spark.sql.rapids.execution.GpuSubqueryBroadcastMeta +import org.apache.spark.sql.rapids.execution.shims.ReuseGpuBroadcastExchangeAndSubquery import org.apache.spark.sql.rapids.shims._ import org.apache.spark.sql.types._ diff --git a/sql-plugin/src/main/spark330db/scala/org/apache/spark/rapids/execution/shims/GpuSubqueryBroadcastMeta.scala b/sql-plugin/src/main/spark330db/scala/org/apache/spark/rapids/execution/GpuSubqueryBroadcastMeta.scala similarity index 96% rename from sql-plugin/src/main/spark330db/scala/org/apache/spark/rapids/execution/shims/GpuSubqueryBroadcastMeta.scala rename to sql-plugin/src/main/spark330db/scala/org/apache/spark/rapids/execution/GpuSubqueryBroadcastMeta.scala index fbc013779c4..76255b3e5a6 100644 --- a/sql-plugin/src/main/spark330db/scala/org/apache/spark/rapids/execution/shims/GpuSubqueryBroadcastMeta.scala +++ b/sql-plugin/src/main/spark330db/scala/org/apache/spark/rapids/execution/GpuSubqueryBroadcastMeta.scala @@ -19,7 +19,7 @@ {"spark": "332db"} {"spark": "341db"} spark-rapids-shim-json-lines ***/ -package org.apache.spark.sql.rapids.execution.shims +package org.apache.spark.sql.rapids.execution import com.nvidia.spark.rapids.{BaseExprMeta, DataFromReplacementRule, GpuExec, RapidsConf, RapidsMeta, SparkPlanMeta} @@ -29,7 +29,6 @@ import org.apache.spark.sql.execution.{SparkPlan, SubqueryBroadcastExec} import org.apache.spark.sql.execution.adaptive.{BroadcastQueryStageExec} import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ReusedExchangeExec} import org.apache.spark.sql.execution.joins.HashedRelationBroadcastMode -import org.apache.spark.sql.rapids.execution._ class GpuSubqueryBroadcastMeta( s: SubqueryBroadcastExec, @@ -94,7 +93,7 @@ class GpuSubqueryBroadcastMeta( override def convertToCpu(): SparkPlan = s override def convertToGpu(): GpuExec = { - GpuSubqueryBroadcastExec(s.name, s.index, s.buildKeys, broadcastBuilder())( + GpuSubqueryBroadcastExec(s.name, Seq(s.index), s.buildKeys, broadcastBuilder())( getBroadcastModeKeyExprs) } @@ -123,4 +122,4 @@ class GpuSubqueryBroadcastMeta( case m => throw new UnsupportedOperationException(s"Unknown broadcast mode $m") } } -} \ No newline at end of file +} From dfcde727a9c744a008e256c25d202336639b2e4c Mon Sep 17 00:00:00 2001 From: Zach Puller Date: Wed, 29 May 2024 09:17:51 -0700 Subject: [PATCH 10/68] Add classloader diagnostics to initShuffleManager error message (#10871) Add classloader diagnostics to initShuffleManager error message --------- Signed-off-by: Zach Puller Co-authored-by: Jason Lowe Co-authored-by: Gera Shegalov Co-authored-by: Alessandro Bellina --- .../apache/spark/sql/rapids/GpuShuffleEnv.scala | 17 ++++++++++++++--- 1 file changed, 14 insertions(+), 3 deletions(-) diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuShuffleEnv.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuShuffleEnv.scala index 1682dd13c22..1b0ee21d494 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuShuffleEnv.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuShuffleEnv.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019-2023, NVIDIA CORPORATION. + * Copyright (c) 2019-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -104,11 +104,22 @@ object GpuShuffleEnv extends Logging { // this forces the initialization when we know we are ready in the driver and executor. // def initShuffleManager(): Unit = { - SparkEnv.get.shuffleManager match { + val shuffleManager = SparkEnv.get.shuffleManager + shuffleManager match { case rapidsShuffleManager: RapidsShuffleManagerLike => rapidsShuffleManager.initialize case _ => - throw new IllegalStateException(s"Cannot initialize the RAPIDS Shuffle Manager") + val rsmLoaderViaShuffleManager = shuffleManager.getClass.getSuperclass.getInterfaces + .collectFirst { + case c if c.getName == classOf[RapidsShuffleManagerLike].getName => c.getClassLoader + } + val rsmLoaderDirect = classOf[RapidsShuffleManagerLike].getClassLoader + + throw new IllegalStateException(s"Cannot initialize the RAPIDS Shuffle Manager " + + s"${shuffleManager}! Expected: an instance of RapidsShuffleManagerLike loaded by " + + s"${rsmLoaderDirect}. Actual: ${shuffleManager} tagged with RapidsShuffleManagerLike " + + s"loaded by: ${rsmLoaderViaShuffleManager}" + ) } } From fe69470fb07281c77383be6dd02a45c39ea5d7e8 Mon Sep 17 00:00:00 2001 From: Raza Jafri Date: Wed, 29 May 2024 22:57:17 -0700 Subject: [PATCH 11/68] Add Support for Multiple Filtering Keys for Subquery Broadcast [databricks] (#10945) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * Revert "Revert "Add Support for Multiple Filtering Keys for Subquery Broadcas…" This reverts commit bb05b179acc7c4a17bbf794d16ce3f0273e0f9dc. * Signing off Signed-off-by: Raza Jafri --------- Signed-off-by: Raza Jafri --- .../execution/GpuSubqueryBroadcastExec.scala | 63 +++++++++---------- .../execution/GpuSubqueryBroadcastMeta.scala | 56 +++++++++++++++++ .../execution/GpuSubqueryBroadcastMeta.scala | 35 +++++++++++ .../spark/rapids/DynamicPruningSuite.scala | 2 +- 4 files changed, 123 insertions(+), 33 deletions(-) create mode 100644 sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastMeta.scala create mode 100644 sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastMeta.scala diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastExec.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastExec.scala index 1861a9f2515..72ed0e79504 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastExec.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2023, NVIDIA CORPORATION. + * Copyright (c) 2021-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -41,14 +41,14 @@ import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.ThreadUtils -class GpuSubqueryBroadcastMeta( +abstract class GpuSubqueryBroadcastMetaBase( s: SubqueryBroadcastExec, conf: RapidsConf, p: Option[RapidsMeta[_, _, _]], r: DataFromReplacementRule) extends SparkPlanMeta[SubqueryBroadcastExec](s, conf, p, r) { - private var broadcastBuilder: () => SparkPlan = _ + protected var broadcastBuilder: () => SparkPlan = _ override val childExprs: Seq[BaseExprMeta[_]] = Nil @@ -140,13 +140,8 @@ class GpuSubqueryBroadcastMeta( */ override def convertToCpu(): SparkPlan = s - override def convertToGpu(): GpuExec = { - GpuSubqueryBroadcastExec(s.name, s.index, s.buildKeys, broadcastBuilder())( - getBroadcastModeKeyExprs) - } - /** Extract the broadcast mode key expressions if there are any. */ - private def getBroadcastModeKeyExprs: Option[Seq[Expression]] = { + protected def getBroadcastModeKeyExprs: Option[Seq[Expression]] = { val broadcastMode = s.child match { case b: BroadcastExchangeExec => b.mode @@ -170,7 +165,7 @@ class GpuSubqueryBroadcastMeta( case class GpuSubqueryBroadcastExec( name: String, - index: Int, + indices: Seq[Int], buildKeys: Seq[Expression], child: SparkPlan)(modeKeys: Option[Seq[Expression]]) extends ShimBaseSubqueryExec with GpuExec with ShimUnaryExecNode { @@ -182,16 +177,18 @@ case class GpuSubqueryBroadcastExec( // correctly report the output length, so that `InSubqueryExec` can know it's the single-column // execution mode, not multi-column. override def output: Seq[Attribute] = { - val key = buildKeys(index) - val name = key match { - case n: NamedExpression => - n.name - case cast: Cast if cast.child.isInstanceOf[NamedExpression] => - cast.child.asInstanceOf[NamedExpression].name - case _ => - "key" + indices.map { index => + val key = buildKeys(index) + val name = key match { + case n: NamedExpression => + n.name + case cast: Cast if cast.child.isInstanceOf[NamedExpression] => + cast.child.asInstanceOf[NamedExpression].name + case _ => + "key" + } + AttributeReference(name, key.dataType, key.nullable)() } - Seq(AttributeReference(name, key.dataType, key.nullable)()) } override lazy val additionalMetrics: Map[String, GpuMetric] = Map( @@ -200,7 +197,7 @@ case class GpuSubqueryBroadcastExec( override def doCanonicalize(): SparkPlan = { val keys = buildKeys.map(k => QueryPlan.normalizeExpressions(k, child.output)) - GpuSubqueryBroadcastExec("dpp", index, keys, child.canonicalized)(modeKeys) + GpuSubqueryBroadcastExec("dpp", indices, keys, child.canonicalized)(modeKeys) } @transient @@ -235,28 +232,30 @@ case class GpuSubqueryBroadcastExec( // are being extracted. The CPU already has the key projections applied in the broadcast // data and thus does not have similar logic here. val broadcastModeProject = modeKeys.map { keyExprs => - val keyExpr = if (GpuHashJoin.canRewriteAsLongType(buildKeys)) { + val exprs = if (GpuHashJoin.canRewriteAsLongType(buildKeys)) { // in this case, there is only 1 key expression since it's a packed version that encompasses // multiple integral values into a single long using bit logic. In CPU Spark, the broadcast // would create a LongHashedRelation instead of a standard HashedRelation. - keyExprs.head + indices.map { _ => keyExprs.head } } else { - keyExprs(index) + indices.map { idx => keyExprs(idx) } } - UnsafeProjection.create(keyExpr) + UnsafeProjection.create(exprs) } - // Use the single output of the broadcast mode projection if it exists - val rowProjectIndex = if (broadcastModeProject.isDefined) 0 else index - val rowExpr = if (GpuHashJoin.canRewriteAsLongType(buildKeys)) { + val rowExprs = if (GpuHashJoin.canRewriteAsLongType(buildKeys)) { // Since this is the expected output for a LongHashedRelation, we can extract the key from the - // long packed key using bit logic, using this method available in HashJoin to give us the - // correct key expression. - HashJoin.extractKeyExprAt(buildKeys, index) + // long packed key using bit logic, using this method available in HashJoin to give us the + // correct key expression. + indices.map { idx => HashJoin.extractKeyExprAt(buildKeys, idx) } } else { - BoundReference(rowProjectIndex, buildKeys(index).dataType, buildKeys(index).nullable) + indices.map { idx => + // Use the single output of the broadcast mode projection if it exists + val rowProjectIndex = if (broadcastModeProject.isDefined) 0 else idx + BoundReference(rowProjectIndex, buildKeys(idx).dataType, buildKeys(idx).nullable) + } } - val rowProject = UnsafeProjection.create(rowExpr) + val rowProject = UnsafeProjection.create(rowExprs) // Deserializes the batch on the host. Then, transforms it to rows and performs row-wise // projection. We should NOT run any device operation on the driver node. diff --git a/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastMeta.scala b/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastMeta.scala new file mode 100644 index 00000000000..9bcfa33ab87 --- /dev/null +++ b/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastMeta.scala @@ -0,0 +1,56 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed 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. + */ +/*** spark-rapids-shim-json-lines +{"spark": "311"} +{"spark": "312"} +{"spark": "313"} +{"spark": "320"} +{"spark": "321"} +{"spark": "321cdh"} +{"spark": "322"} +{"spark": "323"} +{"spark": "324"} +{"spark": "330"} +{"spark": "330cdh"} +{"spark": "331"} +{"spark": "332"} +{"spark": "332cdh"} +{"spark": "333"} +{"spark": "334"} +{"spark": "340"} +{"spark": "341"} +{"spark": "342"} +{"spark": "343"} +{"spark": "350"} +{"spark": "351"} +spark-rapids-shim-json-lines ***/ +package org.apache.spark.sql.rapids.execution + +import com.nvidia.spark.rapids.{DataFromReplacementRule, GpuExec, RapidsConf, RapidsMeta} + +import org.apache.spark.sql.execution.SubqueryBroadcastExec + +class GpuSubqueryBroadcastMeta( + s: SubqueryBroadcastExec, + conf: RapidsConf, + p: Option[RapidsMeta[_, _, _]], + r: DataFromReplacementRule) extends + GpuSubqueryBroadcastMetaBase(s, conf, p, r) { + override def convertToGpu(): GpuExec = { + GpuSubqueryBroadcastExec(s.name, Seq(s.index), s.buildKeys, broadcastBuilder())( + getBroadcastModeKeyExprs) + } +} diff --git a/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastMeta.scala b/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastMeta.scala new file mode 100644 index 00000000000..c16564f523e --- /dev/null +++ b/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/execution/GpuSubqueryBroadcastMeta.scala @@ -0,0 +1,35 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed 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. + */ +/*** spark-rapids-shim-json-lines +{"spark": "400"} +spark-rapids-shim-json-lines ***/ +package org.apache.spark.sql.rapids.execution + +import com.nvidia.spark.rapids.{DataFromReplacementRule, GpuExec, RapidsConf, RapidsMeta} + +import org.apache.spark.sql.execution.SubqueryBroadcastExec + +class GpuSubqueryBroadcastMeta( + s: SubqueryBroadcastExec, + conf: RapidsConf, + p: Option[RapidsMeta[_, _, _]], + r: DataFromReplacementRule) extends + GpuSubqueryBroadcastMetaBase(s, conf, p, r) { + override def convertToGpu(): GpuExec = { + GpuSubqueryBroadcastExec(s.name, s.indices, s.buildKeys, broadcastBuilder())( + getBroadcastModeKeyExprs) + } +} diff --git a/tests/src/test/spark321/scala/com/nvidia/spark/rapids/DynamicPruningSuite.scala b/tests/src/test/spark321/scala/com/nvidia/spark/rapids/DynamicPruningSuite.scala index 2d4156d1b3b..722e5bb215b 100644 --- a/tests/src/test/spark321/scala/com/nvidia/spark/rapids/DynamicPruningSuite.scala +++ b/tests/src/test/spark321/scala/com/nvidia/spark/rapids/DynamicPruningSuite.scala @@ -66,7 +66,7 @@ class DynamicPruningSuite // NOTE: We remove the AdaptiveSparkPlanExec since we can't re-run the new plan // under AQE because that fundamentally requires some rewrite and stage // ordering which we can't do for this test. - case GpuSubqueryBroadcastExec(name, index, buildKeys, child) => + case GpuSubqueryBroadcastExec(name, Seq(index), buildKeys, child) => val newChild = child match { case a @ AdaptiveSparkPlanExec(_, _, _, _, _) => (new GpuTransitionOverrides()).apply(ColumnarToRowExec(a.executedPlan)) From e23bf381a74237f882d94063b790c3e045e23ed8 Mon Sep 17 00:00:00 2001 From: Gera Shegalov Date: Thu, 30 May 2024 08:18:37 -0700 Subject: [PATCH 12/68] Unarchive Spark test jar for spark.read(ability) (#10946) Closes #10875 Contributes to #10773 Unjar, cache, and share the test jar content among all test suites from the same jar Test: ```bash mvn package -Dbuildver=330 -pl tests -am -Dsuffixes='.*\.RapidsJsonSuite' ``` Signed-off-by: Gera Shegalov --- .../sql/rapids/suites/RapidsJsonSuite.scala | 4 -- .../utils/RapidsSQLTestsBaseTrait.scala | 46 ++++++++++++++++++- .../sql/rapids/utils/RapidsTestSettings.scala | 9 ---- 3 files changed, 44 insertions(+), 15 deletions(-) diff --git a/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/suites/RapidsJsonSuite.scala b/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/suites/RapidsJsonSuite.scala index 3e9f685dfdc..ef9ae630dfd 100644 --- a/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/suites/RapidsJsonSuite.scala +++ b/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/suites/RapidsJsonSuite.scala @@ -31,10 +31,6 @@ import org.apache.spark.sql.util.CaseInsensitiveStringMap class RapidsJsonSuite extends JsonSuite with RapidsSQLTestsBaseTrait with RapidsJsonConfTrait { - /** Returns full path to the given file in the resource folder */ - override protected def testFile(fileName: String): String = { - getWorkspaceFilePath("sql", "core", "src", "test", "resources").toString + "/" + fileName - } } class RapidsJsonV1Suite extends RapidsJsonSuite with RapidsSQLTestsBaseTrait { diff --git a/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/utils/RapidsSQLTestsBaseTrait.scala b/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/utils/RapidsSQLTestsBaseTrait.scala index 6db9e8b71a6..f8b9d21d169 100644 --- a/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/utils/RapidsSQLTestsBaseTrait.scala +++ b/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/utils/RapidsSQLTestsBaseTrait.scala @@ -21,27 +21,36 @@ package org.apache.spark.sql.rapids.utils import java.util.{Locale, TimeZone} +import org.apache.hadoop.fs.FileUtil import org.scalactic.source.Position import org.scalatest.Tag import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging import org.apache.spark.internal.config.Tests.IS_TESTING import org.apache.spark.sql.DataFrame import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, ShuffleQueryStageExec} +import org.apache.spark.sql.rapids.execution.TrampolineUtil import org.apache.spark.sql.rapids.utils.RapidsTestConstants.RAPIDS_TEST import org.apache.spark.sql.test.SharedSparkSession /** Basic trait for Rapids SQL test cases. */ trait RapidsSQLTestsBaseTrait extends SharedSparkSession with RapidsTestsBaseTrait { - protected override def afterAll(): Unit = { // SparkFunSuite will set this to true, and forget to reset to false System.clearProperty(IS_TESTING.key) super.afterAll() } + override protected def testFile(fileName: String): String = { + import RapidsSQLTestsBaseTrait.sparkTestResourcesDir + + java.nio.file.Paths.get(sparkTestResourcesDir(getClass).toString, fileName) + .toString + } + protected def testRapids(testName: String, testTag: Tag*)(testFun: => Any)(implicit pos: Position): Unit = { test(RAPIDS_TEST + testName, testTag: _*)(testFun) @@ -107,7 +116,40 @@ trait RapidsSQLTestsBaseTrait extends SharedSparkSession with RapidsTestsBaseTra } } -object RapidsSQLTestsBaseTrait { +object RapidsSQLTestsBaseTrait extends Logging { + private val resourceMap = scala.collection.mutable.Map.empty[String, java.nio.file.Path] + private val testJarUrlRegex = raw"jar:file:(/.*-tests.jar)!.*".r + TrampolineUtil.addShutdownHook(10000, () => { + resourceMap.valuesIterator.foreach { dirPath => + logWarning(s"Deleting expanded test jar dir $dirPath") + FileUtil.fullyDelete(dirPath.toFile) + } + }) + + private def expandJar(jarPath: String): java.nio.file.Path = { + val jarFile = new java.io.File(jarPath) + val destDir = java.nio.file.Files.createTempDirectory(jarFile.getName + ".expanded") + logWarning(s"Registering $destDir for deletion on exit") + FileUtil.unZip(jarFile, destDir.toFile) + destDir + } + + def sparkTestResourcesDir(testClass: Class[_]): java.nio.file.Path = { + var sparkTestClass = testClass + while (sparkTestClass.getName.contains("rapids")) { + sparkTestClass = sparkTestClass.getSuperclass + } + val sparkTestClassResource = "/" + sparkTestClass.getName.replace(".", "/") + ".class" + val resourceURL = sparkTestClass.getResource(sparkTestClassResource).toString + val resourceJar = resourceURL match { + case testJarUrlRegex(testJarPath) => testJarPath + case _ => sys.error(s"Could not extract tests jar path from $resourceURL") + } + this.synchronized { + resourceMap.getOrElseUpdate(resourceJar, expandJar(resourceJar)) + } + } + def nativeSparkConf(origin: SparkConf, warehouse: String): SparkConf = { // Timezone is fixed to UTC to allow timestamps to work by default TimeZone.setDefault(TimeZone.getTimeZone("UTC")) diff --git a/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/utils/RapidsTestSettings.scala b/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/utils/RapidsTestSettings.scala index 3ccd478d368..8fc86cc6dce 100644 --- a/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/utils/RapidsTestSettings.scala +++ b/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/utils/RapidsTestSettings.scala @@ -66,15 +66,6 @@ class RapidsTestSettings extends BackendTestSettings { enableSuite[RapidsJsonSuite] .exclude("Casting long as timestamp", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10773")) .exclude("Write timestamps correctly with timestampFormat option and timeZone option", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10773")) - .exclude("SPARK-23723: json in UTF-16 with BOM", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10773")) - .exclude("SPARK-23723: multi-line json in UTF-32BE with BOM", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10773")) - .exclude("SPARK-23723: Use user's encoding in reading of multi-line json in UTF-16LE", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10773")) - .exclude("SPARK-23723: Unsupported encoding name", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10773")) - .exclude("SPARK-23723: checking that the encoding option is case agnostic", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10773")) - .exclude("SPARK-23723: specified encoding is not matched to actual encoding", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10773")) - .exclude("SPARK-23724: lineSep should be set if encoding if different from UTF-8", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10773")) - .exclude("SPARK-31716: inferring should handle malformed input", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10773")) - .exclude("SPARK-24190: restrictions for JSONOptions in read", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10773")) .exclude("exception mode for parsing date/timestamp string", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10773")) enableSuite[RapidsMathFunctionsSuite] enableSuite[RapidsRegexpExpressionsSuite] From a7cdaa9eff17293cae9cee04d16b07d924fd40eb Mon Sep 17 00:00:00 2001 From: Raza Jafri Date: Thu, 30 May 2024 08:49:53 -0700 Subject: [PATCH 13/68] Added Shim for BatchScanExec to Support Spark 4.0 [databricks] (#10944) * Added shim for BatchScanExec to support Spark 4.0 Signed-off-by: Raza Jafri * fixed the failing shim --------- Signed-off-by: Raza Jafri --- .../spark/rapids/shims/GpuBatchScanExec.scala | 1 - .../rapids/shims/BatchScanExecMeta.scala | 52 +--- .../rapids/shims/BatchScanExecMetaBase.scala | 81 ++++++ .../rapids/shims/BatchScanExecMeta.scala | 38 +++ .../spark/rapids/shims/GpuBatchScanExec.scala | 269 ++++++++++++++++++ 5 files changed, 389 insertions(+), 52 deletions(-) create mode 100644 sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/BatchScanExecMetaBase.scala create mode 100644 sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/BatchScanExecMeta.scala create mode 100644 sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala diff --git a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala index 39f42d8b833..5fb252524fd 100644 --- a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala +++ b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala @@ -22,7 +22,6 @@ {"spark": "343"} {"spark": "350"} {"spark": "351"} -{"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/BatchScanExecMeta.scala b/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/BatchScanExecMeta.scala index 4bbc4644241..4b29de25bf0 100644 --- a/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/BatchScanExecMeta.scala +++ b/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/BatchScanExecMeta.scala @@ -17,68 +17,18 @@ /*** spark-rapids-shim-json-lines {"spark": "350"} {"spark": "351"} -{"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims import com.nvidia.spark.rapids._ -import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.datasources.v2.BatchScanExec class BatchScanExecMeta(p: BatchScanExec, conf: RapidsConf, parent: Option[RapidsMeta[_, _, _]], rule: DataFromReplacementRule) - extends SparkPlanMeta[BatchScanExec](p, conf, parent, rule) { - // Replaces SubqueryBroadcastExec inside dynamic pruning filters with GPU counterpart - // if possible. Instead regarding filters as childExprs of current Meta, we create - // a new meta for SubqueryBroadcastExec. The reason is that the GPU replacement of - // BatchScanExec is independent from the replacement of the runtime filters. It is - // possible that the BatchScanExec is on the CPU, while the dynamic runtime filters - // are on the GPU. And vice versa. - private lazy val runtimeFilters = { - val convertBroadcast = (bc: SubqueryBroadcastExec) => { - val meta = GpuOverrides.wrapAndTagPlan(bc, conf) - meta.tagForExplain() - meta.convertIfNeeded().asInstanceOf[BaseSubqueryExec] - } - wrapped.runtimeFilters.map { filter => - filter.transformDown { - case dpe @ DynamicPruningExpression(inSub: InSubqueryExec) => - inSub.plan match { - case bc: SubqueryBroadcastExec => - dpe.copy(inSub.copy(plan = convertBroadcast(bc))) - case reuse @ ReusedSubqueryExec(bc: SubqueryBroadcastExec) => - dpe.copy(inSub.copy(plan = reuse.copy(convertBroadcast(bc)))) - case _ => - dpe - } - } - } - } - - override val childExprs: Seq[BaseExprMeta[_]] = { - // We want to leave the runtime filters as CPU expressions - p.output.map(GpuOverrides.wrapExpr(_, conf, Some(this))) - } - - override val childScans: scala.Seq[ScanMeta[_]] = - Seq(GpuOverrides.wrapScan(p.scan, conf, Some(this))) - - override def tagPlanForGpu(): Unit = { - if (!p.runtimeFilters.isEmpty && !childScans.head.supportsRuntimeFilters) { - willNotWorkOnGpu("runtime filtering (DPP) is not supported for this scan") - } - } - - override def convertToCpu(): SparkPlan = { - val cpu = wrapped.copy(runtimeFilters = runtimeFilters) - cpu.copyTagsFrom(wrapped) - cpu - } - + extends BatchScanExecMetaBase(p, conf, parent, rule) { override def convertToGpu(): GpuExec = { val spj = p.spjParams GpuBatchScanExec(p.output, childScans.head.convertToGpu(), runtimeFilters, diff --git a/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/BatchScanExecMetaBase.scala b/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/BatchScanExecMetaBase.scala new file mode 100644 index 00000000000..914702a289c --- /dev/null +++ b/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/BatchScanExecMetaBase.scala @@ -0,0 +1,81 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed 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. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "350"} +{"spark": "351"} +{"spark": "400"} +spark-rapids-shim-json-lines ***/ +package com.nvidia.spark.rapids.shims + +import com.nvidia.spark.rapids._ + +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.datasources.v2.BatchScanExec + +abstract class BatchScanExecMetaBase(p: BatchScanExec, + conf: RapidsConf, + parent: Option[RapidsMeta[_, _, _]], + rule: DataFromReplacementRule) + extends SparkPlanMeta[BatchScanExec](p, conf, parent, rule) { + // Replaces SubqueryBroadcastExec inside dynamic pruning filters with GPU counterpart + // if possible. Instead regarding filters as childExprs of current Meta, we create + // a new meta for SubqueryBroadcastExec. The reason is that the GPU replacement of + // BatchScanExec is independent from the replacement of the runtime filters. It is + // possible that the BatchScanExec is on the CPU, while the dynamic runtime filters + // are on the GPU. And vice versa. + protected lazy val runtimeFilters = { + val convertBroadcast = (bc: SubqueryBroadcastExec) => { + val meta = GpuOverrides.wrapAndTagPlan(bc, conf) + meta.tagForExplain() + meta.convertIfNeeded().asInstanceOf[BaseSubqueryExec] + } + wrapped.runtimeFilters.map { filter => + filter.transformDown { + case dpe @ DynamicPruningExpression(inSub: InSubqueryExec) => + inSub.plan match { + case bc: SubqueryBroadcastExec => + dpe.copy(inSub.copy(plan = convertBroadcast(bc))) + case reuse @ ReusedSubqueryExec(bc: SubqueryBroadcastExec) => + dpe.copy(inSub.copy(plan = reuse.copy(convertBroadcast(bc)))) + case _ => + dpe + } + } + } + } + + override val childExprs: Seq[BaseExprMeta[_]] = { + // We want to leave the runtime filters as CPU expressions + p.output.map(GpuOverrides.wrapExpr(_, conf, Some(this))) + } + + override val childScans: scala.Seq[ScanMeta[_]] = + Seq(GpuOverrides.wrapScan(p.scan, conf, Some(this))) + + override def tagPlanForGpu(): Unit = { + if (!p.runtimeFilters.isEmpty && !childScans.head.supportsRuntimeFilters) { + willNotWorkOnGpu("runtime filtering (DPP) is not supported for this scan") + } + } + + override def convertToCpu(): SparkPlan = { + val cpu = wrapped.copy(runtimeFilters = runtimeFilters) + cpu.copyTagsFrom(wrapped) + cpu + } +} diff --git a/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/BatchScanExecMeta.scala b/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/BatchScanExecMeta.scala new file mode 100644 index 00000000000..e6c26eb65b8 --- /dev/null +++ b/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/BatchScanExecMeta.scala @@ -0,0 +1,38 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed 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. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "400"} +spark-rapids-shim-json-lines ***/ +package com.nvidia.spark.rapids.shims + +import com.nvidia.spark.rapids._ + +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.datasources.v2.BatchScanExec + +class BatchScanExecMeta(p: BatchScanExec, + conf: RapidsConf, + parent: Option[RapidsMeta[_, _, _]], + rule: DataFromReplacementRule) + extends BatchScanExecMetaBase(p, conf, parent, rule) { + override def convertToGpu(): GpuExec = { + val spj = p.spjParams + GpuBatchScanExec(p.output, childScans.head.convertToGpu(), runtimeFilters, + p.ordering, p.table, spj) + } +} diff --git a/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala b/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala new file mode 100644 index 00000000000..3c2b649339b --- /dev/null +++ b/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala @@ -0,0 +1,269 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed 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. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "400"} +spark-rapids-shim-json-lines ***/ +package com.nvidia.spark.rapids.shims + +import com.google.common.base.Objects +import com.nvidia.spark.rapids.GpuScan + +import org.apache.spark.SparkException +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, DynamicPruningExpression, Expression, Literal, RowOrdering, SortOrder} +import org.apache.spark.sql.catalyst.plans.QueryPlan +import org.apache.spark.sql.catalyst.plans.physical.{KeyGroupedPartitioning, KeyGroupedShuffleSpec, Partitioning, SinglePartition} +import org.apache.spark.sql.catalyst.util.{truncatedString, InternalRowComparableWrapper} +import org.apache.spark.sql.connector.catalog.Table +import org.apache.spark.sql.connector.read._ +import org.apache.spark.sql.execution.datasources.rapids.DataSourceStrategyUtils +import org.apache.spark.sql.execution.datasources.v2.{DataSourceRDD, StoragePartitionJoinParams} +import org.apache.spark.sql.internal.SQLConf + +case class GpuBatchScanExec( + output: Seq[AttributeReference], + @transient scan: GpuScan, + runtimeFilters: Seq[Expression] = Seq.empty, + ordering: Option[Seq[SortOrder]] = None, + @transient table: Table, + spjParams: StoragePartitionJoinParams = StoragePartitionJoinParams() + ) extends GpuBatchScanExecBase(scan, runtimeFilters) { + + @transient lazy val batch: Batch = if (scan == null) null else scan.toBatch + // TODO: unify the equal/hashCode implementation for all data source v2 query plans. + override def equals(other: Any): Boolean = other match { + case other: GpuBatchScanExec => + this.batch != null && this.batch == other.batch && + this.runtimeFilters == other.runtimeFilters && + this.spjParams == other.spjParams + case _ => + false + } + + override def hashCode(): Int = Objects.hashCode(batch, runtimeFilters) + + @transient override lazy val inputPartitions: Seq[InputPartition] = + batch.planInputPartitions() + + @transient override protected lazy val filteredPartitions: Seq[Seq[InputPartition]] = { + val dataSourceFilters = runtimeFilters.flatMap { + case DynamicPruningExpression(e) => DataSourceStrategyUtils.translateRuntimeFilter(e) + case _ => None + } + + if (dataSourceFilters.nonEmpty) { + val originalPartitioning = outputPartitioning + + // the cast is safe as runtime filters are only assigned if the scan can be filtered + val filterableScan = scan.asInstanceOf[SupportsRuntimeV2Filtering] + filterableScan.filter(dataSourceFilters.toArray) + + // call toBatch again to get filtered partitions + val newPartitions = scan.toBatch.planInputPartitions() + + originalPartitioning match { + case p: KeyGroupedPartitioning => + if (newPartitions.exists(!_.isInstanceOf[HasPartitionKey])) { + throw new SparkException("Data source must have preserved the original partitioning " + + "during runtime filtering: not all partitions implement HasPartitionKey after " + + "filtering") + } + + val newPartitionValues = newPartitions.map(partition => + InternalRowComparableWrapper(partition.asInstanceOf[HasPartitionKey], p.expressions)) + .toSet + val oldPartitionValues = p.partitionValues + .map(partition => InternalRowComparableWrapper(partition, p.expressions)).toSet + // We require the new number of partition values to be equal or less than the old number + // of partition values here. In the case of less than, empty partitions will be added for + // those missing values that are not present in the new input partitions. + if (oldPartitionValues.size < newPartitionValues.size) { + throw new SparkException("During runtime filtering, data source must either report " + + "the same number of partition values, or a subset of partition values from the " + + s"original. Before: ${oldPartitionValues.size} partition values. " + + s"After: ${newPartitionValues.size} partition values") + } + + if (!newPartitionValues.forall(oldPartitionValues.contains)) { + throw new SparkException("During runtime filtering, data source must not report new " + + "partition values that are not present in the original partitioning.") + } + groupPartitions(newPartitions) + .map(_.groupedParts.map(_.parts)).getOrElse(Seq.empty) + + case _ => + // no validation is needed as the data source did not report any specific partitioning + newPartitions.map(Seq(_)) + } + + } else { + partitions + } + } + + override def outputPartitioning: Partitioning = { + super.outputPartitioning match { + case k: KeyGroupedPartitioning if spjParams.commonPartitionValues.isDefined => + // We allow duplicated partition values if + // `spark.sql.sources.v2.bucketing.partiallyClusteredDistribution.enabled` is true + val newPartValues = spjParams.commonPartitionValues.get.flatMap { + case (partValue, numSplits) => Seq.fill(numSplits)(partValue) + } + val expressions = spjParams.joinKeyPositions match { + case Some(projectionPositions) => projectionPositions.map(i => k.expressions(i)) + case _ => k.expressions + } + k.copy(expressions = expressions, numPartitions = newPartValues.length, + partitionValues = newPartValues) + case p => p + } + } + + override lazy val readerFactory: PartitionReaderFactory = batch.createReaderFactory() + + override lazy val inputRDD: RDD[InternalRow] = { + val rdd = if (filteredPartitions.isEmpty && outputPartitioning == SinglePartition) { + // return an empty RDD with 1 partition if dynamic filtering removed the only split + sparkContext.parallelize(Array.empty[InternalRow], 1) + } else { + val finalPartitions = outputPartitioning match { + case p: KeyGroupedPartitioning => + assert(spjParams.keyGroupedPartitioning.isDefined) + val expressions = spjParams.keyGroupedPartitioning.get + + // Re-group the input partitions if we are projecting on a subset of join keys + val (groupedPartitions, partExpressions) = spjParams.joinKeyPositions match { + case Some(projectPositions) => + val projectedExpressions = projectPositions.map(i => expressions(i)) + val parts = filteredPartitions.flatten.groupBy(part => { + val row = part.asInstanceOf[HasPartitionKey].partitionKey() + val projectedRow = KeyGroupedPartitioning.project( + expressions, projectPositions, row) + InternalRowComparableWrapper(projectedRow, projectedExpressions) + }).map { case (wrapper, splits) => (wrapper.row, splits) }.toSeq + (parts, projectedExpressions) + case _ => + val groupedParts = filteredPartitions.map(splits => { + assert(splits.nonEmpty && splits.head.isInstanceOf[HasPartitionKey]) + (splits.head.asInstanceOf[HasPartitionKey].partitionKey(), splits) + }) + (groupedParts, expressions) + } + + // Also re-group the partitions if we are reducing compatible partition expressions + val finalGroupedPartitions = spjParams.reducers match { + case Some(reducers) => + val result = groupedPartitions.groupBy { case (row, _) => + KeyGroupedShuffleSpec.reducePartitionValue(row, partExpressions, reducers) + }.map { case (wrapper, splits) => (wrapper.row, splits.flatMap(_._2)) }.toSeq + val rowOrdering = RowOrdering.createNaturalAscendingOrdering( + partExpressions.map(_.dataType)) + result.sorted(rowOrdering.on((t: (InternalRow, _)) => t._1)) + case _ => groupedPartitions + } + + // When partially clustered, the input partitions are not grouped by partition + // values. Here we'll need to check `commonPartitionValues` and decide how to group + // and replicate splits within a partition. + if (spjParams.commonPartitionValues.isDefined && spjParams.applyPartialClustering) { + // A mapping from the common partition values to how many splits the partition + // should contain. + val commonPartValuesMap = spjParams.commonPartitionValues + .get + .map(t => (InternalRowComparableWrapper(t._1, partExpressions), t._2)) + .toMap + val nestGroupedPartitions = finalGroupedPartitions.map { case (partValue, splits) => + // `commonPartValuesMap` should contain the part value since it's the super set. + val numSplits = commonPartValuesMap + .get(InternalRowComparableWrapper(partValue, partExpressions)) + assert(numSplits.isDefined, s"Partition value $partValue does not exist in " + + "common partition values from Spark plan") + + val newSplits = if (spjParams.replicatePartitions) { + // We need to also replicate partitions according to the other side of join + Seq.fill(numSplits.get)(splits) + } else { + // Not grouping by partition values: this could be the side with partially + // clustered distribution. Because of dynamic filtering, we'll need to check if + // the final number of splits of a partition is smaller than the original + // number, and fill with empty splits if so. This is necessary so that both + // sides of a join will have the same number of partitions & splits. + splits.map(Seq(_)).padTo(numSplits.get, Seq.empty) + } + (InternalRowComparableWrapper(partValue, partExpressions), newSplits) + } + + // Now fill missing partition keys with empty partitions + val partitionMapping = nestGroupedPartitions.toMap + spjParams.commonPartitionValues.get.flatMap { + case (partValue, numSplits) => + // Use empty partition for those partition values that are not present. + partitionMapping.getOrElse( + InternalRowComparableWrapper(partValue, partExpressions), + Seq.fill(numSplits)(Seq.empty)) + } + } else { + // either `commonPartitionValues` is not defined, or it is defined but + // `applyPartialClustering` is false. + val partitionMapping = finalGroupedPartitions.map { case (partValue, splits) => + InternalRowComparableWrapper(partValue, partExpressions) -> splits + }.toMap + + // In case `commonPartitionValues` is not defined (e.g., SPJ is not used), there + // could exist duplicated partition values, as partition grouping is not done + // at the beginning and postponed to this method. It is important to use unique + // partition values here so that grouped partitions won't get duplicated. + p.uniquePartitionValues.map { partValue => + // Use empty partition for those partition values that are not present + partitionMapping.getOrElse( + InternalRowComparableWrapper(partValue, partExpressions), Seq.empty) + } + } + + case _ => filteredPartitions + } + + new DataSourceRDD( + sparkContext, finalPartitions, readerFactory, supportsColumnar, customMetrics) + } + postDriverMetrics() + rdd + } + + override def keyGroupedPartitioning: Option[Seq[Expression]] = + spjParams.keyGroupedPartitioning + + override def doCanonicalize(): GpuBatchScanExec = { + this.copy( + output = output.map(QueryPlan.normalizeExpressions(_, output)), + runtimeFilters = QueryPlan.normalizePredicates( + runtimeFilters.filterNot(_ == DynamicPruningExpression(Literal.TrueLiteral)), + output)) + } + + override def simpleString(maxFields: Int): String = { + val truncatedOutputString = truncatedString(output, "[", ", ", "]", maxFields) + val runtimeFiltersString = s"RuntimeFilters: ${runtimeFilters.mkString("[", ",", "]")}" + val result = s"$nodeName$truncatedOutputString ${scan.description()} $runtimeFiltersString" + redact(result) + } + + override def nodeName: String = { + s"GpuBatchScan ${table.name()}".trim + } +} From 499a45bd2bea4b256eb9b0dbae4720cbedd8a295 Mon Sep 17 00:00:00 2001 From: MithunR Date: Thu, 30 May 2024 14:13:52 -0700 Subject: [PATCH 14/68] [Spark 4.0] Account for `CommandUtils.uncacheTableOrView` signature change. (#10863) * Account for `CommandUtils.uncacheTableOrView` signature change. Fixes #10710. This commit accounts for the changes in the signature of `CommandUtils.uncacheTableOrView` in Apache Spark 4.0. (See [SPARK-47191](https://github.com/apache/spark/pull/45289).) Signed-off-by: MithunR * Removed unnecessary base class. --------- Signed-off-by: MithunR --- .../hive/rapids/shims/CommandUtilsShim.scala | 57 +++++++++++++++++++ .../rapids/shims/GpuInsertIntoHiveTable.scala | 2 +- .../rapids/shims/GpuInsertIntoHiveTable.scala | 2 +- .../hive/rapids/shims/CommandUtilsShim.scala | 33 +++++++++++ 4 files changed, 92 insertions(+), 2 deletions(-) create mode 100644 sql-plugin/src/main/spark311/scala/org/apache/spark/sql/hive/rapids/shims/CommandUtilsShim.scala create mode 100644 sql-plugin/src/main/spark400/scala/org/apache/spark/sql/hive/rapids/shims/CommandUtilsShim.scala diff --git a/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/hive/rapids/shims/CommandUtilsShim.scala b/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/hive/rapids/shims/CommandUtilsShim.scala new file mode 100644 index 00000000000..1e1ac57aa60 --- /dev/null +++ b/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/hive/rapids/shims/CommandUtilsShim.scala @@ -0,0 +1,57 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed 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. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "311"} +{"spark": "312"} +{"spark": "313"} +{"spark": "320"} +{"spark": "321"} +{"spark": "321cdh"} +{"spark": "322"} +{"spark": "323"} +{"spark": "324"} +{"spark": "330"} +{"spark": "330cdh"} +{"spark": "330db"} +{"spark": "331"} +{"spark": "332"} +{"spark": "332cdh"} +{"spark": "332db"} +{"spark": "333"} +{"spark": "334"} +{"spark": "340"} +{"spark": "341"} +{"spark": "341db"} +{"spark": "342"} +{"spark": "343"} +{"spark": "350"} +{"spark": "351"} +spark-rapids-shim-json-lines ***/ +package org.apache.spark.sql.hive.rapids.shims + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.execution.command.CommandUtils + +object CommandUtilsShim { + + // Shim for CommandUtils.uncacheTableOrView, whose signature changed in Apache Spark 4.0. + def uncacheTableOrView(sparkSession: SparkSession, tableId: TableIdentifier): Unit = { + CommandUtils.uncacheTableOrView(sparkSession, tableId.quotedString) + } + +} \ No newline at end of file diff --git a/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/hive/rapids/shims/GpuInsertIntoHiveTable.scala b/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/hive/rapids/shims/GpuInsertIntoHiveTable.scala index 92fb72801c8..2af89bf1170 100644 --- a/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/hive/rapids/shims/GpuInsertIntoHiveTable.scala +++ b/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/hive/rapids/shims/GpuInsertIntoHiveTable.scala @@ -137,7 +137,7 @@ case class GpuInsertIntoHiveTable( } // un-cache this table. - CommandUtils.uncacheTableOrView(sparkSession, table.identifier.quotedString) + CommandUtilsShim.uncacheTableOrView(sparkSession, table.identifier) sparkSession.sessionState.catalog.refreshTable(table.identifier) CommandUtils.updateTableStats(sparkSession, table) diff --git a/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuInsertIntoHiveTable.scala b/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuInsertIntoHiveTable.scala index 9105ab50e1e..28b8033389a 100644 --- a/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuInsertIntoHiveTable.scala +++ b/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuInsertIntoHiveTable.scala @@ -127,7 +127,7 @@ case class GpuInsertIntoHiveTable( } // un-cache this table. - CommandUtils.uncacheTableOrView(sparkSession, table.identifier.quotedString) + CommandUtilsShim.uncacheTableOrView(sparkSession, table.identifier) sparkSession.sessionState.catalog.refreshTable(table.identifier) CommandUtils.updateTableStats(sparkSession, table) diff --git a/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/hive/rapids/shims/CommandUtilsShim.scala b/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/hive/rapids/shims/CommandUtilsShim.scala new file mode 100644 index 00000000000..f5858e4cfd6 --- /dev/null +++ b/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/hive/rapids/shims/CommandUtilsShim.scala @@ -0,0 +1,33 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed 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. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "400"} +spark-rapids-shim-json-lines ***/ +package org.apache.spark.sql.hive.rapids.shims + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.execution.command.CommandUtils + +object CommandUtilsShim { + + // Shim for CommandUtils.uncacheTableOrView, whose signature changed in Apache Spark 4.0. + def uncacheTableOrView(sparkSession: SparkSession, tableId: TableIdentifier): Unit = { + CommandUtils.uncacheTableOrView(sparkSession, tableId) + } + +} \ No newline at end of file From 4024ef66d269864d5d9af5e9d7c9015f6982629c Mon Sep 17 00:00:00 2001 From: Liangcai Li Date: Fri, 31 May 2024 09:47:12 +0800 Subject: [PATCH 15/68] GpuInsertIntoHiveTable supports parquet format (#10912) This is a new feature adding the parquet support for GpuInsertIntoHiveTable, who only supports text write now. And this feature is tested by the new added tests in this PR. --------- Signed-off-by: Firestarman Co-authored-by: Jason Lowe --- .../main/python/hive_parquet_write_test.py | 176 ++++++++++++++++++ .../spark/rapids/GpuParquetFileFormat.scala | 4 +- ...leFormat.scala => GpuHiveFileFormat.scala} | 149 ++++++++++++--- .../rapids/shims/GpuInsertIntoHiveTable.scala | 17 +- .../rapids/shims/GpuInsertIntoHiveTable.scala | 17 +- 5 files changed, 322 insertions(+), 41 deletions(-) create mode 100644 integration_tests/src/main/python/hive_parquet_write_test.py rename sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/{GpuHiveTextFileFormat.scala => GpuHiveFileFormat.scala} (54%) diff --git a/integration_tests/src/main/python/hive_parquet_write_test.py b/integration_tests/src/main/python/hive_parquet_write_test.py new file mode 100644 index 00000000000..e74a99f43c7 --- /dev/null +++ b/integration_tests/src/main/python/hive_parquet_write_test.py @@ -0,0 +1,176 @@ +# Copyright (c) 2024, NVIDIA CORPORATION. +# +# Licensed 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. + +import pytest + +from asserts import assert_gpu_and_cpu_sql_writes_are_equal_collect +from conftest import is_databricks_runtime +from data_gen import * +from hive_write_test import _restricted_timestamp +from marks import allow_non_gpu, ignore_order +from spark_session import with_cpu_session, is_before_spark_320 + +# Disable the meta conversion from Hive write to FrameData write in Spark, to test +# "GpuInsertIntoHiveTable" for Parquet write. +_write_to_hive_conf = {"spark.sql.hive.convertMetastoreParquet": False} + +_hive_basic_gens = [ + byte_gen, short_gen, int_gen, long_gen, float_gen, double_gen, string_gen, boolean_gen, + DateGen(start=date(1590, 1, 1)), _restricted_timestamp(), + DecimalGen(precision=19, scale=1, nullable=True), + DecimalGen(precision=23, scale=5, nullable=True), + DecimalGen(precision=36, scale=3, nullable=True)] + +_hive_basic_struct_gen = StructGen( + [['c'+str(ind), c_gen] for ind, c_gen in enumerate(_hive_basic_gens)]) + +_hive_struct_gens = [ + _hive_basic_struct_gen, + StructGen([['child0', byte_gen], ['child1', _hive_basic_struct_gen]]), + StructGen([['child0', ArrayGen(short_gen)], ['child1', double_gen]])] + +_hive_array_gens = [ArrayGen(sub_gen) for sub_gen in _hive_basic_gens] + [ + ArrayGen(ArrayGen(short_gen, max_length=10), max_length=10), + ArrayGen(ArrayGen(string_gen, max_length=10), max_length=10), + ArrayGen(StructGen([['child0', byte_gen], ['child1', string_gen], ['child2', float_gen]]))] + +_hive_map_gens = [simple_string_to_string_map_gen] + [MapGen(f(nullable=False), f()) for f in [ + BooleanGen, ByteGen, ShortGen, IntegerGen, LongGen, FloatGen, DoubleGen, + lambda nullable=True: _restricted_timestamp(nullable=nullable), + lambda nullable=True: DateGen(start=date(1590, 1, 1), nullable=nullable), + lambda nullable=True: DecimalGen(precision=19, scale=1, nullable=nullable), + lambda nullable=True: DecimalGen(precision=36, scale=5, nullable=nullable)]] + +_hive_write_gens = [_hive_basic_gens, _hive_struct_gens, _hive_array_gens, _hive_map_gens] + +# ProjectExec falls back on databricks due to no GPU version of "MapFromArrays". +fallback_nodes = ['ProjectExec'] if is_databricks_runtime() else [] + + +@allow_non_gpu(*(non_utc_allow + fallback_nodes)) +@ignore_order(local=True) +@pytest.mark.parametrize("is_ctas", [True, False], ids=['CTAS', 'CTTW']) +@pytest.mark.parametrize("gens", _hive_write_gens, ids=idfn) +def test_write_parquet_into_hive_table(spark_tmp_table_factory, is_ctas, gens): + + def gen_table(spark): + gen_list = [('_c' + str(i), gen) for i, gen in enumerate(gens)] + types_sql_str = ','.join('{} {}'.format( + name, gen.data_type.simpleString()) for name, gen in gen_list) + data_table = spark_tmp_table_factory.get() + gen_df(spark, gen_list).createOrReplaceTempView(data_table) + return data_table, types_sql_str + + (input_table, input_schema) = with_cpu_session(gen_table) + + def write_to_hive_sql(spark, output_table): + if is_ctas: + # Create Table As Select + return [ + "CREATE TABLE {} STORED AS PARQUET AS SELECT * FROM {}".format( + output_table, input_table) + ] + else: + # Create Table Then Write + return [ + "CREATE TABLE {} ({}) STORED AS PARQUET".format(output_table, input_schema), + "INSERT OVERWRITE TABLE {} SELECT * FROM {}".format(output_table, input_table) + ] + + assert_gpu_and_cpu_sql_writes_are_equal_collect( + spark_tmp_table_factory, + write_to_hive_sql, + _write_to_hive_conf) + + +@allow_non_gpu(*non_utc_allow) +@ignore_order(local=True) +@pytest.mark.parametrize("is_static", [True, False], ids=['Static_Partition', 'Dynamic_Partition']) +def test_write_parquet_into_partitioned_hive_table(spark_tmp_table_factory, is_static): + # Generate hive table in Parquet format + def gen_table(spark): + # gen_list = [('_c' + str(i), gen) for i, gen in enumerate(gens)] + dates = [date(2024, 2, 28), date(2024, 2, 27), date(2024, 2, 26)] + gen_list = [('a', int_gen), + ('b', long_gen), + ('c', short_gen), + ('d', string_gen), + ('part', SetValuesGen(DateType(), dates))] + data_table = spark_tmp_table_factory.get() + gen_df(spark, gen_list).createOrReplaceTempView(data_table) + return data_table + + input_table = with_cpu_session(gen_table) + + def partitioned_write_to_hive_sql(spark, output_table): + sql_create_part_table = ( + "CREATE TABLE {} (a INT, b LONG, c SHORT, d STRING) " + "PARTITIONED BY (part DATE) STORED AS PARQUET" + ).format(output_table) + if is_static: + return [ + # sql_1: Create partitioned hive table + sql_create_part_table, + # sql_2: Static partition write only to partition 'par2' + "INSERT OVERWRITE TABLE {} PARTITION (part='2024-02-25') " + "SELECT a, b, c, d FROM {}".format(output_table, input_table) + ] + else: + return [ + # sql_1: Create partitioned hive table + sql_create_part_table, + # sql_2: Dynamic partition write + "INSERT OVERWRITE TABLE {} SELECT * FROM {}".format(output_table, input_table) + ] + all_confs = copy_and_update(_write_to_hive_conf, { + "hive.exec.dynamic.partition.mode": "nonstrict"}) + assert_gpu_and_cpu_sql_writes_are_equal_collect( + spark_tmp_table_factory, + partitioned_write_to_hive_sql, + all_confs) + + +zstd_param = pytest.param('ZSTD', + marks=pytest.mark.skipif(is_before_spark_320(), reason="zstd is not supported before 320")) + +@allow_non_gpu(*(non_utc_allow + fallback_nodes)) +@ignore_order(local=True) +@pytest.mark.parametrize("comp_type", ['UNCOMPRESSED', 'SNAPPY', zstd_param]) +def test_write_compressed_parquet_into_hive_table(spark_tmp_table_factory, comp_type): + # Generate hive table in Parquet format + def gen_table(spark): + gens = _hive_basic_gens + _hive_struct_gens + _hive_array_gens + _hive_map_gens + gen_list = [('_c' + str(i), gen) for i, gen in enumerate(gens)] + types_sql_str = ','.join('{} {}'.format( + name, gen.data_type.simpleString()) for name, gen in gen_list) + data_table = spark_tmp_table_factory.get() + gen_df(spark, gen_list).createOrReplaceTempView(data_table) + return data_table, types_sql_str + + input_table, schema_str = with_cpu_session(gen_table) + + def write_to_hive_sql(spark, output_table): + return [ + # Create table with compression type + "CREATE TABLE {} ({}) STORED AS PARQUET " + "TBLPROPERTIES ('parquet.compression'='{}')".format( + output_table, schema_str, comp_type), + # Insert into table + "INSERT OVERWRITE TABLE {} SELECT * FROM {}".format(output_table, input_table) + ] + + assert_gpu_and_cpu_sql_writes_are_equal_collect( + spark_tmp_table_factory, + write_to_hive_sql, + _write_to_hive_conf) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetFileFormat.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetFileFormat.scala index e8ae977b1f6..25105386b3d 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetFileFormat.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetFileFormat.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019-2023, NVIDIA CORPORATION. + * Copyright (c) 2019-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -321,7 +321,7 @@ class GpuParquetWriter( new GpuColumnVector(cv.dataType, deepTransformColumn(cv.getBase, cv.dataType)) .asInstanceOf[org.apache.spark.sql.vectorized.ColumnVector] } - new ColumnarBatch(transformedCols) + new ColumnarBatch(transformedCols, batch.numRows()) } } diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/GpuHiveTextFileFormat.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/GpuHiveFileFormat.scala similarity index 54% rename from sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/GpuHiveTextFileFormat.scala rename to sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/GpuHiveFileFormat.scala index 4595ea87ed3..21437a64481 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/GpuHiveTextFileFormat.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/GpuHiveFileFormat.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023, NVIDIA CORPORATION. + * Copyright (c) 2023-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,8 +17,9 @@ package org.apache.spark.sql.hive.rapids import java.nio.charset.Charset +import java.util.Locale -import ai.rapids.cudf.{CSVWriterOptions, DType, QuoteStyle, Scalar, Table, TableWriter => CudfTableWriter} +import ai.rapids.cudf.{CompressionType, CSVWriterOptions, DType, ParquetWriterOptions, QuoteStyle, Scalar, Table, TableWriter => CudfTableWriter} import com.google.common.base.Charsets import com.nvidia.spark.rapids._ import com.nvidia.spark.rapids.Arm.withResource @@ -27,14 +28,85 @@ import org.apache.hadoop.mapreduce.{Job, TaskAttemptContext} import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.hive.rapids.GpuHiveTextFileUtils._ +import org.apache.spark.sql.execution.datasources.parquet.ParquetOptions import org.apache.spark.sql.hive.rapids.shims.GpuInsertIntoHiveTableMeta -import org.apache.spark.sql.types.{DataType, StringType, StructType} +import org.apache.spark.sql.rapids.execution.TrampolineUtil +import org.apache.spark.sql.types.{DataType, Decimal, DecimalType, StringType, StructType} import org.apache.spark.sql.vectorized.ColumnarBatch -object GpuHiveTextFileFormat extends Logging { +object GpuHiveFileFormat extends Logging { + private val parquetOutputFormatClass = + "org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat" + private val parquetSerdeClass = + "org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe" - private def checkIfEnabled(meta: GpuInsertIntoHiveTableMeta): Unit = { + def tagGpuSupport(meta: GpuInsertIntoHiveTableMeta): Option[ColumnarFileFormat] = { + val insertCmd = meta.wrapped + // Bucketing write + if (insertCmd.table.bucketSpec.isDefined) { + meta.willNotWorkOnGpu("bucketed tables are not supported yet") + } + + // Infer the file format from the serde string, similar as what Spark does in + // RelationConversions for Hive. + val serde = insertCmd.table.storage.serde.getOrElse("").toLowerCase(Locale.ROOT) + val tempFileFormat = if (serde.contains("parquet")) { + // Parquet specific tagging + tagGpuSupportForParquet(meta) + } else { + // Default to text file format + tagGpuSupportForText(meta) + } + + if (meta.canThisBeReplaced) { + Some(tempFileFormat) + } else { + None + } + } + + private def tagGpuSupportForParquet(meta: GpuInsertIntoHiveTableMeta): ColumnarFileFormat = { + val insertCmd = meta.wrapped + val storage = insertCmd.table.storage + + if (storage.outputFormat.getOrElse("") != parquetOutputFormatClass) { + meta.willNotWorkOnGpu(s"unsupported output format found: ${storage.outputFormat}, " + + s"only $parquetOutputFormatClass is currently supported for Parquet") + } + if (storage.serde.getOrElse("") != parquetSerdeClass) { + meta.willNotWorkOnGpu(s"unsupported serde found: ${storage.serde}, " + + s"only $parquetSerdeClass is currently supported for Parquet") + } + + // Decimal type check + val hasIntOrLongBackedDec = insertCmd.query.schema.exists { field => + TrampolineUtil.dataTypeExistsRecursively(field.dataType, { + case dec: DecimalType if dec.precision <= Decimal.MAX_LONG_DIGITS => true + case _ => false + }) + } + if (hasIntOrLongBackedDec) { + meta.willNotWorkOnGpu("decimals that fit in a long are not supported " + + s"for Parquet. Hive always writes decimals as binary arrays but the GPU writes them " + + s"as integral types") + } + + FileFormatChecks.tag(meta, insertCmd.table.schema, ParquetFormatType, WriteFileOp) + + // Compression type + val parquetOptions = new ParquetOptions(insertCmd.table.properties, insertCmd.conf) + val compressionType = + GpuParquetFileFormat.parseCompressionType(parquetOptions.compressionCodecClassName) + .getOrElse { + meta.willNotWorkOnGpu("compression codec " + + s"${parquetOptions.compressionCodecClassName} is not supported for Parquet") + CompressionType.NONE + } + new GpuHiveParquetFileFormat(compressionType) + } + + private def tagGpuSupportForText(meta: GpuInsertIntoHiveTableMeta): ColumnarFileFormat = { + import org.apache.spark.sql.hive.rapids.GpuHiveTextFileUtils._ if (!meta.conf.isHiveDelimitedTextEnabled) { meta.willNotWorkOnGpu("Hive text I/O has been disabled. To enable this, " + s"set ${RapidsConf.ENABLE_HIVE_TEXT} to true") @@ -43,21 +115,16 @@ object GpuHiveTextFileFormat extends Logging { meta.willNotWorkOnGpu("writing Hive delimited text tables has been disabled, " + s"to enable this, set ${RapidsConf.ENABLE_HIVE_TEXT_WRITE} to true") } - } - - def tagGpuSupport(meta: GpuInsertIntoHiveTableMeta) - : Option[ColumnarFileFormat] = { - checkIfEnabled(meta) val insertCommand = meta.wrapped val storage = insertCommand.table.storage if (storage.outputFormat.getOrElse("") != textOutputFormat) { meta.willNotWorkOnGpu(s"unsupported output-format found: ${storage.outputFormat}, " + - s"only $textOutputFormat is currently supported") + s"only $textOutputFormat is currently supported for text") } if (storage.serde.getOrElse("") != lazySimpleSerDe) { meta.willNotWorkOnGpu(s"unsupported serde found: ${storage.serde}, " + - s"only $lazySimpleSerDe is currently supported") + s"only $lazySimpleSerDe is currently supported for text") } val serializationFormat = storage.properties.getOrElse(serializationKey, "1") @@ -86,28 +153,60 @@ object GpuHiveTextFileFormat extends Logging { meta.willNotWorkOnGpu("only UTF-8 is supported as the charset") } - if (insertCommand.table.bucketSpec.isDefined) { - meta.willNotWorkOnGpu("bucketed tables are not supported") - } - - if (insertCommand.conf.getConfString("hive.exec.compress.output", "false").toLowerCase - != "false") { + if (insertCommand.conf.getConfString("hive.exec.compress.output", "false").toBoolean) { meta.willNotWorkOnGpu("compressed output is not supported, " + "set hive.exec.compress.output to false to enable writing Hive text via GPU") } - FileFormatChecks.tag(meta, - insertCommand.table.schema, - HiveDelimitedTextFormatType, - WriteFileOp) + FileFormatChecks.tag(meta, insertCommand.table.schema, HiveDelimitedTextFormatType, + WriteFileOp) - Some(new GpuHiveTextFileFormat()) + new GpuHiveTextFileFormat() } } +class GpuHiveParquetFileFormat(compType: CompressionType) extends ColumnarFileFormat { + + override def prepareWrite(sparkSession: SparkSession, job: Job, + options: Map[String, String], dataSchema: StructType): ColumnarOutputWriterFactory = { + + // Avoid referencing the outer object. + val compressionType = compType + new ColumnarOutputWriterFactory { + override def getFileExtension(context: TaskAttemptContext): String = + compressionType match { + case CompressionType.NONE => ".parquet" + case ct => s".${ct.name().toLowerCase(Locale.ROOT)}.parquet" + } + + override def newInstance(path: String, + dataSchema: StructType, + context: TaskAttemptContext): ColumnarOutputWriter = { + new GpuHiveParquetWriter(path, dataSchema, context, compressionType) + } + } + } +} + +class GpuHiveParquetWriter(override val path: String, dataSchema: StructType, + context: TaskAttemptContext, compType: CompressionType) + extends ColumnarOutputWriter(context, dataSchema, "HiveParquet", true) { + + override protected val tableWriter: CudfTableWriter = { + val optionsBuilder = SchemaUtils + .writerOptionsFromSchema(ParquetWriterOptions.builder(), dataSchema, + writeInt96 = true, // Hive 1.2 write timestamp as INT96 + parquetFieldIdEnabled = false) + .withCompressionType(compType) + Table.writeParquetChunked(optionsBuilder.build(), this) + } + +} + class GpuHiveTextFileFormat extends ColumnarFileFormat with Logging { - override def supportDataType(dataType: DataType): Boolean = isSupportedType(dataType) + override def supportDataType(dataType: DataType): Boolean = + GpuHiveTextFileUtils.isSupportedType(dataType) override def prepareWrite(sparkSession: SparkSession, job: Job, diff --git a/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/hive/rapids/shims/GpuInsertIntoHiveTable.scala b/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/hive/rapids/shims/GpuInsertIntoHiveTable.scala index 2af89bf1170..6d4ca5da7c3 100644 --- a/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/hive/rapids/shims/GpuInsertIntoHiveTable.scala +++ b/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/hive/rapids/shims/GpuInsertIntoHiveTable.scala @@ -57,7 +57,7 @@ import org.apache.spark.sql.hive.HiveShim.{ShimFileSinkDesc => FileSinkDesc} import org.apache.spark.sql.hive.client.HiveClientImpl import org.apache.spark.sql.hive.client.hive._ import org.apache.spark.sql.hive.execution.InsertIntoHiveTable -import org.apache.spark.sql.hive.rapids.{GpuHiveTextFileFormat, GpuSaveAsHiveFile, RapidsHiveErrors} +import org.apache.spark.sql.hive.rapids.{GpuHiveFileFormat, GpuSaveAsHiveFile, RapidsHiveErrors} import org.apache.spark.sql.vectorized.ColumnarBatch final class GpuInsertIntoHiveTableMeta(cmd: InsertIntoHiveTable, @@ -69,16 +69,17 @@ final class GpuInsertIntoHiveTableMeta(cmd: InsertIntoHiveTable, private var fileFormat: Option[ColumnarFileFormat] = None override def tagSelfForGpuInternal(): Unit = { - // Only Hive delimited text writes are currently supported. - // Check whether that is the format currently in play. - fileFormat = GpuHiveTextFileFormat.tagGpuSupport(this) + fileFormat = GpuHiveFileFormat.tagGpuSupport(this) } override def convertToGpu(): GpuDataWritingCommand = { + val format = fileFormat.getOrElse( + throw new IllegalStateException("fileFormat missing, tagSelfForGpu not called?")) + GpuInsertIntoHiveTable( table = wrapped.table, partition = wrapped.partition, - fileFormat = this.fileFormat.get, + fileFormat = format, query = wrapped.query, overwrite = wrapped.overwrite, ifPartitionNotExists = wrapped.ifPartitionNotExists, @@ -326,8 +327,10 @@ case class GpuInsertIntoHiveTable( if (!fs.delete(path, true)) { throw RapidsHiveErrors.cannotRemovePartitionDirError(path) } - // Don't let Hive do overwrite operation since it is slower. - doHiveOverwrite = false + // Don't let Hive do overwrite operation since it is slower. But still give a + // chance to forcely override this for some customized cases when this + // operation is optimized. + doHiveOverwrite = hadoopConf.getBoolean("hive.movetask.enable.dir.move", false) } } } diff --git a/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuInsertIntoHiveTable.scala b/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuInsertIntoHiveTable.scala index 28b8033389a..c8d76f85e5c 100644 --- a/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuInsertIntoHiveTable.scala +++ b/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuInsertIntoHiveTable.scala @@ -47,7 +47,7 @@ import org.apache.spark.sql.execution.command.CommandUtils import org.apache.spark.sql.hive.HiveExternalCatalog import org.apache.spark.sql.hive.client.HiveClientImpl import org.apache.spark.sql.hive.execution.InsertIntoHiveTable -import org.apache.spark.sql.hive.rapids.{GpuHiveTextFileFormat, GpuSaveAsHiveFile, RapidsHiveErrors} +import org.apache.spark.sql.hive.rapids.{GpuHiveFileFormat, GpuSaveAsHiveFile, RapidsHiveErrors} import org.apache.spark.sql.vectorized.ColumnarBatch final class GpuInsertIntoHiveTableMeta(cmd: InsertIntoHiveTable, @@ -59,16 +59,17 @@ final class GpuInsertIntoHiveTableMeta(cmd: InsertIntoHiveTable, private var fileFormat: Option[ColumnarFileFormat] = None override def tagSelfForGpuInternal(): Unit = { - // Only Hive delimited text writes are currently supported. - // Check whether that is the format currently in play. - fileFormat = GpuHiveTextFileFormat.tagGpuSupport(this) + fileFormat = GpuHiveFileFormat.tagGpuSupport(this) } override def convertToGpu(): GpuDataWritingCommand = { + val format = fileFormat.getOrElse( + throw new IllegalStateException("fileFormat missing, tagSelfForGpu not called?")) + GpuInsertIntoHiveTable( table = wrapped.table, partition = wrapped.partition, - fileFormat = this.fileFormat.get, + fileFormat = format, query = wrapped.query, overwrite = wrapped.overwrite, ifPartitionNotExists = wrapped.ifPartitionNotExists, @@ -315,8 +316,10 @@ case class GpuInsertIntoHiveTable( if (!fs.delete(path, true)) { throw RapidsHiveErrors.cannotRemovePartitionDirError(path) } - // Don't let Hive do overwrite operation since it is slower. - doHiveOverwrite = false + // Don't let Hive do overwrite operation since it is slower. But still give a + // chance to forcely override this for some customized cases when this + // operation is optimized. + doHiveOverwrite = hadoopConf.getBoolean("hive.movetask.enable.dir.move", false) } } } From 822ad9bc5a10288e2a1a7af54d9c6f61f9d6e151 Mon Sep 17 00:00:00 2001 From: MithunR Date: Fri, 31 May 2024 00:19:35 -0700 Subject: [PATCH 16/68] [Spark 4.0] Account for `PartitionedFileUtil.splitFiles` signature change. (#10857) * Account for PartitionedFileUtil.splitFiles signature change. Fixes #10299. In Apache Spark 4.0, the signature of `PartitionedFileUtil.splitFiles` was changed to remove unused parameters (apache/spark@eabea643c74). This causes the Spark RAPIDS plugin build to break with Spark 4.0. This commit introduces a shim to account for the signature change. Signed-off-by: MithunR * Common base for PartitionFileUtilsShims. Signed-off-by: MithunR * Reusing existing PartitionedFileUtilsShims. * More refactor, for pre-3.5 compile. * Updated Copyright date. * Fixed style error. * Re-fixed the copyright year. * Added missing import. --------- Signed-off-by: MithunR --- .../shims/PartitionedFileUtilsShim.scala | 22 +-------- .../shims/PartitionedFileUtilsShimBase.scala | 45 +++++++++++++++++++ .../shims/PartitionedFileUtilsShim.scala | 16 ++++++- .../execution/rapids/shims/SplitFiles.scala | 6 +-- .../shims/PartitionedFileUtilsShim.scala | 38 ++++++++++++++++ .../shims/PartitionedFileUtilsShim.scala | 40 +++++++++++++++++ 6 files changed, 141 insertions(+), 26 deletions(-) create mode 100644 sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/PartitionedFileUtilsShimBase.scala create mode 100644 sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/PartitionedFileUtilsShim.scala create mode 100644 sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/PartitionedFileUtilsShim.scala diff --git a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/PartitionedFileUtilsShim.scala b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/PartitionedFileUtilsShim.scala index ca2fa215892..62fe32ae8db 100644 --- a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/PartitionedFileUtilsShim.scala +++ b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/PartitionedFileUtilsShim.scala @@ -19,27 +19,7 @@ {"spark": "341"} {"spark": "342"} {"spark": "343"} -{"spark": "350"} -{"spark": "351"} -{"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims -import org.apache.spark.paths.SparkPath -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.execution.datasources.PartitionedFile - -object PartitionedFileUtilsShim { - // Wrapper for case class constructor so Java code can access - // the default values across Spark versions. - def newPartitionedFile( - partitionValues: InternalRow, - filePath: String, - start: Long, - length: Long): PartitionedFile = PartitionedFile(partitionValues, - SparkPath.fromPathString(filePath), start, length) - - def withNewLocations(pf: PartitionedFile, locations: Seq[String]): PartitionedFile = { - pf.copy(locations = locations.toArray) - } -} +object PartitionedFileUtilsShim extends PartitionedFileUtilsShimBase diff --git a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/PartitionedFileUtilsShimBase.scala b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/PartitionedFileUtilsShimBase.scala new file mode 100644 index 00000000000..a94c76dc083 --- /dev/null +++ b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/PartitionedFileUtilsShimBase.scala @@ -0,0 +1,45 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed 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. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "340"} +{"spark": "341"} +{"spark": "342"} +{"spark": "343"} +{"spark": "350"} +{"spark": "351"} +{"spark": "400"} +spark-rapids-shim-json-lines ***/ +package com.nvidia.spark.rapids.shims + +import org.apache.spark.paths.SparkPath +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.datasources.PartitionedFile + +trait PartitionedFileUtilsShimBase { + + // Wrapper for case class constructor so Java code can access + // the default values across Spark versions. + def newPartitionedFile(partitionValues: InternalRow, + filePath: String, + start: Long, + length: Long): PartitionedFile = PartitionedFile(partitionValues, + SparkPath.fromPathString(filePath), start, length) + + def withNewLocations(pf: PartitionedFile, locations: Seq[String]): PartitionedFile = { + pf.copy(locations = locations.toArray) + } +} diff --git a/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/PartitionedFileUtilsShim.scala b/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/PartitionedFileUtilsShim.scala index 249502f1b49..0f1bdafde7a 100644 --- a/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/PartitionedFileUtilsShim.scala +++ b/sql-plugin/src/main/spark341db/scala/com/nvidia/spark/rapids/shims/PartitionedFileUtilsShim.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023, NVIDIA CORPORATION. + * Copyright (c) 2023-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -20,8 +20,10 @@ spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims import org.apache.spark.paths.SparkPath +import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.execution.datasources.PartitionedFile +import org.apache.spark.sql.execution.PartitionedFileUtil +import org.apache.spark.sql.execution.datasources.{FileStatusWithMetadata, PartitionedFile} object PartitionedFileUtilsShim { // Wrapper for case class constructor so Java code can access @@ -37,4 +39,14 @@ object PartitionedFileUtilsShim { def withNewLocations(pf: PartitionedFile, locations: Seq[String]): PartitionedFile = { pf.copy(locations = locations) } + + // In Spark 4.0, PartitionedFileUtil.splitFiles lost its `sparkSession` parameter. + // This pre-Spark-4.0 shim keeps the `sparkSession` parameter. + def splitFiles(sparkSession: SparkSession, + file: FileStatusWithMetadata, + isSplitable: Boolean, + maxSplitBytes: Long, + partitionValues: InternalRow): Seq[PartitionedFile] = { + PartitionedFileUtil.splitFiles(sparkSession, file, isSplitable, maxSplitBytes, partitionValues) + } } diff --git a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/execution/rapids/shims/SplitFiles.scala b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/execution/rapids/shims/SplitFiles.scala index 3b94d5a5201..1934cb6af9f 100644 --- a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/execution/rapids/shims/SplitFiles.scala +++ b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/execution/rapids/shims/SplitFiles.scala @@ -23,12 +23,12 @@ spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.execution.rapids.shims +import com.nvidia.spark.rapids.shims.PartitionedFileUtilsShim import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.io.compress.{CompressionCodecFactory, SplittableCompressionCodec} import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.execution.PartitionedFileUtil import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, PartitionDirectory, PartitionedFile} trait SplitFiles { @@ -49,7 +49,7 @@ trait SplitFiles { selectedPartitions.flatMap { partition => partition.files.flatMap { f => - PartitionedFileUtil.splitFiles( + PartitionedFileUtilsShim.splitFiles( sparkSession, f, isSplitable = canBeSplit(f.getPath, hadoopConf), @@ -71,7 +71,7 @@ trait SplitFiles { val filePath = file.getPath val isSplitable = relation.fileFormat.isSplitable( relation.sparkSession, relation.options, filePath) - PartitionedFileUtil.splitFiles( + PartitionedFileUtilsShim.splitFiles( sparkSession = relation.sparkSession, file = file, isSplitable = isSplitable, diff --git a/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/PartitionedFileUtilsShim.scala b/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/PartitionedFileUtilsShim.scala new file mode 100644 index 00000000000..71ad5ae1a0f --- /dev/null +++ b/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/PartitionedFileUtilsShim.scala @@ -0,0 +1,38 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed 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. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "350"} +{"spark": "351"} +spark-rapids-shim-json-lines ***/ +package com.nvidia.spark.rapids.shims + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.PartitionedFileUtil +import org.apache.spark.sql.execution.datasources.{FileStatusWithMetadata, PartitionedFile} + +object PartitionedFileUtilsShim extends PartitionedFileUtilsShimBase { + // In Spark 4.0, PartitionedFileUtil.splitFiles lost its `sparkSession` parameter. + // This pre-Spark-4.0 shim keeps the `sparkSession` parameter. + def splitFiles(sparkSession: SparkSession, + file: FileStatusWithMetadata, + isSplitable: Boolean, + maxSplitBytes: Long, + partitionValues: InternalRow): Seq[PartitionedFile] = { + PartitionedFileUtil.splitFiles(sparkSession, file, isSplitable, maxSplitBytes, partitionValues) + } +} diff --git a/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/PartitionedFileUtilsShim.scala b/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/PartitionedFileUtilsShim.scala new file mode 100644 index 00000000000..de8e98962a7 --- /dev/null +++ b/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/PartitionedFileUtilsShim.scala @@ -0,0 +1,40 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed 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. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "400"} +spark-rapids-shim-json-lines ***/ +package com.nvidia.spark.rapids.shims + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.PartitionedFileUtil +import org.apache.spark.sql.execution.datasources.{FileStatusWithMetadata, PartitionedFile} + +object PartitionedFileUtilsShim extends PartitionedFileUtilsShimBase { + + // In Spark 4.0, PartitionedFileUtil.splitFiles lost its `sparkSession` parameter. + // This Spark-4.0+ shim ignores the `sparkSession` parameter. + def splitFiles(sparkSession: SparkSession, + file: FileStatusWithMetadata, + isSplitable: Boolean, + maxSplitBytes: Long, + partitionValues: InternalRow): Seq[PartitionedFile] = { + PartitionedFileUtil.splitFiles(file, isSplitable, maxSplitBytes, partitionValues) + } + +} // object PartitionFileUtilsShim; From 2a86bb5d42495418815b8322e946d02ff62aa145 Mon Sep 17 00:00:00 2001 From: Tim Liu Date: Fri, 31 May 2024 16:11:36 +0800 Subject: [PATCH 17/68] Change dependency version to 24.08.0-SNAPSHOT (#10949) To fix: https://github.com/NVIDIA/spark-rapids/issues/10867 Change rapids private and jni dependency version to 24.08.0-SNAPSHOT Signed-off-by: Tim Liu --- jenkins/databricks/init_cudf_udf.sh | 2 +- jenkins/version-def.sh | 2 +- pom.xml | 5 ++--- scala2.13/pom.xml | 5 ++--- 4 files changed, 6 insertions(+), 8 deletions(-) diff --git a/jenkins/databricks/init_cudf_udf.sh b/jenkins/databricks/init_cudf_udf.sh index d5c440bfbb2..3c3e73ab582 100755 --- a/jenkins/databricks/init_cudf_udf.sh +++ b/jenkins/databricks/init_cudf_udf.sh @@ -20,7 +20,7 @@ set -ex -CUDF_VER=${CUDF_VER:-24.06} # TODO: https://github.com/NVIDIA/spark-rapids/issues/ +CUDF_VER=${CUDF_VER:-24.08} CUDA_VER=${CUDA_VER:-11.8} # Need to explicitly add conda into PATH environment, to activate conda environment. diff --git a/jenkins/version-def.sh b/jenkins/version-def.sh index d3c01e1eba4..dbad6d6fd94 100755 --- a/jenkins/version-def.sh +++ b/jenkins/version-def.sh @@ -27,7 +27,7 @@ done IFS=$PRE_IFS -CUDF_VER=${CUDF_VER:-"24.06.0-SNAPSHOT"} # TODO: https://github.com/NVIDIA/spark-rapids/issues/ +CUDF_VER=${CUDF_VER:-"24.08.0-SNAPSHOT"} CUDA_CLASSIFIER=${CUDA_CLASSIFIER:-"cuda11"} CLASSIFIER=${CLASSIFIER:-"$CUDA_CLASSIFIER"} # default as CUDA_CLASSIFIER for compatibility PROJECT_VER=${PROJECT_VER:-"24.08.0-SNAPSHOT"} diff --git a/pom.xml b/pom.xml index df010a7589e..38f7e8c3812 100644 --- a/pom.xml +++ b/pom.xml @@ -719,9 +719,8 @@ spark${buildver} cuda11 ${cuda.version} - - 24.06.0-SNAPSHOT - 24.06.0-SNAPSHOT + 24.08.0-SNAPSHOT + 24.08.0-SNAPSHOT 2.12 2.8.0 incremental diff --git a/scala2.13/pom.xml b/scala2.13/pom.xml index 711872e8d54..a6d7de172fe 100644 --- a/scala2.13/pom.xml +++ b/scala2.13/pom.xml @@ -719,9 +719,8 @@ spark${buildver} cuda11 ${cuda.version} - - 24.06.0-SNAPSHOT - 24.06.0-SNAPSHOT + 24.08.0-SNAPSHOT + 24.08.0-SNAPSHOT 2.13 2.8.0 incremental From 022fdd1da838e12fb84214a46918eed8cc3f0c28 Mon Sep 17 00:00:00 2001 From: Gera Shegalov Date: Fri, 31 May 2024 11:04:40 -0700 Subject: [PATCH 18/68] Prevent contains-PrefixRange optimization if not preceded by wildcards (#10947) Prevent '^[0-9]{n}' from being processed as `spark_rapids_jni::literal_range_pattern` that currently only supports "contains", not "starts with" Fixes #10928 Also adding missing tailrec annotations to recursive parser methods. Signed-off-by: Gera Shegalov --- .../com/nvidia/spark/rapids/RegexParser.scala | 44 ++++++++++++------- .../spark/sql/rapids/stringFunctions.scala | 2 +- .../RegularExpressionRewriteSuite.scala | 23 +++++++--- 3 files changed, 47 insertions(+), 22 deletions(-) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala index 45d5e07dd73..0f5ada9f7fa 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala @@ -17,7 +17,6 @@ package com.nvidia.spark.rapids import java.sql.SQLException -import scala.collection import scala.collection.mutable.ListBuffer import com.nvidia.spark.rapids.GpuOverrides.regexMetaChars @@ -73,7 +72,7 @@ class RegexParser(pattern: String) { sequence } - def parseReplacementBase(): RegexAST = { + private def parseReplacementBase(): RegexAST = { consume() match { case '\\' => parseBackrefOrEscaped() @@ -782,6 +781,7 @@ class CudfRegexTranspiler(mode: RegexMode) { } } + @scala.annotation.tailrec private def isRepetition(e: RegexAST, checkZeroLength: Boolean): Boolean = { e match { case RegexRepetition(_, _) if !checkZeroLength => true @@ -1648,6 +1648,7 @@ class CudfRegexTranspiler(mode: RegexMode) { } } + @scala.annotation.tailrec private def isEntirely(regex: RegexAST, f: RegexAST => Boolean): Boolean = { regex match { case RegexSequence(parts) if parts.nonEmpty => @@ -1672,6 +1673,7 @@ class CudfRegexTranspiler(mode: RegexMode) { }) } + @scala.annotation.tailrec private def beginsWith(regex: RegexAST, f: RegexAST => Boolean): Boolean = { regex match { case RegexSequence(parts) if parts.nonEmpty => @@ -1687,6 +1689,7 @@ class CudfRegexTranspiler(mode: RegexMode) { } + @scala.annotation.tailrec private def endsWith(regex: RegexAST, f: RegexAST => Boolean): Boolean = { regex match { case RegexSequence(parts) if parts.nonEmpty => @@ -1760,7 +1763,7 @@ sealed case class RegexSequence(parts: ListBuffer[RegexAST]) extends RegexAST { } sealed case class RegexGroup(capture: Boolean, term: RegexAST, - val lookahead: Option[RegexLookahead]) + lookahead: Option[RegexLookahead]) extends RegexAST { def this(capture: Boolean, term: RegexAST) = { this(capture, term, None) @@ -2028,6 +2031,7 @@ object RegexOptimizationType { object RegexRewrite { + @scala.annotation.tailrec private def removeBrackets(astLs: collection.Seq[RegexAST]): collection.Seq[RegexAST] = { astLs match { case collection.Seq(RegexGroup(_, term, None)) => removeBrackets(term.children()) @@ -2044,7 +2048,7 @@ object RegexRewrite { */ private def getPrefixRangePattern(astLs: collection.Seq[RegexAST]): Option[(String, Int, Int, Int)] = { - val haveLiteralPrefix = isliteralString(astLs.dropRight(1)) + val haveLiteralPrefix = isLiteralString(astLs.dropRight(1)) val endsWithRange = astLs.lastOption match { case Some(RegexRepetition( RegexCharacterClass(false, ListBuffer(RegexCharacterRange(a,b))), @@ -2080,9 +2084,9 @@ object RegexRewrite { } } - private def isliteralString(astLs: collection.Seq[RegexAST]): Boolean = { + private def isLiteralString(astLs: collection.Seq[RegexAST]): Boolean = { removeBrackets(astLs).forall { - case RegexChar(ch) if !regexMetaChars.contains(ch) => true + case RegexChar(ch) => !regexMetaChars.contains(ch) case _ => false } } @@ -2120,16 +2124,26 @@ object RegexRewrite { * Matches the given regex ast to a regex optimization type for regex rewrite * optimization. * - * @param ast The Abstract Syntax Tree parsed from a regex pattern. + * @param ast unparsed children of the Abstract Syntax Tree parsed from a regex pattern. * @return The `RegexOptimizationType` for the given pattern. */ - def matchSimplePattern(ast: RegexAST): RegexOptimizationType = { - ast.children() match { - case (RegexChar('^') | RegexEscaped('A')) :: ast - if isliteralString(stripTailingWildcards(ast)) => { - // ^literal.* => startsWith literal - RegexOptimizationType.StartsWith(RegexCharsToString(stripTailingWildcards(ast))) - } + @scala.annotation.tailrec + def matchSimplePattern(ast: Seq[RegexAST]): RegexOptimizationType = { + ast match { + case (RegexChar('^') | RegexEscaped('A')) :: astTail => + val noTrailingWildCards = stripTailingWildcards(astTail) + if (isLiteralString(noTrailingWildCards)) { + // ^literal.* => startsWith literal + RegexOptimizationType.StartsWith(RegexCharsToString(noTrailingWildCards)) + } else { + val noWildCards = stripLeadingWildcards(noTrailingWildCards) + if (noWildCards.length == noTrailingWildCards.length) { + // TODO startsWith with PrefIxRange + RegexOptimizationType.NoOptimization + } else { + matchSimplePattern(astTail) + } + } case astLs => { val noStartsWithAst = stripTailingWildcards(stripLeadingWildcards(astLs)) val prefixRangeInfo = getPrefixRangePattern(noStartsWithAst) @@ -2137,7 +2151,7 @@ object RegexRewrite { val (prefix, length, start, end) = prefixRangeInfo.get // (literal[a-b]{x,y}) => prefix range pattern RegexOptimizationType.PrefixRange(prefix, length, start, end) - } else if (isliteralString(noStartsWithAst)) { + } else if (isLiteralString(noStartsWithAst)) { // literal.* or (literal).* => contains literal RegexOptimizationType.Contains(RegexCharsToString(noStartsWithAst)) } else { diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringFunctions.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringFunctions.scala index b875c84edbf..8fea4014149 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringFunctions.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringFunctions.scala @@ -1073,7 +1073,7 @@ class GpuRLikeMeta( val originalPattern = str.toString val regexAst = new RegexParser(originalPattern).parse() if (conf.isRlikeRegexRewriteEnabled) { - rewriteOptimizationType = RegexRewrite.matchSimplePattern(regexAst) + rewriteOptimizationType = RegexRewrite.matchSimplePattern(regexAst.children()) } val (transpiledAST, _) = new CudfRegexTranspiler(RegexFindMode) .getTranspiledAST(regexAst, None, None) diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionRewriteSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionRewriteSuite.scala index a9ef6364aac..a140f4123f4 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionRewriteSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionRewriteSuite.scala @@ -23,7 +23,7 @@ class RegularExpressionRewriteSuite extends AnyFunSuite { Unit = { val results = patterns.map { pattern => val ast = new RegexParser(pattern).parse() - RegexRewrite.matchSimplePattern(ast) + RegexRewrite.matchSimplePattern(ast.children()) } assert(results == excepted) } @@ -53,12 +53,23 @@ class RegularExpressionRewriteSuite extends AnyFunSuite { "(.*)abc[0-9a-z]{1,3}(.*)", "(.*)abc[0-9]{2}.*", "^abc[0-9]{1,3}", - "火花急流[\u4e00-\u9fa5]{1}") - val excepted = Seq(PrefixRange("abc", 1, 48, 57), - NoOptimization, - PrefixRange("abc", 2, 48, 57), + "火花急流[\u4e00-\u9fa5]{1}", + "^[0-9]{6}", + "^[0-9]{3,10}", + "^.*[0-9]{6}", + "^(.*)[0-9]{3,10}" + ) + val excepted = Seq( PrefixRange("abc", 1, 48, 57), - PrefixRange("火花急流", 1, 19968, 40869)) + NoOptimization, // prefix followed by a multi-range not supported + PrefixRange("abc", 2, 48, 57), + NoOptimization, // starts with PrefixRange not supported + PrefixRange("火花急流", 1, 19968, 40869), + NoOptimization, // starts with PrefixRange not supported + NoOptimization, // starts with PrefixRange not supported + PrefixRange("", 6, 48, 57), + PrefixRange("", 3, 48, 57) + ) verifyRewritePattern(patterns, excepted) } } From 2977c14894e4edea97e9ba08746432e7c875c47e Mon Sep 17 00:00:00 2001 From: Raza Jafri Date: Fri, 31 May 2024 16:27:12 -0700 Subject: [PATCH 19/68] Add Support for Renaming of PythonMapInArrow [databricks] (#10931) * Add support for the renaming of PythonMapInArrow to MapInArrow * Signing off Signed-off-by: Raza Jafri * Removed the unnecessary base class from 400 * addressed review comments --------- Signed-off-by: Raza Jafri --- .../shims/GpuPythonMapInArrowExec.scala | 1 - .../shims/PythonMapInArrowExecShims.scala | 1 - .../shims/GpuPythonMapInArrowExecMeta.scala | 1 - .../rapids/shims/MapInArrowExecShims.scala | 40 ++++++++ .../rapids/shims/GpuMapInArrowExecMeta.scala | 94 +++++++++++++++++++ 5 files changed, 134 insertions(+), 3 deletions(-) create mode 100644 sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/MapInArrowExecShims.scala create mode 100644 sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/shims/GpuMapInArrowExecMeta.scala diff --git a/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/shims/GpuPythonMapInArrowExec.scala b/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/shims/GpuPythonMapInArrowExec.scala index 768261cbc89..5118c21ff2e 100644 --- a/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/shims/GpuPythonMapInArrowExec.scala +++ b/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/shims/GpuPythonMapInArrowExec.scala @@ -31,7 +31,6 @@ {"spark": "343"} {"spark": "350"} {"spark": "351"} -{"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.shims diff --git a/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/PythonMapInArrowExecShims.scala b/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/PythonMapInArrowExecShims.scala index 833767558c6..8f9bc5c1573 100644 --- a/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/PythonMapInArrowExecShims.scala +++ b/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/PythonMapInArrowExecShims.scala @@ -17,7 +17,6 @@ /*** spark-rapids-shim-json-lines {"spark": "350"} {"spark": "351"} -{"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims diff --git a/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/rapids/shims/GpuPythonMapInArrowExecMeta.scala b/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/rapids/shims/GpuPythonMapInArrowExecMeta.scala index a08211f3795..c27f4824c4a 100644 --- a/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/rapids/shims/GpuPythonMapInArrowExecMeta.scala +++ b/sql-plugin/src/main/spark350/scala/org/apache/spark/sql/rapids/shims/GpuPythonMapInArrowExecMeta.scala @@ -17,7 +17,6 @@ /*** spark-rapids-shim-json-lines {"spark": "350"} {"spark": "351"} -{"spark": "400"} spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.shims diff --git a/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/MapInArrowExecShims.scala b/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/MapInArrowExecShims.scala new file mode 100644 index 00000000000..4a1998fa88d --- /dev/null +++ b/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/MapInArrowExecShims.scala @@ -0,0 +1,40 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed 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. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "400"} +spark-rapids-shim-json-lines ***/ +package com.nvidia.spark.rapids.shims + +import com.nvidia.spark.rapids._ + +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.python.MapInArrowExec +import org.apache.spark.sql.rapids.shims.GpuMapInArrowExecMeta + +object PythonMapInArrowExecShims { + + def execs: Map[Class[_ <: SparkPlan], ExecRule[_ <: SparkPlan]] = Seq( + GpuOverrides.exec[MapInArrowExec]( + "The backend for Map Arrow Iterator UDF. Accelerates the data transfer between the" + + " Java process and the Python process. It also supports scheduling GPU resources" + + " for the Python process when enabled.", + ExecChecks((TypeSig.commonCudfTypes + TypeSig.ARRAY + TypeSig.STRUCT).nested(), + TypeSig.all), + (mapPy, conf, p, r) => new GpuMapInArrowExecMeta(mapPy, conf, p, r)) + ).map(r => (r.getClassFor.asSubclass(classOf[SparkPlan]), r)).toMap + +} \ No newline at end of file diff --git a/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/shims/GpuMapInArrowExecMeta.scala b/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/shims/GpuMapInArrowExecMeta.scala new file mode 100644 index 00000000000..f7010099813 --- /dev/null +++ b/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/shims/GpuMapInArrowExecMeta.scala @@ -0,0 +1,94 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed 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. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "400"} +spark-rapids-shim-json-lines ***/ +package org.apache.spark.sql.rapids.shims + +import com.nvidia.spark.rapids._ + +import org.apache.spark.api.python.PythonEvalType +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, PythonUDF} +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.python.MapInArrowExec +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.rapids.execution.TrampolineUtil +import org.apache.spark.sql.rapids.execution.python.GpuMapInBatchExec +import org.apache.spark.sql.types.{BinaryType, StringType} + +class GpuMapInArrowExecMeta( + mapArrow: MapInArrowExec, + conf: RapidsConf, + parent: Option[RapidsMeta[_, _, _]], + rule: DataFromReplacementRule) + extends SparkPlanMeta[MapInArrowExec](mapArrow, conf, parent, rule) { + override def replaceMessage: String = "partially run on GPU" + + override def noReplacementPossibleMessage(reasons: String): String = + s"cannot run even partially on the GPU because $reasons" + + protected val udf: BaseExprMeta[PythonUDF] = GpuOverrides.wrapExpr( + mapArrow.func.asInstanceOf[PythonUDF], conf, Some(this)) + protected val resultAttrs: Seq[BaseExprMeta[Attribute]] = + mapArrow.output.map(GpuOverrides.wrapExpr(_, conf, Some(this))) + + override val childExprs: Seq[BaseExprMeta[_]] = resultAttrs :+ udf + + override def tagPlanForGpu(): Unit = { + super.tagPlanForGpu() + if (SQLConf.get.getConf(SQLConf.ARROW_EXECUTION_USE_LARGE_VAR_TYPES)) { + + val inputTypes = mapArrow.child.schema.fields.map(_.dataType) + val outputTypes = mapArrow.output.map(_.dataType) + + val hasStringOrBinaryTypes = (inputTypes ++ outputTypes).exists(dataType => + TrampolineUtil.dataTypeExistsRecursively(dataType, + dt => dt == StringType || dt == BinaryType)) + + if (hasStringOrBinaryTypes) { + willNotWorkOnGpu(s"${SQLConf.ARROW_EXECUTION_USE_LARGE_VAR_TYPES.key} is " + + s"enabled and the schema contains string or binary types. This is not " + + s"supported on the GPU.") + } + } + } + + override def convertToGpu(): GpuExec = + GpuMapInArrowExec( + udf.convertToGpu(), + resultAttrs.map(_.convertToGpu()).asInstanceOf[Seq[Attribute]], + childPlans.head.convertIfNeeded(), + isBarrier = mapArrow.isBarrier, + ) +} + +/* + * A relation produced by applying a function that takes an iterator of PyArrow's record + * batches and outputs an iterator of PyArrow's record batches. + * + * This GpuMapInPandasExec aims at accelerating the data transfer between + * JVM and Python, and scheduling GPU resources for its Python processes. + * + */ +case class GpuMapInArrowExec( + func: Expression, + output: Seq[Attribute], + child: SparkPlan, + override val isBarrier: Boolean) extends GpuMapInBatchExec { + + override protected val pythonEvalType: Int = PythonEvalType.SQL_MAP_ARROW_ITER_UDF +} From 1be42d48ae64595e8e35ce54b332f54e73d4d8a7 Mon Sep 17 00:00:00 2001 From: Liangcai Li Date: Sat, 1 Jun 2024 08:09:32 +0800 Subject: [PATCH 20/68] fix build errors for 4.0 shim (#10952) Signed-off-by: Firestarman --- .../python/GpuAggregateInPandasExec.scala | 8 +-- .../python/GpuArrowEvalPythonExec.scala | 9 +-- .../GpuFlatMapCoGroupsInPandasExec.scala | 6 +- .../python/GpuFlatMapGroupsInPandasExec.scala | 4 +- .../execution/python/GpuMapInBatchExec.scala | 4 +- .../execution/python/GpuPythonHelper.scala | 7 ++- .../execution/python/GpuPythonUDF.scala | 6 +- .../python/GpuWindowInPandasExecBase.scala | 12 ++-- .../python/shims/GpuArrowPythonRunner.scala | 9 ++- .../shims/GpuCoGroupedArrowPythonRunner.scala | 7 +-- .../shims/GpuGroupedPythonRunnerFactory.scala | 2 +- .../python/shims/WritePythonUDFUtils.scala | 59 +++++++++++++++++++ .../shims/GpuGroupUDFArrowPythonRunner.scala | 9 ++- .../shims/GpuGroupedPythonRunnerFactory.scala | 2 +- .../python/shims/GpuArrowPythonRunner.scala | 9 ++- .../shims/GpuCoGroupedArrowPythonRunner.scala | 7 +-- .../shims/GpuGroupUDFArrowPythonRunner.scala | 9 ++- .../shims/GpuGroupedPythonRunnerFactory.scala | 2 +- .../python/shims/WritePythonUDFUtils.scala | 35 +++++++++++ 19 files changed, 148 insertions(+), 58 deletions(-) create mode 100644 sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/execution/python/shims/WritePythonUDFUtils.scala create mode 100644 sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/execution/python/shims/WritePythonUDFUtils.scala diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/python/GpuAggregateInPandasExec.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/python/GpuAggregateInPandasExec.scala index bc2f30dff2f..639a39bcd38 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/python/GpuAggregateInPandasExec.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/python/GpuAggregateInPandasExec.scala @@ -75,15 +75,15 @@ case class GpuAggregateInPandasExec( } private def collectFunctions(udf: GpuPythonFunction): - (ChainedPythonFunctions, Seq[Expression]) = { + ((ChainedPythonFunctions, Long), Seq[Expression]) = { udf.children match { case Seq(u: GpuPythonFunction) => - val (chained, children) = collectFunctions(u) - (ChainedPythonFunctions(chained.funcs ++ Seq(udf.func)), children) + val ((chained, _), children) = collectFunctions(u) + ((ChainedPythonFunctions(chained.funcs ++ Seq(udf.func)), udf.resultId.id), children) case children => // There should not be any other UDFs, or the children can't be evaluated directly. assert(children.forall(_.find(_.isInstanceOf[GpuPythonFunction]).isEmpty)) - (ChainedPythonFunctions(Seq(udf.func)), udf.children) + ((ChainedPythonFunctions(Seq(udf.func)), udf.resultId.id), udf.children) } } diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/python/GpuArrowEvalPythonExec.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/python/GpuArrowEvalPythonExec.scala index 182d7d1b6c6..c99d0403ed0 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/python/GpuArrowEvalPythonExec.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/python/GpuArrowEvalPythonExec.scala @@ -362,15 +362,16 @@ case class GpuArrowEvalPythonExec( override def producedAttributes: AttributeSet = AttributeSet(resultAttrs) - private def collectFunctions(udf: GpuPythonUDF): (ChainedPythonFunctions, Seq[Expression]) = { + private def collectFunctions( + udf: GpuPythonUDF): ((ChainedPythonFunctions, Long), Seq[Expression]) = { udf.children match { case Seq(u: GpuPythonUDF) => - val (chained, children) = collectFunctions(u) - (ChainedPythonFunctions(chained.funcs ++ Seq(udf.func)), children) + val ((chained, _), children) = collectFunctions(u) + ((ChainedPythonFunctions(chained.funcs ++ Seq(udf.func)), udf.resultId.id), children) case children => // There should not be any other UDFs, or the children can't be evaluated directly. assert(children.forall(_.find(_.isInstanceOf[GpuPythonUDF]).isEmpty)) - (ChainedPythonFunctions(Seq(udf.func)), udf.children) + ((ChainedPythonFunctions(Seq(udf.func)), udf.resultId.id), udf.children) } } diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/python/GpuFlatMapCoGroupsInPandasExec.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/python/GpuFlatMapCoGroupsInPandasExec.scala index b8fa3c1ab69..2e90765e40e 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/python/GpuFlatMapCoGroupsInPandasExec.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/python/GpuFlatMapCoGroupsInPandasExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020-2023, NVIDIA CORPORATION. + * Copyright (c) 2020-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -107,8 +107,8 @@ case class GpuFlatMapCoGroupsInPandasExec( private val sessionLocalTimeZone = conf.sessionLocalTimeZone private val pythonRunnerConf = ArrowUtilsShim.getPythonRunnerConfMap(conf) - private val pandasFunction = udf.asInstanceOf[GpuPythonUDF].func - private val chainedFunc = Seq(ChainedPythonFunctions(Seq(pandasFunction))) + private val pyUDF = udf.asInstanceOf[GpuPythonUDF] + private val chainedFunc = Seq((ChainedPythonFunctions(Seq(pyUDF.func)), pyUDF.resultId.id)) override def producedAttributes: AttributeSet = AttributeSet(output) diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/python/GpuFlatMapGroupsInPandasExec.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/python/GpuFlatMapGroupsInPandasExec.scala index 4a24a449b24..f1596ae7a74 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/python/GpuFlatMapGroupsInPandasExec.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/python/GpuFlatMapGroupsInPandasExec.scala @@ -98,7 +98,7 @@ case class GpuFlatMapGroupsInPandasExec( override def requiredChildOrdering: Seq[Seq[SortOrder]] = Seq(groupingAttributes.map(SortOrder(_, Ascending))) - private val pandasFunction = func.asInstanceOf[GpuPythonUDF].func + private val udf = func.asInstanceOf[GpuPythonUDF] // One batch as input to keep the integrity for each group override def childrenCoalesceGoal: Seq[CoalesceGoal] = Seq(RequireSingleBatch) @@ -111,7 +111,7 @@ case class GpuFlatMapGroupsInPandasExec( val (mNumInputRows, mNumInputBatches, mNumOutputRows, mNumOutputBatches) = commonGpuMetrics() lazy val isPythonOnGpuEnabled = GpuPythonHelper.isPythonOnGpuEnabled(conf) - val chainedFunc = Seq(ChainedPythonFunctions(Seq(pandasFunction))) + val chainedFunc = Seq((ChainedPythonFunctions(Seq(udf.func)), udf.resultId.id)) val localOutput = output val localChildOutput = child.output // Python wraps the resulting columns in a single struct column. diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/python/GpuMapInBatchExec.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/python/GpuMapInBatchExec.scala index 4d41cd32e4f..57c1c7f7114 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/python/GpuMapInBatchExec.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/python/GpuMapInBatchExec.scala @@ -46,7 +46,7 @@ trait GpuMapInBatchExec extends ShimUnaryExecNode with GpuPythonExecBase { protected val isBarrier: Boolean - private val pandasFunction = func.asInstanceOf[GpuPythonUDF].func + private val udf = func.asInstanceOf[GpuPythonUDF] override def producedAttributes: AttributeSet = AttributeSet(output) @@ -58,7 +58,7 @@ trait GpuMapInBatchExec extends ShimUnaryExecNode with GpuPythonExecBase { val (numInputRows, numInputBatches, numOutputRows, numOutputBatches) = commonGpuMetrics() val pyInputTypes = child.schema - val chainedFunc = Seq(ChainedPythonFunctions(Seq(pandasFunction))) + val chainedFunc = Seq((ChainedPythonFunctions(Seq(udf.func)), udf.resultId.id)) val sessionLocalTimeZone = conf.sessionLocalTimeZone val pythonRunnerConf = ArrowUtilsShim.getPythonRunnerConfMap(conf) val isPythonOnGpuEnabled = GpuPythonHelper.isPythonOnGpuEnabled(conf) diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/python/GpuPythonHelper.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/python/GpuPythonHelper.scala index 451ae401891..8564018ad3b 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/python/GpuPythonHelper.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/python/GpuPythonHelper.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020, NVIDIA CORPORATION. + * Copyright (c) 2020-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -86,11 +86,12 @@ object GpuPythonHelper extends Logging { } // Called in each task at the executor side - def injectGpuInfo(funcs: Seq[ChainedPythonFunctions], isPythonOnGpuEnabled: Boolean): Unit = { + def injectGpuInfo(funcs: Seq[(ChainedPythonFunctions, Long)], + isPythonOnGpuEnabled: Boolean): Unit = { // Insert GPU related env(s) into `envVars` for all the PythonFunction(s). // Yes `PythonRunner` will only use the first one, but just make sure it will // take effect no matter the order changes or not. - funcs.foreach(_.funcs.foreach { pyF => + funcs.foreach(_._1.funcs.foreach { pyF => pyF.envVars.put("CUDA_VISIBLE_DEVICES", gpuId) pyF.envVars.put("RAPIDS_PYTHON_ENABLED", isPythonOnGpuEnabled.toString) pyF.envVars.put("RAPIDS_UVM_ENABLED", isPythonUvmEnabled) diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/python/GpuPythonUDF.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/python/GpuPythonUDF.scala index 6cb955a6db8..04367d9f29f 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/python/GpuPythonUDF.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/python/GpuPythonUDF.scala @@ -64,7 +64,7 @@ abstract class GpuPythonFunction( children: Seq[Expression], evalType: Int, udfDeterministic: Boolean, - resultId: ExprId = NamedExpression.newExprId) + val resultId: ExprId = NamedExpression.newExprId) extends Expression with GpuUnevaluable with NonSQLExpression with UserDefinedExpression with GpuAggregateWindowFunction with Serializable { @@ -94,7 +94,7 @@ case class GpuPythonUDF( children: Seq[Expression], evalType: Int, udfDeterministic: Boolean, - resultId: ExprId = NamedExpression.newExprId) + override val resultId: ExprId = NamedExpression.newExprId) extends GpuPythonFunction(name, func, dataType, children, evalType, udfDeterministic, resultId) { override lazy val canonicalized: Expression = { val canonicalizedChildren = children.map(_.canonicalized) @@ -110,7 +110,7 @@ case class GpuPythonUDAF( children: Seq[Expression], evalType: Int, udfDeterministic: Boolean, - resultId: ExprId = NamedExpression.newExprId) + override val resultId: ExprId = NamedExpression.newExprId) extends GpuPythonFunction(name, func, dataType, children, evalType, udfDeterministic, resultId) with GpuAggregateFunction { override lazy val canonicalized: Expression = { diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/python/GpuWindowInPandasExecBase.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/python/GpuWindowInPandasExecBase.scala index 3bc91cd6338..fcf9570a9f7 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/python/GpuWindowInPandasExecBase.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/python/GpuWindowInPandasExecBase.scala @@ -235,16 +235,16 @@ trait GpuWindowInPandasExecBase extends ShimUnaryExecNode with GpuPythonExecBase protected val windowBoundTypeConf = "pandas_window_bound_types" - protected def collectFunctions(udf: GpuPythonFunction): - (ChainedPythonFunctions, Seq[Expression]) = { + protected def collectFunctions( + udf: GpuPythonFunction): ((ChainedPythonFunctions, Long), Seq[Expression]) = { udf.children match { case Seq(u: GpuPythonFunction) => - val (chained, children) = collectFunctions(u) - (ChainedPythonFunctions(chained.funcs ++ Seq(udf.func)), children) + val ((chained, _), children) = collectFunctions(u) + ((ChainedPythonFunctions(chained.funcs ++ Seq(udf.func)), udf.resultId.id), children) case children => // There should not be any other UDFs, or the children can't be evaluated directly. assert(children.forall(_.find(_.isInstanceOf[GpuPythonFunction]).isEmpty)) - (ChainedPythonFunctions(Seq(udf.func)), udf.children) + ((ChainedPythonFunctions(Seq(udf.func)), udf.resultId.id), udf.children) } } @@ -396,7 +396,7 @@ trait GpuWindowInPandasExecBase extends ShimUnaryExecNode with GpuPythonExecBase } }.toArray val dataCVs = GpuColumnVector.extractColumns(batch) - new ColumnarBatch(boundsCVs ++ dataCVs.map(_.incRefCount()), numRows) + new ColumnarBatch((boundsCVs ++ dataCVs.map(_.incRefCount())).toArray, numRows) } override protected def internalDoExecuteColumnar(): RDD[ColumnarBatch] = { diff --git a/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuArrowPythonRunner.scala b/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuArrowPythonRunner.scala index 761d84b4667..977c755712a 100644 --- a/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuArrowPythonRunner.scala +++ b/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuArrowPythonRunner.scala @@ -49,7 +49,6 @@ import com.nvidia.spark.rapids.GpuSemaphore import org.apache.spark.{SparkEnv, TaskContext} import org.apache.spark.api.python.ChainedPythonFunctions -import org.apache.spark.sql.execution.python.PythonUDFRunner import org.apache.spark.sql.rapids.execution.python.{GpuArrowPythonWriter, GpuPythonRunnerCommon} import org.apache.spark.sql.rapids.shims.ArrowUtilsShim import org.apache.spark.sql.types.StructType @@ -60,7 +59,7 @@ import org.apache.spark.util.Utils * Similar to `PythonUDFRunner`, but exchange data with Python worker via Arrow stream. */ class GpuArrowPythonRunner( - funcs: Seq[ChainedPythonFunctions], + funcs: Seq[(ChainedPythonFunctions, Long)], evalType: Int, argOffsets: Array[Array[Int]], pythonInSchema: StructType, @@ -69,8 +68,8 @@ class GpuArrowPythonRunner( maxBatchSize: Long, override val pythonOutSchema: StructType, jobArtifactUUID: Option[String] = None) - extends GpuBasePythonRunner[ColumnarBatch](funcs, evalType, argOffsets, jobArtifactUUID) - with GpuArrowPythonOutput with GpuPythonRunnerCommon { + extends GpuBasePythonRunner[ColumnarBatch](funcs.map(_._1), evalType, argOffsets, + jobArtifactUUID) with GpuArrowPythonOutput with GpuPythonRunnerCommon { protected override def newWriterThread( env: SparkEnv, @@ -82,7 +81,7 @@ class GpuArrowPythonRunner( val arrowWriter = new GpuArrowPythonWriter(pythonInSchema, maxBatchSize) { override protected def writeUDFs(dataOut: DataOutputStream): Unit = { - PythonUDFRunner.writeUDFs(dataOut, funcs, argOffsets) + WritePythonUDFUtils.writeUDFs(dataOut, funcs, argOffsets) } } val isInputNonEmpty = inputIterator.nonEmpty diff --git a/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuCoGroupedArrowPythonRunner.scala b/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuCoGroupedArrowPythonRunner.scala index adb28725ba1..68112676a2b 100644 --- a/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuCoGroupedArrowPythonRunner.scala +++ b/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuCoGroupedArrowPythonRunner.scala @@ -50,7 +50,6 @@ import com.nvidia.spark.rapids.GpuSemaphore import org.apache.spark.{SparkEnv, TaskContext} import org.apache.spark.api.python.{ChainedPythonFunctions, PythonRDD} -import org.apache.spark.sql.execution.python.PythonUDFRunner import org.apache.spark.sql.rapids.execution.python.{GpuArrowWriter, GpuPythonRunnerCommon} import org.apache.spark.sql.types.StructType import org.apache.spark.sql.vectorized.ColumnarBatch @@ -63,7 +62,7 @@ import org.apache.spark.util.Utils * and receive it back in JVM as batches of single DataFrame. */ class GpuCoGroupedArrowPythonRunner( - funcs: Seq[ChainedPythonFunctions], + funcs: Seq[(ChainedPythonFunctions, Long)], evalType: Int, argOffsets: Array[Array[Int]], leftSchema: StructType, @@ -73,7 +72,7 @@ class GpuCoGroupedArrowPythonRunner( batchSize: Int, override val pythonOutSchema: StructType, jobArtifactUUID: Option[String] = None) - extends GpuBasePythonRunner[(ColumnarBatch, ColumnarBatch)](funcs, evalType, + extends GpuBasePythonRunner[(ColumnarBatch, ColumnarBatch)](funcs.map(_._1), evalType, argOffsets, jobArtifactUUID) with GpuArrowPythonOutput with GpuPythonRunnerCommon { protected override def newWriterThread( @@ -90,7 +89,7 @@ class GpuCoGroupedArrowPythonRunner( PythonRDD.writeUTF(k, dataOut) PythonRDD.writeUTF(v, dataOut) } - PythonUDFRunner.writeUDFs(dataOut, funcs, argOffsets) + WritePythonUDFUtils.writeUDFs(dataOut, funcs, argOffsets) } protected override def writeIteratorToStream(dataOut: DataOutputStream): Unit = { diff --git a/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuGroupedPythonRunnerFactory.scala b/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuGroupedPythonRunnerFactory.scala index eba0286e181..9df93a9d11b 100644 --- a/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuGroupedPythonRunnerFactory.scala +++ b/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuGroupedPythonRunnerFactory.scala @@ -48,7 +48,7 @@ import org.apache.spark.sql.vectorized.ColumnarBatch case class GpuGroupedPythonRunnerFactory( conf: org.apache.spark.sql.internal.SQLConf, - chainedFunc: Seq[ChainedPythonFunctions], + chainedFunc: Seq[(ChainedPythonFunctions, Long)], argOffsets: Array[Array[Int]], dedupAttrs: StructType, pythonOutputSchema: StructType, diff --git a/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/execution/python/shims/WritePythonUDFUtils.scala b/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/execution/python/shims/WritePythonUDFUtils.scala new file mode 100644 index 00000000000..aacf972e7e0 --- /dev/null +++ b/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/execution/python/shims/WritePythonUDFUtils.scala @@ -0,0 +1,59 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed 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. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "311"} +{"spark": "312"} +{"spark": "313"} +{"spark": "320"} +{"spark": "321"} +{"spark": "321cdh"} +{"spark": "322"} +{"spark": "323"} +{"spark": "324"} +{"spark": "330"} +{"spark": "330cdh"} +{"spark": "330db"} +{"spark": "331"} +{"spark": "332"} +{"spark": "332cdh"} +{"spark": "332db"} +{"spark": "333"} +{"spark": "334"} +{"spark": "340"} +{"spark": "341"} +{"spark": "341db"} +{"spark": "342"} +{"spark": "343"} +{"spark": "350"} +{"spark": "351"} +spark-rapids-shim-json-lines ***/ +package org.apache.spark.sql.rapids.execution.python.shims + +import java.io.DataOutputStream + +import org.apache.spark.api.python.ChainedPythonFunctions +import org.apache.spark.sql.execution.python.PythonUDFRunner + +object WritePythonUDFUtils { + def writeUDFs( + dataOut: DataOutputStream, + funcs: Seq[(ChainedPythonFunctions, Long)], + argOffsets: Array[Array[Int]], + profiler: Option[String] = None): Unit = { + PythonUDFRunner.writeUDFs(dataOut, funcs.map(_._1), argOffsets) + } +} diff --git a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuGroupUDFArrowPythonRunner.scala b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuGroupUDFArrowPythonRunner.scala index cb8eef809f3..a6338e7adc5 100644 --- a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuGroupUDFArrowPythonRunner.scala +++ b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuGroupUDFArrowPythonRunner.scala @@ -30,7 +30,6 @@ import com.nvidia.spark.rapids.GpuSemaphore import org.apache.spark.{SparkEnv, TaskContext} import org.apache.spark.api.python._ -import org.apache.spark.sql.execution.python.PythonUDFRunner import org.apache.spark.sql.rapids.execution.python.{GpuArrowPythonWriter, GpuPythonRunnerCommon} import org.apache.spark.sql.types.StructType import org.apache.spark.sql.vectorized.ColumnarBatch @@ -50,7 +49,7 @@ import org.apache.spark.util.Utils * more data being sent. */ class GpuGroupUDFArrowPythonRunner( - funcs: Seq[ChainedPythonFunctions], + funcs: Seq[(ChainedPythonFunctions, Long)], evalType: Int, argOffsets: Array[Array[Int]], pythonInSchema: StructType, @@ -59,8 +58,8 @@ class GpuGroupUDFArrowPythonRunner( maxBatchSize: Long, override val pythonOutSchema: StructType, jobArtifactUUID: Option[String] = None) - extends GpuBasePythonRunner[ColumnarBatch](funcs, evalType, argOffsets, jobArtifactUUID) - with GpuArrowPythonOutput with GpuPythonRunnerCommon { + extends GpuBasePythonRunner[ColumnarBatch](funcs.map(_._1), evalType, argOffsets, + jobArtifactUUID) with GpuArrowPythonOutput with GpuPythonRunnerCommon { protected override def newWriterThread( env: SparkEnv, @@ -72,7 +71,7 @@ class GpuGroupUDFArrowPythonRunner( val arrowWriter = new GpuArrowPythonWriter(pythonInSchema, maxBatchSize) { override protected def writeUDFs(dataOut: DataOutputStream): Unit = { - PythonUDFRunner.writeUDFs(dataOut, funcs, argOffsets) + WritePythonUDFUtils.writeUDFs(dataOut, funcs, argOffsets) } } diff --git a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuGroupedPythonRunnerFactory.scala b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuGroupedPythonRunnerFactory.scala index 451de0a2527..313ea6c20a2 100644 --- a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuGroupedPythonRunnerFactory.scala +++ b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuGroupedPythonRunnerFactory.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.vectorized.ColumnarBatch case class GpuGroupedPythonRunnerFactory( conf: org.apache.spark.sql.internal.SQLConf, - chainedFunc: Seq[ChainedPythonFunctions], + chainedFunc: Seq[(ChainedPythonFunctions, Long)], argOffsets: Array[Array[Int]], dedupAttrs: StructType, pythonOutputSchema: StructType, diff --git a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuArrowPythonRunner.scala b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuArrowPythonRunner.scala index ac58baa2eb7..50c5e280e9c 100644 --- a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuArrowPythonRunner.scala +++ b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuArrowPythonRunner.scala @@ -25,7 +25,6 @@ import com.nvidia.spark.rapids.GpuSemaphore import org.apache.spark.{SparkEnv, TaskContext} import org.apache.spark.api.python._ -import org.apache.spark.sql.execution.python.PythonUDFRunner import org.apache.spark.sql.rapids.execution.python.{GpuArrowPythonWriter, GpuPythonRunnerCommon} import org.apache.spark.sql.rapids.shims.ArrowUtilsShim import org.apache.spark.sql.types.StructType @@ -35,7 +34,7 @@ import org.apache.spark.sql.vectorized.ColumnarBatch * Similar to `PythonUDFRunner`, but exchange data with Python worker via Arrow stream. */ class GpuArrowPythonRunner( - funcs: Seq[ChainedPythonFunctions], + funcs: Seq[(ChainedPythonFunctions, Long)], evalType: Int, argOffsets: Array[Array[Int]], pythonInSchema: StructType, @@ -44,8 +43,8 @@ class GpuArrowPythonRunner( maxBatchSize: Long, override val pythonOutSchema: StructType, jobArtifactUUID: Option[String] = None) - extends GpuBasePythonRunner[ColumnarBatch](funcs, evalType, argOffsets, jobArtifactUUID) - with GpuArrowPythonOutput with GpuPythonRunnerCommon { + extends GpuBasePythonRunner[ColumnarBatch](funcs.map(_._1), evalType, argOffsets, + jobArtifactUUID) with GpuArrowPythonOutput with GpuPythonRunnerCommon { protected override def newWriter( env: SparkEnv, @@ -57,7 +56,7 @@ class GpuArrowPythonRunner( val arrowWriter = new GpuArrowPythonWriter(pythonInSchema, maxBatchSize) { override protected def writeUDFs(dataOut: DataOutputStream): Unit = { - PythonUDFRunner.writeUDFs(dataOut, funcs, argOffsets) + WritePythonUDFUtils.writeUDFs(dataOut, funcs, argOffsets) } } val isInputNonEmpty = inputIterator.nonEmpty diff --git a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuCoGroupedArrowPythonRunner.scala b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuCoGroupedArrowPythonRunner.scala index aad1eb52c02..0317a89009e 100644 --- a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuCoGroupedArrowPythonRunner.scala +++ b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuCoGroupedArrowPythonRunner.scala @@ -27,7 +27,6 @@ import com.nvidia.spark.rapids.GpuSemaphore import org.apache.spark.{SparkEnv, TaskContext} import org.apache.spark.api.python.{ChainedPythonFunctions, PythonRDD, PythonWorker} -import org.apache.spark.sql.execution.python.PythonUDFRunner import org.apache.spark.sql.rapids.execution.python.{GpuArrowWriter, GpuPythonRunnerCommon} import org.apache.spark.sql.types.StructType import org.apache.spark.sql.vectorized.ColumnarBatch @@ -39,7 +38,7 @@ import org.apache.spark.sql.vectorized.ColumnarBatch * and receive it back in JVM as batches of single DataFrame. */ class GpuCoGroupedArrowPythonRunner( - funcs: Seq[ChainedPythonFunctions], + funcs: Seq[(ChainedPythonFunctions, Long)], evalType: Int, argOffsets: Array[Array[Int]], leftSchema: StructType, @@ -49,7 +48,7 @@ class GpuCoGroupedArrowPythonRunner( batchSize: Int, override val pythonOutSchema: StructType, jobArtifactUUID: Option[String] = None) - extends GpuBasePythonRunner[(ColumnarBatch, ColumnarBatch)](funcs, evalType, + extends GpuBasePythonRunner[(ColumnarBatch, ColumnarBatch)](funcs.map(_._1), evalType, argOffsets, jobArtifactUUID) with GpuArrowPythonOutput with GpuPythonRunnerCommon { protected override def newWriter( @@ -67,7 +66,7 @@ class GpuCoGroupedArrowPythonRunner( PythonRDD.writeUTF(k, dataOut) PythonRDD.writeUTF(v, dataOut) } - PythonUDFRunner.writeUDFs(dataOut, funcs, argOffsets) + WritePythonUDFUtils.writeUDFs(dataOut, funcs, argOffsets) } override def writeNextInputToStream(dataOut: DataOutputStream): Boolean = { diff --git a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuGroupUDFArrowPythonRunner.scala b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuGroupUDFArrowPythonRunner.scala index 4393c8b7057..42c6178ff83 100644 --- a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuGroupUDFArrowPythonRunner.scala +++ b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuGroupUDFArrowPythonRunner.scala @@ -28,7 +28,6 @@ import com.nvidia.spark.rapids.GpuSemaphore import org.apache.spark.{SparkEnv, TaskContext} import org.apache.spark.api.python._ -import org.apache.spark.sql.execution.python.PythonUDFRunner import org.apache.spark.sql.rapids.execution.python.{GpuArrowPythonWriter, GpuPythonRunnerCommon} import org.apache.spark.sql.types.StructType import org.apache.spark.sql.vectorized.ColumnarBatch @@ -47,7 +46,7 @@ import org.apache.spark.sql.vectorized.ColumnarBatch * more data being sent. */ class GpuGroupUDFArrowPythonRunner( - funcs: Seq[ChainedPythonFunctions], + funcs: Seq[(ChainedPythonFunctions, Long)], evalType: Int, argOffsets: Array[Array[Int]], pythonInSchema: StructType, @@ -56,8 +55,8 @@ class GpuGroupUDFArrowPythonRunner( batchSize: Long, override val pythonOutSchema: StructType, jobArtifactUUID: Option[String] = None) - extends GpuBasePythonRunner[ColumnarBatch](funcs, evalType, argOffsets, jobArtifactUUID) - with GpuArrowPythonOutput with GpuPythonRunnerCommon { + extends GpuBasePythonRunner[ColumnarBatch](funcs.map(_._1), evalType, argOffsets, + jobArtifactUUID) with GpuArrowPythonOutput with GpuPythonRunnerCommon { protected override def newWriter( env: SparkEnv, @@ -69,7 +68,7 @@ class GpuGroupUDFArrowPythonRunner( val arrowWriter = new GpuArrowPythonWriter(pythonInSchema, batchSize) { override protected def writeUDFs(dataOut: DataOutputStream): Unit = { - PythonUDFRunner.writeUDFs(dataOut, funcs, argOffsets) + WritePythonUDFUtils.writeUDFs(dataOut, funcs, argOffsets) } } diff --git a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuGroupedPythonRunnerFactory.scala b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuGroupedPythonRunnerFactory.scala index b1dabbf5b5e..63a4289c5b0 100644 --- a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuGroupedPythonRunnerFactory.scala +++ b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/execution/python/shims/GpuGroupedPythonRunnerFactory.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.vectorized.ColumnarBatch case class GpuGroupedPythonRunnerFactory( conf: org.apache.spark.sql.internal.SQLConf, - chainedFunc: Seq[ChainedPythonFunctions], + chainedFunc: Seq[(ChainedPythonFunctions, Long)], argOffsets: Array[Array[Int]], dedupAttrs: StructType, pythonOutputSchema: StructType, diff --git a/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/execution/python/shims/WritePythonUDFUtils.scala b/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/execution/python/shims/WritePythonUDFUtils.scala new file mode 100644 index 00000000000..4650d998fd7 --- /dev/null +++ b/sql-plugin/src/main/spark400/scala/org/apache/spark/sql/rapids/execution/python/shims/WritePythonUDFUtils.scala @@ -0,0 +1,35 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed 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. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "400"} +spark-rapids-shim-json-lines ***/ +package org.apache.spark.sql.rapids.execution.python.shims + +import java.io.DataOutputStream + +import org.apache.spark.api.python.ChainedPythonFunctions +import org.apache.spark.sql.execution.python.PythonUDFRunner + +object WritePythonUDFUtils { + def writeUDFs( + dataOut: DataOutputStream, + funcs: Seq[(ChainedPythonFunctions, Long)], + argOffsets: Array[Array[Int]], + profiler: Option[String] = None): Unit = { + PythonUDFRunner.writeUDFs(dataOut, funcs, argOffsets, profiler) + } +} From 5750ace5bcece0aeba1d3b515bb8f70a9f423878 Mon Sep 17 00:00:00 2001 From: Peixin Date: Mon, 3 Jun 2024 15:15:38 +0800 Subject: [PATCH 21/68] Add new blossom-ci allowed user (#10959) Signed-off-by: Peixin Li --- .github/workflows/blossom-ci.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.github/workflows/blossom-ci.yml b/.github/workflows/blossom-ci.yml index b3cbbb6ad14..6f597d6baf3 100644 --- a/.github/workflows/blossom-ci.yml +++ b/.github/workflows/blossom-ci.yml @@ -71,6 +71,7 @@ jobs: liurenjie1024,\ binmahone,\ zpuller,\ + pxLi,\ ', format('{0},', github.actor)) && github.event.comment.body == 'build' steps: - name: Check if comment is issued by authorized person From 8d3a8ced98d2f04b0021fe5759df3984e6340991 Mon Sep 17 00:00:00 2001 From: Peixin Date: Tue, 4 Jun 2024 12:02:24 +0800 Subject: [PATCH 22/68] Add default value for REF of premerge jenkinsfile to avoid bad overwritten [skip ci] (#10966) * DO NOT REVIEW Signed-off-by: Peixin Li * Add default value for REF to avoid overwritten while unexpected manual trigger Signed-off-by: Peixin Li --------- Signed-off-by: Peixin Li --- jenkins/Jenkinsfile-blossom.premerge | 3 ++- jenkins/Jenkinsfile-blossom.premerge-databricks | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/jenkins/Jenkinsfile-blossom.premerge b/jenkins/Jenkinsfile-blossom.premerge index e7bb8af2cdd..fb3a5c2f328 100755 --- a/jenkins/Jenkinsfile-blossom.premerge +++ b/jenkins/Jenkinsfile-blossom.premerge @@ -57,7 +57,8 @@ pipeline { } parameters { - string(name: 'REF', defaultValue: '', + // Put a default value for REF to avoid error when running the pipeline manually + string(name: 'REF', defaultValue: 'main', description: 'Merged commit of specific PR') string(name: 'GITHUB_DATA', defaultValue: '', description: 'Json-formatted github data from upstream blossom-ci') diff --git a/jenkins/Jenkinsfile-blossom.premerge-databricks b/jenkins/Jenkinsfile-blossom.premerge-databricks index a13170f7162..cd872fd9153 100644 --- a/jenkins/Jenkinsfile-blossom.premerge-databricks +++ b/jenkins/Jenkinsfile-blossom.premerge-databricks @@ -46,7 +46,8 @@ pipeline { } parameters { - string(name: 'REF', defaultValue: '', + // Put a default value for REF to avoid error when running the pipeline manually + string(name: 'REF', defaultValue: 'main', description: 'Merged commit of specific PR') string(name: 'GITHUB_DATA', defaultValue: '', description: 'Json-formatted github data from upstream blossom-ci') From 47074062b92f02cfd89f37766aefd949c2095900 Mon Sep 17 00:00:00 2001 From: Raza Jafri Date: Tue, 4 Jun 2024 12:30:15 -0700 Subject: [PATCH 23/68] Use ErrorClass to Throw AnalysisException [databricks] (#10830) * AnalysisException child class Signed-off-by: Raza Jafri * Use errorClass for reporting AnalysisException * POM changes Signed-off-by: Raza Jafri * Reuse the RapidsErrorUtils to throw the AnalysisException * Revert "POM changes" This reverts commit 0f765c9b3910aa3c174de709850d9a5fbc4aea89. * Updated copyrights * Added the TrampolineUtil method back to handle cases which don't use errorClass * Add doc to the RapidsAnalysisException * addressed review comments * Fixed imports * Moved the RapidsAnalysisException out of TrampolineUtil * fixed imports * addressed review comments * fixed unused import * Removed the TrampolineUtil method for throwing RapidsAnalysisException --------- Signed-off-by: Raza Jafri --- .../rapids/GpuDataWritingCommandExec.scala | 11 +- .../spark/rapids/GpuRunnableCommandExec.scala | 11 +- .../sql/hive/rapids/RapidsHiveErrors.scala | 7 +- .../spark/sql/rapids/GpuDataSourceBase.scala | 51 +++---- ...GpuInsertIntoHadoopFsRelationCommand.scala | 8 +- .../expressions/GpuRandomExpressions.scala | 6 +- .../sql/rapids/execution/TrampolineUtil.scala | 12 +- .../rapids/shims/ParquetSchemaClipShims.scala | 16 +-- .../rapids/shims/GpuInsertIntoHiveTable.scala | 5 +- .../sql/rapids/shims/RapidsErrorUtils.scala | 4 +- .../rapids/shims/RapidsQueryErrorUtils.scala | 125 +++++++++++++++++ .../rapids/shims/ParquetSchemaClipShims.scala | 13 +- .../sql/rapids/shims/RapidsErrorUtils.scala | 4 +- .../rapids/shims/RapidsQueryErrorUtils.scala | 127 ++++++++++++++++++ .../rapids/shims/ParquetSchemaClipShims.scala | 14 +- .../sql/rapids/shims/RapidsErrorUtils.scala | 4 +- .../sql/rapids/shims/RapidsErrorUtils.scala | 4 +- .../rapids/shims/GpuInsertIntoHiveTable.scala | 5 +- .../sql/rapids/GpuFileFormatWriter.scala | 5 +- ...eDataSourceTableAsSelectCommandShims.scala | 2 +- .../sql/rapids/shims/RapidsErrorUtils.scala | 2 +- .../sql/rapids/shims/RapidsErrorUtils.scala | 4 +- 22 files changed, 342 insertions(+), 98 deletions(-) create mode 100644 sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala create mode 100644 sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuDataWritingCommandExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuDataWritingCommandExec.scala index 5a54d0b2f66..019f9b2e6b0 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuDataWritingCommandExec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuDataWritingCommandExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019-2023, NVIDIA CORPORATION. + * Copyright (c) 2019-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -31,7 +31,7 @@ import org.apache.spark.sql.execution.command.DataWritingCommand import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.rapids.GpuWriteJobStatsTracker -import org.apache.spark.sql.rapids.execution.TrampolineUtil +import org.apache.spark.sql.rapids.shims.RapidsErrorUtils import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.SerializableConfiguration @@ -84,10 +84,9 @@ object GpuDataWritingCommand { if (fs.exists(filePath) && fs.getFileStatus(filePath).isDirectory && fs.listStatus(filePath).length != 0) { - TrampolineUtil.throwAnalysisException( - s"CREATE-TABLE-AS-SELECT cannot create table with location to a non-empty directory " + - s"${tablePath} . To allow overwriting the existing non-empty directory, " + - s"set '$allowNonEmptyLocationInCTASKey' to true.") + throw RapidsErrorUtils. + createTableAsSelectWithNonEmptyDirectoryError(tablePath.toString, + allowNonEmptyLocationInCTASKey) } } } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuRunnableCommandExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuRunnableCommandExec.scala index e3869960fc4..43bd593c0b5 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuRunnableCommandExec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuRunnableCommandExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023, NVIDIA CORPORATION. + * Copyright (c) 2023-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -31,7 +31,7 @@ import org.apache.spark.sql.execution.command.RunnableCommand import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.rapids.GpuWriteJobStatsTracker -import org.apache.spark.sql.rapids.execution.TrampolineUtil +import org.apache.spark.sql.rapids.shims.RapidsErrorUtils import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.SerializableConfiguration @@ -82,10 +82,9 @@ object GpuRunnableCommand { if (fs.exists(filePath) && fs.getFileStatus(filePath).isDirectory && fs.listStatus(filePath).length != 0) { - TrampolineUtil.throwAnalysisException( - s"CREATE-TABLE-AS-SELECT cannot create table with location to a non-empty directory " + - s"${tablePath} . To allow overwriting the existing non-empty directory, " + - s"set '$allowNonEmptyLocationInCTASKey' to true.") + throw RapidsErrorUtils. + createTableAsSelectWithNonEmptyDirectoryError(tablePath.toString, + allowNonEmptyLocationInCTASKey) } } } diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/RapidsHiveErrors.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/RapidsHiveErrors.scala index 259a04ec318..40cac90680f 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/RapidsHiveErrors.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/RapidsHiveErrors.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023, NVIDIA CORPORATION. + * Copyright (c) 2023-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -19,9 +19,9 @@ package org.apache.spark.sql.hive.rapids import org.apache.hadoop.fs.Path import org.apache.spark.SparkException -import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.catalog.CatalogTable import org.apache.spark.sql.catalyst.expressions.Literal +import org.apache.spark.sql.rapids.shims.RapidsErrorUtils import org.apache.spark.sql.types.{DataType, DoubleType, FloatType, StringType} object RapidsHiveErrors { @@ -53,8 +53,7 @@ object RapidsHiveErrors { } def cannotResolveAttributeError(name: String, outputStr: String): Throwable = { - new AnalysisException( - s"Unable to resolve $name given [$outputStr]") + throw RapidsErrorUtils.cannotResolveAttributeError(name, outputStr) } def writePartitionExceedConfigSizeWhenDynamicPartitionError( diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuDataSourceBase.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuDataSourceBase.scala index 0ec720733e8..5589bca0435 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuDataSourceBase.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuDataSourceBase.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020-2023, NVIDIA CORPORATION. + * Copyright (c) 2020-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -44,7 +44,7 @@ import org.apache.spark.sql.execution.datasources.v2.orc.OrcDataSourceV2 import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.sources.{RateStreamProvider, TextSocketSourceProvider} import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.rapids.shims.SchemaUtilsShims +import org.apache.spark.sql.rapids.shims.{RapidsErrorUtils, SchemaUtilsShims} import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.{DataType, StructType} import org.apache.spark.util.{HadoopFSUtils, ThreadUtils, Utils} @@ -144,8 +144,8 @@ abstract class GpuDataSourceBase( } inferredOpt }.getOrElse { - throw new AnalysisException(s"Failed to resolve the schema for $format for " + - s"the partition column: $partitionColumn. It must be specified manually.") + throw RapidsErrorUtils. + partitionColumnNotSpecifiedError(format.toString, partitionColumn) } } StructType(partitionFields) @@ -162,8 +162,7 @@ abstract class GpuDataSourceBase( caseInsensitiveOptions - "path", SparkShimImpl.filesFromFileIndex(tempFileIndex)) }.getOrElse { - throw new AnalysisException( - s"Unable to infer schema for $format. It must be specified manually.") + throw RapidsErrorUtils.dataSchemaNotSpecifiedError(format.toString) } // We just print a waring message if the data schema and partition schema have the duplicate @@ -201,17 +200,13 @@ abstract class GpuDataSourceBase( case (dataSource: RelationProvider, None) => dataSource.createRelation(sparkSession.sqlContext, caseInsensitiveOptions) case (_: SchemaRelationProvider, None) => - throw new AnalysisException(s"A schema needs to be specified when using $className.") + throw RapidsErrorUtils.schemaNotSpecifiedForSchemaRelationProviderError(className) case (dataSource: RelationProvider, Some(schema)) => val baseRelation = dataSource.createRelation(sparkSession.sqlContext, caseInsensitiveOptions) if (!DataType.equalsIgnoreCompatibleNullability(baseRelation.schema, schema)) { - throw new AnalysisException( - "The user-specified schema doesn't match the actual schema: " + - s"user-specified: ${schema.toDDL}, actual: ${baseRelation.schema.toDDL}. If " + - "you're using DataFrameReader.schema API or creating a table, please do not " + - "specify the schema. Or if you're scanning an existed table, please drop " + - "it and re-create it.") + throw RapidsErrorUtils.userSpecifiedSchemaMismatchActualSchemaError(schema, + baseRelation.schema) } baseRelation @@ -233,9 +228,8 @@ abstract class GpuDataSourceBase( caseInsensitiveOptions - "path", SparkShimImpl.filesFromFileIndex(fileCatalog)) }.getOrElse { - throw new AnalysisException( - s"Unable to infer schema for $format at ${fileCatalog.allFiles().mkString(",")}. " + - "It must be specified manually") + throw RapidsErrorUtils. + dataSchemaNotSpecifiedError(format.toString, fileCatalog.allFiles().mkString(",")) } HadoopFsRelation( @@ -276,8 +270,7 @@ abstract class GpuDataSourceBase( caseInsensitiveOptions)(sparkSession) case _ => - throw new AnalysisException( - s"$className is not a valid Spark SQL Data Source.") + throw RapidsErrorUtils.invalidDataSourceError(className) } relation match { @@ -411,22 +404,13 @@ object GpuDataSourceBase extends Logging { dataSource case Failure(error) => if (provider1.startsWith("org.apache.spark.sql.hive.orc")) { - throw new AnalysisException( - "Hive built-in ORC data source must be used with Hive support enabled. " + - "Please use the native ORC data source by setting 'spark.sql.orc.impl' to " + - "'native'") + throw RapidsErrorUtils.orcNotUsedWithHiveEnabledError() } else if (provider1.toLowerCase(Locale.ROOT) == "avro" || provider1 == "com.databricks.spark.avro" || provider1 == "org.apache.spark.sql.avro") { - throw new AnalysisException( - s"Failed to find data source: $provider1. Avro is built-in but external data " + - "source module since Spark 2.4. Please deploy the application as per " + - "the deployment section of \"Apache Avro Data Source Guide\".") + throw RapidsErrorUtils.failedToFindAvroDataSourceError(provider1) } else if (provider1.toLowerCase(Locale.ROOT) == "kafka") { - throw new AnalysisException( - s"Failed to find data source: $provider1. Please deploy the application as " + - "per the deployment section of " + - "\"Structured Streaming + Kafka Integration Guide\".") + throw RapidsErrorUtils.failedToFindKafkaDataSourceError(provider1) } else { throw new ClassNotFoundException( s"Failed to find data source: $provider1. Please find packages at " + @@ -459,8 +443,7 @@ object GpuDataSourceBase extends Logging { s"defaulting to the internal datasource (${internalSources.head.getClass.getName}).") internalSources.head.getClass } else { - throw new AnalysisException(s"Multiple sources found for $provider1 " + - s"(${sourceNames.mkString(", ")}), please specify the fully qualified class name.") + throw RapidsErrorUtils.findMultipleDataSourceError(provider1, sourceNames) } } } catch { @@ -513,7 +496,7 @@ object GpuDataSourceBase extends Logging { } if (checkEmptyGlobPath && globResult.isEmpty) { - throw new AnalysisException(s"Path does not exist: $globPath") + throw RapidsErrorUtils.dataPathNotExistError(globPath.toString) } globResult @@ -527,7 +510,7 @@ object GpuDataSourceBase extends Logging { ThreadUtils.parmap(nonGlobPaths, "checkPathsExist", numThreads) { path => val fs = path.getFileSystem(hadoopConf) if (!fs.exists(path)) { - throw new AnalysisException(s"Path does not exist: $path") + throw RapidsErrorUtils.dataPathNotExistError(path.toString) } } } catch { diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuInsertIntoHadoopFsRelationCommand.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuInsertIntoHadoopFsRelationCommand.scala index 2b7974fd1a6..ece5ef5acf5 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuInsertIntoHadoopFsRelationCommand.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuInsertIntoHadoopFsRelationCommand.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019-2022, NVIDIA CORPORATION. + * Copyright (c) 2019-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -22,7 +22,7 @@ import com.nvidia.spark.rapids.{ColumnarFileFormat, GpuDataWritingCommand} import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.spark.internal.io.FileCommitProtocol -import org.apache.spark.sql.{AnalysisException, SaveMode, SparkSession} +import org.apache.spark.sql.{SaveMode, SparkSession} import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogTable, CatalogTablePartition} import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.getPartitionPathString @@ -33,7 +33,7 @@ import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.command.{AlterTableAddPartitionCommand, AlterTableDropPartitionCommand, CommandUtils} import org.apache.spark.sql.execution.datasources.{FileFormatWriter, FileIndex, PartitioningUtils} import org.apache.spark.sql.internal.SQLConf.PartitionOverwriteMode -import org.apache.spark.sql.rapids.shims.SchemaUtilsShims +import org.apache.spark.sql.rapids.shims.{RapidsErrorUtils, SchemaUtilsShims} import org.apache.spark.sql.vectorized.ColumnarBatch case class GpuInsertIntoHadoopFsRelationCommand( @@ -121,7 +121,7 @@ case class GpuInsertIntoHadoopFsRelationCommand( val pathExists = fs.exists(qualifiedOutputPath) (mode, pathExists) match { case (SaveMode.ErrorIfExists, true) => - throw new AnalysisException(s"path $qualifiedOutputPath already exists.") + throw RapidsErrorUtils.outputPathAlreadyExistsError(qualifiedOutputPath) case (SaveMode.Overwrite, true) => if (ifPartitionNotExists && matchingPartitions.nonEmpty) { false diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/catalyst/expressions/GpuRandomExpressions.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/catalyst/expressions/GpuRandomExpressions.scala index 6675f678f6d..f9d0be81505 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/catalyst/expressions/GpuRandomExpressions.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/catalyst/expressions/GpuRandomExpressions.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020-2023, NVIDIA CORPORATION. + * Copyright (c) 2020-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -23,8 +23,8 @@ import com.nvidia.spark.rapids.Arm.withResource import com.nvidia.spark.rapids.shims.ShimUnaryExpression import org.apache.spark.TaskContext -import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, ExpressionWithRandomSeed} +import org.apache.spark.sql.rapids.execution.RapidsAnalysisException import org.apache.spark.sql.types._ import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.Utils @@ -52,7 +52,7 @@ case class GpuRand(child: Expression) extends ShimUnaryExpression with GpuExpres @transient protected lazy val seed: Long = child match { case GpuLiteral(s, IntegerType) => s.asInstanceOf[Int] case GpuLiteral(s, LongType) => s.asInstanceOf[Long] - case _ => throw new AnalysisException( + case _ => throw new RapidsAnalysisException( s"Input argument to $prettyName must be an integer, long or null literal.") } diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/TrampolineUtil.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/TrampolineUtil.scala index 5ffe08348f1..8a88cc4024d 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/TrampolineUtil.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/TrampolineUtil.scala @@ -157,9 +157,6 @@ object TrampolineUtil { TaskContext.get.taskMemoryManager() } - /** Throw a Spark analysis exception */ - def throwAnalysisException(msg: String) = throw new AnalysisException(msg) - /** Set the task context for the current thread */ def setTaskContext(tc: TaskContext): Unit = TaskContext.setTaskContext(tc) @@ -241,4 +238,13 @@ object TrampolineUtil { } def getSparkHadoopUtilConf: Configuration = SparkHadoopUtil.get.conf + } + +/** + * This class is to only be used to throw errors specific to the + * RAPIDS Accelerator or errors mirroring Spark where a raw + * AnalysisException is thrown directly rather than via an error + * utility class (this should be rare). + */ +class RapidsAnalysisException(msg: String) extends AnalysisException(msg) diff --git a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala index fd48b8b6375..4d6d4967a80 100644 --- a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala +++ b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2023, NVIDIA CORPORATION. + * Copyright (c) 2022-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -26,7 +26,8 @@ import org.apache.parquet.schema.OriginalType._ import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.rapids.execution.TrampolineUtil +import org.apache.spark.sql.rapids.execution.RapidsAnalysisException +import org.apache.spark.sql.rapids.shims.RapidsErrorUtils import org.apache.spark.sql.types._ object ParquetSchemaClipShims { @@ -64,13 +65,13 @@ object ParquetSchemaClipShims { if (originalType == null) s"$typeName" else s"$typeName ($originalType)" def typeNotSupported() = - TrampolineUtil.throwAnalysisException(s"Parquet type not supported: $typeString") + throw new RapidsAnalysisException(s"Parquet type not supported: $typeString") def typeNotImplemented() = - TrampolineUtil.throwAnalysisException(s"Parquet type not yet supported: $typeString") + throw RapidsErrorUtils.parquetTypeUnsupportedYetError(typeString) def illegalType() = - TrampolineUtil.throwAnalysisException(s"Illegal Parquet type: $typeString") + throw RapidsErrorUtils.illegalParquetTypeError(typeString) // When maxPrecision = -1, we skip precision range check, and always respect the precision // specified in field.getDecimalMetadata. This is useful when interpreting decimal types stored @@ -80,8 +81,7 @@ object ParquetSchemaClipShims { val scale = field.getDecimalMetadata.getScale if (!(maxPrecision == -1 || 1 <= precision && precision <= maxPrecision)) { - TrampolineUtil.throwAnalysisException( - s"Invalid decimal precision: $typeName " + + throw new RapidsAnalysisException(s"Invalid decimal precision: $typeName " + s"cannot store $precision digits (max $maxPrecision)") } @@ -121,7 +121,7 @@ object ParquetSchemaClipShims { case INT96 => if (!SQLConf.get.isParquetINT96AsTimestamp) { - TrampolineUtil.throwAnalysisException( + throw new RapidsAnalysisException( "INT96 is not supported unless it's interpreted as timestamp. " + s"Please try to set ${SQLConf.PARQUET_INT96_AS_TIMESTAMP.key} to true.") } diff --git a/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/hive/rapids/shims/GpuInsertIntoHiveTable.scala b/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/hive/rapids/shims/GpuInsertIntoHiveTable.scala index 6d4ca5da7c3..2ea0301fa2c 100644 --- a/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/hive/rapids/shims/GpuInsertIntoHiveTable.scala +++ b/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/hive/rapids/shims/GpuInsertIntoHiveTable.scala @@ -45,7 +45,7 @@ import org.apache.hadoop.hive.ql.ErrorMsg import org.apache.hadoop.hive.ql.plan.TableDesc import org.apache.spark.SparkException -import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogTableType, ExternalCatalog, ExternalCatalogUtils, ExternalCatalogWithListener} import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan @@ -58,6 +58,7 @@ import org.apache.spark.sql.hive.client.HiveClientImpl import org.apache.spark.sql.hive.client.hive._ import org.apache.spark.sql.hive.execution.InsertIntoHiveTable import org.apache.spark.sql.hive.rapids.{GpuHiveFileFormat, GpuSaveAsHiveFile, RapidsHiveErrors} +import org.apache.spark.sql.rapids.shims.RapidsErrorUtils import org.apache.spark.sql.vectorized.ColumnarBatch final class GpuInsertIntoHiveTableMeta(cmd: InsertIntoHiveTable, @@ -193,7 +194,7 @@ case class GpuInsertIntoHiveTable( // Report error if any static partition appears after a dynamic partition val isDynamic = partitionColumnNames.map(partitionSpec(_).isEmpty) if (isDynamic.init.zip(isDynamic.tail).contains((true, false))) { - throw new AnalysisException(ErrorMsg.PARTITION_DYN_STA_ORDER.getMsg) + throw RapidsErrorUtils.dynamicPartitionParentError } } diff --git a/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala b/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala index f23229e0956..7fa269db71a 100644 --- a/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala +++ b/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2023, NVIDIA CORPORATION. + * Copyright (c) 2022-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -26,7 +26,7 @@ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.trees.Origin import org.apache.spark.sql.types.{DataType, Decimal, DecimalType} -object RapidsErrorUtils { +object RapidsErrorUtils extends RapidsQueryErrorUtils { def invalidArrayIndexError(index: Int, numElements: Int, isElementAtF: Boolean = false): ArrayIndexOutOfBoundsException = { // Follow the Spark string format before 3.3.0 diff --git a/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala b/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala new file mode 100644 index 00000000000..266cb4ef54f --- /dev/null +++ b/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala @@ -0,0 +1,125 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed 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. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "311"} +{"spark": "312"} +{"spark": "313"} +spark-rapids-shim-json-lines ***/ + +package org.apache.spark.sql.rapids.shims + +import org.apache.hadoop.fs.Path +import org.apache.hadoop.hive.ql.ErrorMsg + +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.rapids.execution.RapidsAnalysisException +import org.apache.spark.sql.types.StructType + +trait RapidsQueryErrorUtils { + + def outputPathAlreadyExistsError(qualifiedOutputPath: Path): Throwable = { + new AnalysisException(s"path $qualifiedOutputPath already exists.") + } + + def createTableAsSelectWithNonEmptyDirectoryError(tablePath: String, conf: String): Throwable = { + new AnalysisException(s"CREATE-TABLE-AS-SELECT cannot create table with location to a " + + s"non-empty directory $tablePath. To allow overwriting the existing non-empty directory, " + + s"set '$conf' to true.") + } + + def cannotResolveAttributeError(name: String, outputStr: String): Throwable = { + new AnalysisException(s"Unable to resolve $name given [$outputStr]") + } + + def partitionColumnNotSpecifiedError(format: String, partitionColumn: String): Throwable = { + new AnalysisException(s"Failed to resolve the schema for $format for the partition column: " + + s"$partitionColumn. It must be specified manually.") + } + + def dataSchemaNotSpecifiedError(format: String): Throwable = { + new AnalysisException(s"Unable to infer schema for $format. It must be specified manually.") + } + + def schemaNotSpecifiedForSchemaRelationProviderError(className: String): Throwable = { + new AnalysisException(s"A schema needs to be specified when using $className.") + } + + def userSpecifiedSchemaMismatchActualSchemaError( + schema: StructType, + actualSchema: StructType): Throwable = { + new AnalysisException("The user-specified schema doesn't match the actual schema: " + + s"user-specified: ${schema.toDDL}, actual: ${actualSchema.toDDL}. If " + + "you're using DataFrameReader.schema API or creating a table, please do not " + + "specify the schema. Or if you're scanning an existed table, please drop " + + "it and re-create it.") + } + + def dataSchemaNotSpecifiedError(format: String, fileCatalog: String): Throwable = { + new AnalysisException(s"Unable to infer schema for $format at $fileCatalog. " + + "It must be specified manually") + } + + def invalidDataSourceError(className: String): Throwable = { + new AnalysisException(s"$className is not a valid Spark SQL Data Source.") + } + + def orcNotUsedWithHiveEnabledError(): Throwable = { + new AnalysisException( + s"Hive built-in ORC data source must be used with Hive support enabled. " + + s"Please use the native ORC data source by setting 'spark.sql.orc.impl' to 'native'.") + } + + def failedToFindAvroDataSourceError(provider: String): Throwable = { + new AnalysisException( + s"Failed to find data source: $provider. Avro is built-in but external data " + + "source module since Spark 2.4. Please deploy the application as per " + + "the deployment section of \"Apache Avro Data Source Guide\".") + } + + def failedToFindKafkaDataSourceError(provider: String): Throwable = { + new AnalysisException( + s"Failed to find data source: $provider. Please deploy the application as " + + "per the deployment section of " + + "\"Structured Streaming + Kafka Integration Guide\".") + } + + def findMultipleDataSourceError(provider: String, sourceNames: Seq[String]): Throwable = { + new AnalysisException( + s"Multiple sources found for $provider " + + s"(${sourceNames.mkString(", ")}), please specify the fully qualified class name.") + } + + def dataPathNotExistError(path: String): Throwable = { + new AnalysisException(s"Path does not exist: $path") + } + + def dynamicPartitionParentError: Throwable = { + throw new RapidsAnalysisException(ErrorMsg.PARTITION_DYN_STA_ORDER.getMsg) + } + + def tableOrViewAlreadyExistsError(tableName: String): Throwable = { + new AnalysisException(s"Table $tableName already exists. You need to drop it first.") + } + + def parquetTypeUnsupportedYetError(parquetType: String): Throwable = { + new AnalysisException(s"Parquet type not yet supported: $parquetType.") + } + + def illegalParquetTypeError(parquetType: String): Throwable = { + new AnalysisException(s"Illegal Parquet type: $parquetType.") + } +} \ No newline at end of file diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala index c3152a8a235..bba205f267f 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala @@ -29,7 +29,8 @@ import org.apache.parquet.schema.LogicalTypeAnnotation._ import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.rapids.execution.TrampolineUtil +import org.apache.spark.sql.rapids.execution.RapidsAnalysisException +import org.apache.spark.sql.rapids.shims.RapidsErrorUtils import org.apache.spark.sql.types._ object ParquetSchemaClipShims { @@ -67,10 +68,10 @@ object ParquetSchemaClipShims { if (typeAnnotation == null) s"$typeName" else s"$typeName ($typeAnnotation)" def typeNotImplemented() = - TrampolineUtil.throwAnalysisException(s"Parquet type not yet supported: $typeString") + throw RapidsErrorUtils.parquetTypeUnsupportedYetError(typeString) def illegalType() = - TrampolineUtil.throwAnalysisException(s"Illegal Parquet type: $typeString") + throw RapidsErrorUtils.illegalParquetTypeError(typeString) // When maxPrecision = -1, we skip precision range check, and always respect the precision // specified in field.getDecimalMetadata. This is useful when interpreting decimal types stored @@ -82,7 +83,7 @@ object ParquetSchemaClipShims { val scale = decimalLogicalTypeAnnotation.getScale if (!(maxPrecision == -1 || 1 <= precision && precision <= maxPrecision)) { - TrampolineUtil.throwAnalysisException( + throw new RapidsAnalysisException( s"Invalid decimal precision: $typeName " + s"cannot store $precision digits (max $maxPrecision)") } @@ -143,14 +144,14 @@ object ParquetSchemaClipShims { TimestampType case timestamp: TimestampLogicalTypeAnnotation if timestamp.getUnit == TimeUnit.NANOS && ParquetLegacyNanoAsLongShims.legacyParquetNanosAsLong => - TrampolineUtil.throwAnalysisException( + throw new RapidsAnalysisException( "GPU does not support spark.sql.legacy.parquet.nanosAsLong") case _ => illegalType() } case INT96 => if (!SQLConf.get.isParquetINT96AsTimestamp) { - TrampolineUtil.throwAnalysisException( + throw new RapidsAnalysisException( "INT96 is not supported unless it's interpreted as timestamp. " + s"Please try to set ${SQLConf.PARQUET_INT96_AS_TIMESTAMP.key} to true.") } diff --git a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala index b301397255a..68a6ce30569 100644 --- a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala +++ b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2023, NVIDIA CORPORATION. + * Copyright (c) 2022-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -29,7 +29,7 @@ import org.apache.spark.sql.catalyst.trees.Origin import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} import org.apache.spark.sql.types.{DataType, Decimal, DecimalType} -object RapidsErrorUtils { +object RapidsErrorUtils extends RapidsQueryErrorUtils { def invalidArrayIndexError(index: Int, numElements: Int, isElementAtF: Boolean = false): ArrayIndexOutOfBoundsException = { // Follow the Spark string format before 3.3.0 diff --git a/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala new file mode 100644 index 00000000000..dbc4145ee54 --- /dev/null +++ b/sql-plugin/src/main/spark320/scala/org/apache/spark/sql/rapids/shims/RapidsQueryErrorUtils.scala @@ -0,0 +1,127 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed 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. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "320"} +{"spark": "321"} +{"spark": "321cdh"} +{"spark": "322"} +{"spark": "323"} +{"spark": "324"} +{"spark": "330"} +{"spark": "330cdh"} +{"spark": "330db"} +{"spark": "331"} +{"spark": "332"} +{"spark": "332cdh"} +{"spark": "332db"} +{"spark": "333"} +{"spark": "334"} +{"spark": "340"} +{"spark": "341"} +{"spark": "341db"} +{"spark": "342"} +{"spark": "343"} +{"spark": "350"} +{"spark": "351"} +{"spark": "400"} +spark-rapids-shim-json-lines ***/ + +package org.apache.spark.sql.rapids.shims + +import org.apache.hadoop.fs.Path +import org.apache.hadoop.hive.ql.ErrorMsg + +import org.apache.spark.sql.errors.QueryCompilationErrors +import org.apache.spark.sql.rapids.execution.RapidsAnalysisException +import org.apache.spark.sql.types.StructType + +trait RapidsQueryErrorUtils { + + def outputPathAlreadyExistsError(qualifiedOutputPath: Path): Throwable = { + QueryCompilationErrors.outputPathAlreadyExistsError(qualifiedOutputPath) + } + + def createTableAsSelectWithNonEmptyDirectoryError(tablePath: String, conf: String): Throwable = { + QueryCompilationErrors.createTableAsSelectWithNonEmptyDirectoryError(tablePath) + } + + def cannotResolveAttributeError(name: String, outputStr: String): Throwable = { + QueryCompilationErrors.cannotResolveAttributeError(name, outputStr) + } + + def partitionColumnNotSpecifiedError(format: String, partitionColumn: String): Throwable = { + QueryCompilationErrors.partitionColumnNotSpecifiedError(format, partitionColumn) + } + + def dataSchemaNotSpecifiedError(format: String): Throwable = { + QueryCompilationErrors.dataSchemaNotSpecifiedError(format) + } + + def schemaNotSpecifiedForSchemaRelationProviderError(className: String): Throwable = { + QueryCompilationErrors.schemaNotSpecifiedForSchemaRelationProviderError(className) + } + + def userSpecifiedSchemaMismatchActualSchemaError( + schema: StructType, + actualSchema: StructType): Throwable = { + QueryCompilationErrors.userSpecifiedSchemaMismatchActualSchemaError(schema, actualSchema) + } + + def dataSchemaNotSpecifiedError(format: String, fileCatalog: String): Throwable = { + QueryCompilationErrors.dataSchemaNotSpecifiedError(format, fileCatalog) + } + + def invalidDataSourceError(className: String): Throwable = { + QueryCompilationErrors.invalidDataSourceError(className) + } + + def orcNotUsedWithHiveEnabledError(): Throwable = { + QueryCompilationErrors.orcNotUsedWithHiveEnabledError() + } + + def failedToFindAvroDataSourceError(provider: String): Throwable = { + QueryCompilationErrors.failedToFindAvroDataSourceError(provider) + } + + def failedToFindKafkaDataSourceError(provider: String): Throwable = { + QueryCompilationErrors.failedToFindKafkaDataSourceError(provider) + } + + def findMultipleDataSourceError(provider: String, sourceNames: Seq[String]): Throwable = { + QueryCompilationErrors.findMultipleDataSourceError(provider, sourceNames) + } + + def dataPathNotExistError(path: String): Throwable = { + QueryCompilationErrors.dataPathNotExistError(path) + } + + def tableOrViewAlreadyExistsError(tableName: String): Throwable = { + QueryCompilationErrors.tableOrViewAlreadyExistsError(tableName) + } + + def parquetTypeUnsupportedYetError(parquetType: String): Throwable = { + QueryCompilationErrors.parquetTypeUnsupportedYetError(parquetType) + } + + def illegalParquetTypeError(parquetType: String): Throwable = { + QueryCompilationErrors.illegalParquetTypeError(parquetType) + } + + def dynamicPartitionParentError: Throwable = { + throw new RapidsAnalysisException(ErrorMsg.PARTITION_DYN_STA_ORDER.getMsg) + } +} \ No newline at end of file diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala index 56708017a23..8c395274e07 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/ParquetSchemaClipShims.scala @@ -44,7 +44,8 @@ import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName._ import org.apache.spark.sql.execution.datasources.parquet.ParquetReadSupport.containsFieldIds import org.apache.spark.sql.execution.datasources.parquet.ParquetUtils import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.rapids.execution.TrampolineUtil +import org.apache.spark.sql.rapids.execution.RapidsAnalysisException +import org.apache.spark.sql.rapids.shims.RapidsErrorUtils import org.apache.spark.sql.types._ object ParquetSchemaClipShims { @@ -109,10 +110,11 @@ object ParquetSchemaClipShims { if (typeAnnotation == null) s"$typeName" else s"$typeName ($typeAnnotation)" def typeNotImplemented() = - TrampolineUtil.throwAnalysisException(s"Parquet type not yet supported: $typeString") + throw RapidsErrorUtils.parquetTypeUnsupportedYetError(typeString) def illegalType() = - TrampolineUtil.throwAnalysisException(s"Illegal Parquet type: $parquetType") + throw RapidsErrorUtils.illegalParquetTypeError(typeString) + // When maxPrecision = -1, we skip precision range check, and always respect the precision // specified in field.getDecimalMetadata. This is useful when interpreting decimal types stored @@ -124,7 +126,7 @@ object ParquetSchemaClipShims { val scale = decimalLogicalTypeAnnotation.getScale if (!(maxPrecision == -1 || 1 <= precision && precision <= maxPrecision)) { - TrampolineUtil.throwAnalysisException(s"Invalid decimal precision: $typeName " + + throw new RapidsAnalysisException(s"Invalid decimal precision: $typeName " + s"cannot store $precision digits (max $maxPrecision)") } @@ -183,14 +185,14 @@ object ParquetSchemaClipShims { ParquetTimestampAnnotationShims.timestampTypeForMillisOrMicros(timestamp) case timestamp: TimestampLogicalTypeAnnotation if timestamp.getUnit == TimeUnit.NANOS && ParquetLegacyNanoAsLongShims.legacyParquetNanosAsLong => - TrampolineUtil.throwAnalysisException( + throw new RapidsAnalysisException( "GPU does not support spark.sql.legacy.parquet.nanosAsLong") case _ => illegalType() } case INT96 => if (!SQLConf.get.isParquetINT96AsTimestamp) { - TrampolineUtil.throwAnalysisException( + throw new RapidsAnalysisException( "INT96 is not supported unless it's interpreted as timestamp. " + s"Please try to set ${SQLConf.PARQUET_INT96_AS_TIMESTAMP.key} to true.") } diff --git a/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala b/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala index bb28c370749..e5cdcd43568 100644 --- a/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala +++ b/sql-plugin/src/main/spark330/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2023, NVIDIA CORPORATION. + * Copyright (c) 2022-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -31,7 +31,7 @@ import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{DataType, Decimal, DecimalType} -object RapidsErrorUtils extends RapidsErrorUtilsFor330plus { +object RapidsErrorUtils extends RapidsErrorUtilsFor330plus with RapidsQueryErrorUtils { def mapKeyNotExistError( key: String, diff --git a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala index 1012b28d8b7..7e58a54c921 100644 --- a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala +++ b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2023, NVIDIA CORPORATION. + * Copyright (c) 2022-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -22,7 +22,7 @@ package org.apache.spark.sql.rapids.shims import org.apache.spark.sql.errors.QueryExecutionErrors -object RapidsErrorUtils extends RapidsErrorUtilsBase { +object RapidsErrorUtils extends RapidsErrorUtilsBase with RapidsQueryErrorUtils { def sqlArrayIndexNotStartAtOneError(): RuntimeException = { QueryExecutionErrors.elementAtByIndexZeroError(context = null) } diff --git a/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuInsertIntoHiveTable.scala b/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuInsertIntoHiveTable.scala index c8d76f85e5c..42fd5941025 100644 --- a/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuInsertIntoHiveTable.scala +++ b/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuInsertIntoHiveTable.scala @@ -37,7 +37,7 @@ import org.apache.hadoop.hive.ql.ErrorMsg import org.apache.hadoop.hive.ql.plan.TableDesc import org.apache.spark.SparkException -import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogTableType, ExternalCatalog, ExternalCatalogUtils, ExternalCatalogWithListener} import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan @@ -48,6 +48,7 @@ import org.apache.spark.sql.hive.HiveExternalCatalog import org.apache.spark.sql.hive.client.HiveClientImpl import org.apache.spark.sql.hive.execution.InsertIntoHiveTable import org.apache.spark.sql.hive.rapids.{GpuHiveFileFormat, GpuSaveAsHiveFile, RapidsHiveErrors} +import org.apache.spark.sql.rapids.shims.RapidsErrorUtils import org.apache.spark.sql.vectorized.ColumnarBatch final class GpuInsertIntoHiveTableMeta(cmd: InsertIntoHiveTable, @@ -182,7 +183,7 @@ case class GpuInsertIntoHiveTable( // Report error if any static partition appears after a dynamic partition val isDynamic = partitionColumnNames.map(partitionSpec(_).isEmpty) if (isDynamic.init.zip(isDynamic.tail).contains((true, false))) { - throw new AnalysisException(ErrorMsg.PARTITION_DYN_STA_ORDER.getMsg) + throw RapidsErrorUtils.dynamicPartitionParentError } } diff --git a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/GpuFileFormatWriter.scala b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/GpuFileFormatWriter.scala index 78daa0bf6f1..e7b3561f5fd 100644 --- a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/GpuFileFormatWriter.scala +++ b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/GpuFileFormatWriter.scala @@ -42,7 +42,7 @@ import org.apache.spark.{SparkException, TaskContext} import org.apache.spark.internal.Logging import org.apache.spark.internal.io.{FileCommitProtocol, SparkHadoopWriterUtils} import org.apache.spark.shuffle.FetchFailedException -import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.expressions.{Ascending, Attribute, AttributeSet, Expression, SortOrder} @@ -51,6 +51,7 @@ import org.apache.spark.sql.connector.write.WriterCommitMessage import org.apache.spark.sql.execution.{SparkPlan, SQLExecution} import org.apache.spark.sql.execution.datasources.{GpuWriteFiles, GpuWriteFilesExec, GpuWriteFilesSpec, WriteTaskResult, WriteTaskStats} import org.apache.spark.sql.execution.datasources.FileFormatWriter.OutputSpec +import org.apache.spark.sql.rapids.execution.RapidsAnalysisException import org.apache.spark.sql.types.StructType import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.{SerializableConfiguration, Utils} @@ -61,7 +62,7 @@ object GpuFileFormatWriter extends Logging { private def verifySchema(format: ColumnarFileFormat, schema: StructType): Unit = { schema.foreach { field => if (!format.supportDataType(field.dataType)) { - throw new AnalysisException( + throw new RapidsAnalysisException( s"$format data source does not support ${field.dataType.catalogString} data type.") } } diff --git a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/GpuCreateDataSourceTableAsSelectCommandShims.scala b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/GpuCreateDataSourceTableAsSelectCommandShims.scala index 9e36cf41fad..6308f24c552 100644 --- a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/GpuCreateDataSourceTableAsSelectCommandShims.scala +++ b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/shims/GpuCreateDataSourceTableAsSelectCommandShims.scala @@ -64,7 +64,7 @@ case class GpuCreateDataSourceTableAsSelectCommand( s"Expect the table $tableName has been dropped when the save mode is Overwrite") if (mode == SaveMode.ErrorIfExists) { - throw new AnalysisException(s"Table $tableName already exists. You need to drop it first.") + throw RapidsErrorUtils.tableOrViewAlreadyExistsError(tableName) } if (mode == SaveMode.Ignore) { // Since the table already exists and the save mode is Ignore, we will just return. diff --git a/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala b/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala index 8ee0485ab36..e6f8886f19c 100644 --- a/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala +++ b/sql-plugin/src/main/spark340/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{DataType, Decimal, DecimalType} -object RapidsErrorUtils extends RapidsErrorUtilsFor330plus { +object RapidsErrorUtils extends RapidsErrorUtilsFor330plus with RapidsQueryErrorUtils { def mapKeyNotExistError( key: String, diff --git a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala index a0ba17f9bd4..9b800d4e51a 100644 --- a/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala +++ b/sql-plugin/src/main/spark341db/scala/org/apache/spark/sql/rapids/shims/RapidsErrorUtils.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2023, NVIDIA CORPORATION. + * Copyright (c) 2022-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -21,7 +21,7 @@ package org.apache.spark.sql.rapids.shims import org.apache.spark.sql.errors.QueryExecutionErrors -object RapidsErrorUtils extends RapidsErrorUtilsBase { +object RapidsErrorUtils extends RapidsErrorUtilsBase with RapidsQueryErrorUtils { def sqlArrayIndexNotStartAtOneError(): RuntimeException = { QueryExecutionErrors.invalidIndexOfZeroError(context = null) } From 3111e2ba20c45ecb245e55b141a5dc2f9fe41d7d Mon Sep 17 00:00:00 2001 From: Raza Jafri Date: Tue, 4 Jun 2024 14:36:53 -0700 Subject: [PATCH 24/68] Move Support for `RaiseError` to a Shim Excluding Spark 4.0.0 [databricks] (#10970) * Incomplete impl of RaiseError for 400 * Removed RaiseError from 400 * Signing off Signed-off-by: Raza Jafri --------- Signed-off-by: Raza Jafri --- .../nvidia/spark/rapids/GpuOverrides.scala | 11 +--- .../spark/rapids/shims/RaiseErrorShim.scala | 62 +++++++++++++++++++ .../apache/spark/sql/rapids/shims}/misc.scala | 32 +++++++++- .../spark/rapids/shims/RaiseErrorShim.scala | 28 +++++++++ 4 files changed, 121 insertions(+), 12 deletions(-) create mode 100644 sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/RaiseErrorShim.scala rename sql-plugin/src/main/{scala/org/apache/spark/sql/rapids => spark311/scala/org/apache/spark/sql/rapids/shims}/misc.scala (75%) create mode 100644 sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/RaiseErrorShim.scala diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala index 55b883a479b..295480d24cc 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala @@ -3797,14 +3797,6 @@ object GpuOverrides extends Logging { TypeSig.ARRAY.nested(TypeSig.all)), (e, conf, p, r) => new GpuGetArrayStructFieldsMeta(e, conf, p, r) ), - expr[RaiseError]( - "Throw an exception", - ExprChecks.unaryProject( - TypeSig.NULL, TypeSig.NULL, - TypeSig.STRING, TypeSig.STRING), - (a, conf, p, r) => new UnaryExprMeta[RaiseError](a, conf, p, r) { - override def convertToGpu(child: Expression): GpuExpression = GpuRaiseError(child) - }), expr[DynamicPruningExpression]( "Dynamic pruning expression marker", ExprChecks.unaryProject(TypeSig.all, TypeSig.all, TypeSig.BOOLEAN, TypeSig.BOOLEAN), @@ -3820,7 +3812,8 @@ object GpuOverrides extends Logging { val expressions: Map[Class[_ <: Expression], ExprRule[_ <: Expression]] = commonExpressions ++ TimeStamp.getExprs ++ GpuHiveOverrides.exprs ++ ZOrderRules.exprs ++ DecimalArithmeticOverrides.exprs ++ - BloomFilterShims.exprs ++ InSubqueryShims.exprs ++ SparkShimImpl.getExprs + BloomFilterShims.exprs ++ InSubqueryShims.exprs ++ RaiseErrorShim.exprs ++ + SparkShimImpl.getExprs def wrapScan[INPUT <: Scan]( scan: INPUT, diff --git a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/RaiseErrorShim.scala b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/RaiseErrorShim.scala new file mode 100644 index 00000000000..de433d5f270 --- /dev/null +++ b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/RaiseErrorShim.scala @@ -0,0 +1,62 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed 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. + */ +/*** spark-rapids-shim-json-lines +{"spark": "311"} +{"spark": "312"} +{"spark": "313"} +{"spark": "320"} +{"spark": "321"} +{"spark": "321cdh"} +{"spark": "322"} +{"spark": "323"} +{"spark": "324"} +{"spark": "330"} +{"spark": "330cdh"} +{"spark": "330db"} +{"spark": "331"} +{"spark": "332"} +{"spark": "332cdh"} +{"spark": "332db"} +{"spark": "333"} +{"spark": "334"} +{"spark": "340"} +{"spark": "341"} +{"spark": "341db"} +{"spark": "342"} +{"spark": "343"} +{"spark": "350"} +{"spark": "351"} +spark-rapids-shim-json-lines ***/ +package com.nvidia.spark.rapids.shims + +import com.nvidia.spark.rapids.{ExprRule, GpuOverrides} +import com.nvidia.spark.rapids.{ExprChecks, GpuExpression, TypeSig, UnaryExprMeta} + +import org.apache.spark.sql.catalyst.expressions.{Expression, RaiseError} +import org.apache.spark.sql.rapids.shims.GpuRaiseError + +object RaiseErrorShim { + val exprs: Map[Class[_ <: Expression], ExprRule[_ <: Expression]] = { + Seq(GpuOverrides.expr[RaiseError]( + "Throw an exception", + ExprChecks.unaryProject( + TypeSig.NULL, TypeSig.NULL, + TypeSig.STRING, TypeSig.STRING), + (a, conf, p, r) => new UnaryExprMeta[RaiseError](a, conf, p, r) { + override def convertToGpu(child: Expression): GpuExpression = GpuRaiseError(child) + })).map(r => (r.getClassFor.asSubclass(classOf[Expression]), r)).toMap + } +} diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/misc.scala b/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/shims/misc.scala similarity index 75% rename from sql-plugin/src/main/scala/org/apache/spark/sql/rapids/misc.scala rename to sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/shims/misc.scala index b32bdfa207c..1ab58ddcbb6 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/misc.scala +++ b/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/shims/misc.scala @@ -13,10 +13,36 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +/*** spark-rapids-shim-json-lines +{"spark": "311"} +{"spark": "312"} +{"spark": "313"} +{"spark": "320"} +{"spark": "321"} +{"spark": "321cdh"} +{"spark": "322"} +{"spark": "323"} +{"spark": "324"} +{"spark": "330"} +{"spark": "330cdh"} +{"spark": "330db"} +{"spark": "331"} +{"spark": "332"} +{"spark": "332cdh"} +{"spark": "332db"} +{"spark": "333"} +{"spark": "334"} +{"spark": "340"} +{"spark": "341"} +{"spark": "341db"} +{"spark": "342"} +{"spark": "343"} +{"spark": "350"} +{"spark": "351"} +spark-rapids-shim-json-lines ***/ +package org.apache.spark.sql.rapids.shims -package org.apache.spark.sql.rapids - -import ai.rapids.cudf.{ColumnVector} +import ai.rapids.cudf.ColumnVector import com.nvidia.spark.rapids.{GpuColumnVector, GpuUnaryExpression} import com.nvidia.spark.rapids.Arm.withResource diff --git a/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/RaiseErrorShim.scala b/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/RaiseErrorShim.scala new file mode 100644 index 00000000000..70d40fc19a0 --- /dev/null +++ b/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/RaiseErrorShim.scala @@ -0,0 +1,28 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed 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. + */ +/*** spark-rapids-shim-json-lines +{"spark": "400"} +spark-rapids-shim-json-lines ***/ +package com.nvidia.spark.rapids.shims + +import com.nvidia.spark.rapids.{ExprRule, GpuOverrides} +import com.nvidia.spark.rapids.{ExprChecks, GpuExpression, TypeSig, UnaryExprMeta} + +import org.apache.spark.sql.catalyst.expressions.{Expression, RaiseError} + +object RaiseErrorShim { + val exprs: Map[Class[_ <: Expression], ExprRule[_ <: Expression]] = Map.empty +} From 149e0d58e50d815eb741c225f79aa60c2ea805b2 Mon Sep 17 00:00:00 2001 From: Liangcai Li Date: Wed, 5 Jun 2024 09:48:26 +0800 Subject: [PATCH 25/68] Fix a hive write test failure (#10958) This is a bug fix for the hive write tests. In some of the tests on Spak 351, the ProjectExec will fall back to CPU due to missing the GPU version of the MapFromArrays expression. This PR adds the ProjectExec to the allowed list of fallback for Spark 351 and the laters. Signed-off-by: Firestarman --- integration_tests/src/main/python/hive_parquet_write_test.py | 4 ++-- integration_tests/src/main/python/spark_session.py | 5 ++++- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/integration_tests/src/main/python/hive_parquet_write_test.py b/integration_tests/src/main/python/hive_parquet_write_test.py index e74a99f43c7..96976c3a356 100644 --- a/integration_tests/src/main/python/hive_parquet_write_test.py +++ b/integration_tests/src/main/python/hive_parquet_write_test.py @@ -19,7 +19,7 @@ from data_gen import * from hive_write_test import _restricted_timestamp from marks import allow_non_gpu, ignore_order -from spark_session import with_cpu_session, is_before_spark_320 +from spark_session import with_cpu_session, is_before_spark_320, is_spark_351_or_later # Disable the meta conversion from Hive write to FrameData write in Spark, to test # "GpuInsertIntoHiveTable" for Parquet write. @@ -55,7 +55,7 @@ _hive_write_gens = [_hive_basic_gens, _hive_struct_gens, _hive_array_gens, _hive_map_gens] # ProjectExec falls back on databricks due to no GPU version of "MapFromArrays". -fallback_nodes = ['ProjectExec'] if is_databricks_runtime() else [] +fallback_nodes = ['ProjectExec'] if is_databricks_runtime() or is_spark_351_or_later() else [] @allow_non_gpu(*(non_utc_allow + fallback_nodes)) diff --git a/integration_tests/src/main/python/spark_session.py b/integration_tests/src/main/python/spark_session.py index 78e0b08a651..c55f1976497 100644 --- a/integration_tests/src/main/python/spark_session.py +++ b/integration_tests/src/main/python/spark_session.py @@ -1,4 +1,4 @@ -# Copyright (c) 2020-2023, NVIDIA CORPORATION. +# Copyright (c) 2020-2024, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -220,6 +220,9 @@ def is_spark_341(): def is_spark_350_or_later(): return spark_version() >= "3.5.0" +def is_spark_351_or_later(): + return spark_version() >= "3.5.1" + def is_spark_330(): return spark_version() == "3.3.0" From d51499973e54d2dde61855c24d512cb95ae65223 Mon Sep 17 00:00:00 2001 From: Tim Liu Date: Wed, 5 Jun 2024 12:15:51 +0800 Subject: [PATCH 26/68] Speed up the integration tests by running them in parallel on the Databricks cluster (#10953) * Speed up the integration tests by running them in parallel on the Databricks cluster Separate the integration tests into two parts to speed up the testing process. "part2" includes integration tests : "cache_test, delta_lake_test, shuffle_test and pyarrow_test" "part1" includes all other tests except for the ones in "part1" Run each part in parallel on separate Databricks clusters. Signed-off-by: Tim Liu * TEST_MODE variables on Databrick to run integration tests for CI jobs Signed-off-by: Tim Liu --------- Signed-off-by: Tim Liu --- jenkins/Jenkinsfile-blossom.premerge | 33 ++++++++++++++++--- .../Jenkinsfile-blossom.premerge-databricks | 4 ++- jenkins/databricks/test.sh | 23 +++++++------ 3 files changed, 43 insertions(+), 17 deletions(-) diff --git a/jenkins/Jenkinsfile-blossom.premerge b/jenkins/Jenkinsfile-blossom.premerge index fb3a5c2f328..d61638d901a 100755 --- a/jenkins/Jenkinsfile-blossom.premerge +++ b/jenkins/Jenkinsfile-blossom.premerge @@ -274,7 +274,7 @@ git --no-pager diff --name-only HEAD \$BASE -- ${PREMERGE_DOCKERFILE} || true""" } } // end of Unit Test stage - stage('Databricks') { + stage('Databricks IT part1') { when { expression { db_build } } @@ -285,17 +285,42 @@ git --no-pager diff --name-only HEAD \$BASE -- ${PREMERGE_DOCKERFILE} || true""" propagate: false, wait: true, parameters: [ string(name: 'REF', value: params.REF), - string(name: 'GITHUB_DATA', value: params.GITHUB_DATA) + string(name: 'GITHUB_DATA', value: params.GITHUB_DATA), + string(name: 'TEST_MODE', value: 'CI_PART1') ]) if ( DBJob.result != 'SUCCESS' ) { // Output Databricks failure logs to uploaded onto the pre-merge PR print(DBJob.getRawBuild().getLog()) // Fail the pipeline - error "Databricks build result : " + DBJob.result + error "Databricks part1 result : " + DBJob.result } } } - } // end of Databricks + } // end of Databricks IT part1 + + stage('Databricks IT part2') { + when { + expression { db_build } + } + steps { + script { + githubHelper.updateCommitStatus("", "Running - includes databricks", GitHubCommitState.PENDING) + def DBJob = build(job: 'rapids-databricks_premerge-github', + propagate: false, wait: true, + parameters: [ + string(name: 'REF', value: params.REF), + string(name: 'GITHUB_DATA', value: params.GITHUB_DATA), + string(name: 'TEST_MODE', value: 'CI_PART2') + ]) + if ( DBJob.result != 'SUCCESS' ) { + // Output Databricks failure logs to uploaded onto the pre-merge PR + print(DBJob.getRawBuild().getLog()) + // Fail the pipeline + error "Databricks part2 result : " + DBJob.result + } + } + } + } // end of Databricks IT part2 stage('Dummy stage: blue ocean log view') { steps { diff --git a/jenkins/Jenkinsfile-blossom.premerge-databricks b/jenkins/Jenkinsfile-blossom.premerge-databricks index cd872fd9153..5b0a2bf1226 100644 --- a/jenkins/Jenkinsfile-blossom.premerge-databricks +++ b/jenkins/Jenkinsfile-blossom.premerge-databricks @@ -51,6 +51,8 @@ pipeline { description: 'Merged commit of specific PR') string(name: 'GITHUB_DATA', defaultValue: '', description: 'Json-formatted github data from upstream blossom-ci') + choice(name: 'TEST_MODE', choices: ['CI_PART1', 'CI_PART2'], + description: 'Separate integration tests into 2 parts, and run each part in parallell') } environment { @@ -178,7 +180,7 @@ void databricksBuild() { container('cpu') { try { withCredentials([file(credentialsId: 'SPARK_DATABRICKS_PRIVKEY', variable: 'DATABRICKS_PRIVKEY')]) { - def TEST_PARAMS = " -w $DATABRICKS_HOST -t $DATABRICKS_TOKEN -c $CLUSTER_ID" + + def TEST_PARAMS = " -w $DATABRICKS_HOST -t $DATABRICKS_TOKEN -c $CLUSTER_ID -e TEST_MODE=$TEST_MODE" + " -p $DATABRICKS_PRIVKEY -l ./jenkins/databricks/test.sh -v $BASE_SPARK_VERSION -d /home/ubuntu/test.sh" if (params.SPARK_CONF) { TEST_PARAMS += " -f ${params.SPARK_CONF}" diff --git a/jenkins/databricks/test.sh b/jenkins/databricks/test.sh index 404dcd97578..f71f69844f7 100755 --- a/jenkins/databricks/test.sh +++ b/jenkins/databricks/test.sh @@ -1,6 +1,6 @@ #!/bin/bash # -# Copyright (c) 2020-2023, NVIDIA CORPORATION. All rights reserved. +# Copyright (c) 2020-2024, NVIDIA CORPORATION. All rights reserved. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -59,6 +59,7 @@ IS_SPARK_321_OR_LATER=0 # - DELTA_LAKE_ONLY: delta_lake tests only # - MULTITHREADED_SHUFFLE: shuffle tests only # - PYARROW_ONLY: pyarrow tests only +# - CI_PART1 or CI_PART2 : part1 or part2 of the tests run in parallel from CI TEST_MODE=${TEST_MODE:-'DEFAULT'} # Classloader config is here to work around classloader issues with @@ -89,32 +90,30 @@ run_pyarrow_tests() { bash integration_tests/run_pyspark_from_build.sh -m pyarrow_test --pyarrow_test --runtime_env="databricks" --test_type=$TEST_TYPE } -## limit parallelism to avoid OOM kill -export TEST_PARALLEL=${TEST_PARALLEL:-4} - -if [[ $TEST_MODE == "DEFAULT" ]]; then +## Separate the integration tests into "CI_PART1" and "CI_PART2", run each part in parallel on separate Databricks clusters to speed up the testing process. +if [[ $TEST_MODE == "DEFAULT" || $TEST_MODE == "CI_PART1" ]]; then bash integration_tests/run_pyspark_from_build.sh --runtime_env="databricks" --test_type=$TEST_TYPE +fi +## Run tests with jars building from the spark-rapids source code +if [[ "$(pwd)" == "$SOURCE_PATH" ]]; then ## Run cache tests - if [[ "$IS_SPARK_321_OR_LATER" -eq "1" ]]; then + if [[ "$IS_SPARK_321_OR_LATER" -eq "1" && ("$TEST_MODE" == "DEFAULT" || $TEST_MODE == "CI_PART2") ]]; then PYSP_TEST_spark_sql_cache_serializer=${PCBS_CONF} \ bash integration_tests/run_pyspark_from_build.sh --runtime_env="databricks" --test_type=$TEST_TYPE -k cache_test fi -fi -## Run tests with jars building from the spark-rapids source code -if [ "$(pwd)" == "$SOURCE_PATH" ]; then - if [[ "$TEST_MODE" == "DEFAULT" || "$TEST_MODE" == "DELTA_LAKE_ONLY" ]]; then + if [[ "$TEST_MODE" == "DEFAULT" || $TEST_MODE == "CI_PART2" || "$TEST_MODE" == "DELTA_LAKE_ONLY" ]]; then ## Run Delta Lake tests SPARK_SUBMIT_FLAGS="$SPARK_CONF $DELTA_LAKE_CONFS" TEST_PARALLEL=1 \ bash integration_tests/run_pyspark_from_build.sh --runtime_env="databricks" -m "delta_lake" --delta_lake --test_type=$TEST_TYPE fi - if [[ "$TEST_MODE" == "DEFAULT" || "$TEST_MODE" == "MULTITHREADED_SHUFFLE" ]]; then + if [[ "$TEST_MODE" == "DEFAULT" || $TEST_MODE == "CI_PART2" || "$TEST_MODE" == "MULTITHREADED_SHUFFLE" ]]; then ## Mutithreaded Shuffle test rapids_shuffle_smoke_test fi - if [[ "$TEST_MODE" == "DEFAULT" || "$TEST_MODE" == "PYARROW_ONLY" ]]; then + if [[ "$TEST_MODE" == "DEFAULT" || $TEST_MODE == "CI_PART2" || "$TEST_MODE" == "PYARROW_ONLY" ]]; then # Pyarrow tests run_pyarrow_tests fi From 4d3b34679bceb9c3b515ffdb024fa9b52033824f Mon Sep 17 00:00:00 2001 From: Raza Jafri Date: Wed, 5 Jun 2024 01:09:09 -0700 Subject: [PATCH 27/68] More compilation fixes for Spark 4.0.0 [databricks] (#10978) * More compilation fixes * Signing off Signed-off-by: Raza Jafri * Added shim for UnsafeCudfRow * POM changes Signed-off-by: Raza Jafri * Added shim for UnsafeCudfRow * Revert "POM changes" This reverts commit dc4823b2c87d748dea50a76623d673f5adead8c3. --------- Signed-off-by: Raza Jafri --- .../nvidia/spark/rapids/CudfUnsafeRow.java | 400 ------------------ .../InternalRowToColumnarBatchIterator.java | 1 + .../spark/rapids/GpuColumnarToRowExec.scala | 4 +- .../spark/rapids/GpuRowToColumnarExec.scala | 2 +- .../spark/rapids/shims/CudfUnsafeRow.scala | 51 +++ .../rapids/shims/CudfUnsafeRowBase.scala | 245 +++++++++++ .../rapids/shims/GpuShuffleExchangeExec.scala | 1 + .../spark/rapids/shims/GpuOrcDataReader.scala | 88 +--- .../shims/GpuOrcDataReader320Plus.scala | 130 ++++++ .../spark/rapids/shims/CudfUnsafeRow.scala | 33 ++ .../spark/rapids/shims/GpuBatchScanExec.scala | 2 +- .../spark/rapids/shims/GpuOrcDataReader.scala | 38 ++ 12 files changed, 505 insertions(+), 490 deletions(-) delete mode 100644 sql-plugin/src/main/java/com/nvidia/spark/rapids/CudfUnsafeRow.java create mode 100644 sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/CudfUnsafeRow.scala create mode 100644 sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/CudfUnsafeRowBase.scala create mode 100644 sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuOrcDataReader320Plus.scala create mode 100644 sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/CudfUnsafeRow.scala create mode 100644 sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/GpuOrcDataReader.scala diff --git a/sql-plugin/src/main/java/com/nvidia/spark/rapids/CudfUnsafeRow.java b/sql-plugin/src/main/java/com/nvidia/spark/rapids/CudfUnsafeRow.java deleted file mode 100644 index d25500a77b2..00000000000 --- a/sql-plugin/src/main/java/com/nvidia/spark/rapids/CudfUnsafeRow.java +++ /dev/null @@ -1,400 +0,0 @@ -/* - * Copyright (c) 2020-2021, NVIDIA CORPORATION. - * - * Licensed 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 com.nvidia.spark.rapids; - -import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.catalyst.expressions.Attribute; -import org.apache.spark.sql.catalyst.expressions.SpecializedGettersReader; -import org.apache.spark.sql.catalyst.util.ArrayData; -import org.apache.spark.sql.catalyst.util.MapData; -import org.apache.spark.sql.types.DataType; -import org.apache.spark.sql.types.Decimal; -import org.apache.spark.unsafe.Platform; -import org.apache.spark.unsafe.array.ByteArrayMethods; -import org.apache.spark.unsafe.hash.Murmur3_x86_32; -import org.apache.spark.unsafe.types.CalendarInterval; -import org.apache.spark.unsafe.types.UTF8String; - -import java.util.Arrays; - -/** - * This is an InternalRow implementation based off of UnsafeRow, but follows a format for use with - * the row format supported by cudf. In this format each column is padded to match the alignment - * needed by it, and validity is placed at the end one byte at a time. - * - * It also supports remapping the columns so that if the columns were re-ordered to reduce packing - * in the format, then they can be mapped back to their original positions. - * - * This class is likely to go away once we move to code generation when going directly to an - * UnsafeRow through code generation. This is rather difficult because of some details in how - * UnsafeRow works. - */ -public final class CudfUnsafeRow extends InternalRow { - public static int alignOffset(int offset, int alignment) { - return (offset + alignment - 1) & -alignment; - } - - public static int calculateBitSetWidthInBytes(int numFields) { - return (numFields + 7)/ 8; - } - - public static int getRowSizeEstimate(Attribute[] attributes) { - // This needs to match what is in cudf and what is in the constructor. - int offset = 0; - for (Attribute attr : attributes) { - int length = GpuColumnVector.getNonNestedRapidsType(attr.dataType()).getSizeInBytes(); - offset = alignOffset(offset, length); - offset += length; - } - int bitSetWidthInBytes = calculateBitSetWidthInBytes(attributes.length); - // Each row is 64-bit aligned - return alignOffset(offset + bitSetWidthInBytes, 8); - } - - ////////////////////////////////////////////////////////////////////////////// - // Private fields and methods - ////////////////////////////////////////////////////////////////////////////// - - /** - * Address of where the row is stored in off heap memory. - */ - private long address; - - /** - * For each column the starting location to read from. The index to the is the position in - * the row bytes, not the user faceing ordinal. - */ - private int[] startOffsets; - - /** - * At what point validity data starts. - */ - private int fixedWidthSizeInBytes; - - /** - * The size of this row's backing data, in bytes. - */ - private int sizeInBytes; - - /** - * A mapping from the user facing ordinal to the index in the underlying row. - */ - private int[] remapping; - - /** - * Get the address where a field is stored. - * @param ordinal the user facing ordinal. - * @return the address of the field. - */ - private long getFieldAddressFromOrdinal(int ordinal) { - assertIndexIsValid(ordinal); - int i = remapping[ordinal]; - return address + startOffsets[i]; - } - - /** - * Verify that index is valid for this row. - * @param index in this case the index can be either the user facing ordinal or the index into the - * row. - */ - private void assertIndexIsValid(int index) { - assert index >= 0 : "index (" + index + ") should >= 0"; - assert index < startOffsets.length : "index (" + index + ") should < " + startOffsets.length; - } - - ////////////////////////////////////////////////////////////////////////////// - // Public methods - ////////////////////////////////////////////////////////////////////////////// - - /** - * Construct a new Row. The resulting row won't be usable until `pointTo()` has been called, - * since the value returned by this constructor is equivalent to a null pointer. - * - * @param attributes the schema of what this will hold. This is the schema of the underlying - * row, so if columns were re-ordered it is the attributes of the reordered - * data. - * @param remapping a mapping from the user requested column to the underlying column in the - * backing row. - */ - public CudfUnsafeRow(Attribute[] attributes, int[] remapping) { - int offset = 0; - startOffsets = new int[attributes.length]; - for (int i = 0; i < attributes.length; i++) { - Attribute attr = attributes[i]; - int length = GpuColumnVector.getNonNestedRapidsType(attr.dataType()).getSizeInBytes(); - assert length > 0 : "Only fixed width types are currently supported."; - offset = alignOffset(offset, length); - startOffsets[i] = offset; - offset += length; - } - fixedWidthSizeInBytes = offset; - this.remapping = remapping; - assert startOffsets.length == remapping.length; - } - - // for serializer - public CudfUnsafeRow() {} - - @Override - public int numFields() { return startOffsets.length; } - - /** - * Update this CudfUnsafeRow to point to different backing data. - * - * @param address the address in host memory for this. We should change this to be a - * MemoryBuffer class or something like that. - * @param sizeInBytes the size of this row's backing data, in bytes - */ - public void pointTo(long address, int sizeInBytes) { - assert startOffsets != null && startOffsets.length > 0 : "startOffsets not properly initialized"; - assert sizeInBytes % 8 == 0 : "sizeInBytes (" + sizeInBytes + ") should be a multiple of 8"; - this.address = address; - this.sizeInBytes = sizeInBytes; - } - - @Override - public void update(int ordinal, Object value) { - throw new UnsupportedOperationException(); - } - - @Override - public Object get(int ordinal, DataType dataType) { - // Don't remap the ordinal because it will be remapped in each of the other backing APIs - return SpecializedGettersReader.read(this, ordinal, dataType, true, true); - } - - @Override - public boolean isNullAt(int ordinal) { - int i = remapping[ordinal]; - assertIndexIsValid(i); - int validByteIndex = i / 8; - int validBitIndex = i % 8; - byte b = Platform.getByte(null, address + fixedWidthSizeInBytes + validByteIndex); - return ((1 << validBitIndex) & b) == 0; - } - - @Override - public void setNullAt(int ordinal) { - int i = remapping[ordinal]; - assertIndexIsValid(i); - int validByteIndex = i / 8; - int validBitIndex = i % 8; - byte b = Platform.getByte(null, address + fixedWidthSizeInBytes + validByteIndex); - b = (byte)((b & ~(1 << validBitIndex)) & 0xFF); - Platform.putByte(null, address + fixedWidthSizeInBytes + validByteIndex, b); - } - - @Override - public boolean getBoolean(int ordinal) { - return Platform.getBoolean(null, getFieldAddressFromOrdinal(ordinal)); - } - - @Override - public byte getByte(int ordinal) { - return Platform.getByte(null, getFieldAddressFromOrdinal(ordinal)); - } - - @Override - public short getShort(int ordinal) { - return Platform.getShort(null, getFieldAddressFromOrdinal(ordinal)); - } - - @Override - public int getInt(int ordinal) { - return Platform.getInt(null, getFieldAddressFromOrdinal(ordinal)); - } - - @Override - public long getLong(int ordinal) { - return Platform.getLong(null, getFieldAddressFromOrdinal(ordinal)); - } - - @Override - public float getFloat(int ordinal) { - return Platform.getFloat(null, getFieldAddressFromOrdinal(ordinal)); - } - - @Override - public double getDouble(int ordinal) { - return Platform.getDouble(null, getFieldAddressFromOrdinal(ordinal)); - } - - @Override - public Decimal getDecimal(int ordinal, int precision, int scale) { - if (isNullAt(ordinal)) { - return null; - } - if (precision <= Decimal.MAX_INT_DIGITS()) { - return Decimal.createUnsafe(getInt(ordinal), precision, scale); - } else if (precision <= Decimal.MAX_LONG_DIGITS()) { - return Decimal.createUnsafe(getLong(ordinal), precision, scale); - } else { - throw new IllegalArgumentException("NOT IMPLEMENTED YET"); -// byte[] bytes = getBinary(ordinal); -// BigInteger bigInteger = new BigInteger(bytes); -// BigDecimal javaDecimal = new BigDecimal(bigInteger, scale); -// return Decimal.apply(javaDecimal, precision, scale); - } - } - - @Override - public UTF8String getUTF8String(int ordinal) { -// if (isNullAt(ordinal)) return null; -// final long offsetAndSize = getLong(ordinal); -// final int offset = (int) (offsetAndSize >> 32); -// final int size = (int) offsetAndSize; -// return UTF8String.fromAddress(null, address + offset, size); - throw new IllegalArgumentException("NOT IMPLEMENTED YET"); - } - - @Override - public byte[] getBinary(int ordinal) { -// if (isNullAt(ordinal)) { -// return null; -// } else { -// final long offsetAndSize = getLong(ordinal); -// final int offset = (int) (offsetAndSize >> 32); -// final int size = (int) offsetAndSize; -// final byte[] bytes = new byte[size]; -// Platform.copyMemory( -// null, -// address + offset, -// bytes, -// Platform.BYTE_ARRAY_OFFSET, -// size -// ); -// return bytes; -// } - throw new IllegalArgumentException("NOT IMPLEMENTED YET"); - } - - @Override - public CalendarInterval getInterval(int ordinal) { -// if (isNullAt(ordinal)) { -// return null; -// } else { -// final long offsetAndSize = getLong(ordinal); -// final int offset = (int) (offsetAndSize >> 32); -// final int months = Platform.getInt(baseObject, address + offset); -// final int days = Platform.getInt(baseObject, address + offset + 4); -// final long microseconds = Platform.getLong(baseObject, address + offset + 8); -// return new CalendarInterval(months, days, microseconds); -// } - throw new IllegalArgumentException("NOT IMPLEMENTED YET"); - } - - @Override - public CudfUnsafeRow getStruct(int ordinal, int numFields) { -// if (isNullAt(ordinal)) { -// return null; -// } else { -// final long offsetAndSize = getLong(ordinal); -// final int offset = (int) (offsetAndSize >> 32); -// final int size = (int) offsetAndSize; -// final UnsafeRow row = new UnsafeRow(numFields); -// row.pointTo(baseObject, address + offset, size); -// return row; -// } - throw new IllegalArgumentException("NOT IMPLEMENTED YET"); - } - - @Override - public ArrayData getArray(int ordinal) { -// if (isNullAt(ordinal)) { -// return null; -// } else { -// final long offsetAndSize = getLong(ordinal); -// final int offset = (int) (offsetAndSize >> 32); -// final int size = (int) offsetAndSize; -// final UnsafeArrayData array = new UnsafeArrayData(); -// array.pointTo(baseObject, address + offset, size); -// return array; -// } - throw new IllegalArgumentException("NOT IMPLEMENTED YET"); - } - - @Override - public MapData getMap(int ordinal) { -// if (isNullAt(ordinal)) { -// return null; -// } else { -// final long offsetAndSize = getLong(ordinal); -// final int offset = (int) (offsetAndSize >> 32); -// final int size = (int) offsetAndSize; -// final UnsafeMapData map = new UnsafeMapData(); -// map.pointTo(baseObject, address + offset, size); -// return map; -// } - throw new IllegalArgumentException("NOT IMPLEMENTED YET"); - } - - /** - * Copies this row, returning a self-contained UnsafeRow that stores its data in an internal - * byte array rather than referencing data stored in a data page. - */ - @Override - public CudfUnsafeRow copy() { -// UnsafeRow rowCopy = new UnsafeRow(numFields); -// final byte[] rowDataCopy = new byte[sizeInBytes]; -// Platform.copyMemory( -// baseObject, -// address, -// rowDataCopy, -// Platform.BYTE_ARRAY_OFFSET, -// sizeInBytes -// ); -// rowCopy.pointTo(rowDataCopy, Platform.BYTE_ARRAY_OFFSET, sizeInBytes); -// return rowCopy; - throw new IllegalArgumentException("NOT IMPLEMENTED YET"); - } - - @Override - public int hashCode() { - return Murmur3_x86_32.hashUnsafeWords(null, address, sizeInBytes, 42); - } - - @Override - public boolean equals(Object other) { - if (other instanceof CudfUnsafeRow) { - CudfUnsafeRow o = (CudfUnsafeRow) other; - return (sizeInBytes == o.sizeInBytes) && - ByteArrayMethods.arrayEquals(null, address, null, o.address, sizeInBytes) && - Arrays.equals(remapping, o.remapping); - } - return false; - } - - // This is for debugging - @Override - public String toString() { - StringBuilder build = new StringBuilder("["); - for (int i = 0; i < sizeInBytes; i += 8) { - if (i != 0) build.append(','); - build.append(java.lang.Long.toHexString(Platform.getLong(null, address + i))); - } - build.append(']'); - build.append(" remapped with "); - build.append(Arrays.toString(remapping)); - return build.toString(); - } - - @Override - public boolean anyNull() { - throw new IllegalArgumentException("NOT IMPLEMENTED YET"); -// return BitSetMethods.anySet(baseObject, address, bitSetWidthInBytes / 8); - } -} \ No newline at end of file diff --git a/sql-plugin/src/main/java/com/nvidia/spark/rapids/InternalRowToColumnarBatchIterator.java b/sql-plugin/src/main/java/com/nvidia/spark/rapids/InternalRowToColumnarBatchIterator.java index 9e532ba394a..0aa3f0978e9 100644 --- a/sql-plugin/src/main/java/com/nvidia/spark/rapids/InternalRowToColumnarBatchIterator.java +++ b/sql-plugin/src/main/java/com/nvidia/spark/rapids/InternalRowToColumnarBatchIterator.java @@ -35,6 +35,7 @@ import ai.rapids.cudf.NvtxRange; import ai.rapids.cudf.Table; import com.nvidia.spark.rapids.jni.RowConversion; +import com.nvidia.spark.rapids.shims.CudfUnsafeRow; import org.apache.spark.TaskContext; import org.apache.spark.sql.catalyst.InternalRow; diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuColumnarToRowExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuColumnarToRowExec.scala index 38b235a36f6..694d6dabbd6 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuColumnarToRowExec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuColumnarToRowExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019-2023, NVIDIA CORPORATION. + * Copyright (c) 2019-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -25,7 +25,7 @@ import com.nvidia.spark.rapids.RapidsPluginImplicits._ import com.nvidia.spark.rapids.RmmRapidsRetryIterator.{splitSpillableInHalfByRows, withRetryNoSplit} import com.nvidia.spark.rapids.ScalableTaskCompletion.onTaskCompletion import com.nvidia.spark.rapids.jni.RowConversion -import com.nvidia.spark.rapids.shims.ShimUnaryExecNode +import com.nvidia.spark.rapids.shims.{CudfUnsafeRow, ShimUnaryExecNode} import org.apache.spark.TaskContext import org.apache.spark.rdd.RDD diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuRowToColumnarExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuRowToColumnarExec.scala index 99f17cf341a..51b6645d7b7 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuRowToColumnarExec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuRowToColumnarExec.scala @@ -19,7 +19,7 @@ package com.nvidia.spark.rapids import ai.rapids.cudf.{NvtxColor, NvtxRange} import com.nvidia.spark.rapids.Arm.withResource import com.nvidia.spark.rapids.GpuColumnVector.GpuColumnarBatchBuilder -import com.nvidia.spark.rapids.shims.{GpuTypeShims, ShimUnaryExecNode} +import com.nvidia.spark.rapids.shims.{CudfUnsafeRow, GpuTypeShims, ShimUnaryExecNode} import org.apache.spark.TaskContext import org.apache.spark.broadcast.Broadcast diff --git a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/CudfUnsafeRow.scala b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/CudfUnsafeRow.scala new file mode 100644 index 00000000000..c04d3b2db29 --- /dev/null +++ b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/CudfUnsafeRow.scala @@ -0,0 +1,51 @@ +/* + * Copyright (c) 2020-2024, NVIDIA CORPORATION. + * + * Licensed 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. + */ +/*** spark-rapids-shim-json-lines +{"spark": "311"} +{"spark": "312"} +{"spark": "313"} +{"spark": "320"} +{"spark": "321"} +{"spark": "321cdh"} +{"spark": "322"} +{"spark": "323"} +{"spark": "324"} +{"spark": "330"} +{"spark": "330cdh"} +{"spark": "330db"} +{"spark": "331"} +{"spark": "332"} +{"spark": "332cdh"} +{"spark": "332db"} +{"spark": "333"} +{"spark": "334"} +{"spark": "340"} +{"spark": "341"} +{"spark": "341db"} +{"spark": "342"} +{"spark": "343"} +{"spark": "350"} +{"spark": "351"} +spark-rapids-shim-json-lines ***/ +package com.nvidia.spark.rapids.shims + +import org.apache.spark.sql.catalyst.expressions.Attribute + +final class CudfUnsafeRow( + attributes: Array[Attribute], + remapping: Array[Int]) extends CudfUnsafeRowBase(attributes, remapping) + +object CudfUnsafeRow extends CudfUnsafeRowTrait \ No newline at end of file diff --git a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/CudfUnsafeRowBase.scala b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/CudfUnsafeRowBase.scala new file mode 100644 index 00000000000..e5e0bbd3dc6 --- /dev/null +++ b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/CudfUnsafeRowBase.scala @@ -0,0 +1,245 @@ +/* + * Copyright (c) 2020-2024, NVIDIA CORPORATION. + * + * Licensed 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. + */ +/*** spark-rapids-shim-json-lines +{"spark": "311"} +{"spark": "312"} +{"spark": "313"} +{"spark": "320"} +{"spark": "321"} +{"spark": "321cdh"} +{"spark": "322"} +{"spark": "323"} +{"spark": "324"} +{"spark": "330"} +{"spark": "330cdh"} +{"spark": "330db"} +{"spark": "331"} +{"spark": "332"} +{"spark": "332cdh"} +{"spark": "332db"} +{"spark": "333"} +{"spark": "334"} +{"spark": "340"} +{"spark": "341"} +{"spark": "341db"} +{"spark": "342"} +{"spark": "343"} +{"spark": "350"} +{"spark": "351"} +{"spark": "400"} +spark-rapids-shim-json-lines ***/ +package com.nvidia.spark.rapids.shims + +import java.util.Arrays + +import com.nvidia.spark.rapids.GpuColumnVector + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.expressions.SpecializedGettersReader +import org.apache.spark.sql.catalyst.util.ArrayData +import org.apache.spark.sql.catalyst.util.MapData +import org.apache.spark.sql.types.DataType +import org.apache.spark.sql.types.Decimal +import org.apache.spark.unsafe.Platform +import org.apache.spark.unsafe.array.ByteArrayMethods +import org.apache.spark.unsafe.hash.Murmur3_x86_32 +import org.apache.spark.unsafe.types.CalendarInterval +import org.apache.spark.unsafe.types.UTF8String + +abstract class CudfUnsafeRowBase( + protected val attributes: Array[Attribute], + protected val remapping: Array[Int]) extends InternalRow { + protected var address: Long = _ + private var startOffsets: Array[Int] = _ + private var fixedWidthSizeInBytes: Int = _ + protected var sizeInBytes: Int = _ + + def this() = this(null, null) + + init(attributes, remapping) + + private def init(attributes: Array[Attribute], remapping: Array[Int]): Unit = { + var offset = 0 + startOffsets = new Array[Int](attributes.length) + for (i <- attributes.indices) { + val attr = attributes(i) + val length = GpuColumnVector.getNonNestedRapidsType(attr.dataType).getSizeInBytes + assert(length > 0, "Only fixed width types are currently supported.") + offset = CudfUnsafeRow.alignOffset(offset, length) + startOffsets(i) = offset + offset += length + } + fixedWidthSizeInBytes = offset + assert(startOffsets.length == remapping.length) + } + + override def numFields: Int = startOffsets.length + + def pointTo(address: Long, sizeInBytes: Int): Unit = { + assert(startOffsets != null && startOffsets.length > 0, "startOffsets not properly initialized") + assert(sizeInBytes % 8 == 0, s"sizeInBytes ($sizeInBytes) should be a multiple of 8") + this.address = address + this.sizeInBytes = sizeInBytes + } + + override def update(ordinal: Int, value: Any): Unit = throw new UnsupportedOperationException() + + override def get(ordinal: Int, dataType: DataType): Object = { + SpecializedGettersReader.read(this, ordinal, dataType, true, true) + } + + override def isNullAt(ordinal: Int): Boolean = { + val i = remapping(ordinal) + assertIndexIsValid(i) + val validByteIndex = i / 8 + val validBitIndex = i % 8 + val b = Platform.getByte(null, address + fixedWidthSizeInBytes + validByteIndex) + ((1 << validBitIndex) & b) == 0 + } + + override def setNullAt(ordinal: Int): Unit = { + val i = remapping(ordinal) + assertIndexIsValid(i) + val validByteIndex = i / 8 + val validBitIndex = i % 8 + var b = Platform.getByte(null, address + fixedWidthSizeInBytes + validByteIndex) + b = (b & ~(1 << validBitIndex)).toByte + Platform.putByte(null, address + fixedWidthSizeInBytes + validByteIndex, b) + } + + override def getBoolean(ordinal: Int): Boolean = { + Platform.getBoolean(null, getFieldAddressFromOrdinal(ordinal)) + } + + override def getByte(ordinal: Int): Byte = { + Platform.getByte(null, getFieldAddressFromOrdinal(ordinal)) + } + + override def getShort(ordinal: Int): Short = { + Platform.getShort(null, getFieldAddressFromOrdinal(ordinal)) + } + + override def getInt(ordinal: Int): Int = { + Platform.getInt(null, getFieldAddressFromOrdinal(ordinal)) + } + + override def getLong(ordinal: Int): Long = { + Platform.getLong(null, getFieldAddressFromOrdinal(ordinal)) + } + + override def getFloat(ordinal: Int): Float = { + Platform.getFloat(null, getFieldAddressFromOrdinal(ordinal)) + } + + override def getDouble(ordinal: Int): Double = { + Platform.getDouble(null, getFieldAddressFromOrdinal(ordinal)) + } + + override def getDecimal(ordinal: Int, precision: Int, scale: Int): Decimal = { + if (isNullAt(ordinal)) { + null + } else if (precision <= Decimal.MAX_INT_DIGITS) { + Decimal.createUnsafe(getInt(ordinal), precision, scale) + } else if (precision <= Decimal.MAX_LONG_DIGITS) { + Decimal.createUnsafe(getLong(ordinal), precision, scale) + } else { + throw new IllegalArgumentException("NOT IMPLEMENTED YET") + } + } + + override def getUTF8String(ordinal: Int): UTF8String = { + throw new IllegalArgumentException("NOT IMPLEMENTED YET") + } + + override def getBinary(ordinal: Int): Array[Byte] = { + throw new IllegalArgumentException("NOT IMPLEMENTED YET") + } + + override def getInterval(ordinal: Int): CalendarInterval = { + throw new IllegalArgumentException("NOT IMPLEMENTED YET") + } + + override def getStruct(ordinal: Int, numFields: Int): CudfUnsafeRow = { + throw new IllegalArgumentException("NOT IMPLEMENTED YET") + } + + override def getArray(ordinal: Int): ArrayData = { + throw new IllegalArgumentException("NOT IMPLEMENTED YET") + } + + override def getMap(ordinal: Int): MapData = { + throw new IllegalArgumentException("NOT IMPLEMENTED YET") + } + + override def copy(): CudfUnsafeRow = { + throw new IllegalArgumentException("NOT IMPLEMENTED YET") + } + + override def hashCode(): Int = { + Murmur3_x86_32.hashUnsafeWords(null, address, sizeInBytes, 42) + } + + override def equals(other: Any): Boolean = other match { + case o: CudfUnsafeRow => + sizeInBytes == o.sizeInBytes && + ByteArrayMethods.arrayEquals(null, address, null, o.address, sizeInBytes) && + Arrays.equals(this.remapping, o.remapping) + case _ => false + } + + override def toString: String = { + val build = new StringBuilder("[") + for (i <- 0 until sizeInBytes by 8) { + if (i != 0) build.append(',') + build.append(java.lang.Long.toHexString(Platform.getLong(null, address + i))) + } + build.append(']') + build.append(" remapped with ") + build.append(Arrays.toString(remapping)) + build.toString() + } + + override def anyNull(): Boolean = throw new IllegalArgumentException("NOT IMPLEMENTED YET") + + private def getFieldAddressFromOrdinal(ordinal: Int): Long = { + assertIndexIsValid(ordinal) + val i = remapping(ordinal) + address + startOffsets(i) + } + + private def assertIndexIsValid(index: Int): Unit = { + assert(index >= 0, s"index ($index) should >= 0") + assert(index < startOffsets.length, s"index ($index) should < ${startOffsets.length}") + } +} + +trait CudfUnsafeRowTrait { + def alignOffset(offset: Int, alignment: Int): Int = (offset + alignment - 1) & -alignment + + def calculateBitSetWidthInBytes(numFields: Int): Int = (numFields + 7) / 8 + + def getRowSizeEstimate(attributes: Array[Attribute]): Int = { + var offset = 0 + for (attr <- attributes) { + val length = GpuColumnVector.getNonNestedRapidsType(attr.dataType).getSizeInBytes + offset = alignOffset(offset, length) + offset += length + } + val bitSetWidthInBytes = calculateBitSetWidthInBytes(attributes.length) + alignOffset(offset + bitSetWidthInBytes, 8) + } +} \ No newline at end of file diff --git a/sql-plugin/src/main/spark311/scala/org/apache/spark/rapids/shims/GpuShuffleExchangeExec.scala b/sql-plugin/src/main/spark311/scala/org/apache/spark/rapids/shims/GpuShuffleExchangeExec.scala index d94c8e54683..2dcad0d4226 100644 --- a/sql-plugin/src/main/spark311/scala/org/apache/spark/rapids/shims/GpuShuffleExchangeExec.scala +++ b/sql-plugin/src/main/spark311/scala/org/apache/spark/rapids/shims/GpuShuffleExchangeExec.scala @@ -58,6 +58,7 @@ case class GpuShuffleExchangeExec( cpuOutputPartitioning: Partitioning) extends GpuShuffleExchangeExecBaseWithMetrics(gpuOutputPartitioning, child) with ShuffleExchangeLike { + def shuffleId: Int = shuffleDependencyColumnar.shuffleId override def otherCopyArgs: Seq[AnyRef] = cpuOutputPartitioning :: Nil diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuOrcDataReader.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuOrcDataReader.scala index 8c82074b8f5..aec35945b4e 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuOrcDataReader.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuOrcDataReader.scala @@ -33,101 +33,17 @@ {"spark": "343"} {"spark": "350"} {"spark": "351"} -{"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims -import java.io.EOFException -import java.nio.ByteBuffer -import java.nio.channels.SeekableByteChannel - -import ai.rapids.cudf.HostMemoryBuffer -import com.nvidia.spark.rapids.Arm.closeOnExcept import com.nvidia.spark.rapids.GpuMetric -import com.nvidia.spark.rapids.filecache.FileCache import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.hive.common.io.DiskRangeList -import org.apache.orc.OrcProto -import org.apache.orc.impl.{BufferChunk, BufferChunkList, DataReaderProperties, InStream, OrcCodecPool} +import org.apache.orc.impl.DataReaderProperties class GpuOrcDataReader( props: DataReaderProperties, conf: Configuration, - metrics: Map[String, GpuMetric]) extends GpuOrcDataReaderBase(props, conf, metrics) { - - private class BufferChunkLoader(useDirect: Boolean) extends BlockLoader { - override def loadRemoteBlocks( - baseOffset: Long, - first: DiskRangeList, - last: DiskRangeList, - data: ByteBuffer): DiskRangeList = { - var current = first - val offset = current.getOffset - while (current ne last.next) { - val buffer = if (current eq last) data else data.duplicate() - buffer.position((current.getOffset - offset).toInt) - buffer.limit((current.getEnd - offset).toInt) - current.asInstanceOf[BufferChunk].setChunk(buffer) - // see if the filecache wants any of this data - val cacheToken = FileCache.get.startDataRangeCache(filePathString, - baseOffset + current.getOffset, current.getLength, conf) - cacheToken.foreach { token => - val hmb = closeOnExcept(HostMemoryBuffer.allocate(current.getLength, false)) { hmb => - hmb.setBytes(0, buffer.array(), - buffer.arrayOffset() + buffer.position(), current.getLength) - hmb - } - token.complete(hmb) - } - current = current.next - } - current - } - - override def loadCachedBlock( - chunk: DiskRangeList, - channel: SeekableByteChannel): DiskRangeList = { - val buffer = if (useDirect) { - ByteBuffer.allocateDirect(chunk.getLength) - } else { - ByteBuffer.allocate(chunk.getLength) - } - while (buffer.remaining() > 0) { - if (channel.read(buffer) < 0) { - throw new EOFException(s"Unexpected EOF while reading cached block for $filePathString") - } - } - buffer.flip() - chunk.asInstanceOf[BufferChunk].setChunk(buffer) - chunk - } - } - - override protected def parseStripeFooter(buf: ByteBuffer, size: Int): OrcProto.StripeFooter = { - OrcProto.StripeFooter.parseFrom( - InStream.createCodedInputStream(InStream.create("footer", - new BufferChunk(buf, 0), 0, size, compression))) - } - - override def getCompressionOptions: InStream.StreamOptions = compression - - override def readFileData(chunks: BufferChunkList, forceDirect: Boolean): BufferChunkList = { - if (chunks != null) { - readDiskRanges(chunks.get, 0, new BufferChunkLoader(forceDirect)) - } - chunks - } - - override def close(): Unit = { - if (compression.getCodec != null) { - if (compression.getCodec != null) { - OrcCodecPool.returnCodec(compression.getCodec.getKind, compression.getCodec) - compression.withCodec(null) - } - } - super.close() - } -} + metrics: Map[String, GpuMetric]) extends GpuOrcDataReader320Plus(props, conf, metrics) object GpuOrcDataReader { // File cache is being used, so we want read ranges that can be cached separately diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuOrcDataReader320Plus.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuOrcDataReader320Plus.scala new file mode 100644 index 00000000000..e28f7001a2b --- /dev/null +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuOrcDataReader320Plus.scala @@ -0,0 +1,130 @@ +/* + * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * + * Licensed 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. + */ +/*** spark-rapids-shim-json-lines +{"spark": "320"} +{"spark": "321"} +{"spark": "322"} +{"spark": "323"} +{"spark": "324"} +{"spark": "330"} +{"spark": "330db"} +{"spark": "331"} +{"spark": "332"} +{"spark": "332db"} +{"spark": "333"} +{"spark": "334"} +{"spark": "340"} +{"spark": "341"} +{"spark": "341db"} +{"spark": "342"} +{"spark": "343"} +{"spark": "350"} +{"spark": "351"} +{"spark": "400"} +spark-rapids-shim-json-lines ***/ +package com.nvidia.spark.rapids.shims + +import java.io.EOFException +import java.nio.ByteBuffer +import java.nio.channels.SeekableByteChannel + +import ai.rapids.cudf.HostMemoryBuffer +import com.nvidia.spark.rapids.Arm.closeOnExcept +import com.nvidia.spark.rapids.GpuMetric +import com.nvidia.spark.rapids.filecache.FileCache +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.hive.common.io.DiskRangeList +import org.apache.orc.OrcProto +import org.apache.orc.impl.{BufferChunk, BufferChunkList, DataReaderProperties, InStream, OrcCodecPool} + +abstract class GpuOrcDataReader320Plus( + props: DataReaderProperties, + conf: Configuration, + metrics: Map[String, GpuMetric]) extends GpuOrcDataReaderBase(props, conf, metrics) { + + private class BufferChunkLoader(useDirect: Boolean) extends BlockLoader { + override def loadRemoteBlocks( + baseOffset: Long, + first: DiskRangeList, + last: DiskRangeList, + data: ByteBuffer): DiskRangeList = { + var current = first + val offset = current.getOffset + while (current ne last.next) { + val buffer = if (current eq last) data else data.duplicate() + buffer.position((current.getOffset - offset).toInt) + buffer.limit((current.getEnd - offset).toInt) + current.asInstanceOf[BufferChunk].setChunk(buffer) + // see if the filecache wants any of this data + val cacheToken = FileCache.get.startDataRangeCache(filePathString, + baseOffset + current.getOffset, current.getLength, conf) + cacheToken.foreach { token => + val hmb = closeOnExcept(HostMemoryBuffer.allocate(current.getLength, false)) { hmb => + hmb.setBytes(0, buffer.array(), + buffer.arrayOffset() + buffer.position(), current.getLength) + hmb + } + token.complete(hmb) + } + current = current.next + } + current + } + + override def loadCachedBlock( + chunk: DiskRangeList, + channel: SeekableByteChannel): DiskRangeList = { + val buffer = if (useDirect) { + ByteBuffer.allocateDirect(chunk.getLength) + } else { + ByteBuffer.allocate(chunk.getLength) + } + while (buffer.remaining() > 0) { + if (channel.read(buffer) < 0) { + throw new EOFException(s"Unexpected EOF while reading cached block for $filePathString") + } + } + buffer.flip() + chunk.asInstanceOf[BufferChunk].setChunk(buffer) + chunk + } + } + + override protected def parseStripeFooter(buf: ByteBuffer, size: Int): OrcProto.StripeFooter = { + OrcProto.StripeFooter.parseFrom( + InStream.createCodedInputStream(InStream.create("footer", + new BufferChunk(buf, 0), 0, size, compression))) + } + + override def getCompressionOptions: InStream.StreamOptions = compression + + override def readFileData(chunks: BufferChunkList, forceDirect: Boolean): BufferChunkList = { + if (chunks != null) { + readDiskRanges(chunks.get, 0, new BufferChunkLoader(forceDirect)) + } + chunks + } + + override def close(): Unit = { + if (compression.getCodec != null) { + if (compression.getCodec != null) { + OrcCodecPool.returnCodec(compression.getCodec.getKind, compression.getCodec) + compression.withCodec(null) + } + } + super.close() + } +} \ No newline at end of file diff --git a/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/CudfUnsafeRow.scala b/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/CudfUnsafeRow.scala new file mode 100644 index 00000000000..623005654fc --- /dev/null +++ b/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/CudfUnsafeRow.scala @@ -0,0 +1,33 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed 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. + */ +/*** spark-rapids-shim-json-lines +{"spark": "400"} +spark-rapids-shim-json-lines ***/ +package com.nvidia.spark.rapids.shims + +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.unsafe.types.VariantVal + + +final class CudfUnsafeRow( + attributes: Array[Attribute], + remapping: Array[Int]) extends CudfUnsafeRowBase(attributes, remapping) { + def getVariant(ordinal: Int) = { + throw new UnsupportedOperationException("VariantVal is not supported") + } +} + +object CudfUnsafeRow extends CudfUnsafeRowTrait \ No newline at end of file diff --git a/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala b/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala index 3c2b649339b..4fc62d82df3 100644 --- a/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala +++ b/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala @@ -44,7 +44,7 @@ case class GpuBatchScanExec( spjParams: StoragePartitionJoinParams = StoragePartitionJoinParams() ) extends GpuBatchScanExecBase(scan, runtimeFilters) { - @transient lazy val batch: Batch = if (scan == null) null else scan.toBatch + @transient override lazy val batch: Batch = if (scan == null) null else scan.toBatch // TODO: unify the equal/hashCode implementation for all data source v2 query plans. override def equals(other: Any): Boolean = other match { case other: GpuBatchScanExec => diff --git a/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/GpuOrcDataReader.scala b/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/GpuOrcDataReader.scala new file mode 100644 index 00000000000..8c6a9c793f2 --- /dev/null +++ b/sql-plugin/src/main/spark400/scala/com/nvidia/spark/rapids/shims/GpuOrcDataReader.scala @@ -0,0 +1,38 @@ +/* + * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * + * Licensed 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. + */ +/*** spark-rapids-shim-json-lines +{"spark": "400"} +spark-rapids-shim-json-lines ***/ +package com.nvidia.spark.rapids.shims + +import com.nvidia.spark.rapids.GpuMetric +import org.apache.hadoop.conf.Configuration +import org.apache.orc.impl.DataReaderProperties + +class GpuOrcDataReader( + props: DataReaderProperties, + conf: Configuration, + metrics: Map[String, GpuMetric]) extends GpuOrcDataReader320Plus(props, conf, metrics) { + override def releaseAllBuffers(): Unit = { + throw new IllegalStateException("should not be trying to release buffers") + } +} + + +object GpuOrcDataReader { + // File cache is being used, so we want read ranges that can be cached separately + val shouldMergeDiskRanges: Boolean = false +} From c7129f5a006e19e4dc592dfd3810c1ac99f7f263 Mon Sep 17 00:00:00 2001 From: Gary Shen Date: Fri, 7 Jun 2024 11:30:11 +0800 Subject: [PATCH 28/68] Add rapids configs to enable GPU running (#10963) Exclude 2 new failed cases Signed-off-by: Gary Shen --- .../sql/rapids/utils/RapidsTestSettings.scala | 2 ++ .../sql/rapids/utils/RapidsTestsTrait.scala | 24 +++++++++++++++++++ 2 files changed, 26 insertions(+) diff --git a/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/utils/RapidsTestSettings.scala b/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/utils/RapidsTestSettings.scala index 8fc86cc6dce..181fb14fb00 100644 --- a/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/utils/RapidsTestSettings.scala +++ b/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/utils/RapidsTestSettings.scala @@ -34,6 +34,8 @@ class RapidsTestSettings extends BackendTestSettings { .exclude("SPARK-35735: Take into account day-time interval fields in cast", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10771")) .exclude("casting to fixed-precision decimals", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10771")) .exclude("SPARK-32828: cast from a derived user-defined type to a base type", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10771")) + .exclude("cast string to timestamp", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10771")) + .exclude("cast string to date", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10771")) enableSuite[RapidsDataFrameAggregateSuite] .exclude("collect functions", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10772")) .exclude("collect functions structs", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10772")) diff --git a/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/utils/RapidsTestsTrait.scala b/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/utils/RapidsTestsTrait.scala index bcac0b8fe2d..69bd4532c71 100644 --- a/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/utils/RapidsTestsTrait.scala +++ b/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/utils/RapidsTestsTrait.scala @@ -110,6 +110,30 @@ trait RapidsTestsTrait extends RapidsTestsCommonTrait { // .config("spark.rapids.sql.test.enabled", "true") // .config("spark.rapids.sql.test.allowedNonGpu", // "SerializeFromObjectExec,DeserializeToObjectExec,ExternalRDDScanExec") + .config("spark.rapids.sql.castStringToTimestamp.enabled", "true") + .config("spark.rapids.sql.csv.read.decimal.enabled", "true") + .config("spark.rapids.sql.format.avro.enabled", "true") + .config("spark.rapids.sql.format.avro.read.enabled", "true") + .config("spark.rapids.sql.format.hive.text.write.enabled", "true") + .config("spark.rapids.sql.format.json.enabled", "true") + .config("spark.rapids.sql.format.json.read.enabled", "true") + .config("spark.rapids.sql.incompatibleDateFormats.enabled", "true") + .config("spark.rapids.sql.python.gpu.enabled", "true") + .config("spark.rapids.sql.rowBasedUDF.enabled", "true") + .config("spark.rapids.sql.window.collectList.enabled", "true") + .config("spark.rapids.sql.window.collectSet.enabled", "true") + .config("spark.rapids.sql.window.range.byte.enabled", "true") + .config("spark.rapids.sql.window.range.short.enabled", "true") + .config("spark.rapids.sql.expression.Ascii", "true") + .config("spark.rapids.sql.expression.Conv", "true") + .config("spark.rapids.sql.expression.GetJsonObject", "true") + .config("spark.rapids.sql.expression.JsonToStructs", "true") + .config("spark.rapids.sql.expression.JsonTuple", "true") + .config("spark.rapids.sql.expression.StructsToJson", "true") + .config("spark.rapids.sql.exec.CollectLimitExec", "true") + .config("spark.rapids.sql.exec.FlatMapCoGroupsInPandasExec", "true") + .config("spark.rapids.sql.exec.WindowInPandasExec", "true") + .config("spark.rapids.sql.hasExtendedYearValues", "false") .appName("rapids spark plugin running Vanilla Spark UT") _spark = sparkBuilder From 18c2579f5dfd1ed592f9287d936e3597b45a10b0 Mon Sep 17 00:00:00 2001 From: Haoyang Li Date: Sat, 8 Jun 2024 11:48:48 +0800 Subject: [PATCH 29/68] Fix Spark UT issues in RapidsDataFrameAggregateSuite (#10943) * Fix Spark UT issues in RapidsDataFrameAggregateSuite Signed-off-by: Haoyang Li * Added SPARK-24788 back Signed-off-by: Haoyang Li --------- Signed-off-by: Haoyang Li --- .../RapidsDataFrameAggregateSuite.scala | 63 +++++++++++++++++-- .../rapids/utils/BackendTestSettings.scala | 1 + .../sql/rapids/utils/RapidsTestSettings.scala | 10 +-- 3 files changed, 65 insertions(+), 9 deletions(-) diff --git a/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/suites/RapidsDataFrameAggregateSuite.scala b/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/suites/RapidsDataFrameAggregateSuite.scala index 5a394a5b0e8..dba811c073c 100644 --- a/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/suites/RapidsDataFrameAggregateSuite.scala +++ b/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/suites/RapidsDataFrameAggregateSuite.scala @@ -19,12 +19,67 @@ spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.suites -import org.apache.spark.sql.DataFrameAggregateSuite +import org.apache.spark.sql.{DataFrameAggregateSuite, Row} +import org.apache.spark.sql.functions._ import org.apache.spark.sql.rapids.utils.RapidsSQLTestsTrait +import org.apache.spark.sql.types._ class RapidsDataFrameAggregateSuite extends DataFrameAggregateSuite with RapidsSQLTestsTrait { - // example to show how to replace the logic of an excluded test case in Vanilla Spark - testRapids("collect functions" ) { // "collect functions" was excluded at RapidsTestSettings - // println("...") + import testImplicits._ + + testRapids("collect functions") { + val df = Seq((1, 2), (2, 2), (3, 4)).toDF("a", "b") + checkAnswer( + df.select(sort_array(collect_list($"a")), sort_array(collect_list($"b"))), + Seq(Row(Seq(1, 2, 3), Seq(2, 2, 4))) + ) + checkAnswer( + df.select(sort_array(collect_set($"a")), sort_array(collect_set($"b"))), + Seq(Row(Seq(1, 2, 3), Seq(2, 4))) + ) + + checkDataset( + df.select(sort_array(collect_set($"a")).as("aSet")).as[Set[Int]], + Set(1, 2, 3)) + checkDataset( + df.select(sort_array(collect_set($"b")).as("bSet")).as[Set[Int]], + Set(2, 4)) + checkDataset( + df.select(sort_array(collect_set($"a")), sort_array(collect_set($"b"))) + .as[(Set[Int], Set[Int])], Seq(Set(1, 2, 3) -> Set(2, 4)): _*) + } + + testRapids("collect functions structs") { + val df = Seq((1, 2, 2), (2, 2, 2), (3, 4, 1)) + .toDF("a", "x", "y") + .select($"a", struct($"x", $"y").as("b")) + checkAnswer( + df.select(sort_array(collect_list($"a")), sort_array(collect_list($"b"))), + Seq(Row(Seq(1, 2, 3), Seq(Row(2, 2), Row(2, 2), Row(4, 1)))) + ) + checkAnswer( + df.select(sort_array(collect_set($"a")), sort_array(collect_set($"b"))), + Seq(Row(Seq(1, 2, 3), Seq(Row(2, 2), Row(4, 1)))) + ) + } + + testRapids("SPARK-17641: collect functions should not collect null values") { + val df = Seq(("1", 2), (null, 2), ("1", 4)).toDF("a", "b") + checkAnswer( + df.select(sort_array(collect_list($"a")), sort_array(collect_list($"b"))), + Seq(Row(Seq("1", "1"), Seq(2, 2, 4))) + ) + checkAnswer( + df.select(sort_array(collect_set($"a")), sort_array(collect_set($"b"))), + Seq(Row(Seq("1"), Seq(2, 4))) + ) + } + + testRapids("collect functions should be able to cast to array type with no null values") { + val df = Seq(1, 2).toDF("a") + checkAnswer(df.select(sort_array(collect_list("a")) cast ArrayType(IntegerType, false)), + Seq(Row(Seq(1, 2)))) + checkAnswer(df.select(sort_array(collect_set("a")) cast ArrayType(FloatType, false)), + Seq(Row(Seq(1.0, 2.0)))) } } diff --git a/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/utils/BackendTestSettings.scala b/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/utils/BackendTestSettings.scala index 83396e977fa..e1aec1ffebc 100644 --- a/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/utils/BackendTestSettings.scala +++ b/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/utils/BackendTestSettings.scala @@ -83,6 +83,7 @@ abstract class BackendTestSettings { // or a description like "This simply can't work on GPU". // It should never be "unknown" or "need investigation" case class KNOWN_ISSUE(reason: String) extends ExcludeReason + case class ADJUST_UT(reason: String) extends ExcludeReason case class WONT_FIX_ISSUE(reason: String) extends ExcludeReason diff --git a/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/utils/RapidsTestSettings.scala b/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/utils/RapidsTestSettings.scala index 181fb14fb00..ad93c4dd2e9 100644 --- a/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/utils/RapidsTestSettings.scala +++ b/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/utils/RapidsTestSettings.scala @@ -37,11 +37,11 @@ class RapidsTestSettings extends BackendTestSettings { .exclude("cast string to timestamp", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10771")) .exclude("cast string to date", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10771")) enableSuite[RapidsDataFrameAggregateSuite] - .exclude("collect functions", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10772")) - .exclude("collect functions structs", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10772")) - .exclude("collect functions should be able to cast to array type with no null values", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10772")) - .exclude("SPARK-17641: collect functions should not collect null values", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10772")) - .exclude("SPARK-19471: AggregationIterator does not initialize the generated result projection before using it", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10772")) + .exclude("collect functions", ADJUST_UT("order of elements in the array is non-deterministic in collect")) + .exclude("collect functions structs", ADJUST_UT("order of elements in the array is non-deterministic in collect")) + .exclude("collect functions should be able to cast to array type with no null values", ADJUST_UT("order of elements in the array is non-deterministic in collect")) + .exclude("SPARK-17641: collect functions should not collect null values", ADJUST_UT("order of elements in the array is non-deterministic in collect")) + .exclude("SPARK-19471: AggregationIterator does not initialize the generated result projection before using it", WONT_FIX_ISSUE("Codegen related UT, not applicable for GPU")) .exclude("SPARK-24788: RelationalGroupedDataset.toString with unresolved exprs should not fail", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10801")) enableSuite[RapidsJsonExpressionsSuite] .exclude("from_json - invalid data", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10849")) From 9030b133c5770f791da19f6610b9fde1e85a9d50 Mon Sep 17 00:00:00 2001 From: Raza Jafri Date: Fri, 7 Jun 2024 22:44:41 -0700 Subject: [PATCH 30/68] Addressing the Spark change of renaming the named parameter (#10992) Signed-off-by: Raza Jafri --- .../spark/sql/rapids/GpuInSubqueryExecSuite.scala | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/GpuInSubqueryExecSuite.scala b/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/GpuInSubqueryExecSuite.scala index 82ce1073e13..a606dba0572 100644 --- a/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/GpuInSubqueryExecSuite.scala +++ b/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/GpuInSubqueryExecSuite.scala @@ -65,7 +65,7 @@ class GpuInSubqueryExecSuite extends SparkQueryCompareTestSuite { private def buildCpuInSubqueryPlan( spark: SparkSession, - shouldBroadcast: Boolean): SparkPlan = { + shouldBroadcastOrDpp: Boolean): SparkPlan = { val df1ReadExec = readToPhysicalPlan(nullableStringsIntsDf(spark)) val df2ReadExec = readToPhysicalPlan(subqueryTable(spark)) val inSubquery = InSubqueryExec( @@ -73,16 +73,19 @@ class GpuInSubqueryExecSuite extends SparkQueryCompareTestSuite { SubqueryExec("sbe", ProjectExec(Seq(df2ReadExec.output.head), df2ReadExec)), ExprId(7), - shouldBroadcast=shouldBroadcast) + shouldBroadcastOrDpp) FilterExec(DynamicPruningExpression(inSubquery), df1ReadExec) } - for (shouldBroadcast <- Seq(false, true)) { - test(s"InSubqueryExec shouldBroadcast=$shouldBroadcast") { + /** + * The named parameter shouldBroadcast was renamed to isDynamicPruning in Spark 4.0.0+ + */ + for (shouldBroadcastOrDpp <- Seq(false, true)) { + test(s"InSubqueryExec shouldBroadcastOrDpp=$shouldBroadcastOrDpp") { val gpuResults = withGpuSparkSession({ spark => val overrides = new GpuOverrides() val transitionOverrides = new GpuTransitionOverrides() - val cpuPlan = buildCpuInSubqueryPlan(spark, shouldBroadcast) + val cpuPlan = buildCpuInSubqueryPlan(spark, shouldBroadcastOrDpp) val gpuPlan = transitionOverrides(overrides(cpuPlan)) gpuPlan.execute().collect() }) From f921c4dcfc9455171c37ebf7db27682123623b06 Mon Sep 17 00:00:00 2001 From: Jenkins Automation <70000568+nvauto@users.noreply.github.com> Date: Mon, 10 Jun 2024 08:14:52 +0800 Subject: [PATCH 31/68] Update rapids JNI and private dependency to 24.06.0 (#10974) Wait for the pre-merge CI job to SUCCEED Signed-off-by: jenkins Co-authored-by: jenkins --- pom.xml | 4 ++-- scala2.13/pom.xml | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/pom.xml b/pom.xml index c939f8d5891..942905ea427 100644 --- a/pom.xml +++ b/pom.xml @@ -719,8 +719,8 @@ spark${buildver} cuda11 ${cuda.version} - 24.06.0-SNAPSHOT - 24.06.0-SNAPSHOT + 24.06.0 + 24.06.0 2.12 2.8.0 incremental diff --git a/scala2.13/pom.xml b/scala2.13/pom.xml index cf3bfb48373..783aa38cf50 100644 --- a/scala2.13/pom.xml +++ b/scala2.13/pom.xml @@ -719,8 +719,8 @@ spark${buildver} cuda11 ${cuda.version} - 24.06.0-SNAPSHOT - 24.06.0-SNAPSHOT + 24.06.0 + 24.06.0 2.13 2.8.0 incremental From e9ccf97e453ff6ba1e97aa867f7f80c516d13189 Mon Sep 17 00:00:00 2001 From: Suraj Aralihalli Date: Mon, 10 Jun 2024 07:58:52 -0700 Subject: [PATCH 32/68] [DOC] Update docs for 24.06.0 release [skip ci] (#10984) * add 24.06 Signed-off-by: Suraj Aralihalli * update release notes Signed-off-by: Suraj Aralihalli * update release notes Signed-off-by: Suraj Aralihalli --------- Signed-off-by: Suraj Aralihalli --- docs/archive.md | 136 ++++++++++++++++++++++++++++++++++++++++++++++- docs/download.md | 33 ++++++------ 2 files changed, 149 insertions(+), 20 deletions(-) diff --git a/docs/archive.md b/docs/archive.md index 6cce30557f4..f4eeab11a40 100644 --- a/docs/archive.md +++ b/docs/archive.md @@ -5,11 +5,143 @@ nav_order: 15 --- Below are archived releases for RAPIDS Accelerator for Apache Spark. +## Release v24.04.1 +### Hardware Requirements: + +The plugin is tested on the following architectures: + + GPU Models: NVIDIA V100, T4, A10/A100, L4 and H100 GPUs + +### Software Requirements: + + OS: Ubuntu 20.04, Ubuntu 22.04, CentOS 7, or Rocky Linux 8 + + NVIDIA Driver*: R470+ + + Runtime: + Scala 2.12, 2.13 + Python, Java Virtual Machine (JVM) compatible with your spark-version. + + * Check the Spark documentation for Python and Java version compatibility with your specific + Spark version. For instance, visit `https://spark.apache.org/docs/3.4.1` for Spark 3.4.1. + + Supported Spark versions: + Apache Spark 3.2.0, 3.2.1, 3.2.2, 3.2.3, 3.2.4 + Apache Spark 3.3.0, 3.3.1, 3.3.2, 3.3.3, 3.3.4 + Apache Spark 3.4.0, 3.4.1, 3.4.2 + Apache Spark 3.5.0, 3.5.1 + + Supported Databricks runtime versions for Azure and AWS: + Databricks 11.3 ML LTS (GPU, Scala 2.12, Spark 3.3.0) + Databricks 12.2 ML LTS (GPU, Scala 2.12, Spark 3.3.2) + Databricks 13.3 ML LTS (GPU, Scala 2.12, Spark 3.4.1) + + Supported Dataproc versions (Debian/Ubuntu): + GCP Dataproc 2.0 + GCP Dataproc 2.1 + + Supported Dataproc Serverless versions: + Spark runtime 1.1 LTS + Spark runtime 2.0 + Spark runtime 2.1 + +*Some hardware may have a minimum driver version greater than R470. Check the GPU spec sheet +for your hardware's minimum driver version. + +*For Cloudera and EMR support, please refer to the +[Distributions](https://docs.nvidia.com/spark-rapids/user-guide/latest/faq.html#which-distributions-are-supported) section of the FAQ. + +### RAPIDS Accelerator's Support Policy for Apache Spark +The RAPIDS Accelerator maintains support for Apache Spark versions available for download from [Apache Spark](https://spark.apache.org/downloads.html) + +### Download RAPIDS Accelerator for Apache Spark v24.04.1 + +| Processor | Scala Version | Download Jar | Download Signature | +|-----------|---------------|--------------|--------------------| +| x86_64 | Scala 2.12 | [RAPIDS Accelerator v24.04.1](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/24.04.1/rapids-4-spark_2.12-24.04.1.jar) | [Signature](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/24.04.1/rapids-4-spark_2.12-24.04.1.jar.asc) | +| x86_64 | Scala 2.13 | [RAPIDS Accelerator v24.04.1](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.13/24.04.1/rapids-4-spark_2.13-24.04.1.jar) | [Signature](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.13/24.04.1/rapids-4-spark_2.13-24.04.1.jar.asc) | +| arm64 | Scala 2.12 | [RAPIDS Accelerator v24.04.1](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/24.04.1/rapids-4-spark_2.12-24.04.1-cuda11-arm64.jar) | [Signature](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/24.04.1/rapids-4-spark_2.12-24.04.1-cuda11-arm64.jar.asc) | +| arm64 | Scala 2.13 | [RAPIDS Accelerator v24.04.1](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.13/24.04.1/rapids-4-spark_2.13-24.04.1-cuda11-arm64.jar) | [Signature](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.13/24.04.1/rapids-4-spark_2.13-24.04.1-cuda11-arm64.jar.asc) | + +This package is built against CUDA 11.8. It is tested on V100, T4, A10, A100, L4 and H100 GPUs with +CUDA 11.8 through CUDA 12.0. + +### Verify signature +* Download the [PUB_KEY](https://keys.openpgp.org/search?q=sw-spark@nvidia.com). +* Import the public key: `gpg --import PUB_KEY` +* Verify the signature for Scala 2.12 jar: + `gpg --verify rapids-4-spark_2.12-24.04.1.jar.asc rapids-4-spark_2.12-24.04.1.jar` +* Verify the signature for Scala 2.13 jar: + `gpg --verify rapids-4-spark_2.13-24.04.1.jar.asc rapids-4-spark_2.13-24.04.1.jar` + +The output of signature verify: + + gpg: Good signature from "NVIDIA Spark (For the signature of spark-rapids release jars) " + +### Release Notes +* New functionality and performance improvements for this release include: +* Performance improvements for S3 reading. +Refer to perfio.s3.enabled in [advanced_configs](./additional-functionality/advanced_configs.md) for more details. +* Performance improvements when doing a joins on unique keys. +* Enhanced decompression kernels for zstd and snappy. +* Enhanced Parquet reading performance with modular kernels. +* Added compatibility with Spark version 3.5.1. +* Deprecated support for Databricks 10.4 ML LTS. +* For updates on RAPIDS Accelerator Tools, please visit [this link](https://github.com/NVIDIA/spark-rapids-tools/releases). + +For a detailed list of changes, please refer to the +[CHANGELOG](https://github.com/NVIDIA/spark-rapids/blob/main/CHANGELOG.md). + +## Archived releases + +As new releases come out, previous ones will still be available in [archived releases](./archive.md). + ## Release v24.04.0 ### Hardware Requirements: The plugin is tested on the following architectures: - @@ -67,14 +67,14 @@ for your hardware's minimum driver version. + + GPU Models: NVIDIA V100, T4, A10/A100, L4 and H100 GPUs + +### Software Requirements: + + OS: Ubuntu 20.04, Ubuntu 22.04, CentOS 7, or Rocky Linux 8 + + NVIDIA Driver*: R470+ + + Runtime: + Scala 2.12, 2.13 + Python, Java Virtual Machine (JVM) compatible with your spark-version. + + * Check the Spark documentation for Python and Java version compatibility with your specific + Spark version. For instance, visit `https://spark.apache.org/docs/3.4.1` for Spark 3.4.1. + + Supported Spark versions: + Apache Spark 3.2.0, 3.2.1, 3.2.2, 3.2.3, 3.2.4 + Apache Spark 3.3.0, 3.3.1, 3.3.2, 3.3.3, 3.3.4 + Apache Spark 3.4.0, 3.4.1, 3.4.2 + Apache Spark 3.5.0, 3.5.1 + + Supported Databricks runtime versions for Azure and AWS: + Databricks 11.3 ML LTS (GPU, Scala 2.12, Spark 3.3.0) + Databricks 12.2 ML LTS (GPU, Scala 2.12, Spark 3.3.2) + Databricks 13.3 ML LTS (GPU, Scala 2.12, Spark 3.4.1) + + Supported Dataproc versions (Debian/Ubuntu): + GCP Dataproc 2.0 + GCP Dataproc 2.1 + + Supported Dataproc Serverless versions: + Spark runtime 1.1 LTS + Spark runtime 2.0 + Spark runtime 2.1 + +*Some hardware may have a minimum driver version greater than R470. Check the GPU spec sheet +for your hardware's minimum driver version. + +*For Cloudera and EMR support, please refer to the +[Distributions](https://docs.nvidia.com/spark-rapids/user-guide/latest/faq.html#which-distributions-are-supported) section of the FAQ. + ### RAPIDS Accelerator's Support Policy for Apache Spark The RAPIDS Accelerator maintains support for Apache Spark versions available for download from [Apache Spark](https://spark.apache.org/downloads.html) @@ -74,7 +206,7 @@ The plugin is tested on the following architectures: Supported Spark versions: Apache Spark 3.2.0, 3.2.1, 3.2.2, 3.2.3, 3.2.4 Apache Spark 3.3.0, 3.3.1, 3.3.2, 3.3.3 - Apache Spark 3.4.0, 3.4.1 + Apache Spark 3.4.0, 3.4.1, 3.4.2 Apache Spark 3.5.0 Supported Databricks runtime versions for Azure and AWS: diff --git a/docs/download.md b/docs/download.md index a7b6bd23a4a..f786f5a217d 100644 --- a/docs/download.md +++ b/docs/download.md @@ -18,7 +18,7 @@ cuDF jar, that is either preinstalled in the Spark classpath on all nodes or sub that uses the RAPIDS Accelerator For Apache Spark. See the [getting-started guide](https://docs.nvidia.com/spark-rapids/user-guide/latest/getting-started/overview.html) for more details. -## Release v24.04.1 +## Release v24.06.0 ### Hardware Requirements: The plugin is tested on the following architectures: @@ -41,7 +41,7 @@ The plugin is tested on the following architectures: Supported Spark versions: Apache Spark 3.2.0, 3.2.1, 3.2.2, 3.2.3, 3.2.4 Apache Spark 3.3.0, 3.3.1, 3.3.2, 3.3.3, 3.3.4 - Apache Spark 3.4.0, 3.4.1, 3.4.2 + Apache Spark 3.4.0, 3.4.1, 3.4.2, 3.4.3 Apache Spark 3.5.0, 3.5.1 Supported Databricks runtime versions for Azure and AWS: @@ -57,6 +57,7 @@ The plugin is tested on the following architectures: Spark runtime 1.1 LTS Spark runtime 2.0 Spark runtime 2.1 + Spark runtime 2.2 *Some hardware may have a minimum driver version greater than R470. Check the GPU spec sheet for your hardware's minimum driver version. @@ -67,14 +68,14 @@ for your hardware's minimum driver version. ### RAPIDS Accelerator's Support Policy for Apache Spark The RAPIDS Accelerator maintains support for Apache Spark versions available for download from [Apache Spark](https://spark.apache.org/downloads.html) -### Download RAPIDS Accelerator for Apache Spark v24.04.1 +### Download RAPIDS Accelerator for Apache Spark v24.06.0 | Processor | Scala Version | Download Jar | Download Signature | |-----------|---------------|--------------|--------------------| -| x86_64 | Scala 2.12 | [RAPIDS Accelerator v24.04.1](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/24.04.1/rapids-4-spark_2.12-24.04.1.jar) | [Signature](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/24.04.1/rapids-4-spark_2.12-24.04.1.jar.asc) | -| x86_64 | Scala 2.13 | [RAPIDS Accelerator v24.04.1](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.13/24.04.1/rapids-4-spark_2.13-24.04.1.jar) | [Signature](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.13/24.04.1/rapids-4-spark_2.13-24.04.1.jar.asc) | -| arm64 | Scala 2.12 | [RAPIDS Accelerator v24.04.1](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/24.04.1/rapids-4-spark_2.12-24.04.1-cuda11-arm64.jar) | [Signature](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/24.04.1/rapids-4-spark_2.12-24.04.1-cuda11-arm64.jar.asc) | -| arm64 | Scala 2.13 | [RAPIDS Accelerator v24.04.1](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.13/24.04.1/rapids-4-spark_2.13-24.04.1-cuda11-arm64.jar) | [Signature](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.13/24.04.1/rapids-4-spark_2.13-24.04.1-cuda11-arm64.jar.asc) | +| x86_64 | Scala 2.12 | [RAPIDS Accelerator v24.06.0](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/24.06.0/rapids-4-spark_2.12-24.06.0.jar) | [Signature](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/24.06.0/rapids-4-spark_2.12-24.06.0.jar.asc) | +| x86_64 | Scala 2.13 | [RAPIDS Accelerator v24.06.0](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.13/24.06.0/rapids-4-spark_2.13-24.06.0.jar) | [Signature](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.13/24.06.0/rapids-4-spark_2.13-24.06.0.jar.asc) | +| arm64 | Scala 2.12 | [RAPIDS Accelerator v24.06.0](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/24.06.0/rapids-4-spark_2.12-24.06.0-cuda11-arm64.jar) | [Signature](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.12/24.06.0/rapids-4-spark_2.12-24.06.0-cuda11-arm64.jar.asc) | +| arm64 | Scala 2.13 | [RAPIDS Accelerator v24.06.0](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.13/24.06.0/rapids-4-spark_2.13-24.06.0-cuda11-arm64.jar) | [Signature](https://repo1.maven.org/maven2/com/nvidia/rapids-4-spark_2.13/24.06.0/rapids-4-spark_2.13-24.06.0-cuda11-arm64.jar.asc) | This package is built against CUDA 11.8. It is tested on V100, T4, A10, A100, L4 and H100 GPUs with CUDA 11.8 through CUDA 12.0. @@ -83,24 +84,20 @@ CUDA 11.8 through CUDA 12.0. * Download the [PUB_KEY](https://keys.openpgp.org/search?q=sw-spark@nvidia.com). * Import the public key: `gpg --import PUB_KEY` * Verify the signature for Scala 2.12 jar: - `gpg --verify rapids-4-spark_2.12-24.04.1.jar.asc rapids-4-spark_2.12-24.04.1.jar` + `gpg --verify rapids-4-spark_2.12-24.06.0.jar.asc rapids-4-spark_2.12-24.06.0.jar` * Verify the signature for Scala 2.13 jar: - `gpg --verify rapids-4-spark_2.13-24.04.1.jar.asc rapids-4-spark_2.13-24.04.1.jar` + `gpg --verify rapids-4-spark_2.13-24.06.0.jar.asc rapids-4-spark_2.13-24.06.0.jar` The output of signature verify: gpg: Good signature from "NVIDIA Spark (For the signature of spark-rapids release jars) " ### Release Notes -* New functionality and performance improvements for this release include: -* Performance improvements for S3 reading. -Refer to perfio.s3.enabled in [advanced_configs](./additional-functionality/advanced_configs.md) for more details. -* Performance improvements when doing a joins on unique keys. -* Enhanced decompression kernels for zstd and snappy. -* Enhanced Parquet reading performance with modular kernels. -* Added compatibility with Spark version 3.5.1. -* Deprecated support for Databricks 10.4 ML LTS. -* For updates on RAPIDS Accelerator Tools, please visit [this link](https://github.com/NVIDIA/spark-rapids-tools/releases). +* Improve support for Unity Catalog on Databricks +* Added support for parse_url PATH +* Added support for array_filter +* Added support for Spark 3.4.3 +* For updates on RAPIDS Accelerator Tools, please visit [this link](https://github.com/NVIDIA/spark-rapids-tools/releases) For a detailed list of changes, please refer to the [CHANGELOG](https://github.com/NVIDIA/spark-rapids/blob/main/CHANGELOG.md). From 7887113aa080c247fd0a6f581879838012fcf30c Mon Sep 17 00:00:00 2001 From: Jenkins Automation <70000568+nvauto@users.noreply.github.com> Date: Mon, 10 Jun 2024 23:28:53 +0800 Subject: [PATCH 33/68] Update latest changelog [skip ci] (#10981) * Update latest changelog [skip ci] Update change log with CLI: \n\n scripts/generate-changelog --token= --releases=24.04,24.06 Signed-off-by: jenkins * Update the Changelog.md for the change review Signed-off-by: Tim Liu * Update change log Signed-off-by: Tim Liu * Update changelog Signed-off-by: Tim Liu --------- Signed-off-by: jenkins Signed-off-by: Tim Liu Co-authored-by: jenkins Co-authored-by: Tim Liu --- CHANGELOG.md | 420 +++++++++---------------------- docs/archives/CHANGELOG_24.02.md | 300 ++++++++++++++++++++++ 2 files changed, 419 insertions(+), 301 deletions(-) create mode 100644 docs/archives/CHANGELOG_24.02.md diff --git a/CHANGELOG.md b/CHANGELOG.md index 0340afa5931..d534d9398b6 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,5 +1,119 @@ # Change log -Generated on 2024-05-20 +Generated on 2024-06-10 + +## Release 24.06 + +### Features +||| +|:---|:---| +|[#10850](https://github.com/NVIDIA/spark-rapids/issues/10850)|[FEA] Refine the test framework introduced in #10745| +|[#6969](https://github.com/NVIDIA/spark-rapids/issues/6969)|[FEA] Support parse_url | +|[#10496](https://github.com/NVIDIA/spark-rapids/issues/10496)|[FEA] Drop support for CentOS7| +|[#10760](https://github.com/NVIDIA/spark-rapids/issues/10760)|[FEA]Support ArrayFilter| +|[#10721](https://github.com/NVIDIA/spark-rapids/issues/10721)|[FEA] Dump the complete set of build-info properties to the Spark eventLog| +|[#10666](https://github.com/NVIDIA/spark-rapids/issues/10666)|[FEA] Create Spark 3.4.3 shim| + +### Performance +||| +|:---|:---| +|[#8963](https://github.com/NVIDIA/spark-rapids/issues/8963)|[FEA] Use custom kernel for parse_url| +|[#10817](https://github.com/NVIDIA/spark-rapids/issues/10817)|[FOLLOW ON] Combining regex parsing in transpiling and regex rewrite in `rlike`| +|[#10821](https://github.com/NVIDIA/spark-rapids/issues/10821)|Rewrite `pattern[A-B]{X,Y}` (a pattern string followed by X to Y chars in range A - B) in `RLIKE` to a custom kernel| + +### Bugs Fixed +||| +|:---|:---| +|[#10928](https://github.com/NVIDIA/spark-rapids/issues/10928)|[BUG] 24.06 test_conditional_with_side_effects_case_when test failed on Scala 2.13 with DATAGEN_SEED=1716656294| +|[#10941](https://github.com/NVIDIA/spark-rapids/issues/10941)|[BUG] Failed to build on databricks due to GpuOverrides.scala:4264: not found: type GpuSubqueryBroadcastMeta| +|[#10902](https://github.com/NVIDIA/spark-rapids/issues/10902)|Spark UT failed: SPARK-37360: Timestamp type inference for a mix of TIMESTAMP_NTZ and TIMESTAMP_LTZ| +|[#10899](https://github.com/NVIDIA/spark-rapids/issues/10899)|[BUG] format_number Spark UT failed because Type conversion is not allowed| +|[#10913](https://github.com/NVIDIA/spark-rapids/issues/10913)|[BUG] rlike with empty pattern failed with 'NoSuchElementException' when enabling regex rewrite| +|[#10774](https://github.com/NVIDIA/spark-rapids/issues/10774)|[BUG] Issues found by Spark UT Framework on RapidsRegexpExpressionsSuite| +|[#10606](https://github.com/NVIDIA/spark-rapids/issues/10606)|[BUG] Update Plugin to use the new `getPartitionedFile` method| +|[#10806](https://github.com/NVIDIA/spark-rapids/issues/10806)|[BUG] orc_write_test.py::test_write_round_trip_corner failed with DATAGEN_SEED=1715517863| +|[#10831](https://github.com/NVIDIA/spark-rapids/issues/10831)|[BUG] Failed to read data from iceberg| +|[#10810](https://github.com/NVIDIA/spark-rapids/issues/10810)|[BUG] NPE when running `ParseUrl` tests in `RapidsStringExpressionsSuite`| +|[#10797](https://github.com/NVIDIA/spark-rapids/issues/10797)|[BUG] udf_test test_single_aggregate_udf, test_group_aggregate_udf and test_group_apply_udf_more_types failed on DB 13.3| +|[#10719](https://github.com/NVIDIA/spark-rapids/issues/10719)|[BUG] test_exact_percentile_groupby FAILED: hash_aggregate_test.py::test_exact_percentile_groupby with DATAGEN seed 1713362217| +|[#10738](https://github.com/NVIDIA/spark-rapids/issues/10738)|[BUG] test_exact_percentile_groupby_partial_fallback_to_cpu failed with DATAGEN_SEED=1713928179| +|[#10768](https://github.com/NVIDIA/spark-rapids/issues/10768)|[DOC] Dead links with tools pages| +|[#10751](https://github.com/NVIDIA/spark-rapids/issues/10751)|[BUG] Cascaded Pandas UDFs not working as expected on Databricks when plugin is enabled| +|[#10318](https://github.com/NVIDIA/spark-rapids/issues/10318)|[BUG] `fs.azure.account.keyInvalid` configuration issue while reading from Unity Catalog Tables on Azure DB| +|[#10722](https://github.com/NVIDIA/spark-rapids/issues/10722)|[BUG] "Could not find any rapids-4-spark jars in classpath" error when debugging UT in IDEA| +|[#10724](https://github.com/NVIDIA/spark-rapids/issues/10724)|[BUG] Failed to convert string with invisible characters to float| +|[#10633](https://github.com/NVIDIA/spark-rapids/issues/10633)|[BUG] ScanJson and JsonToStructs can give almost random errors| +|[#10659](https://github.com/NVIDIA/spark-rapids/issues/10659)|[BUG] from_json ArrayIndexOutOfBoundsException in 24.02| +|[#10656](https://github.com/NVIDIA/spark-rapids/issues/10656)|[BUG] Databricks cache tests failing with host memory OOM| + +### PRs +||| +|:---|:---| +|[#10984](https://github.com/NVIDIA/spark-rapids/pull/10984)|[DOC] Update docs for 24.06.0 release [skip ci]| +|[#10974](https://github.com/NVIDIA/spark-rapids/pull/10974)|Update rapids JNI and private dependency to 24.06.0| +|[#10947](https://github.com/NVIDIA/spark-rapids/pull/10947)|Prevent contains-PrefixRange optimization if not preceded by wildcards| +|[#10934](https://github.com/NVIDIA/spark-rapids/pull/10934)|Revert "Add Support for Multiple Filtering Keys for Subquery Broadcast "| +|[#10870](https://github.com/NVIDIA/spark-rapids/pull/10870)|Add support for self-contained profiling| +|[#10903](https://github.com/NVIDIA/spark-rapids/pull/10903)|Use upper case for LEGACY_TIME_PARSER_POLICY to fix a spark UT| +|[#10900](https://github.com/NVIDIA/spark-rapids/pull/10900)|Fix type convert error in format_number scalar input| +|[#10868](https://github.com/NVIDIA/spark-rapids/pull/10868)|Disable default cuDF pinned pool| +|[#10914](https://github.com/NVIDIA/spark-rapids/pull/10914)|Fix NoSuchElementException when rlike with empty pattern| +|[#10858](https://github.com/NVIDIA/spark-rapids/pull/10858)|Add Support for Multiple Filtering Keys for Subquery Broadcast | +|[#10861](https://github.com/NVIDIA/spark-rapids/pull/10861)|refine ut framework including Part 1 and Part 2| +|[#10872](https://github.com/NVIDIA/spark-rapids/pull/10872)|[DOC] ignore released plugin links to reduce the bother info [skip ci]| +|[#10839](https://github.com/NVIDIA/spark-rapids/pull/10839)|Replace anonymous classes for SortOrder and FIlterExec overrides| +|[#10873](https://github.com/NVIDIA/spark-rapids/pull/10873)|Auto merge PRs to branch-24.08 from branch-24.06 [skip ci]| +|[#10860](https://github.com/NVIDIA/spark-rapids/pull/10860)|[Spark 4.0] Account for `PartitionedFileUtil.getPartitionedFile` signature change.| +|[#10822](https://github.com/NVIDIA/spark-rapids/pull/10822)|Rewrite regex pattern `literal[a-b]{x}` to custom kernel in rlike| +|[#10833](https://github.com/NVIDIA/spark-rapids/pull/10833)|Filter out unused json_path tokens| +|[#10855](https://github.com/NVIDIA/spark-rapids/pull/10855)|Fix auto merge conflict 10845 [[skip ci]]| +|[#10826](https://github.com/NVIDIA/spark-rapids/pull/10826)|Add NVTX ranges to identify Spark stages and tasks| +|[#10846](https://github.com/NVIDIA/spark-rapids/pull/10846)|Update latest changelog [skip ci]| +|[#10836](https://github.com/NVIDIA/spark-rapids/pull/10836)|Catch exceptions when trying to examine Iceberg scan for metadata queries| +|[#10824](https://github.com/NVIDIA/spark-rapids/pull/10824)|Support zstd for GPU shuffle compression| +|[#10828](https://github.com/NVIDIA/spark-rapids/pull/10828)|Added DateTimeUtilsShims [Databricks]| +|[#10829](https://github.com/NVIDIA/spark-rapids/pull/10829)|Fix `Inheritance Shadowing` to add support for Spark 4.0.0| +|[#10811](https://github.com/NVIDIA/spark-rapids/pull/10811)|Fix NPE in GpuParseUrl for null keys.| +|[#10723](https://github.com/NVIDIA/spark-rapids/pull/10723)|Implement chunked ORC reader| +|[#10715](https://github.com/NVIDIA/spark-rapids/pull/10715)|Rewrite some rlike expression to StartsWith/Contains| +|[#10820](https://github.com/NVIDIA/spark-rapids/pull/10820)|workaround #10801 temporally| +|[#10812](https://github.com/NVIDIA/spark-rapids/pull/10812)|Replace ThreadPoolExecutor creation with ThreadUtils API| +|[#10816](https://github.com/NVIDIA/spark-rapids/pull/10816)|Fix a test error for DB13.3| +|[#10813](https://github.com/NVIDIA/spark-rapids/pull/10813)|Fix the errors for Pandas UDF tests on DB13.3| +|[#10795](https://github.com/NVIDIA/spark-rapids/pull/10795)|Remove fixed seed for exact `percentile` integration tests| +|[#10805](https://github.com/NVIDIA/spark-rapids/pull/10805)|Drop Support for CentOS 7| +|[#10800](https://github.com/NVIDIA/spark-rapids/pull/10800)|Add number normalization test and address followup for getJsonObject| +|[#10796](https://github.com/NVIDIA/spark-rapids/pull/10796)|fixing build break on DBR| +|[#10791](https://github.com/NVIDIA/spark-rapids/pull/10791)|Fix auto merge conflict 10779 [skip ci]| +|[#10636](https://github.com/NVIDIA/spark-rapids/pull/10636)|Update actions version [skip ci]| +|[#10743](https://github.com/NVIDIA/spark-rapids/pull/10743)|initial PR for the framework reusing Vanilla Spark's unit tests| +|[#10767](https://github.com/NVIDIA/spark-rapids/pull/10767)|Add rows-only batches support to RebatchingRoundoffIterator| +|[#10763](https://github.com/NVIDIA/spark-rapids/pull/10763)|Add in the GpuArrayFilter command| +|[#10766](https://github.com/NVIDIA/spark-rapids/pull/10766)|Fix dead links related to tools documentation [skip ci]| +|[#10644](https://github.com/NVIDIA/spark-rapids/pull/10644)|Add logging to Integration test runs in local and local-cluster mode| +|[#10756](https://github.com/NVIDIA/spark-rapids/pull/10756)|Fix Authorization Failure While Reading Tables From Unity Catalog| +|[#10752](https://github.com/NVIDIA/spark-rapids/pull/10752)|Add SparkRapidsBuildInfoEvent to the event log| +|[#10754](https://github.com/NVIDIA/spark-rapids/pull/10754)|Substitute whoami for $USER| +|[#10755](https://github.com/NVIDIA/spark-rapids/pull/10755)|[DOC] Update README for prioritize-commits script [skip ci]| +|[#10728](https://github.com/NVIDIA/spark-rapids/pull/10728)|Let big data gen set nullability recursively| +|[#10740](https://github.com/NVIDIA/spark-rapids/pull/10740)|Use parse_url kernel for PATH parsing| +|[#10734](https://github.com/NVIDIA/spark-rapids/pull/10734)|Add short circuit path for get-json-object when there is separate wildcard path| +|[#10725](https://github.com/NVIDIA/spark-rapids/pull/10725)|Initial definition for Spark 4.0.0 shim| +|[#10635](https://github.com/NVIDIA/spark-rapids/pull/10635)|Use new getJsonObject kernel for json_tuple| +|[#10739](https://github.com/NVIDIA/spark-rapids/pull/10739)|Use fixed seed for some random failed tests| +|[#10720](https://github.com/NVIDIA/spark-rapids/pull/10720)|Add Shims for Spark 3.4.3| +|[#10716](https://github.com/NVIDIA/spark-rapids/pull/10716)|Remove the mixedType config for JSON as it has no downsides any longer| +|[#10733](https://github.com/NVIDIA/spark-rapids/pull/10733)|Fix "Could not find any rapids-4-spark jars in classpath" error when debugging UT in IDEA| +|[#10718](https://github.com/NVIDIA/spark-rapids/pull/10718)|Change parameters for memory limit in Parquet chunked reader| +|[#10292](https://github.com/NVIDIA/spark-rapids/pull/10292)|Upgrade to UCX 1.16.0| +|[#10709](https://github.com/NVIDIA/spark-rapids/pull/10709)|Removing some authorizations for departed users [skip ci]| +|[#10726](https://github.com/NVIDIA/spark-rapids/pull/10726)|Append new authorized user to blossom-ci whitelist [skip ci]| +|[#10708](https://github.com/NVIDIA/spark-rapids/pull/10708)|Updated dump tool to verify get_json_object| +|[#10706](https://github.com/NVIDIA/spark-rapids/pull/10706)|Fix auto merge conflict 10704 [skip ci]| +|[#10675](https://github.com/NVIDIA/spark-rapids/pull/10675)|Fix merge conflict with branch-24.04 [skip ci]| +|[#10678](https://github.com/NVIDIA/spark-rapids/pull/10678)|Append new authorized user to blossom-ci whitelist [skip ci]| +|[#10662](https://github.com/NVIDIA/spark-rapids/pull/10662)|Audit script - Check commits from shuffle and storage directories [skip ci]| +|[#10655](https://github.com/NVIDIA/spark-rapids/pull/10655)|Update rapids jni/private dependency to 24.06| +|[#10652](https://github.com/NVIDIA/spark-rapids/pull/10652)|Substitute murmurHash32 for spark32BitMurmurHash3| ## Release 24.04 @@ -85,8 +199,12 @@ Generated on 2024-05-20 ### PRs ||| |:---|:---| +|[#10844](https://github.com/NVIDIA/spark-rapids/pull/10844)|Update rapids private dependency to 24.04.3| +|[#10788](https://github.com/NVIDIA/spark-rapids/pull/10788)|[DOC] Update archive page for v24.04.1 [skip ci]| +|[#10784](https://github.com/NVIDIA/spark-rapids/pull/10784)|Update latest changelog [skip ci]| |[#10782](https://github.com/NVIDIA/spark-rapids/pull/10782)|Update latest changelog [skip ci]| |[#10780](https://github.com/NVIDIA/spark-rapids/pull/10780)|[DOC]Update download page for v24.04.1 [skip ci]| +|[#10778](https://github.com/NVIDIA/spark-rapids/pull/10778)|Update version to 24.04.1-SNAPSHOT| |[#10777](https://github.com/NVIDIA/spark-rapids/pull/10777)|Update rapids JNI dependency: private to 24.04.2| |[#10683](https://github.com/NVIDIA/spark-rapids/pull/10683)|Update latest changelog [skip ci]| |[#10681](https://github.com/NVIDIA/spark-rapids/pull/10681)|Update rapids JNI dependency to 24.04.0, private to 24.04.1| @@ -172,307 +290,7 @@ Generated on 2024-05-20 |[#10348](https://github.com/NVIDIA/spark-rapids/pull/10348)|Remove redundant joinOutputRows metric| |[#10321](https://github.com/NVIDIA/spark-rapids/pull/10321)|Bump up dependency version to 24.04.0-SNAPSHOT| |[#10330](https://github.com/NVIDIA/spark-rapids/pull/10330)|Add tryAcquire to GpuSemaphore| -|[#10331](https://github.com/NVIDIA/spark-rapids/pull/10331)|Revert "Update to libcudf unsigned sum aggregation types change (#10267)"| |[#10258](https://github.com/NVIDIA/spark-rapids/pull/10258)|Init project version 24.04.0-SNAPSHOT| -## Release 24.02 - -### Features -||| -|:---|:---| -|[#9926](https://github.com/NVIDIA/spark-rapids/issues/9926)|[FEA] Add config option for the parquet reader input read limit.| -|[#10270](https://github.com/NVIDIA/spark-rapids/issues/10270)|[FEA] Add support for single quotes when reading JSON| -|[#10253](https://github.com/NVIDIA/spark-rapids/issues/10253)|[FEA] Enable mixed types as string in GpuJsonToStruct| -|[#9692](https://github.com/NVIDIA/spark-rapids/issues/9692)|[FEA] Remove Pascal support| -|[#8806](https://github.com/NVIDIA/spark-rapids/issues/8806)|[FEA] Support lazy quantifier and specified group index in regexp_extract function| -|[#10079](https://github.com/NVIDIA/spark-rapids/issues/10079)|[FEA] Add string parameter support for `unix_timestamp` for non-UTC time zones| -|[#9667](https://github.com/NVIDIA/spark-rapids/issues/9667)|[FEA][JSON] Add support for non default `dateFormat` in `from_json`| -|[#9173](https://github.com/NVIDIA/spark-rapids/issues/9173)|[FEA] Support format_number | -|[#10145](https://github.com/NVIDIA/spark-rapids/issues/10145)|[FEA] Support to_utc_timestamp| -|[#9927](https://github.com/NVIDIA/spark-rapids/issues/9927)|[FEA] Support to_date with non-UTC timezones without DST| -|[#10006](https://github.com/NVIDIA/spark-rapids/issues/10006)|[FEA] Support ```ParseToTimestamp``` for non-UTC time zones| -|[#9096](https://github.com/NVIDIA/spark-rapids/issues/9096)|[FEA] Add Spark 3.3.4 support| -|[#9585](https://github.com/NVIDIA/spark-rapids/issues/9585)|[FEA] support ascii function| -|[#9260](https://github.com/NVIDIA/spark-rapids/issues/9260)|[FEA] Create Spark 3.4.2 shim and build env| -|[#10076](https://github.com/NVIDIA/spark-rapids/issues/10076)|[FEA] Add performance test framework for non-UTC time zone features.| -|[#9881](https://github.com/NVIDIA/spark-rapids/issues/9881)|[TASK] Remove `spark.rapids.sql.nonUTC.enabled` configuration option| -|[#9801](https://github.com/NVIDIA/spark-rapids/issues/9801)|[FEA] Support DateFormat on GPU with a non-UTC timezone| -|[#6834](https://github.com/NVIDIA/spark-rapids/issues/6834)|[FEA] Support GpuHour expression for timezones other than UTC| -|[#6842](https://github.com/NVIDIA/spark-rapids/issues/6842)|[FEA] Support TimeZone aware operations for value extraction| -|[#1860](https://github.com/NVIDIA/spark-rapids/issues/1860)|[FEA] Optimize row based window operations for BOUNDED ranges| -|[#9606](https://github.com/NVIDIA/spark-rapids/issues/9606)|[FEA] Support unix_timestamp with CST(China Time Zone) support| -|[#9815](https://github.com/NVIDIA/spark-rapids/issues/9815)|[FEA] Support ```unix_timestamp``` for non-DST timezones| -|[#8807](https://github.com/NVIDIA/spark-rapids/issues/8807)|[FEA] support ‘yyyyMMdd’ format in from_unixtime function| -|[#9605](https://github.com/NVIDIA/spark-rapids/issues/9605)|[FEA] Support from_unixtime with CST(China Time Zone) support| -|[#6836](https://github.com/NVIDIA/spark-rapids/issues/6836)|[FEA] Support FromUnixTime for non UTC timezones| -|[#9175](https://github.com/NVIDIA/spark-rapids/issues/9175)|[FEA] Support Databricks 13.3| -|[#6881](https://github.com/NVIDIA/spark-rapids/issues/6881)|[FEA] Support RAPIDS Spark plugin on ARM| -|[#9274](https://github.com/NVIDIA/spark-rapids/issues/9274)|[FEA] Regular deploy process to include arm artifacts| -|[#9844](https://github.com/NVIDIA/spark-rapids/issues/9844)|[FEA] Let Gpu arrow python runners support writing one batch one time for the single threaded model.| -|[#7309](https://github.com/NVIDIA/spark-rapids/issues/7309)|[FEA] Detect multiple versions of the RAPIDS jar on the classpath at the same time| - -### Performance -||| -|:---|:---| -|[#9442](https://github.com/NVIDIA/spark-rapids/issues/9442)|[FEA] For hash joins where the build side can change use the smaller table for the build side| -|[#10142](https://github.com/NVIDIA/spark-rapids/issues/10142)|[TASK] Benchmark existing timestamp functions that work in non-UTC time zone (non-DST)| - -### Bugs Fixed -||| -|:---|:---| -|[#10548](https://github.com/NVIDIA/spark-rapids/issues/10548)|[BUG] test_dpp_bypass / test_dpp_via_aggregate_subquery failures in CI Databricks 13.3| -|[#10530](https://github.com/NVIDIA/spark-rapids/issues/10530)|test_delta_merge_match_delete_only java.lang.OutOfMemoryError: GC overhead limit exceeded| -|[#10464](https://github.com/NVIDIA/spark-rapids/issues/10464)|[BUG] spark334 and spark342 shims missed in scala2.13 dist jar| -|[#10473](https://github.com/NVIDIA/spark-rapids/issues/10473)|[BUG] Leak when running RANK query| -|[#10432](https://github.com/NVIDIA/spark-rapids/issues/10432)|Plug-in Build Failing for Databricks 11.3 | -|[#9974](https://github.com/NVIDIA/spark-rapids/issues/9974)|[BUG] host memory Leak in MultiFileCoalescingPartitionReaderBase in UTC time zone| -|[#10359](https://github.com/NVIDIA/spark-rapids/issues/10359)|[BUG] Build failure on Databricks nightly run with `GpuMapInPandasExecMeta`| -|[#10327](https://github.com/NVIDIA/spark-rapids/issues/10327)|[BUG] Unit test FAILED against : SPARK-24957: average with decimal followed by aggregation returning wrong result | -|[#10324](https://github.com/NVIDIA/spark-rapids/issues/10324)|[BUG] hash_aggregate_test.py test FAILED: Type conversion is not allowed from Table {...}| -|[#10291](https://github.com/NVIDIA/spark-rapids/issues/10291)|[BUG] SIGSEGV in libucp.so| -|[#9212](https://github.com/NVIDIA/spark-rapids/issues/9212)|[BUG] `from_json` fails with cuDF error `Invalid list size computation error`| -|[#10264](https://github.com/NVIDIA/spark-rapids/issues/10264)|[BUG] hash aggregate test failures due to type conversion errors| -|[#10262](https://github.com/NVIDIA/spark-rapids/issues/10262)|[BUG] Test "SPARK-24957: average with decimal followed by aggregation returning wrong result" failed.| -|[#9353](https://github.com/NVIDIA/spark-rapids/issues/9353)|[BUG] [JSON] A mix of lists and structs within the same column is not supported| -|[#10099](https://github.com/NVIDIA/spark-rapids/issues/10099)|[BUG] orc_test.py::test_orc_scan_with_aggregate_pushdown fails with a standalone cluster on spark 3.3.0| -|[#10047](https://github.com/NVIDIA/spark-rapids/issues/10047)|[BUG] CudfException during conditional hash join while running nds query64| -|[#9779](https://github.com/NVIDIA/spark-rapids/issues/9779)|[BUG] 330cdh failed test_hash_reduction_sum_full_decimal on CI| -|[#10197](https://github.com/NVIDIA/spark-rapids/issues/10197)|[BUG] Disable GetJsonObject by default and update docs| -|[#10165](https://github.com/NVIDIA/spark-rapids/issues/10165)|[BUG] Databricks 13.3 executor side broadcast failure| -|[#10224](https://github.com/NVIDIA/spark-rapids/issues/10224)|[BUG] DBR builds fails when installing Maven| -|[#10222](https://github.com/NVIDIA/spark-rapids/issues/10222)|[BUG] to_utc_timestamp and from_utc_timestamp fallback when TZ is supported time zone| -|[#10195](https://github.com/NVIDIA/spark-rapids/issues/10195)|[BUG] test_window_aggs_for_negative_rows_partitioned failure in CI| -|[#10182](https://github.com/NVIDIA/spark-rapids/issues/10182)|[BUG] test_dpp_bypass / test_dpp_via_aggregate_subquery failures in CI (databricks)| -|[#10169](https://github.com/NVIDIA/spark-rapids/issues/10169)|[BUG] Host column vector leaks when running `test_cast_timestamp_to_date`| -|[#10050](https://github.com/NVIDIA/spark-rapids/issues/10050)|[BUG] test_cast_decimal_to_decimal[to:DecimalType(1,-1)-from:Decimal(5,-3)] fails with DATAGEN_SEED=1702439569| -|[#10088](https://github.com/NVIDIA/spark-rapids/issues/10088)|[BUG] GpuExplode single row split to fit cuDF limits| -|[#10174](https://github.com/NVIDIA/spark-rapids/issues/10174)|[BUG] json_test.py::test_from_json_struct_timestamp failed on: Part of the plan is not columnar | -|[#10186](https://github.com/NVIDIA/spark-rapids/issues/10186)|[BUG] test_to_date_with_window_functions failed in non-UTC nightly CI| -|[#10154](https://github.com/NVIDIA/spark-rapids/issues/10154)|[BUG] 'spark-test.sh' integration tests FAILED on 'ps: command not found" in Rocky Docker environment| -|[#10175](https://github.com/NVIDIA/spark-rapids/issues/10175)|[BUG] string_test.py::test_format_number_float_special FAILED : AssertionError 'NaN' == | -|[#10166](https://github.com/NVIDIA/spark-rapids/issues/10166)|Detect Undeclared Shim in POM.xml| -|[#10170](https://github.com/NVIDIA/spark-rapids/issues/10170)|[BUG] `test_cast_timestamp_to_date` fails with `TZ=Asia/Hebron`| -|[#10149](https://github.com/NVIDIA/spark-rapids/issues/10149)|[BUG] GPU illegal access detected during delta_byte_array.parquet read| -|[#9905](https://github.com/NVIDIA/spark-rapids/issues/9905)|[BUG] GpuJsonScan incorrect behavior when parsing dates| -|[#10163](https://github.com/NVIDIA/spark-rapids/issues/10163)|Spark 3.3.4 Shim Build Failure| -|[#10105](https://github.com/NVIDIA/spark-rapids/issues/10105)|[BUG] scala:compile is not thread safe unless compiler bridge already exists | -|[#10026](https://github.com/NVIDIA/spark-rapids/issues/10026)|[BUG] test_hash_agg_with_nan_keys failed with a DATAGEN_SEED=1702335559| -|[#10075](https://github.com/NVIDIA/spark-rapids/issues/10075)|[BUG] `non-pinned blocking alloc with spill` unit test failed in HostAllocSuite| -|[#10134](https://github.com/NVIDIA/spark-rapids/issues/10134)|[BUG] test_window_aggs_for_batched_finite_row_windows_partitioned failed on Scala 2.13 with DATAGEN_SEED=1704033145| -|[#10118](https://github.com/NVIDIA/spark-rapids/issues/10118)|[BUG] non-UTC Nightly CI failed| -|[#10136](https://github.com/NVIDIA/spark-rapids/issues/10136)|[BUG] The canonicalized version of `GpuFileSourceScanExec`s that suppose to be semantic-equal can be different | -|[#10110](https://github.com/NVIDIA/spark-rapids/issues/10110)|[BUG] disable collect_list and collect_set for window operations by default.| -|[#10129](https://github.com/NVIDIA/spark-rapids/issues/10129)|[BUG] Unit test suite fails with `Null data pointer` in GpuTimeZoneDB| -|[#10089](https://github.com/NVIDIA/spark-rapids/issues/10089)|[BUG] DATAGEN_SEED= environment does not override the marker datagen_overrides| -|[#10108](https://github.com/NVIDIA/spark-rapids/issues/10108)|[BUG] @datagen_overrides seed is sticky when it shouldn't be| -|[#10064](https://github.com/NVIDIA/spark-rapids/issues/10064)|[BUG] test_unsupported_fallback_regexp_replace failed with DATAGEN_SEED=1702662063| -|[#10117](https://github.com/NVIDIA/spark-rapids/issues/10117)|[BUG] test_from_utc_timestamp failed on Cloudera Env when TZ is Iran| -|[#9914](https://github.com/NVIDIA/spark-rapids/issues/9914)|[BUG] Report GPU OOM on recent passed CI premerges.| -|[#10094](https://github.com/NVIDIA/spark-rapids/issues/10094)|[BUG] spark351 PR check failure MockTaskContext method isFailed in class TaskContext of type ()Boolean is not defined| -|[#10017](https://github.com/NVIDIA/spark-rapids/issues/10017)|[BUG] test_casting_from_double_to_timestamp failed for DATAGEN_SEED=1702329497| -|[#9992](https://github.com/NVIDIA/spark-rapids/issues/9992)|[BUG] conditionals_test.py::test_conditional_with_side_effects_cast[String] failed with DATAGEN_SEED=1701976979| -|[#9743](https://github.com/NVIDIA/spark-rapids/issues/9743)|[BUG][AUDIT] SPARK-45652 - SPJ: Handle empty input partitions after dynamic filtering| -|[#9859](https://github.com/NVIDIA/spark-rapids/issues/9859)|[AUDIT] [SPARK-45786] Inaccurate Decimal multiplication and division results| -|[#9555](https://github.com/NVIDIA/spark-rapids/issues/9555)|[BUG] Scala 2.13 build with JDK 11 or 17 fails OpcodeSuite tests| -|[#10073](https://github.com/NVIDIA/spark-rapids/issues/10073)|[BUG] test_csv_prefer_date_with_infer_schema failed with DATAGEN_SEED=1702847907| -|[#10004](https://github.com/NVIDIA/spark-rapids/issues/10004)|[BUG] If a host memory buffer is spilled, it cannot be unspilled| -|[#10063](https://github.com/NVIDIA/spark-rapids/issues/10063)|[BUG] CI build failure with 341db: method getKillReason has weaker access privileges; it should be public| -|[#10055](https://github.com/NVIDIA/spark-rapids/issues/10055)|[BUG] array_test.py::test_array_transform_non_deterministic failed with non-UTC time zone| -|[#10056](https://github.com/NVIDIA/spark-rapids/issues/10056)|[BUG] Unit tests ToPrettyStringSuite FAILED on spark-3.5.0| -|[#10048](https://github.com/NVIDIA/spark-rapids/issues/10048)|[BUG] Fix ```out of range``` error from ```pySpark``` in ```test_timestamp_millis``` and other two integration test cases| -|[#4204](https://github.com/NVIDIA/spark-rapids/issues/4204)|casting double to string does not match Spark| -|[#9938](https://github.com/NVIDIA/spark-rapids/issues/9938)|Better to do some refactor for the Python UDF code| -|[#10018](https://github.com/NVIDIA/spark-rapids/issues/10018)|[BUG] `GpuToUnixTimestampImproved` off by 1 on GPU when handling timestamp before epoch| -|[#10012](https://github.com/NVIDIA/spark-rapids/issues/10012)|[BUG] test_str_to_map_expr_random_delimiters with DATAGEN_SEED=1702166057 hangs| -|[#10029](https://github.com/NVIDIA/spark-rapids/issues/10029)|[BUG] doc links fail with 404 for shims.md| -|[#9472](https://github.com/NVIDIA/spark-rapids/issues/9472)|[BUG] Non-Deterministic expressions in an array_transform can cause errors| -|[#9884](https://github.com/NVIDIA/spark-rapids/issues/9884)|[BUG] delta_lake_delete_test.py failed assertion [DATAGEN_SEED=1701225104, IGNORE_ORDER...| -|[#9977](https://github.com/NVIDIA/spark-rapids/issues/9977)|[BUG] test_cast_date_integral fails on databricks 3.4.1| -|[#9936](https://github.com/NVIDIA/spark-rapids/issues/9936)|[BUG] Nightly CI of non-UTC time zone reports 'year 0 is out of range' error| -|[#9941](https://github.com/NVIDIA/spark-rapids/issues/9941)|[BUG] A potential data corruption in Pandas UDFs| -|[#9897](https://github.com/NVIDIA/spark-rapids/issues/9897)|[BUG] Error message for multiple jars on classpath is wrong| -|[#9916](https://github.com/NVIDIA/spark-rapids/issues/9916)|[BUG] ```test_cast_string_ts_valid_format``` failed at ```seed = 1701362564```| -|[#9559](https://github.com/NVIDIA/spark-rapids/issues/9559)|[BUG] precommit regularly fails with error trying to download a dependency| -|[#9708](https://github.com/NVIDIA/spark-rapids/issues/9708)|[BUG] test_cast_string_ts_valid_format fails with DATAGEN_SEED=1699978422| - -### PRs -||| -|:---|:---| -|[#10555](https://github.com/NVIDIA/spark-rapids/pull/10555)|Update change log [skip ci]| -|[#10551](https://github.com/NVIDIA/spark-rapids/pull/10551)|Try to make degenerative joins here impossible for these tests| -|[#10546](https://github.com/NVIDIA/spark-rapids/pull/10546)|Update changelog [skip ci]| -|[#10541](https://github.com/NVIDIA/spark-rapids/pull/10541)|Fix Delta log cache size settings during integration tests| -|[#10525](https://github.com/NVIDIA/spark-rapids/pull/10525)|Update changelog for v24.02.0 release [skip ci]| -|[#10465](https://github.com/NVIDIA/spark-rapids/pull/10465)|Add missed shims for scala2.13| -|[#10511](https://github.com/NVIDIA/spark-rapids/pull/10511)|Update rapids jni and private dependency version to 24.02.1| -|[#10513](https://github.com/NVIDIA/spark-rapids/pull/10513)|Fix scalar leak in SumBinaryFixer (#10510)| -|[#10475](https://github.com/NVIDIA/spark-rapids/pull/10475)|Fix scalar leak in RankFixer| -|[#10461](https://github.com/NVIDIA/spark-rapids/pull/10461)|Preserve tags on FileSourceScanExec| -|[#10459](https://github.com/NVIDIA/spark-rapids/pull/10459)|[DOC] Fix table rendering issue in github.io download UI page on branch-24.02 [skip ci] | -|[#10443](https://github.com/NVIDIA/spark-rapids/pull/10443)|Update change log for v24.02.0 release [skip ci]| -|[#10439](https://github.com/NVIDIA/spark-rapids/pull/10439)|Reverts NVIDIA/spark-rapids#10232 and fixes the plugin build on Databricks 11.3| -|[#10380](https://github.com/NVIDIA/spark-rapids/pull/10380)|Init changelog 24.02 [skip ci]| -|[#10367](https://github.com/NVIDIA/spark-rapids/pull/10367)|Update rapids JNI and private version to release 24.02.0| -|[#10414](https://github.com/NVIDIA/spark-rapids/pull/10414)|[DOC] Fix 24.02.0 documentation errors [skip ci]| -|[#10403](https://github.com/NVIDIA/spark-rapids/pull/10403)|Cherry-pick: Fix a memory leak in json tuple (#10360)| -|[#10387](https://github.com/NVIDIA/spark-rapids/pull/10387)|[DOC] Update docs for 24.02.0 release [skip ci]| -|[#10399](https://github.com/NVIDIA/spark-rapids/pull/10399)|Update NOTICE-binary| -|[#10389](https://github.com/NVIDIA/spark-rapids/pull/10389)|Change version and branch to 24.02 in docs [skip ci]| -|[#10384](https://github.com/NVIDIA/spark-rapids/pull/10384)|[DOC] Update docs for 23.12.2 release [skip ci] | -|[#10309](https://github.com/NVIDIA/spark-rapids/pull/10309)|[DOC] add custom 404 page and fix some document issue [skip ci]| -|[#10352](https://github.com/NVIDIA/spark-rapids/pull/10352)|xfail mixed type test| -|[#10355](https://github.com/NVIDIA/spark-rapids/pull/10355)|Revert "Support barrier mode for mapInPandas/mapInArrow (#10343)"| -|[#10353](https://github.com/NVIDIA/spark-rapids/pull/10353)|Use fixed seed for test_from_json_struct_decimal| -|[#10343](https://github.com/NVIDIA/spark-rapids/pull/10343)|Support barrier mode for mapInPandas/mapInArrow| -|[#10345](https://github.com/NVIDIA/spark-rapids/pull/10345)|Fix auto merge conflict 10339 [skip ci]| -|[#9991](https://github.com/NVIDIA/spark-rapids/pull/9991)|Start to use explicit memory limits in the parquet chunked reader| -|[#10328](https://github.com/NVIDIA/spark-rapids/pull/10328)|Fix typo in spark-tests.sh [skip ci]| -|[#10279](https://github.com/NVIDIA/spark-rapids/pull/10279)|Run '--packages' only with default cuda11 jar| -|[#10273](https://github.com/NVIDIA/spark-rapids/pull/10273)|Support reading JSON data with single quotes around attribute names and values| -|[#10306](https://github.com/NVIDIA/spark-rapids/pull/10306)|Fix performance regression in from_json| -|[#10272](https://github.com/NVIDIA/spark-rapids/pull/10272)|Add FullOuter support to GpuShuffledSymmetricHashJoinExec| -|[#10260](https://github.com/NVIDIA/spark-rapids/pull/10260)|Add perf test for time zone operators| -|[#10275](https://github.com/NVIDIA/spark-rapids/pull/10275)|Add tests for window Python udf with array input| -|[#10278](https://github.com/NVIDIA/spark-rapids/pull/10278)|Clean up $M2_CACHE to avoid side-effect of previous dependency:get [skip ci]| -|[#10268](https://github.com/NVIDIA/spark-rapids/pull/10268)|Add config to enable mixed types as string in GpuJsonToStruct & GpuJsonScan| -|[#10297](https://github.com/NVIDIA/spark-rapids/pull/10297)|Revert "UCX 1.16.0 upgrade (#10190)"| -|[#10289](https://github.com/NVIDIA/spark-rapids/pull/10289)|Add gerashegalov to CODEOWNERS [skip ci]| -|[#10290](https://github.com/NVIDIA/spark-rapids/pull/10290)|Fix merge conflict with 23.12 [skip ci]| -|[#10190](https://github.com/NVIDIA/spark-rapids/pull/10190)|UCX 1.16.0 upgrade| -|[#10211](https://github.com/NVIDIA/spark-rapids/pull/10211)|Use parse_url kernel for QUERY literal and column key| -|[#10267](https://github.com/NVIDIA/spark-rapids/pull/10267)|Update to libcudf unsigned sum aggregation types change| -|[#10208](https://github.com/NVIDIA/spark-rapids/pull/10208)|Added Support for Lazy Quantifier| -|[#9993](https://github.com/NVIDIA/spark-rapids/pull/9993)|Enable mixed types as string in GpuJsonScan| -|[#10246](https://github.com/NVIDIA/spark-rapids/pull/10246)|Refactor full join iterator to allow access to build tracker| -|[#10257](https://github.com/NVIDIA/spark-rapids/pull/10257)|Enable auto-merge from branch-24.02 to branch-24.04 [skip CI]| -|[#10178](https://github.com/NVIDIA/spark-rapids/pull/10178)|Mark hash reduction decimal overflow test as a permanent seed override| -|[#10244](https://github.com/NVIDIA/spark-rapids/pull/10244)|Use POSIX mode in assembly plugin to avoid issues with large UID/GID| -|[#10238](https://github.com/NVIDIA/spark-rapids/pull/10238)|Smoke test with '--package' to fetch the plugin jar| -|[#10201](https://github.com/NVIDIA/spark-rapids/pull/10201)|Deploy release candidates to local maven repo for dependency check[skip ci]| -|[#10240](https://github.com/NVIDIA/spark-rapids/pull/10240)|Improved inner joins with large build side| -|[#10220](https://github.com/NVIDIA/spark-rapids/pull/10220)|Disable GetJsonObject by default and add tests for as many issues with it as possible| -|[#10230](https://github.com/NVIDIA/spark-rapids/pull/10230)|Fix Databricks 13.3 BroadcastHashJoin using executor side broadcast fed by ColumnarToRow [Databricks]| -|[#10232](https://github.com/NVIDIA/spark-rapids/pull/10232)|Fixed 330db Shims to Adopt the PythonRunner Changes| -|[#10225](https://github.com/NVIDIA/spark-rapids/pull/10225)|Download Maven from apache.org archives [skip ci]| -|[#10210](https://github.com/NVIDIA/spark-rapids/pull/10210)|Add string parameter support for unix_timestamp for non-UTC time zones| -|[#10223](https://github.com/NVIDIA/spark-rapids/pull/10223)|Fix to_utc_timestamp and from_utc_timestamp fallback when TZ is supported time zone| -|[#10205](https://github.com/NVIDIA/spark-rapids/pull/10205)|Deterministic ordering in window tests| -|[#10204](https://github.com/NVIDIA/spark-rapids/pull/10204)|Further prevent degenerative joins in dpp_test| -|[#10156](https://github.com/NVIDIA/spark-rapids/pull/10156)|Update string to float compatibility doc[skip ci]| -|[#10193](https://github.com/NVIDIA/spark-rapids/pull/10193)|Fix explode with carry-along columns on GpuExplode single row retry handling| -|[#10191](https://github.com/NVIDIA/spark-rapids/pull/10191)|Updating the config documentation for filecache configs [skip ci]| -|[#10131](https://github.com/NVIDIA/spark-rapids/pull/10131)|With a single row GpuExplode tries to split the generator array| -|[#10179](https://github.com/NVIDIA/spark-rapids/pull/10179)|Fix build regression against Spark 3.2.x| -|[#10189](https://github.com/NVIDIA/spark-rapids/pull/10189)|test needs marks for non-UTC and for non_supported timezones| -|[#10176](https://github.com/NVIDIA/spark-rapids/pull/10176)|Fix format_number NaN symbol in high jdk version| -|[#10074](https://github.com/NVIDIA/spark-rapids/pull/10074)|Update the legacy mode check: only take effect when reading date/timestamp column| -|[#10167](https://github.com/NVIDIA/spark-rapids/pull/10167)|Defined Shims Should Be Declared In POM | -|[#10168](https://github.com/NVIDIA/spark-rapids/pull/10168)|Prevent a degenerative join in test_dpp_reuse_broadcast_exchange| -|[#10171](https://github.com/NVIDIA/spark-rapids/pull/10171)|Fix `test_cast_timestamp_to_date` when running in a DST time zone| -|[#9975](https://github.com/NVIDIA/spark-rapids/pull/9975)|Improve dateFormat support in GpuJsonScan and make tests consistent with GpuStructsToJson| -|[#9790](https://github.com/NVIDIA/spark-rapids/pull/9790)|Support float case of format_number with format_float kernel| -|[#10144](https://github.com/NVIDIA/spark-rapids/pull/10144)|Support to_utc_timestamp| -|[#10162](https://github.com/NVIDIA/spark-rapids/pull/10162)|Fix Spark 334 Build| -|[#10146](https://github.com/NVIDIA/spark-rapids/pull/10146)|Refactor the window code so it is not mostly kept in a few very large files| -|[#10155](https://github.com/NVIDIA/spark-rapids/pull/10155)|Install procps tools for rocky docker images [skip ci]| -|[#10153](https://github.com/NVIDIA/spark-rapids/pull/10153)|Disable multi-threaded Maven | -|[#10100](https://github.com/NVIDIA/spark-rapids/pull/10100)|Enable to_date (via gettimestamp and casting timestamp to date) for non-UTC time zones| -|[#10140](https://github.com/NVIDIA/spark-rapids/pull/10140)|Removed Unnecessary Whitespaces From Spark 3.3.4 Shim [skip ci]| -|[#10148](https://github.com/NVIDIA/spark-rapids/pull/10148)|fix test_hash_agg_with_nan_keys floating point sum failure| -|[#10150](https://github.com/NVIDIA/spark-rapids/pull/10150)|Increase timeouts in HostAllocSuite to avoid timeout failures on slow machines| -|[#10143](https://github.com/NVIDIA/spark-rapids/pull/10143)|Fix `test_window_aggs_for_batched_finite_row_windows_partitioned` fail| -|[#9887](https://github.com/NVIDIA/spark-rapids/pull/9887)|Reduce time-consuming of pre-merge| -|[#10130](https://github.com/NVIDIA/spark-rapids/pull/10130)|Change unit tests that force ooms to specify the oom type (gpu|cpu)| -|[#10138](https://github.com/NVIDIA/spark-rapids/pull/10138)|Update copyright dates in NOTICE files [skip ci]| -|[#10139](https://github.com/NVIDIA/spark-rapids/pull/10139)|Add Delta Lake 2.3.0 to list of versions to test for Spark 3.3.x| -|[#10135](https://github.com/NVIDIA/spark-rapids/pull/10135)|Fix CI: can't find script when there is pushd in script [skip ci]| -|[#10137](https://github.com/NVIDIA/spark-rapids/pull/10137)|Fix the canonicalizing for GPU file scan| -|[#10132](https://github.com/NVIDIA/spark-rapids/pull/10132)|Disable collect_list and collect_set for window by default| -|[#10084](https://github.com/NVIDIA/spark-rapids/pull/10084)|Refactor GpuJsonToStruct to reduce code duplication and manage resources more efficiently| -|[#10087](https://github.com/NVIDIA/spark-rapids/pull/10087)|Additional unit tests for GeneratedInternalRowToCudfRowIterator| -|[#10082](https://github.com/NVIDIA/spark-rapids/pull/10082)|Add Spark 3.3.4 Shim| -|[#10054](https://github.com/NVIDIA/spark-rapids/pull/10054)|Support Ascii function for ascii and latin-1| -|[#10127](https://github.com/NVIDIA/spark-rapids/pull/10127)|Fix merge conflict with branch-23.12| -|[#10097](https://github.com/NVIDIA/spark-rapids/pull/10097)|[DOC] Update docs for 23.12.1 release [skip ci]| -|[#10109](https://github.com/NVIDIA/spark-rapids/pull/10109)|Fixes a bug where datagen seed overrides were sticky and adds datagen_seed_override_disabled| -|[#10093](https://github.com/NVIDIA/spark-rapids/pull/10093)|Fix test_unsupported_fallback_regexp_replace| -|[#10119](https://github.com/NVIDIA/spark-rapids/pull/10119)|Fix from_utc_timestamp case failure on Cloudera when TZ is Iran| -|[#10106](https://github.com/NVIDIA/spark-rapids/pull/10106)|Add `isFailed()` to MockTaskContext and Remove MockTaskContextBase.scala| -|[#10112](https://github.com/NVIDIA/spark-rapids/pull/10112)|Remove datagen seed override for test_conditional_with_side_effects_cast| -|[#10104](https://github.com/NVIDIA/spark-rapids/pull/10104)|[DOC] Add in docs about memory debugging [skip ci]| -|[#9925](https://github.com/NVIDIA/spark-rapids/pull/9925)|Use threads, cache Scala compiler in GH mvn workflow| -|[#9967](https://github.com/NVIDIA/spark-rapids/pull/9967)|Added Spark-3.4.2 Shims| -|[#10061](https://github.com/NVIDIA/spark-rapids/pull/10061)|Use parse_url kernel for QUERY parsing| -|[#10101](https://github.com/NVIDIA/spark-rapids/pull/10101)|[DOC] Add column order error docs [skip ci]| -|[#10078](https://github.com/NVIDIA/spark-rapids/pull/10078)|Add perf test for non-UTC operators| -|[#10096](https://github.com/NVIDIA/spark-rapids/pull/10096)|Shim MockTaskContext to fix Spark 3.5.1 build| -|[#10092](https://github.com/NVIDIA/spark-rapids/pull/10092)|Implement Math.round using floor on GPU| -|[#10085](https://github.com/NVIDIA/spark-rapids/pull/10085)|Update tests that originally restricted the Spark timestamp range| -|[#10090](https://github.com/NVIDIA/spark-rapids/pull/10090)|Replace GPU-unsupported `\z` with an alternative RLIKE expression| -|[#10095](https://github.com/NVIDIA/spark-rapids/pull/10095)|Temporarily fix date format failed cases for non-UTC time zone.| -|[#9999](https://github.com/NVIDIA/spark-rapids/pull/9999)|Add some odd time zones for timezone transition tests| -|[#9962](https://github.com/NVIDIA/spark-rapids/pull/9962)|Add 3.5.1-SNAPSHOT Shim| -|[#10071](https://github.com/NVIDIA/spark-rapids/pull/10071)|Cleanup usage of non-utc configuration here| -|[#10057](https://github.com/NVIDIA/spark-rapids/pull/10057)|Add support for StringConcatFactory.makeConcatWithConstants (#9555)| -|[#9996](https://github.com/NVIDIA/spark-rapids/pull/9996)|Test full timestamp output range in PySpark| -|[#10081](https://github.com/NVIDIA/spark-rapids/pull/10081)|Add a fallback Cloudera Maven repo URL [skip ci]| -|[#10065](https://github.com/NVIDIA/spark-rapids/pull/10065)|Improve host memory spill interfaces| -|[#10069](https://github.com/NVIDIA/spark-rapids/pull/10069)|Revert "Support split broadcast join condition into ast and non-ast […| -|[#10070](https://github.com/NVIDIA/spark-rapids/pull/10070)|Fix 332db build failure| -|[#10060](https://github.com/NVIDIA/spark-rapids/pull/10060)|Fix failed cases for non-utc time zone| -|[#10038](https://github.com/NVIDIA/spark-rapids/pull/10038)|Remove spark.rapids.sql.nonUTC.enabled configuration option| -|[#10059](https://github.com/NVIDIA/spark-rapids/pull/10059)|Fixed Failing ToPrettyStringSuite Test for 3.5.0| -|[#10013](https://github.com/NVIDIA/spark-rapids/pull/10013)|Extended configuration of OOM injection mode| -|[#10052](https://github.com/NVIDIA/spark-rapids/pull/10052)|Set seed=0 for some integration test cases| -|[#10053](https://github.com/NVIDIA/spark-rapids/pull/10053)|Remove invalid user from CODEOWNER file [skip ci]| -|[#10049](https://github.com/NVIDIA/spark-rapids/pull/10049)|Fix out of range error from pySpark in test_timestamp_millis and other two integration test cases| -|[#9721](https://github.com/NVIDIA/spark-rapids/pull/9721)|Support date_format via Gpu for non-UTC time zone| -|[#9470](https://github.com/NVIDIA/spark-rapids/pull/9470)|Use float to string kernel| -|[#9845](https://github.com/NVIDIA/spark-rapids/pull/9845)|Use parse_url kernel for HOST parsing| -|[#10024](https://github.com/NVIDIA/spark-rapids/pull/10024)|Support hour minute second for non-UTC time zone| -|[#9973](https://github.com/NVIDIA/spark-rapids/pull/9973)|Batching support for row-based bounded window functions | -|[#10042](https://github.com/NVIDIA/spark-rapids/pull/10042)|Update tests to not have hard coded fallback when not needed| -|[#9816](https://github.com/NVIDIA/spark-rapids/pull/9816)|Support unix_timestamp and to_unix_timestamp with non-UTC timezones (non-DST)| -|[#9902](https://github.com/NVIDIA/spark-rapids/pull/9902)|Some refactor for the Python UDF code| -|[#10023](https://github.com/NVIDIA/spark-rapids/pull/10023)|GPU supports `yyyyMMdd` format by post process for the `from_unixtime` function| -|[#10033](https://github.com/NVIDIA/spark-rapids/pull/10033)|Remove GpuToTimestampImproved and spark.rapids.sql.improvedTimeOps.enabled| -|[#10016](https://github.com/NVIDIA/spark-rapids/pull/10016)|Fix infinite loop in test_str_to_map_expr_random_delimiters| -|[#9481](https://github.com/NVIDIA/spark-rapids/pull/9481)|Use parse_url kernel for PROTOCOL parsing| -|[#10030](https://github.com/NVIDIA/spark-rapids/pull/10030)|Update links in shims.md| -|[#10015](https://github.com/NVIDIA/spark-rapids/pull/10015)|Fix array_transform to not recompute the argument| -|[#10011](https://github.com/NVIDIA/spark-rapids/pull/10011)|Add cpu oom retry split handling to InternalRowToColumnarBatchIterator| -|[#10019](https://github.com/NVIDIA/spark-rapids/pull/10019)|Fix auto merge conflict 10010 [skip ci]| -|[#9760](https://github.com/NVIDIA/spark-rapids/pull/9760)|Support split broadcast join condition into ast and non-ast| -|[#9827](https://github.com/NVIDIA/spark-rapids/pull/9827)|Enable ORC timestamp and decimal predicate push down tests| -|[#10002](https://github.com/NVIDIA/spark-rapids/pull/10002)|Use Spark 3.3.3 instead of 3.3.2 for Scala 2.13 premerge builds| -|[#10000](https://github.com/NVIDIA/spark-rapids/pull/10000)|Optimize from_unixtime| -|[#10003](https://github.com/NVIDIA/spark-rapids/pull/10003)|Fix merge conflict with branch-23.12| -|[#9984](https://github.com/NVIDIA/spark-rapids/pull/9984)|Fix 340+(including DB341+) does not support casting date to integral/float| -|[#9972](https://github.com/NVIDIA/spark-rapids/pull/9972)|Fix year 0 is out of range in test_from_json_struct_timestamp | -|[#9814](https://github.com/NVIDIA/spark-rapids/pull/9814)|Support from_unixtime via Gpu for non-UTC time zone| -|[#9929](https://github.com/NVIDIA/spark-rapids/pull/9929)|Add host memory retries for GeneratedInternalRowToCudfRowIterator| -|[#9957](https://github.com/NVIDIA/spark-rapids/pull/9957)|Update cases for cast between integral and (date/time)| -|[#9959](https://github.com/NVIDIA/spark-rapids/pull/9959)|Append new authorized user to blossom-ci whitelist [skip ci]| -|[#9942](https://github.com/NVIDIA/spark-rapids/pull/9942)|Fix a potential data corruption for Pandas UDF| -|[#9922](https://github.com/NVIDIA/spark-rapids/pull/9922)|Fix `allowMultipleJars` recommend setting message| -|[#9947](https://github.com/NVIDIA/spark-rapids/pull/9947)|Fix merge conflict with branch-23.12| -|[#9908](https://github.com/NVIDIA/spark-rapids/pull/9908)|Register default allocator for host memory| -|[#9944](https://github.com/NVIDIA/spark-rapids/pull/9944)|Fix Java OOM caused by incorrect state of shouldCapture when exception occurred| -|[#9937](https://github.com/NVIDIA/spark-rapids/pull/9937)|Refactor to use CLASSIFIER instead of CUDA_CLASSIFIER [skip ci]| -|[#9904](https://github.com/NVIDIA/spark-rapids/pull/9904)|Params for build and test CI scripts on Databricks| -|[#9719](https://github.com/NVIDIA/spark-rapids/pull/9719)|Support fine grained timezone checker instead of type based| -|[#9918](https://github.com/NVIDIA/spark-rapids/pull/9918)|Prevent generation of 'year 0 is out of range' strings in IT| -|[#9852](https://github.com/NVIDIA/spark-rapids/pull/9852)|Avoid generating duplicate nan keys with MapGen(FloatGen)| -|[#9674](https://github.com/NVIDIA/spark-rapids/pull/9674)|Add cache action to speed up mvn workflow [skip ci]| -|[#9900](https://github.com/NVIDIA/spark-rapids/pull/9900)|Revert "Remove Databricks 13.3 from release 23.12 (#9890)"| -|[#9889](https://github.com/NVIDIA/spark-rapids/pull/9889)|Fix test_cast_string_ts_valid_format test| -|[#9888](https://github.com/NVIDIA/spark-rapids/pull/9888)|Update nightly build and deploy script for arm artifacts [skip ci]| -|[#9833](https://github.com/NVIDIA/spark-rapids/pull/9833)|Fix a hang for Pandas UDFs on DB 13.3| -|[#9656](https://github.com/NVIDIA/spark-rapids/pull/9656)|Update for new retry state machine JNI APIs| -|[#9654](https://github.com/NVIDIA/spark-rapids/pull/9654)|Detect multiple jars on the classpath when init plugin| -|[#9857](https://github.com/NVIDIA/spark-rapids/pull/9857)|Skip redundant steps in nightly build [skip ci]| -|[#9812](https://github.com/NVIDIA/spark-rapids/pull/9812)|Update JNI and private dep version to 24.02.0-SNAPSHOT| -|[#9716](https://github.com/NVIDIA/spark-rapids/pull/9716)|Initiate project version 24.02.0-SNAPSHOT| - ## Older Releases Changelog of older releases can be found at [docs/archives](/docs/archives) diff --git a/docs/archives/CHANGELOG_24.02.md b/docs/archives/CHANGELOG_24.02.md new file mode 100644 index 00000000000..732035502f0 --- /dev/null +++ b/docs/archives/CHANGELOG_24.02.md @@ -0,0 +1,300 @@ +# Change log\nGenerated on 2024-06-05 +## Release 24.02 + +### Features +||| +|:---|:---| +|[#9926](https://github.com/NVIDIA/spark-rapids/issues/9926)|[FEA] Add config option for the parquet reader input read limit.| +|[#10270](https://github.com/NVIDIA/spark-rapids/issues/10270)|[FEA] Add support for single quotes when reading JSON| +|[#10253](https://github.com/NVIDIA/spark-rapids/issues/10253)|[FEA] Enable mixed types as string in GpuJsonToStruct| +|[#9692](https://github.com/NVIDIA/spark-rapids/issues/9692)|[FEA] Remove Pascal support| +|[#8806](https://github.com/NVIDIA/spark-rapids/issues/8806)|[FEA] Support lazy quantifier and specified group index in regexp_extract function| +|[#10079](https://github.com/NVIDIA/spark-rapids/issues/10079)|[FEA] Add string parameter support for `unix_timestamp` for non-UTC time zones| +|[#9667](https://github.com/NVIDIA/spark-rapids/issues/9667)|[FEA][JSON] Add support for non default `dateFormat` in `from_json`| +|[#9173](https://github.com/NVIDIA/spark-rapids/issues/9173)|[FEA] Support format_number | +|[#10145](https://github.com/NVIDIA/spark-rapids/issues/10145)|[FEA] Support to_utc_timestamp| +|[#9927](https://github.com/NVIDIA/spark-rapids/issues/9927)|[FEA] Support to_date with non-UTC timezones without DST| +|[#10006](https://github.com/NVIDIA/spark-rapids/issues/10006)|[FEA] Support ```ParseToTimestamp``` for non-UTC time zones| +|[#9096](https://github.com/NVIDIA/spark-rapids/issues/9096)|[FEA] Add Spark 3.3.4 support| +|[#9585](https://github.com/NVIDIA/spark-rapids/issues/9585)|[FEA] support ascii function| +|[#9260](https://github.com/NVIDIA/spark-rapids/issues/9260)|[FEA] Create Spark 3.4.2 shim and build env| +|[#10076](https://github.com/NVIDIA/spark-rapids/issues/10076)|[FEA] Add performance test framework for non-UTC time zone features.| +|[#9881](https://github.com/NVIDIA/spark-rapids/issues/9881)|[TASK] Remove `spark.rapids.sql.nonUTC.enabled` configuration option| +|[#9801](https://github.com/NVIDIA/spark-rapids/issues/9801)|[FEA] Support DateFormat on GPU with a non-UTC timezone| +|[#6834](https://github.com/NVIDIA/spark-rapids/issues/6834)|[FEA] Support GpuHour expression for timezones other than UTC| +|[#6842](https://github.com/NVIDIA/spark-rapids/issues/6842)|[FEA] Support TimeZone aware operations for value extraction| +|[#1860](https://github.com/NVIDIA/spark-rapids/issues/1860)|[FEA] Optimize row based window operations for BOUNDED ranges| +|[#9606](https://github.com/NVIDIA/spark-rapids/issues/9606)|[FEA] Support unix_timestamp with CST(China Time Zone) support| +|[#9815](https://github.com/NVIDIA/spark-rapids/issues/9815)|[FEA] Support ```unix_timestamp``` for non-DST timezones| +|[#8807](https://github.com/NVIDIA/spark-rapids/issues/8807)|[FEA] support ‘yyyyMMdd’ format in from_unixtime function| +|[#9605](https://github.com/NVIDIA/spark-rapids/issues/9605)|[FEA] Support from_unixtime with CST(China Time Zone) support| +|[#6836](https://github.com/NVIDIA/spark-rapids/issues/6836)|[FEA] Support FromUnixTime for non UTC timezones| +|[#9175](https://github.com/NVIDIA/spark-rapids/issues/9175)|[FEA] Support Databricks 13.3| +|[#6881](https://github.com/NVIDIA/spark-rapids/issues/6881)|[FEA] Support RAPIDS Spark plugin on ARM| +|[#9274](https://github.com/NVIDIA/spark-rapids/issues/9274)|[FEA] Regular deploy process to include arm artifacts| +|[#9844](https://github.com/NVIDIA/spark-rapids/issues/9844)|[FEA] Let Gpu arrow python runners support writing one batch one time for the single threaded model.| +|[#7309](https://github.com/NVIDIA/spark-rapids/issues/7309)|[FEA] Detect multiple versions of the RAPIDS jar on the classpath at the same time| + +### Performance +||| +|:---|:---| +|[#9442](https://github.com/NVIDIA/spark-rapids/issues/9442)|[FEA] For hash joins where the build side can change use the smaller table for the build side| +|[#10142](https://github.com/NVIDIA/spark-rapids/issues/10142)|[TASK] Benchmark existing timestamp functions that work in non-UTC time zone (non-DST)| + +### Bugs Fixed +||| +|:---|:---| +|[#10548](https://github.com/NVIDIA/spark-rapids/issues/10548)|[BUG] test_dpp_bypass / test_dpp_via_aggregate_subquery failures in CI Databricks 13.3| +|[#10530](https://github.com/NVIDIA/spark-rapids/issues/10530)|test_delta_merge_match_delete_only java.lang.OutOfMemoryError: GC overhead limit exceeded| +|[#10464](https://github.com/NVIDIA/spark-rapids/issues/10464)|[BUG] spark334 and spark342 shims missed in scala2.13 dist jar| +|[#10473](https://github.com/NVIDIA/spark-rapids/issues/10473)|[BUG] Leak when running RANK query| +|[#10432](https://github.com/NVIDIA/spark-rapids/issues/10432)|Plug-in Build Failing for Databricks 11.3 | +|[#9974](https://github.com/NVIDIA/spark-rapids/issues/9974)|[BUG] host memory Leak in MultiFileCoalescingPartitionReaderBase in UTC time zone| +|[#10359](https://github.com/NVIDIA/spark-rapids/issues/10359)|[BUG] Build failure on Databricks nightly run with `GpuMapInPandasExecMeta`| +|[#10327](https://github.com/NVIDIA/spark-rapids/issues/10327)|[BUG] Unit test FAILED against : SPARK-24957: average with decimal followed by aggregation returning wrong result | +|[#10324](https://github.com/NVIDIA/spark-rapids/issues/10324)|[BUG] hash_aggregate_test.py test FAILED: Type conversion is not allowed from Table {...}| +|[#10291](https://github.com/NVIDIA/spark-rapids/issues/10291)|[BUG] SIGSEGV in libucp.so| +|[#9212](https://github.com/NVIDIA/spark-rapids/issues/9212)|[BUG] `from_json` fails with cuDF error `Invalid list size computation error`| +|[#10264](https://github.com/NVIDIA/spark-rapids/issues/10264)|[BUG] hash aggregate test failures due to type conversion errors| +|[#10262](https://github.com/NVIDIA/spark-rapids/issues/10262)|[BUG] Test "SPARK-24957: average with decimal followed by aggregation returning wrong result" failed.| +|[#9353](https://github.com/NVIDIA/spark-rapids/issues/9353)|[BUG] [JSON] A mix of lists and structs within the same column is not supported| +|[#10099](https://github.com/NVIDIA/spark-rapids/issues/10099)|[BUG] orc_test.py::test_orc_scan_with_aggregate_pushdown fails with a standalone cluster on spark 3.3.0| +|[#10047](https://github.com/NVIDIA/spark-rapids/issues/10047)|[BUG] CudfException during conditional hash join while running nds query64| +|[#9779](https://github.com/NVIDIA/spark-rapids/issues/9779)|[BUG] 330cdh failed test_hash_reduction_sum_full_decimal on CI| +|[#10197](https://github.com/NVIDIA/spark-rapids/issues/10197)|[BUG] Disable GetJsonObject by default and update docs| +|[#10165](https://github.com/NVIDIA/spark-rapids/issues/10165)|[BUG] Databricks 13.3 executor side broadcast failure| +|[#10224](https://github.com/NVIDIA/spark-rapids/issues/10224)|[BUG] DBR builds fails when installing Maven| +|[#10222](https://github.com/NVIDIA/spark-rapids/issues/10222)|[BUG] to_utc_timestamp and from_utc_timestamp fallback when TZ is supported time zone| +|[#10195](https://github.com/NVIDIA/spark-rapids/issues/10195)|[BUG] test_window_aggs_for_negative_rows_partitioned failure in CI| +|[#10182](https://github.com/NVIDIA/spark-rapids/issues/10182)|[BUG] test_dpp_bypass / test_dpp_via_aggregate_subquery failures in CI (databricks)| +|[#10169](https://github.com/NVIDIA/spark-rapids/issues/10169)|[BUG] Host column vector leaks when running `test_cast_timestamp_to_date`| +|[#10050](https://github.com/NVIDIA/spark-rapids/issues/10050)|[BUG] test_cast_decimal_to_decimal[to:DecimalType(1,-1)-from:Decimal(5,-3)] fails with DATAGEN_SEED=1702439569| +|[#10088](https://github.com/NVIDIA/spark-rapids/issues/10088)|[BUG] GpuExplode single row split to fit cuDF limits| +|[#10174](https://github.com/NVIDIA/spark-rapids/issues/10174)|[BUG] json_test.py::test_from_json_struct_timestamp failed on: Part of the plan is not columnar | +|[#10186](https://github.com/NVIDIA/spark-rapids/issues/10186)|[BUG] test_to_date_with_window_functions failed in non-UTC nightly CI| +|[#10154](https://github.com/NVIDIA/spark-rapids/issues/10154)|[BUG] 'spark-test.sh' integration tests FAILED on 'ps: command not found" in Rocky Docker environment| +|[#10175](https://github.com/NVIDIA/spark-rapids/issues/10175)|[BUG] string_test.py::test_format_number_float_special FAILED : AssertionError 'NaN' == | +|[#10166](https://github.com/NVIDIA/spark-rapids/issues/10166)|Detect Undeclared Shim in POM.xml| +|[#10170](https://github.com/NVIDIA/spark-rapids/issues/10170)|[BUG] `test_cast_timestamp_to_date` fails with `TZ=Asia/Hebron`| +|[#10149](https://github.com/NVIDIA/spark-rapids/issues/10149)|[BUG] GPU illegal access detected during delta_byte_array.parquet read| +|[#9905](https://github.com/NVIDIA/spark-rapids/issues/9905)|[BUG] GpuJsonScan incorrect behavior when parsing dates| +|[#10163](https://github.com/NVIDIA/spark-rapids/issues/10163)|Spark 3.3.4 Shim Build Failure| +|[#10105](https://github.com/NVIDIA/spark-rapids/issues/10105)|[BUG] scala:compile is not thread safe unless compiler bridge already exists | +|[#10026](https://github.com/NVIDIA/spark-rapids/issues/10026)|[BUG] test_hash_agg_with_nan_keys failed with a DATAGEN_SEED=1702335559| +|[#10075](https://github.com/NVIDIA/spark-rapids/issues/10075)|[BUG] `non-pinned blocking alloc with spill` unit test failed in HostAllocSuite| +|[#10134](https://github.com/NVIDIA/spark-rapids/issues/10134)|[BUG] test_window_aggs_for_batched_finite_row_windows_partitioned failed on Scala 2.13 with DATAGEN_SEED=1704033145| +|[#10118](https://github.com/NVIDIA/spark-rapids/issues/10118)|[BUG] non-UTC Nightly CI failed| +|[#10136](https://github.com/NVIDIA/spark-rapids/issues/10136)|[BUG] The canonicalized version of `GpuFileSourceScanExec`s that suppose to be semantic-equal can be different | +|[#10110](https://github.com/NVIDIA/spark-rapids/issues/10110)|[BUG] disable collect_list and collect_set for window operations by default.| +|[#10129](https://github.com/NVIDIA/spark-rapids/issues/10129)|[BUG] Unit test suite fails with `Null data pointer` in GpuTimeZoneDB| +|[#10089](https://github.com/NVIDIA/spark-rapids/issues/10089)|[BUG] DATAGEN_SEED= environment does not override the marker datagen_overrides| +|[#10108](https://github.com/NVIDIA/spark-rapids/issues/10108)|[BUG] @datagen_overrides seed is sticky when it shouldn't be| +|[#10064](https://github.com/NVIDIA/spark-rapids/issues/10064)|[BUG] test_unsupported_fallback_regexp_replace failed with DATAGEN_SEED=1702662063| +|[#10117](https://github.com/NVIDIA/spark-rapids/issues/10117)|[BUG] test_from_utc_timestamp failed on Cloudera Env when TZ is Iran| +|[#9914](https://github.com/NVIDIA/spark-rapids/issues/9914)|[BUG] Report GPU OOM on recent passed CI premerges.| +|[#10094](https://github.com/NVIDIA/spark-rapids/issues/10094)|[BUG] spark351 PR check failure MockTaskContext method isFailed in class TaskContext of type ()Boolean is not defined| +|[#10017](https://github.com/NVIDIA/spark-rapids/issues/10017)|[BUG] test_casting_from_double_to_timestamp failed for DATAGEN_SEED=1702329497| +|[#9992](https://github.com/NVIDIA/spark-rapids/issues/9992)|[BUG] conditionals_test.py::test_conditional_with_side_effects_cast[String] failed with DATAGEN_SEED=1701976979| +|[#9743](https://github.com/NVIDIA/spark-rapids/issues/9743)|[BUG][AUDIT] SPARK-45652 - SPJ: Handle empty input partitions after dynamic filtering| +|[#9859](https://github.com/NVIDIA/spark-rapids/issues/9859)|[AUDIT] [SPARK-45786] Inaccurate Decimal multiplication and division results| +|[#9555](https://github.com/NVIDIA/spark-rapids/issues/9555)|[BUG] Scala 2.13 build with JDK 11 or 17 fails OpcodeSuite tests| +|[#10073](https://github.com/NVIDIA/spark-rapids/issues/10073)|[BUG] test_csv_prefer_date_with_infer_schema failed with DATAGEN_SEED=1702847907| +|[#10004](https://github.com/NVIDIA/spark-rapids/issues/10004)|[BUG] If a host memory buffer is spilled, it cannot be unspilled| +|[#10063](https://github.com/NVIDIA/spark-rapids/issues/10063)|[BUG] CI build failure with 341db: method getKillReason has weaker access privileges; it should be public| +|[#10055](https://github.com/NVIDIA/spark-rapids/issues/10055)|[BUG] array_test.py::test_array_transform_non_deterministic failed with non-UTC time zone| +|[#10056](https://github.com/NVIDIA/spark-rapids/issues/10056)|[BUG] Unit tests ToPrettyStringSuite FAILED on spark-3.5.0| +|[#10048](https://github.com/NVIDIA/spark-rapids/issues/10048)|[BUG] Fix ```out of range``` error from ```pySpark``` in ```test_timestamp_millis``` and other two integration test cases| +|[#4204](https://github.com/NVIDIA/spark-rapids/issues/4204)|casting double to string does not match Spark| +|[#9938](https://github.com/NVIDIA/spark-rapids/issues/9938)|Better to do some refactor for the Python UDF code| +|[#10018](https://github.com/NVIDIA/spark-rapids/issues/10018)|[BUG] `GpuToUnixTimestampImproved` off by 1 on GPU when handling timestamp before epoch| +|[#10012](https://github.com/NVIDIA/spark-rapids/issues/10012)|[BUG] test_str_to_map_expr_random_delimiters with DATAGEN_SEED=1702166057 hangs| +|[#10029](https://github.com/NVIDIA/spark-rapids/issues/10029)|[BUG] doc links fail with 404 for shims.md| +|[#9472](https://github.com/NVIDIA/spark-rapids/issues/9472)|[BUG] Non-Deterministic expressions in an array_transform can cause errors| +|[#9884](https://github.com/NVIDIA/spark-rapids/issues/9884)|[BUG] delta_lake_delete_test.py failed assertion [DATAGEN_SEED=1701225104, IGNORE_ORDER...| +|[#9977](https://github.com/NVIDIA/spark-rapids/issues/9977)|[BUG] test_cast_date_integral fails on databricks 3.4.1| +|[#9936](https://github.com/NVIDIA/spark-rapids/issues/9936)|[BUG] Nightly CI of non-UTC time zone reports 'year 0 is out of range' error| +|[#9941](https://github.com/NVIDIA/spark-rapids/issues/9941)|[BUG] A potential data corruption in Pandas UDFs| +|[#9897](https://github.com/NVIDIA/spark-rapids/issues/9897)|[BUG] Error message for multiple jars on classpath is wrong| +|[#9916](https://github.com/NVIDIA/spark-rapids/issues/9916)|[BUG] ```test_cast_string_ts_valid_format``` failed at ```seed = 1701362564```| +|[#9559](https://github.com/NVIDIA/spark-rapids/issues/9559)|[BUG] precommit regularly fails with error trying to download a dependency| +|[#9708](https://github.com/NVIDIA/spark-rapids/issues/9708)|[BUG] test_cast_string_ts_valid_format fails with DATAGEN_SEED=1699978422| + +### PRs +||| +|:---|:---| +|[#10555](https://github.com/NVIDIA/spark-rapids/pull/10555)|Update change log [skip ci]| +|[#10551](https://github.com/NVIDIA/spark-rapids/pull/10551)|Try to make degenerative joins here impossible for these tests| +|[#10546](https://github.com/NVIDIA/spark-rapids/pull/10546)|Update changelog [skip ci]| +|[#10541](https://github.com/NVIDIA/spark-rapids/pull/10541)|Fix Delta log cache size settings during integration tests| +|[#10525](https://github.com/NVIDIA/spark-rapids/pull/10525)|Update changelog for v24.02.0 release [skip ci]| +|[#10465](https://github.com/NVIDIA/spark-rapids/pull/10465)|Add missed shims for scala2.13| +|[#10511](https://github.com/NVIDIA/spark-rapids/pull/10511)|Update rapids jni and private dependency version to 24.02.1| +|[#10513](https://github.com/NVIDIA/spark-rapids/pull/10513)|Fix scalar leak in SumBinaryFixer (#10510)| +|[#10475](https://github.com/NVIDIA/spark-rapids/pull/10475)|Fix scalar leak in RankFixer| +|[#10461](https://github.com/NVIDIA/spark-rapids/pull/10461)|Preserve tags on FileSourceScanExec| +|[#10459](https://github.com/NVIDIA/spark-rapids/pull/10459)|[DOC] Fix table rendering issue in github.io download UI page on branch-24.02 [skip ci] | +|[#10443](https://github.com/NVIDIA/spark-rapids/pull/10443)|Update change log for v24.02.0 release [skip ci]| +|[#10439](https://github.com/NVIDIA/spark-rapids/pull/10439)|Reverts NVIDIA/spark-rapids#10232 and fixes the plugin build on Databricks 11.3| +|[#10380](https://github.com/NVIDIA/spark-rapids/pull/10380)|Init changelog 24.02 [skip ci]| +|[#10367](https://github.com/NVIDIA/spark-rapids/pull/10367)|Update rapids JNI and private version to release 24.02.0| +|[#10414](https://github.com/NVIDIA/spark-rapids/pull/10414)|[DOC] Fix 24.02.0 documentation errors [skip ci]| +|[#10403](https://github.com/NVIDIA/spark-rapids/pull/10403)|Cherry-pick: Fix a memory leak in json tuple (#10360)| +|[#10387](https://github.com/NVIDIA/spark-rapids/pull/10387)|[DOC] Update docs for 24.02.0 release [skip ci]| +|[#10399](https://github.com/NVIDIA/spark-rapids/pull/10399)|Update NOTICE-binary| +|[#10389](https://github.com/NVIDIA/spark-rapids/pull/10389)|Change version and branch to 24.02 in docs [skip ci]| +|[#10384](https://github.com/NVIDIA/spark-rapids/pull/10384)|[DOC] Update docs for 23.12.2 release [skip ci] | +|[#10309](https://github.com/NVIDIA/spark-rapids/pull/10309)|[DOC] add custom 404 page and fix some document issue [skip ci]| +|[#10352](https://github.com/NVIDIA/spark-rapids/pull/10352)|xfail mixed type test| +|[#10355](https://github.com/NVIDIA/spark-rapids/pull/10355)|Revert "Support barrier mode for mapInPandas/mapInArrow (#10343)"| +|[#10353](https://github.com/NVIDIA/spark-rapids/pull/10353)|Use fixed seed for test_from_json_struct_decimal| +|[#10343](https://github.com/NVIDIA/spark-rapids/pull/10343)|Support barrier mode for mapInPandas/mapInArrow| +|[#10345](https://github.com/NVIDIA/spark-rapids/pull/10345)|Fix auto merge conflict 10339 [skip ci]| +|[#9991](https://github.com/NVIDIA/spark-rapids/pull/9991)|Start to use explicit memory limits in the parquet chunked reader| +|[#10328](https://github.com/NVIDIA/spark-rapids/pull/10328)|Fix typo in spark-tests.sh [skip ci]| +|[#10279](https://github.com/NVIDIA/spark-rapids/pull/10279)|Run '--packages' only with default cuda11 jar| +|[#10273](https://github.com/NVIDIA/spark-rapids/pull/10273)|Support reading JSON data with single quotes around attribute names and values| +|[#10306](https://github.com/NVIDIA/spark-rapids/pull/10306)|Fix performance regression in from_json| +|[#10272](https://github.com/NVIDIA/spark-rapids/pull/10272)|Add FullOuter support to GpuShuffledSymmetricHashJoinExec| +|[#10260](https://github.com/NVIDIA/spark-rapids/pull/10260)|Add perf test for time zone operators| +|[#10275](https://github.com/NVIDIA/spark-rapids/pull/10275)|Add tests for window Python udf with array input| +|[#10278](https://github.com/NVIDIA/spark-rapids/pull/10278)|Clean up $M2_CACHE to avoid side-effect of previous dependency:get [skip ci]| +|[#10268](https://github.com/NVIDIA/spark-rapids/pull/10268)|Add config to enable mixed types as string in GpuJsonToStruct & GpuJsonScan| +|[#10297](https://github.com/NVIDIA/spark-rapids/pull/10297)|Revert "UCX 1.16.0 upgrade (#10190)"| +|[#10289](https://github.com/NVIDIA/spark-rapids/pull/10289)|Add gerashegalov to CODEOWNERS [skip ci]| +|[#10290](https://github.com/NVIDIA/spark-rapids/pull/10290)|Fix merge conflict with 23.12 [skip ci]| +|[#10190](https://github.com/NVIDIA/spark-rapids/pull/10190)|UCX 1.16.0 upgrade| +|[#10211](https://github.com/NVIDIA/spark-rapids/pull/10211)|Use parse_url kernel for QUERY literal and column key| +|[#10267](https://github.com/NVIDIA/spark-rapids/pull/10267)|Update to libcudf unsigned sum aggregation types change| +|[#10208](https://github.com/NVIDIA/spark-rapids/pull/10208)|Added Support for Lazy Quantifier| +|[#9993](https://github.com/NVIDIA/spark-rapids/pull/9993)|Enable mixed types as string in GpuJsonScan| +|[#10246](https://github.com/NVIDIA/spark-rapids/pull/10246)|Refactor full join iterator to allow access to build tracker| +|[#10257](https://github.com/NVIDIA/spark-rapids/pull/10257)|Enable auto-merge from branch-24.02 to branch-24.04 [skip CI]| +|[#10178](https://github.com/NVIDIA/spark-rapids/pull/10178)|Mark hash reduction decimal overflow test as a permanent seed override| +|[#10244](https://github.com/NVIDIA/spark-rapids/pull/10244)|Use POSIX mode in assembly plugin to avoid issues with large UID/GID| +|[#10238](https://github.com/NVIDIA/spark-rapids/pull/10238)|Smoke test with '--package' to fetch the plugin jar| +|[#10201](https://github.com/NVIDIA/spark-rapids/pull/10201)|Deploy release candidates to local maven repo for dependency check[skip ci]| +|[#10240](https://github.com/NVIDIA/spark-rapids/pull/10240)|Improved inner joins with large build side| +|[#10220](https://github.com/NVIDIA/spark-rapids/pull/10220)|Disable GetJsonObject by default and add tests for as many issues with it as possible| +|[#10230](https://github.com/NVIDIA/spark-rapids/pull/10230)|Fix Databricks 13.3 BroadcastHashJoin using executor side broadcast fed by ColumnarToRow [Databricks]| +|[#10232](https://github.com/NVIDIA/spark-rapids/pull/10232)|Fixed 330db Shims to Adopt the PythonRunner Changes| +|[#10225](https://github.com/NVIDIA/spark-rapids/pull/10225)|Download Maven from apache.org archives [skip ci]| +|[#10210](https://github.com/NVIDIA/spark-rapids/pull/10210)|Add string parameter support for unix_timestamp for non-UTC time zones| +|[#10223](https://github.com/NVIDIA/spark-rapids/pull/10223)|Fix to_utc_timestamp and from_utc_timestamp fallback when TZ is supported time zone| +|[#10205](https://github.com/NVIDIA/spark-rapids/pull/10205)|Deterministic ordering in window tests| +|[#10204](https://github.com/NVIDIA/spark-rapids/pull/10204)|Further prevent degenerative joins in dpp_test| +|[#10156](https://github.com/NVIDIA/spark-rapids/pull/10156)|Update string to float compatibility doc[skip ci]| +|[#10193](https://github.com/NVIDIA/spark-rapids/pull/10193)|Fix explode with carry-along columns on GpuExplode single row retry handling| +|[#10191](https://github.com/NVIDIA/spark-rapids/pull/10191)|Updating the config documentation for filecache configs [skip ci]| +|[#10131](https://github.com/NVIDIA/spark-rapids/pull/10131)|With a single row GpuExplode tries to split the generator array| +|[#10179](https://github.com/NVIDIA/spark-rapids/pull/10179)|Fix build regression against Spark 3.2.x| +|[#10189](https://github.com/NVIDIA/spark-rapids/pull/10189)|test needs marks for non-UTC and for non_supported timezones| +|[#10176](https://github.com/NVIDIA/spark-rapids/pull/10176)|Fix format_number NaN symbol in high jdk version| +|[#10074](https://github.com/NVIDIA/spark-rapids/pull/10074)|Update the legacy mode check: only take effect when reading date/timestamp column| +|[#10167](https://github.com/NVIDIA/spark-rapids/pull/10167)|Defined Shims Should Be Declared In POM | +|[#10168](https://github.com/NVIDIA/spark-rapids/pull/10168)|Prevent a degenerative join in test_dpp_reuse_broadcast_exchange| +|[#10171](https://github.com/NVIDIA/spark-rapids/pull/10171)|Fix `test_cast_timestamp_to_date` when running in a DST time zone| +|[#9975](https://github.com/NVIDIA/spark-rapids/pull/9975)|Improve dateFormat support in GpuJsonScan and make tests consistent with GpuStructsToJson| +|[#9790](https://github.com/NVIDIA/spark-rapids/pull/9790)|Support float case of format_number with format_float kernel| +|[#10144](https://github.com/NVIDIA/spark-rapids/pull/10144)|Support to_utc_timestamp| +|[#10162](https://github.com/NVIDIA/spark-rapids/pull/10162)|Fix Spark 334 Build| +|[#10146](https://github.com/NVIDIA/spark-rapids/pull/10146)|Refactor the window code so it is not mostly kept in a few very large files| +|[#10155](https://github.com/NVIDIA/spark-rapids/pull/10155)|Install procps tools for rocky docker images [skip ci]| +|[#10153](https://github.com/NVIDIA/spark-rapids/pull/10153)|Disable multi-threaded Maven | +|[#10100](https://github.com/NVIDIA/spark-rapids/pull/10100)|Enable to_date (via gettimestamp and casting timestamp to date) for non-UTC time zones| +|[#10140](https://github.com/NVIDIA/spark-rapids/pull/10140)|Removed Unnecessary Whitespaces From Spark 3.3.4 Shim [skip ci]| +|[#10148](https://github.com/NVIDIA/spark-rapids/pull/10148)|fix test_hash_agg_with_nan_keys floating point sum failure| +|[#10150](https://github.com/NVIDIA/spark-rapids/pull/10150)|Increase timeouts in HostAllocSuite to avoid timeout failures on slow machines| +|[#10143](https://github.com/NVIDIA/spark-rapids/pull/10143)|Fix `test_window_aggs_for_batched_finite_row_windows_partitioned` fail| +|[#9887](https://github.com/NVIDIA/spark-rapids/pull/9887)|Reduce time-consuming of pre-merge| +|[#10130](https://github.com/NVIDIA/spark-rapids/pull/10130)|Change unit tests that force ooms to specify the oom type (gpu|cpu)| +|[#10138](https://github.com/NVIDIA/spark-rapids/pull/10138)|Update copyright dates in NOTICE files [skip ci]| +|[#10139](https://github.com/NVIDIA/spark-rapids/pull/10139)|Add Delta Lake 2.3.0 to list of versions to test for Spark 3.3.x| +|[#10135](https://github.com/NVIDIA/spark-rapids/pull/10135)|Fix CI: can't find script when there is pushd in script [skip ci]| +|[#10137](https://github.com/NVIDIA/spark-rapids/pull/10137)|Fix the canonicalizing for GPU file scan| +|[#10132](https://github.com/NVIDIA/spark-rapids/pull/10132)|Disable collect_list and collect_set for window by default| +|[#10084](https://github.com/NVIDIA/spark-rapids/pull/10084)|Refactor GpuJsonToStruct to reduce code duplication and manage resources more efficiently| +|[#10087](https://github.com/NVIDIA/spark-rapids/pull/10087)|Additional unit tests for GeneratedInternalRowToCudfRowIterator| +|[#10082](https://github.com/NVIDIA/spark-rapids/pull/10082)|Add Spark 3.3.4 Shim| +|[#10054](https://github.com/NVIDIA/spark-rapids/pull/10054)|Support Ascii function for ascii and latin-1| +|[#10127](https://github.com/NVIDIA/spark-rapids/pull/10127)|Fix merge conflict with branch-23.12| +|[#10097](https://github.com/NVIDIA/spark-rapids/pull/10097)|[DOC] Update docs for 23.12.1 release [skip ci]| +|[#10109](https://github.com/NVIDIA/spark-rapids/pull/10109)|Fixes a bug where datagen seed overrides were sticky and adds datagen_seed_override_disabled| +|[#10093](https://github.com/NVIDIA/spark-rapids/pull/10093)|Fix test_unsupported_fallback_regexp_replace| +|[#10119](https://github.com/NVIDIA/spark-rapids/pull/10119)|Fix from_utc_timestamp case failure on Cloudera when TZ is Iran| +|[#10106](https://github.com/NVIDIA/spark-rapids/pull/10106)|Add `isFailed()` to MockTaskContext and Remove MockTaskContextBase.scala| +|[#10112](https://github.com/NVIDIA/spark-rapids/pull/10112)|Remove datagen seed override for test_conditional_with_side_effects_cast| +|[#10104](https://github.com/NVIDIA/spark-rapids/pull/10104)|[DOC] Add in docs about memory debugging [skip ci]| +|[#9925](https://github.com/NVIDIA/spark-rapids/pull/9925)|Use threads, cache Scala compiler in GH mvn workflow| +|[#9967](https://github.com/NVIDIA/spark-rapids/pull/9967)|Added Spark-3.4.2 Shims| +|[#10061](https://github.com/NVIDIA/spark-rapids/pull/10061)|Use parse_url kernel for QUERY parsing| +|[#10101](https://github.com/NVIDIA/spark-rapids/pull/10101)|[DOC] Add column order error docs [skip ci]| +|[#10078](https://github.com/NVIDIA/spark-rapids/pull/10078)|Add perf test for non-UTC operators| +|[#10096](https://github.com/NVIDIA/spark-rapids/pull/10096)|Shim MockTaskContext to fix Spark 3.5.1 build| +|[#10092](https://github.com/NVIDIA/spark-rapids/pull/10092)|Implement Math.round using floor on GPU| +|[#10085](https://github.com/NVIDIA/spark-rapids/pull/10085)|Update tests that originally restricted the Spark timestamp range| +|[#10090](https://github.com/NVIDIA/spark-rapids/pull/10090)|Replace GPU-unsupported `\z` with an alternative RLIKE expression| +|[#10095](https://github.com/NVIDIA/spark-rapids/pull/10095)|Temporarily fix date format failed cases for non-UTC time zone.| +|[#9999](https://github.com/NVIDIA/spark-rapids/pull/9999)|Add some odd time zones for timezone transition tests| +|[#9962](https://github.com/NVIDIA/spark-rapids/pull/9962)|Add 3.5.1-SNAPSHOT Shim| +|[#10071](https://github.com/NVIDIA/spark-rapids/pull/10071)|Cleanup usage of non-utc configuration here| +|[#10057](https://github.com/NVIDIA/spark-rapids/pull/10057)|Add support for StringConcatFactory.makeConcatWithConstants (#9555)| +|[#9996](https://github.com/NVIDIA/spark-rapids/pull/9996)|Test full timestamp output range in PySpark| +|[#10081](https://github.com/NVIDIA/spark-rapids/pull/10081)|Add a fallback Cloudera Maven repo URL [skip ci]| +|[#10065](https://github.com/NVIDIA/spark-rapids/pull/10065)|Improve host memory spill interfaces| +|[#10069](https://github.com/NVIDIA/spark-rapids/pull/10069)|Revert "Support split broadcast join condition into ast and non-ast […| +|[#10070](https://github.com/NVIDIA/spark-rapids/pull/10070)|Fix 332db build failure| +|[#10060](https://github.com/NVIDIA/spark-rapids/pull/10060)|Fix failed cases for non-utc time zone| +|[#10038](https://github.com/NVIDIA/spark-rapids/pull/10038)|Remove spark.rapids.sql.nonUTC.enabled configuration option| +|[#10059](https://github.com/NVIDIA/spark-rapids/pull/10059)|Fixed Failing ToPrettyStringSuite Test for 3.5.0| +|[#10013](https://github.com/NVIDIA/spark-rapids/pull/10013)|Extended configuration of OOM injection mode| +|[#10052](https://github.com/NVIDIA/spark-rapids/pull/10052)|Set seed=0 for some integration test cases| +|[#10053](https://github.com/NVIDIA/spark-rapids/pull/10053)|Remove invalid user from CODEOWNER file [skip ci]| +|[#10049](https://github.com/NVIDIA/spark-rapids/pull/10049)|Fix out of range error from pySpark in test_timestamp_millis and other two integration test cases| +|[#9721](https://github.com/NVIDIA/spark-rapids/pull/9721)|Support date_format via Gpu for non-UTC time zone| +|[#9470](https://github.com/NVIDIA/spark-rapids/pull/9470)|Use float to string kernel| +|[#9845](https://github.com/NVIDIA/spark-rapids/pull/9845)|Use parse_url kernel for HOST parsing| +|[#10024](https://github.com/NVIDIA/spark-rapids/pull/10024)|Support hour minute second for non-UTC time zone| +|[#9973](https://github.com/NVIDIA/spark-rapids/pull/9973)|Batching support for row-based bounded window functions | +|[#10042](https://github.com/NVIDIA/spark-rapids/pull/10042)|Update tests to not have hard coded fallback when not needed| +|[#9816](https://github.com/NVIDIA/spark-rapids/pull/9816)|Support unix_timestamp and to_unix_timestamp with non-UTC timezones (non-DST)| +|[#9902](https://github.com/NVIDIA/spark-rapids/pull/9902)|Some refactor for the Python UDF code| +|[#10023](https://github.com/NVIDIA/spark-rapids/pull/10023)|GPU supports `yyyyMMdd` format by post process for the `from_unixtime` function| +|[#10033](https://github.com/NVIDIA/spark-rapids/pull/10033)|Remove GpuToTimestampImproved and spark.rapids.sql.improvedTimeOps.enabled| +|[#10016](https://github.com/NVIDIA/spark-rapids/pull/10016)|Fix infinite loop in test_str_to_map_expr_random_delimiters| +|[#9481](https://github.com/NVIDIA/spark-rapids/pull/9481)|Use parse_url kernel for PROTOCOL parsing| +|[#10030](https://github.com/NVIDIA/spark-rapids/pull/10030)|Update links in shims.md| +|[#10015](https://github.com/NVIDIA/spark-rapids/pull/10015)|Fix array_transform to not recompute the argument| +|[#10011](https://github.com/NVIDIA/spark-rapids/pull/10011)|Add cpu oom retry split handling to InternalRowToColumnarBatchIterator| +|[#10019](https://github.com/NVIDIA/spark-rapids/pull/10019)|Fix auto merge conflict 10010 [skip ci]| +|[#9760](https://github.com/NVIDIA/spark-rapids/pull/9760)|Support split broadcast join condition into ast and non-ast| +|[#9827](https://github.com/NVIDIA/spark-rapids/pull/9827)|Enable ORC timestamp and decimal predicate push down tests| +|[#10002](https://github.com/NVIDIA/spark-rapids/pull/10002)|Use Spark 3.3.3 instead of 3.3.2 for Scala 2.13 premerge builds| +|[#10000](https://github.com/NVIDIA/spark-rapids/pull/10000)|Optimize from_unixtime| +|[#10003](https://github.com/NVIDIA/spark-rapids/pull/10003)|Fix merge conflict with branch-23.12| +|[#9984](https://github.com/NVIDIA/spark-rapids/pull/9984)|Fix 340+(including DB341+) does not support casting date to integral/float| +|[#9972](https://github.com/NVIDIA/spark-rapids/pull/9972)|Fix year 0 is out of range in test_from_json_struct_timestamp | +|[#9814](https://github.com/NVIDIA/spark-rapids/pull/9814)|Support from_unixtime via Gpu for non-UTC time zone| +|[#9929](https://github.com/NVIDIA/spark-rapids/pull/9929)|Add host memory retries for GeneratedInternalRowToCudfRowIterator| +|[#9957](https://github.com/NVIDIA/spark-rapids/pull/9957)|Update cases for cast between integral and (date/time)| +|[#9959](https://github.com/NVIDIA/spark-rapids/pull/9959)|Append new authorized user to blossom-ci whitelist [skip ci]| +|[#9942](https://github.com/NVIDIA/spark-rapids/pull/9942)|Fix a potential data corruption for Pandas UDF| +|[#9922](https://github.com/NVIDIA/spark-rapids/pull/9922)|Fix `allowMultipleJars` recommend setting message| +|[#9947](https://github.com/NVIDIA/spark-rapids/pull/9947)|Fix merge conflict with branch-23.12| +|[#9908](https://github.com/NVIDIA/spark-rapids/pull/9908)|Register default allocator for host memory| +|[#9944](https://github.com/NVIDIA/spark-rapids/pull/9944)|Fix Java OOM caused by incorrect state of shouldCapture when exception occurred| +|[#9937](https://github.com/NVIDIA/spark-rapids/pull/9937)|Refactor to use CLASSIFIER instead of CUDA_CLASSIFIER [skip ci]| +|[#9904](https://github.com/NVIDIA/spark-rapids/pull/9904)|Params for build and test CI scripts on Databricks| +|[#9719](https://github.com/NVIDIA/spark-rapids/pull/9719)|Support fine grained timezone checker instead of type based| +|[#9918](https://github.com/NVIDIA/spark-rapids/pull/9918)|Prevent generation of 'year 0 is out of range' strings in IT| +|[#9852](https://github.com/NVIDIA/spark-rapids/pull/9852)|Avoid generating duplicate nan keys with MapGen(FloatGen)| +|[#9674](https://github.com/NVIDIA/spark-rapids/pull/9674)|Add cache action to speed up mvn workflow [skip ci]| +|[#9900](https://github.com/NVIDIA/spark-rapids/pull/9900)|Revert "Remove Databricks 13.3 from release 23.12 (#9890)"| +|[#9889](https://github.com/NVIDIA/spark-rapids/pull/9889)|Fix test_cast_string_ts_valid_format test| +|[#9888](https://github.com/NVIDIA/spark-rapids/pull/9888)|Update nightly build and deploy script for arm artifacts [skip ci]| +|[#9833](https://github.com/NVIDIA/spark-rapids/pull/9833)|Fix a hang for Pandas UDFs on DB 13.3| +|[#9656](https://github.com/NVIDIA/spark-rapids/pull/9656)|Update for new retry state machine JNI APIs| +|[#9654](https://github.com/NVIDIA/spark-rapids/pull/9654)|Detect multiple jars on the classpath when init plugin| +|[#9857](https://github.com/NVIDIA/spark-rapids/pull/9857)|Skip redundant steps in nightly build [skip ci]| +|[#9812](https://github.com/NVIDIA/spark-rapids/pull/9812)|Update JNI and private dep version to 24.02.0-SNAPSHOT| +|[#9716](https://github.com/NVIDIA/spark-rapids/pull/9716)|Initiate project version 24.02.0-SNAPSHOT| + From 586e6f4532cac49ab77e3d3d7c75545ada66fd77 Mon Sep 17 00:00:00 2001 From: Advait Chandorkar <110400437+AdvaitChandorkar07@users.noreply.github.com> Date: Tue, 11 Jun 2024 00:04:59 +0530 Subject: [PATCH 34/68] Increase the console output of buildall upon build failures (#10998) Signed-off-by: AdvaitChandorkar07 --- build/buildall | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build/buildall b/build/buildall index e8c0610deb7..b3c473be141 100755 --- a/build/buildall +++ b/build/buildall @@ -265,7 +265,7 @@ function build_single_shim() { -Dmaven.scaladoc.skip \ -Dmaven.scalastyle.skip="$SKIP_CHECKS" \ -pl tools -am > "$LOG_FILE" 2>&1 || { - [[ "$LOG_FILE" != "/dev/tty" ]] && echo "$LOG_FILE:" && tail -20 "$LOG_FILE" || true + [[ "$LOG_FILE" != "/dev/tty" ]] && echo "$LOG_FILE:" && tail -500 "$LOG_FILE" || true exit 255 } } From f47c205041eb6efc78b162903fc5c2aaf2be30b5 Mon Sep 17 00:00:00 2001 From: Liangcai Li Date: Tue, 11 Jun 2024 09:38:32 +0800 Subject: [PATCH 35/68] Allow ProjectExec fall fallback to CPU for 350 (#11032) This is a bug fix for the hive write tests. This is simliar as #10958 but it fixes the failures for Spark 350. The tests fail because the ProjectExec will fall back to CPU due to missing the GPU version of the MapFromArrays expression. This PR adds the ProjectExec to the allowed list of fallback for Spark 350 and the laters. Signed-off-by: Firestarman --- integration_tests/src/main/python/hive_parquet_write_test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/integration_tests/src/main/python/hive_parquet_write_test.py b/integration_tests/src/main/python/hive_parquet_write_test.py index 96976c3a356..f62439a39af 100644 --- a/integration_tests/src/main/python/hive_parquet_write_test.py +++ b/integration_tests/src/main/python/hive_parquet_write_test.py @@ -19,7 +19,7 @@ from data_gen import * from hive_write_test import _restricted_timestamp from marks import allow_non_gpu, ignore_order -from spark_session import with_cpu_session, is_before_spark_320, is_spark_351_or_later +from spark_session import with_cpu_session, is_before_spark_320, is_spark_350_or_later # Disable the meta conversion from Hive write to FrameData write in Spark, to test # "GpuInsertIntoHiveTable" for Parquet write. @@ -55,7 +55,7 @@ _hive_write_gens = [_hive_basic_gens, _hive_struct_gens, _hive_array_gens, _hive_map_gens] # ProjectExec falls back on databricks due to no GPU version of "MapFromArrays". -fallback_nodes = ['ProjectExec'] if is_databricks_runtime() or is_spark_351_or_later() else [] +fallback_nodes = ['ProjectExec'] if is_databricks_runtime() or is_spark_350_or_later() else [] @allow_non_gpu(*(non_utc_allow + fallback_nodes)) From 1ca4c44408e02df3710bb559794861b3ee861afb Mon Sep 17 00:00:00 2001 From: Peixin Date: Tue, 11 Jun 2024 10:01:32 +0800 Subject: [PATCH 36/68] Update blossom-ci ACL to secure format (#11036) Signed-off-by: Peixin Li --- .github/workflows/blossom-ci.yml | 82 ++++++++++++++++---------------- 1 file changed, 42 insertions(+), 40 deletions(-) diff --git a/.github/workflows/blossom-ci.yml b/.github/workflows/blossom-ci.yml index 6f597d6baf3..3da5fae084d 100644 --- a/.github/workflows/blossom-ci.yml +++ b/.github/workflows/blossom-ci.yml @@ -33,46 +33,48 @@ jobs: args: ${{ env.args }} # This job only runs for pull request comments - if: contains( '\ - abellina,\ - anfeng,\ - firestarman,\ - GaryShen2008,\ - jlowe,\ - kuhushukla,\ - mythrocks,\ - nartal1,\ - nvdbaranec,\ - NvTimLiu,\ - razajafri,\ - revans2,\ - rwlee,\ - sameerz,\ - tgravescs,\ - wbo4958,\ - wjxiz1992,\ - sperlingxx,\ - hyperbolic2346,\ - gerashegalov,\ - ttnghia,\ - nvliyuan,\ - res-life,\ - HaoYang670,\ - NVnavkumar,\ - amahussein,\ - mattahrens,\ - YanxuanLiu,\ - cindyyuanjiang,\ - thirtiseven,\ - winningsix,\ - viadea,\ - yinqingh,\ - parthosa,\ - liurenjie1024,\ - binmahone,\ - zpuller,\ - pxLi,\ - ', format('{0},', github.actor)) && github.event.comment.body == 'build' + if: | + github.event.comment.body == 'build' && + ( + github.actor == 'abellina' || + github.actor == 'anfeng' || + github.actor == 'firestarman' || + github.actor == 'GaryShen2008' || + github.actor == 'jlowe' || + github.actor == 'kuhushukla' || + github.actor == 'mythrocks' || + github.actor == 'nartal1' || + github.actor == 'nvdbaranec' || + github.actor == 'NvTimLiu' || + github.actor == 'razajafri' || + github.actor == 'revans2' || + github.actor == 'rwlee' || + github.actor == 'sameerz' || + github.actor == 'tgravescs' || + github.actor == 'wbo4958' || + github.actor == 'wjxiz1992' || + github.actor == 'sperlingxx' || + github.actor == 'hyperbolic2346' || + github.actor == 'gerashegalov' || + github.actor == 'ttnghia' || + github.actor == 'nvliyuan' || + github.actor == 'res-life' || + github.actor == 'HaoYang670' || + github.actor == 'NVnavkumar' || + github.actor == 'amahussein' || + github.actor == 'mattahrens' || + github.actor == 'YanxuanLiu' || + github.actor == 'cindyyuanjiang' || + github.actor == 'thirtiseven' || + github.actor == 'winningsix' || + github.actor == 'viadea' || + github.actor == 'yinqingh' || + github.actor == 'parthosa' || + github.actor == 'liurenjie1024' || + github.actor == 'binmahone' || + github.actor == 'zpuller' || + github.actor == 'pxLi' + ) steps: - name: Check if comment is issued by authorized person run: blossom-ci From 4e9b961e700726a4515bf3c44e88cfbce4efff9a Mon Sep 17 00:00:00 2001 From: Feng Jiang <106386742+Feng-Jiang28@users.noreply.github.com> Date: Tue, 11 Jun 2024 16:47:15 +0800 Subject: [PATCH 37/68] Append new authorized user to blossom-ci whitelist [skip ci] (#11040) Signed-off-by: fejiang --- .github/workflows/blossom-ci.yml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/.github/workflows/blossom-ci.yml b/.github/workflows/blossom-ci.yml index 3da5fae084d..4b8071303c1 100644 --- a/.github/workflows/blossom-ci.yml +++ b/.github/workflows/blossom-ci.yml @@ -73,7 +73,8 @@ jobs: github.actor == 'liurenjie1024' || github.actor == 'binmahone' || github.actor == 'zpuller' || - github.actor == 'pxLi' + github.actor == 'pxLi' || + github.actor == 'Feng-Jiang28' ) steps: - name: Check if comment is issued by authorized person From 2cf59346fd66e1d500a16e8107068dc9e20c3585 Mon Sep 17 00:00:00 2001 From: Haoyang Li Date: Wed, 12 Jun 2024 08:03:27 +0800 Subject: [PATCH 38/68] Rewrite multiple literal choice regex to multiple contains in rlike (#10977) * rewrite multiple literal choice to multiple contains, wip Signed-off-by: Haoyang Li * fix bug Signed-off-by: Haoyang Li * optimize memory Signed-off-by: Haoyang Li * remove debug log Signed-off-by: Haoyang Li * address comments Signed-off-by: Haoyang Li * Apply suggestions from code review Co-authored-by: Gera Shegalov * support abc|def case Signed-off-by: Haoyang Li * fix 2.13 Signed-off-by: Haoyang Li * fix 2.13 build Signed-off-by: Haoyang Li --------- Signed-off-by: Haoyang Li Co-authored-by: Gera Shegalov --- .../src/main/python/regexp_test.py | 8 +- .../com/nvidia/spark/rapids/RegexParser.scala | 94 +++++++++++-------- .../spark/sql/rapids/stringFunctions.scala | 30 +++++- .../RegularExpressionRewriteSuite.scala | 31 ++++-- 4 files changed, 118 insertions(+), 45 deletions(-) diff --git a/integration_tests/src/main/python/regexp_test.py b/integration_tests/src/main/python/regexp_test.py index 89929eb6762..18a83870d83 100644 --- a/integration_tests/src/main/python/regexp_test.py +++ b/integration_tests/src/main/python/regexp_test.py @@ -454,6 +454,7 @@ def test_rlike_rewrite_optimization(): 'rlike(a, "(.*)(abb)(.*)")', 'rlike(a, "^(abb)(.*)")', 'rlike(a, "^abb")', + 'rlike(a, "^.*(aaa)")', 'rlike(a, "\\\\A(abb)(.*)")', 'rlike(a, "\\\\Aabb")', 'rlike(a, "^(abb)\\\\Z")', @@ -466,7 +467,12 @@ def test_rlike_rewrite_optimization(): 'rlike(a, "ab[a-c]{3}")', 'rlike(a, "a[a-c]{1,3}")', 'rlike(a, "a[a-c]{1,}")', - 'rlike(a, "a[a-c]+")'), + 'rlike(a, "a[a-c]+")', + 'rlike(a, "(aaa|bbb|ccc)")', + 'rlike(a, ".*.*(aaa|bbb).*.*")', + 'rlike(a, "^.*(aaa|bbb|ccc)")', + 'rlike(a, "aaa|bbb")', + 'rlike(a, "aaa|(bbb|ccc)")'), conf=_regexp_conf) def test_regexp_replace_character_set_negated(): diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala index 0f5ada9f7fa..1ca155f8a52 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala @@ -2026,6 +2026,7 @@ object RegexOptimizationType { case class Contains(literal: String) extends RegexOptimizationType case class PrefixRange(literal: String, length: Int, rangeStart: Int, rangeEnd: Int) extends RegexOptimizationType + case class MultipleContains(literals: Seq[String]) extends RegexOptimizationType case object NoOptimization extends RegexOptimizationType } @@ -2091,6 +2092,20 @@ object RegexRewrite { } } + private def getMultipleContainsLiterals(ast: RegexAST): Seq[String] = { + ast match { + case RegexGroup(_, term, _) => getMultipleContainsLiterals(term) + case RegexChoice(RegexSequence(parts), ls) if isLiteralString(parts) => { + getMultipleContainsLiterals(ls) match { + case Seq() => Seq.empty + case literals => RegexCharsToString(parts) +: literals + } + } + case RegexSequence(parts) if (isLiteralString(parts)) => Seq(RegexCharsToString(parts)) + case _ => Seq.empty + } + } + private def isWildcard(ast: RegexAST): Boolean = { ast match { case RegexRepetition(RegexChar('.'), SimpleQuantifier('*')) => true @@ -2101,11 +2116,8 @@ object RegexRewrite { } private def stripLeadingWildcards(astLs: collection.Seq[RegexAST]): - collection.Seq[RegexAST] = astLs match { - case (RegexChar('^') | RegexEscaped('A')) :: tail => - // if the pattern starts with ^ or \A, strip it too - tail.dropWhile(isWildcard) - case _ => astLs.dropWhile(isWildcard) + collection.Seq[RegexAST] = { + astLs.dropWhile(isWildcard) } private def stripTailingWildcards(astLs: collection.Seq[RegexAST]): @@ -2124,40 +2136,48 @@ object RegexRewrite { * Matches the given regex ast to a regex optimization type for regex rewrite * optimization. * - * @param ast unparsed children of the Abstract Syntax Tree parsed from a regex pattern. + * @param ast Abstract Syntax Tree parsed from a regex pattern. * @return The `RegexOptimizationType` for the given pattern. */ - @scala.annotation.tailrec - def matchSimplePattern(ast: Seq[RegexAST]): RegexOptimizationType = { - ast match { - case (RegexChar('^') | RegexEscaped('A')) :: astTail => - val noTrailingWildCards = stripTailingWildcards(astTail) - if (isLiteralString(noTrailingWildCards)) { - // ^literal.* => startsWith literal - RegexOptimizationType.StartsWith(RegexCharsToString(noTrailingWildCards)) - } else { - val noWildCards = stripLeadingWildcards(noTrailingWildCards) - if (noWildCards.length == noTrailingWildCards.length) { - // TODO startsWith with PrefIxRange - RegexOptimizationType.NoOptimization - } else { - matchSimplePattern(astTail) - } - } - case astLs => { - val noStartsWithAst = stripTailingWildcards(stripLeadingWildcards(astLs)) - val prefixRangeInfo = getPrefixRangePattern(noStartsWithAst) - if (prefixRangeInfo.isDefined) { - val (prefix, length, start, end) = prefixRangeInfo.get - // (literal[a-b]{x,y}) => prefix range pattern - RegexOptimizationType.PrefixRange(prefix, length, start, end) - } else if (isLiteralString(noStartsWithAst)) { - // literal.* or (literal).* => contains literal - RegexOptimizationType.Contains(RegexCharsToString(noStartsWithAst)) - } else { - RegexOptimizationType.NoOptimization - } + def matchSimplePattern(ast: RegexAST): RegexOptimizationType = { + val astLs = ast match { + case RegexSequence(_) => ast.children() + case _ => Seq(ast) + } + val noTailingWildcards = stripTailingWildcards(astLs) + if (noTailingWildcards.headOption.exists( + ast => ast == RegexChar('^') || ast == RegexEscaped('A'))) { + val possibleLiteral = noTailingWildcards.drop(1) + if (isLiteralString(possibleLiteral)) { + return RegexOptimizationType.StartsWith(RegexCharsToString(possibleLiteral)) + } + } + + val noStartsWithAst = stripLeadingWildcards(noTailingWildcards) + + // Check if the pattern is a contains literal pattern + if (isLiteralString(noStartsWithAst)) { + // literal or .*(literal).* => contains literal + return RegexOptimizationType.Contains(RegexCharsToString(noStartsWithAst)) + } + + // Check if the pattern is a multiple contains literal pattern (e.g. "abc|def|ghi") + if (noStartsWithAst.length == 1) { + val containsLiterals = getMultipleContainsLiterals(noStartsWithAst.head) + if (!containsLiterals.isEmpty) { + return RegexOptimizationType.MultipleContains(containsLiterals) } } + + // Check if the pattern is a prefix range pattern (e.g. "abc[a-z]{3}") + val prefixRangeInfo = getPrefixRangePattern(noStartsWithAst) + if (prefixRangeInfo.isDefined) { + val (prefix, length, start, end) = prefixRangeInfo.get + // (literal[a-b]{x,y}) => prefix range pattern + return RegexOptimizationType.PrefixRange(prefix, length, start, end) + } + + // return NoOptimization if the pattern is not a simple pattern and use cuDF + RegexOptimizationType.NoOptimization } -} \ No newline at end of file +} diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringFunctions.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringFunctions.scala index 8fea4014149..dc2845e4461 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringFunctions.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringFunctions.scala @@ -1073,7 +1073,7 @@ class GpuRLikeMeta( val originalPattern = str.toString val regexAst = new RegexParser(originalPattern).parse() if (conf.isRlikeRegexRewriteEnabled) { - rewriteOptimizationType = RegexRewrite.matchSimplePattern(regexAst.children()) + rewriteOptimizationType = RegexRewrite.matchSimplePattern(regexAst) } val (transpiledAST, _) = new CudfRegexTranspiler(RegexFindMode) .getTranspiledAST(regexAst, None, None) @@ -1097,6 +1097,7 @@ class GpuRLikeMeta( } case StartsWith(s) => GpuStartsWith(lhs, GpuLiteral(s, StringType)) case Contains(s) => GpuContains(lhs, GpuLiteral(s, StringType)) + case MultipleContains(ls) => GpuMultipleContains(lhs, ls) case PrefixRange(s, length, start, end) => GpuLiteralRangePattern(lhs, GpuLiteral(s, StringType), length, start, end) case _ => throw new IllegalStateException("Unexpected optimization type") @@ -1126,6 +1127,33 @@ case class GpuRLike(left: Expression, right: Expression, pattern: String) override def dataType: DataType = BooleanType } +case class GpuMultipleContains(input: Expression, searchList: Seq[String]) + extends GpuUnaryExpression with ImplicitCastInputTypes with NullIntolerant { + + override def dataType: DataType = BooleanType + + override def child: Expression = input + + override def inputTypes: Seq[AbstractDataType] = Seq(StringType) + + override def doColumnar(input: GpuColumnVector): ColumnVector = { + assert(searchList.length > 1) + val accInit = withResource(Scalar.fromString(searchList.head)) { searchScalar => + input.getBase.stringContains(searchScalar) + } + searchList.tail.foldLeft(accInit) { (acc, search) => + val containsSearch = withResource(Scalar.fromString(search)) { searchScalar => + input.getBase.stringContains(searchScalar) + } + withResource(acc) { _ => + withResource(containsSearch) { _ => + acc.or(containsSearch) + } + } + } + } +} + case class GpuLiteralRangePattern(left: Expression, right: Expression, length: Int, start: Int, end: Int) extends GpuBinaryExpressionArgsAnyScalar with ImplicitCastInputTypes with NullIntolerant { diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionRewriteSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionRewriteSuite.scala index a140f4123f4..7626c1450c1 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionRewriteSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionRewriteSuite.scala @@ -23,7 +23,7 @@ class RegularExpressionRewriteSuite extends AnyFunSuite { Unit = { val results = patterns.map { pattern => val ast = new RegexParser(pattern).parse() - RegexRewrite.matchSimplePattern(ast.children()) + RegexRewrite.matchSimplePattern(ast) } assert(results == excepted) } @@ -40,9 +40,9 @@ class RegularExpressionRewriteSuite extends AnyFunSuite { test("regex rewrite contains") { import RegexOptimizationType._ val patterns = Seq(".*abc.*", ".*(abc).*", "^.*(abc).*$", "^.*(.*)(abc).*.*", - raw".*\w.*\Z", raw".*..*\Z") - val excepted = Seq(Contains("abc"), Contains("abc"), NoOptimization, Contains("abc"), - NoOptimization, NoOptimization) + raw".*\w.*\Z", raw".*..*\Z", "^(.*)(abc)") + val excepted = Seq(Contains("abc"), Contains("abc"), NoOptimization, NoOptimization, + NoOptimization, NoOptimization, NoOptimization) verifyRewritePattern(patterns, excepted) } @@ -67,8 +67,27 @@ class RegularExpressionRewriteSuite extends AnyFunSuite { PrefixRange("火花急流", 1, 19968, 40869), NoOptimization, // starts with PrefixRange not supported NoOptimization, // starts with PrefixRange not supported - PrefixRange("", 6, 48, 57), - PrefixRange("", 3, 48, 57) + NoOptimization, // .* can't match line break so can't be optimized + NoOptimization // .* can't match line break so can't be optimized + ) + verifyRewritePattern(patterns, excepted) + } + + test("regex rewrite multiple contains") { + import RegexOptimizationType._ + val patterns = Seq( + "(abc|def).*", + ".*(abc|def|ghi).*", + "((abc)|(def))", + "(abc)|(def)", + "(火花|急流)" + ) + val excepted = Seq( + MultipleContains(Seq("abc", "def")), + MultipleContains(Seq("abc", "def", "ghi")), + MultipleContains(Seq("abc", "def")), + MultipleContains(Seq("abc", "def")), + MultipleContains(Seq("火花", "急流")) ) verifyRewritePattern(patterns, excepted) } From d9686d4dba4afd6e2c061fd2822d40d9a55d88d4 Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Wed, 12 Jun 2024 16:26:28 -0500 Subject: [PATCH 39/68] Add in the ability to fingerprint JSON columns (#11002) Signed-off-by: Robert (Bobby) Evans --- .../spark/sql/tests/datagen/bigDataGen.scala | 1181 ++++++++++++++--- 1 file changed, 1002 insertions(+), 179 deletions(-) diff --git a/datagen/src/main/scala/org/apache/spark/sql/tests/datagen/bigDataGen.scala b/datagen/src/main/scala/org/apache/spark/sql/tests/datagen/bigDataGen.scala index 91335afe4e6..14e0d4e0970 100644 --- a/datagen/src/main/scala/org/apache/spark/sql/tests/datagen/bigDataGen.scala +++ b/datagen/src/main/scala/org/apache/spark/sql/tests/datagen/bigDataGen.scala @@ -16,21 +16,22 @@ package org.apache.spark.sql.tests.datagen +import com.fasterxml.jackson.core.{JsonFactoryBuilder, JsonParser, JsonToken} +import com.fasterxml.jackson.core.json.JsonReadFeature import java.math.{BigDecimal => JavaBigDecimal} import java.sql.{Date, Timestamp} import java.time.{Duration, Instant, LocalDate, LocalDateTime} import java.util - import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import scala.math.BigDecimal.RoundingMode import scala.util.Random -import org.apache.spark.sql.{Column, DataFrame, SparkSession} +import org.apache.spark.sql.{Column, DataFrame, Row, SparkSession} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Expression, XXH64} import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, ArrayData, DateTimeUtils} -import org.apache.spark.sql.functions.col +import org.apache.spark.sql.functions.{approx_count_distinct, avg, coalesce, col, count, lit, stddev, struct, transform, udf, when} import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.random.XORShiftRandom @@ -79,22 +80,28 @@ class RowLocation(val rowNum: Long, val subRows: Array[Int] = null) { * hash. This makes the generated data correlated for all column/child columns. * @param tableNum a unique ID for the table this is a part of. * @param columnNum the location of the column in the data being generated + * @param substringNum the location of the substring column * @param correlatedKeyGroup the correlated key group this column is a part of, if any. */ -case class ColumnLocation(tableNum: Int, columnNum: Int, correlatedKeyGroup: Option[Long] = None) { - def forNextColumn(): ColumnLocation = ColumnLocation(tableNum, columnNum + 1) +case class ColumnLocation(tableNum: Int, + columnNum: Int, + substringNum: Int, + correlatedKeyGroup: Option[Long] = None) { + def forNextColumn(): ColumnLocation = ColumnLocation(tableNum, columnNum + 1, 0) + def forNextSubstring: ColumnLocation = ColumnLocation(tableNum, columnNum, substringNum + 1) /** * Create a new ColumnLocation that is specifically for a given key group */ def forCorrelatedKeyGroup(keyGroup: Long): ColumnLocation = - ColumnLocation(tableNum, columnNum, Some(keyGroup)) + ColumnLocation(tableNum, columnNum, substringNum, Some(keyGroup)) /** * Hash the location into a single long value. */ - lazy val hashLoc: Long = XXH64.hashLong(tableNum, correlatedKeyGroup.getOrElse(columnNum)) + lazy val hashLoc: Long = XXH64.hashLong(tableNum, + correlatedKeyGroup.getOrElse(XXH64.hashLong(columnNum, substringNum))) } /** @@ -115,6 +122,9 @@ case class ColumnConf(columnLoc: ColumnLocation, def forNextColumn(nullable: Boolean): ColumnConf = ColumnConf(columnLoc.forNextColumn(), nullable, numTableRows) + def forNextSubstring: ColumnConf = + ColumnConf(columnLoc.forNextSubstring, nullable = true, numTableRows) + /** * Create a new configuration based on this, but for a given correlated key group. */ @@ -303,6 +313,23 @@ case class VarLengthGeneratorFunction(minLength: Int, maxLength: Int) extends } } +case class StdDevLengthGen(mean: Double, + stdDev: Double, + mapping: LocationToSeedMapping = null) extends + LengthGeneratorFunction { + override def withLocationToSeedMapping(mapping: LocationToSeedMapping): LengthGeneratorFunction = + StdDevLengthGen(mean, stdDev, mapping) + + override def apply(rowLoc: RowLocation): Int = { + val r = DataGen.getRandomFor(rowLoc, mapping) + val g = r.nextGaussian() // g has a mean of 0 and a stddev of 1.0 + val adjusted = mean + (g * stdDev) + // If the range of seed is too small compared to the stddev and mean we will + // end up with an invalid distribution, but they asked for it. + math.max(0, math.round(adjusted).toInt) + } +} + /** * Generate nulls with a given probability. * @param prob 0.0 to 1.0 for how often nulls should appear in the output. @@ -562,11 +589,8 @@ case class DataGenExpr(child: Expression, } } -/** - * Base class for generating a column/sub-column. This holds configuration for the column, - * and handles what is needed to convert it into GeneratorFunction - */ -abstract class DataGen(var conf: ColumnConf, +abstract class CommonDataGen( + var conf: ColumnConf, defaultValueRange: Option[(Any, Any)], var seedMapping: LocationToSeedMapping = FlatDistribution(), var nullMapping: LocationToSeedMapping = FlatDistribution(), @@ -576,26 +600,25 @@ abstract class DataGen(var conf: ColumnConf, protected var valueRange: Option[(Any, Any)] = defaultValueRange /** - * Set a value range for this data gen. + * Set a value range */ - def setValueRange(min: Any, max: Any): DataGen = { + def setValueRange(min: Any, max: Any): CommonDataGen = { valueRange = Some((min, max)) this } /** - * Set a custom GeneratorFunction to use for this column. + * Set a custom GeneratorFunction */ - def setValueGen(f: GeneratorFunction): DataGen = { + def setValueGen(f: GeneratorFunction): CommonDataGen = { userProvidedValueGen = Some(f) this } /** - * Set a NullGeneratorFunction for this column. This will not be used - * if the column is not nullable. + * Set a NullGeneratorFunction */ - def setNullGen(f: NullGeneratorFunction): DataGen = { + def setNullGen(f: NullGeneratorFunction): CommonDataGen = { this.userProvidedNullGen = Some(f) this } @@ -604,12 +627,12 @@ abstract class DataGen(var conf: ColumnConf, * Set the probability of a null appearing in the output. The probability should be * 0.0 to 1.0. */ - def setNullProbability(probability: Double): DataGen = { + def setNullProbability(probability: Double): CommonDataGen = { this.userProvidedNullGen = Some(NullProbabilityGenerationFunction(probability)) this } - def setNullProbabilityRecursively(probability: Double): DataGen = { + def setNullProbabilityRecursively(probability: Double): CommonDataGen = { this.userProvidedNullGen = Some(NullProbabilityGenerationFunction(probability)) children.foreach { case (_, dataGen) => @@ -621,7 +644,7 @@ abstract class DataGen(var conf: ColumnConf, /** * Set a specific location to seed mapping for the value generation. */ - def setSeedMapping(seedMapping: LocationToSeedMapping): DataGen = { + def setSeedMapping(seedMapping: LocationToSeedMapping): CommonDataGen = { this.seedMapping = seedMapping this } @@ -629,7 +652,7 @@ abstract class DataGen(var conf: ColumnConf, /** * Set a specific location to seed mapping for the null generation. */ - def setNullMapping(nullMapping: LocationToSeedMapping): DataGen = { + def setNullMapping(nullMapping: LocationToSeedMapping): CommonDataGen = { this.nullMapping = nullMapping this } @@ -638,7 +661,7 @@ abstract class DataGen(var conf: ColumnConf, * Set a specific LengthGeneratorFunction to use. This will only be used if * the datatype needs a length. */ - def setLengthGen(lengthGen: LengthGeneratorFunction): DataGen = { + def setLengthGen(lengthGen: LengthGeneratorFunction): CommonDataGen = { this.lengthGen = lengthGen this } @@ -646,25 +669,30 @@ abstract class DataGen(var conf: ColumnConf, /** * Set the length generation to be a fixed length. */ - def setLength(len: Int): DataGen = { + def setLength(len: Int): CommonDataGen = { this.lengthGen = FixedLengthGeneratorFunction(len) this } - def setLength(minLen: Int, maxLen: Int) = { + def setLength(minLen: Int, maxLen: Int): CommonDataGen = { this.lengthGen = VarLengthGeneratorFunction(minLen, maxLen) this } + def setGaussianLength(mean: Double, stdDev: Double): CommonDataGen = { + this.lengthGen = StdDevLengthGen(mean, stdDev) + this + } + /** * Add this column to a specific correlated key group. This should not be * called directly by users. */ def setCorrelatedKeyGroup(keyGroup: Long, - minSeed: Long, maxSeed: Long, - seedMapping: LocationToSeedMapping): DataGen = { + minSeed: Long, maxSeed: Long, + seedMapping: LocationToSeedMapping): CommonDataGen = { conf = conf.forCorrelatedKeyGroup(keyGroup) - .forSeedRange(minSeed, maxSeed) + .forSeedRange(minSeed, maxSeed) this.seedMapping = seedMapping this } @@ -672,7 +700,7 @@ abstract class DataGen(var conf: ColumnConf, /** * Set a range of seed values that should be returned by the LocationToSeedMapping */ - def setSeedRange(min: Long, max: Long): DataGen = { + def setSeedRange(min: Long, max: Long): CommonDataGen = { conf = conf.forSeedRange(min, max) this } @@ -681,7 +709,7 @@ abstract class DataGen(var conf: ColumnConf, * Get the default value generator for this specific data gen. */ protected def getValGen: GeneratorFunction - def children: Seq[(String, DataGen)] + def children: Seq[(String, CommonDataGen)] /** * Get the final ready to use GeneratorFunction for the data generator. @@ -690,8 +718,8 @@ abstract class DataGen(var conf: ColumnConf, val sm = seedMapping.withColumnConf(conf) val lg = lengthGen.withLocationToSeedMapping(sm) var valGen = userProvidedValueGen.getOrElse(getValGen) - .withLocationToSeedMapping(sm) - .withLengthGeneratorFunction(lg) + .withLocationToSeedMapping(sm) + .withLengthGeneratorFunction(lg) valueRange.foreach { case (min, max) => valGen = valGen.withValueRange(min, max) @@ -700,35 +728,75 @@ abstract class DataGen(var conf: ColumnConf, val nullColConf = conf.forNulls val nm = nullMapping.withColumnConf(nullColConf) userProvidedNullGen.get - .withWrapped(valGen) - .withLocationToSeedMapping(nm) + .withWrapped(valGen) + .withLocationToSeedMapping(nm) } else { valGen } } - /** - * Get the data type for this column - */ - def dataType: DataType - /** * Is this column nullable or not. */ def nullable: Boolean = conf.nullable /** - * Get a child column for a given name, if it has one. + * Get a child for a given name, if it has one. */ - final def apply(name: String): DataGen = { + final def apply(name: String): CommonDataGen = { get(name).getOrElse{ throw new IllegalStateException(s"Could not find a child $name for $this") } } - def get(name: String): Option[DataGen] = None + def get(name: String): Option[CommonDataGen] = None +} + + +/** + * Base class for generating a column/sub-column. This holds configuration + * for the column, and handles what is needed to convert it into GeneratorFunction + */ +abstract class DataGen( + conf: ColumnConf, + defaultValueRange: Option[(Any, Any)], + seedMapping: LocationToSeedMapping = FlatDistribution(), + nullMapping: LocationToSeedMapping = FlatDistribution(), + lengthGen: LengthGeneratorFunction = FixedLengthGeneratorFunction(10)) extends + CommonDataGen(conf, defaultValueRange, seedMapping, nullMapping, lengthGen) { + + /** + * Get the data type for this column + */ + def dataType: DataType + + override def get(name: String): Option[DataGen] = None + + def getSubstringGen: Option[SubstringDataGen] = None + + def substringGen: SubstringDataGen = + getSubstringGen.getOrElse( + throw new IllegalArgumentException("substring data gen was not set")) + + def setSubstringGen(f : ColumnConf => SubstringDataGen): Unit = + setSubstringGen(Option(f(conf.forNextSubstring))) + + def setSubstringGen(subgen: Option[SubstringDataGen]): Unit = + throw new IllegalArgumentException("substring data gens can only be set for a STRING") } +/** + * Base class for generating a sub-string. This holds configuration + * for the substring, and handles what is needed to convert it into a GeneratorFunction + */ +abstract class SubstringDataGen( + conf: ColumnConf, + defaultValueRange: Option[(Any, Any)], + seedMapping: LocationToSeedMapping = FlatDistribution(), + nullMapping: LocationToSeedMapping = FlatDistribution(), + lengthGen: LengthGeneratorFunction = FixedLengthGeneratorFunction(10)) extends + CommonDataGen(conf, defaultValueRange, seedMapping, nullMapping, lengthGen) {} + /** * A special GeneratorFunction that just returns the computed seed. This is helpful for * debugging distributions or if you want long values without any abstraction in between. @@ -1494,155 +1562,866 @@ class FloatGen(conf: ColumnConf, defaultValueRange: Option[(Any, Any)]) override def children: Seq[(String, DataGen)] = Seq.empty } -trait JSONType { - def appendRandomValue(sb: StringBuilder, - index: Int, - maxStringLength: Int, - maxArrayLength: Int, - maxObjectLength: Int, - depth: Int, - maxDepth: Int, - r: Random): Unit -} +case class JsonPathElement(name: String, is_array: Boolean) +case class JsonLevel(path: Array[JsonPathElement], data_type: String, length: Int, value: String) {} + +object JsonColumnStats { + private def printHelp(): Unit = { + println("JSON Fingerprinting Tool:") + println("PARAMS: ") + println(" is a path to a Spark dataframe to read in") + println(" is a path in a Spark file system to write out fingerprint data to.") + println() + println("OPTIONS:") + println(" --json= where is the name of a top level String column") + println(" --anon= where is a SEED used to anonymize the JSON keys ") + println(" and column names.") + println(" --input_format= where is parquet or ORC. Defaults to parquet.") + println(" --overwrite to enable overwriting the fingerprint output.") + println(" --debug to enable some debug information to be printed out") + println(" --help to print out this help message") + println() + } + + def main(args: Array[String]): Unit = { + var inputPath = Option.empty[String] + var outputPath = Option.empty[String] + val jsonColumns = ArrayBuffer.empty[String] + var anonSeed = Option.empty[Long] + var debug = false + var argsDone = false + var format = "parquet" + var overwrite = false + + args.foreach { + case a if !argsDone && a.startsWith("--json=") => + jsonColumns += a.substring("--json=".length) + case a if !argsDone && a.startsWith("--anon=") => + anonSeed = Some(a.substring("--anon=".length).toLong) + case a if !argsDone && a.startsWith("--input_format=") => + format = a.substring("--input_format=".length).toLowerCase(java.util.Locale.US) + case "--overwrite" if !argsDone => + overwrite = true + case "--debug" if !argsDone => + debug = true + case "--help" if !argsDone => + printHelp() + System.exit(0) + case "--" if !argsDone => + argsDone = true + case a if !argsDone && a.startsWith("--") => // "--" was covered above already + println(s"ERROR $a is not a supported argument") + printHelp() + System.exit(-1) + case a if inputPath.isEmpty => + inputPath = Some(a) + case a if outputPath.isEmpty => + outputPath = Some(a) + case a => + println(s"ERROR only two arguments are supported. Found $a") + printHelp() + System.exit(-1) + } + if (outputPath.isEmpty) { + println("ERROR both an inputPath and an outputPath are required") + printHelp() + System.exit(-1) + } + + val spark = SparkSession.builder.getOrCreate() + spark.sparkContext.setLogLevel("WARN") + + val df = spark.read.format(format).load(inputPath.get) + jsonColumns.foreach { column => + val fp = fingerPrint(df, df(column), anonSeed) + val name = anonSeed.map(s => anonymizeString(column, s)).getOrElse(column) + val fullOutPath = s"${outputPath.get}/$name" + var writer = fp.write + if (overwrite) { + writer = writer.mode("overwrite") + } + if (debug) { + anonSeed.foreach { s => + println(s"Keys and columns will be anonymized with seed $s") + } + println(s"Writing $column fingerprint to $fullOutPath") + spark.time(writer.parquet(fullOutPath)) + println(s"Wrote ${spark.read.parquet(fullOutPath).count} rows") + spark.read.parquet(fullOutPath).show() + } else { + writer.parquet(fullOutPath) + } + } + } -object JSONType { - def selectType(depth: Int, - maxDepth: Int, - r: Random): JSONType = { - val toSelectFrom = if (depth < maxDepth) { - Seq(QuotedJSONString, JSONLong, JSONDouble, JSONArray, JSONObject) - } else { - Seq(QuotedJSONString, JSONLong, JSONDouble) - } - val index = r.nextInt(toSelectFrom.length) - toSelectFrom(index) - } -} - -object QuotedJSONString extends JSONType { - override def appendRandomValue(sb: StringBuilder, - index: Int, - maxStringLength: Int, - maxArrayLength: Int, - maxObjectLength: Int, - depth: Int, - maxDepth: Int, - r: Random): Unit = { - val strValue = r.nextString(r.nextInt(maxStringLength + 1)) - .replace("\\", "\\\\") - .replace("\"", "\\\"") - .replace("\n", "\\n") - .replace("\r", "\\r") - .replace("\b", "\\b") - .replace("\f", "\\f") - sb.append('"') - sb.append(strValue) - sb.append('"') - } -} - -object JSONLong extends JSONType { - override def appendRandomValue(sb: StringBuilder, - index: Int, - maxStringLength: Int, - maxArrayLength: Int, - maxObjectLength: Int, - depth: Int, - maxDepth: Int, - r: Random): Unit = { - sb.append(r.nextLong()) - } -} - -object JSONDouble extends JSONType { - override def appendRandomValue(sb: StringBuilder, - index: Int, - maxStringLength: Int, - maxArrayLength: Int, - maxObjectLength: Int, - depth: Int, - maxDepth: Int, - r: Random): Unit = { - sb.append(r.nextDouble() * 4096.0) - } -} - -object JSONArray extends JSONType { - override def appendRandomValue(sb: StringBuilder, - index: Int, - maxStringLength: Int, - maxArrayLength: Int, - maxObjectLength: Int, - depth: Int, - maxDepth: Int, - r: Random): Unit = { - val childType = JSONType.selectType(depth, maxDepth, r) - val length = r.nextInt(maxArrayLength + 1) - sb.append("[") + case class JsonNodeStats(count: Long, meanLen: Double, stdDevLength: Double, dc: Long) + + class JsonNode() { + private val forDataType = + mutable.HashMap[String, (JsonNodeStats, mutable.HashMap[String, JsonNode])]() + + def getChild(name: String, isArray: Boolean): JsonNode = { + val dt = if (isArray) { "ARRAY" } else { "OBJECT" } + val typed = forDataType.getOrElse(dt, + throw new IllegalArgumentException(s"$dt is not a set data type yet.")) + typed._2.getOrElse(name, + throw new IllegalArgumentException(s"$name is not a child when the type is $dt")) + } + + def contains(name: String, isArray: Boolean): Boolean = { + val dt = if (isArray) { "ARRAY" } else { "OBJECT" } + forDataType.get(dt).exists { children => + children._2.contains(name) + } + } + + def addChild(name: String, isArray: Boolean): JsonNode = { + val dt = if (isArray) { "ARRAY" } else { "OBJECT" } + val found = forDataType.getOrElse(dt, + throw new IllegalArgumentException(s"$dt was not already added as a data type")) + if (found._2.contains(name)) { + throw new IllegalArgumentException(s"$dt already has a child named $name") + } + val node = new JsonNode() + found._2.put(name, node) + node + } + + def addChoice(dt: String, stats: JsonNodeStats): Unit = { + if (forDataType.contains(dt)) { + throw new IllegalArgumentException(s"$dt was already added as a data type") + } + forDataType.put(dt, (stats, new mutable.HashMap[String, JsonNode]())) + } + + override def toString: String = { + forDataType.toString() + } + + def totalCount: Long = { + forDataType.values.map{ case (stats, _) => stats.count}.sum + } + + private def makeNoChoiceGenRecursive(dt: String, + children: mutable.HashMap[String, JsonNode], + cc: ColumnConf): (SubstringDataGen, ColumnConf) = { + var c = cc + val ret = dt match { + case "LONG" => new JSONLongGen(c) + case "DOUBLE" => new JSONDoubleGen(c) + case "BOOLEAN" => new JSONBoolGen(c) + case "NULL" => new JSONNullGen(false, c) + case "VALUE_NULL" => new JSONNullGen(true, c) + case "ERROR" => new JSONErrorGen(c) + case "STRING" => new JSONStringGen(c) + case "ARRAY" => + val child = if (children.isEmpty) { + // A corner case, we will just make it a BOOL column and it will be ignored + val tmp = new JSONBoolGen(c) + c = c.forNextSubstring + tmp + } else { + val tmp = children.values.head.makeGenRecursive(c) + c = tmp._2 + tmp._1 + } + new JSONArrayGen(child, c) + case "OBJECT" => + val childGens = if (children.isEmpty) { + Seq.empty + } else { + children.toSeq.map { + case (k, node) => + val tmp = node.makeGenRecursive(c) + c = tmp._2 + (k, tmp._1) + } + } + new JSONObjectGen(childGens, c) + case other => + throw new IllegalArgumentException(s"$other is not a leaf node type") + } + (ret, c.forNextSubstring) + } + + private def makeGenRecursive(cc: ColumnConf): (SubstringDataGen, ColumnConf) = { + var c = cc + // We are going to recursively walk the tree for all of the values. + if (forDataType.size == 1) { + // We don't need a choice at all. This makes it simpler.. + val (dt, (_, children)) = forDataType.head + makeNoChoiceGenRecursive(dt, children, c) + } else { + val totalSum = forDataType.map(f => f._2._1.count).sum.toDouble + var runningSum = 0L + val allChoices = ArrayBuffer[(Double, String, SubstringDataGen)]() + forDataType.foreach { + case (dt, (stats, children)) => + val tmp = makeNoChoiceGenRecursive(dt, children, c) + c = tmp._2 + runningSum += stats.count + allChoices.append((runningSum/totalSum, dt, tmp._1)) + } + + val ret = new JSONChoiceGen(allChoices.toSeq, c) + (ret, c.forNextSubstring) + } + } + + def makeGen(cc: ColumnConf): SubstringDataGen = { + val (ret, _) = makeGenRecursive(cc) + ret + } + + def setStatsSingle(dg: CommonDataGen, + dt: String, + stats: JsonNodeStats, + nullPct: Double): Unit = { + + val includeLength = dt != "OBJECT" && dt != "BOOLEAN" && dt != "NULL" && dt != "VALUE_NULL" + val includeNullPct = nullPct > 0.0 + if (includeLength) { + dg.setGaussianLength(stats.meanLen, stats.stdDevLength) + } + if (includeNullPct) { + dg.setNullProbability(nullPct) + } + dg.setSeedRange(1, stats.dc) + } + + def setStats(dg: CommonDataGen, + parentCount: Option[Long]): Unit = { + // We are going to recursively walk the tree... + if (forDataType.size == 1) { + // We don't need a choice at all. This makes it simpler.. + val (dt, (stats, children)) = forDataType.head + val nullPct = parentCount.map { pc => + (pc - stats.count).toDouble/pc + }.getOrElse(0.0) + setStatsSingle(dg, dt, stats, nullPct) + val myCount = if (dt == "OBJECT") { + Some(totalCount) + } else { + None + } + children.foreach { + case (name, node) => + node.setStats(dg(name), myCount) + } + } else { + // We have choices to make between different types. + // The null percent cannot be calculated for each individual choice + // but is calculated on the group as a whole instead + parentCount.foreach { pc => + val tc = totalCount + val choiceNullPct = (pc - tc).toDouble / pc + if (choiceNullPct > 0.0) { + dg.setNullProbability(choiceNullPct) + } + } + forDataType.foreach { + case (dt, (stats, children)) => + // When there is a choice the name to access it is the data type + val choiceDg = dg(dt) + setStatsSingle(choiceDg, dt, stats, 0.0) + children.foreach { + case (name, node) => + val myCount = if (dt == "OBJECT") { + // Here we only want the count for the OBJECTs + Some(stats.count) + } else { + None + } + node.setStats(choiceDg(name), myCount) + } + } + } + } + } + + private lazy val jsonFactory = new JsonFactoryBuilder() + // The two options below enabled for Hive compatibility + .enable(JsonReadFeature.ALLOW_UNESCAPED_CONTROL_CHARS) + .enable(JsonReadFeature.ALLOW_SINGLE_QUOTES) + .build() + + private def processNext(parser: JsonParser, + currentPath: ArrayBuffer[JsonPathElement], + output: ArrayBuffer[JsonLevel]): Unit = { + parser.currentToken() match { + case JsonToken.START_OBJECT => + parser.nextToken() + while (parser.currentToken() != JsonToken.END_OBJECT) { + processNext(parser, currentPath, output) + } + output.append(JsonLevel(currentPath.toArray, "OBJECT", 0, "")) + parser.nextToken() + case JsonToken.START_ARRAY => + currentPath.append(JsonPathElement("data", is_array = true)) + parser.nextToken() + var length = 0 + while (parser.currentToken() != JsonToken.END_ARRAY) { + length += 1 + processNext(parser, currentPath, output) + } + currentPath.remove(currentPath.length - 1) + output.append(JsonLevel(currentPath.toArray, "ARRAY", length, "")) + parser.nextToken() + case JsonToken.FIELD_NAME => + currentPath.append(JsonPathElement(parser.getCurrentName, is_array = false)) + parser.nextToken() + processNext(parser, currentPath, output) + currentPath.remove(currentPath.length - 1) + case JsonToken.VALUE_NUMBER_INT => + val length = parser.getValueAsString.getBytes("UTF-8").length + output.append(JsonLevel(currentPath.toArray, "LONG", length, parser.getValueAsString)) + parser.nextToken() + case JsonToken.VALUE_NUMBER_FLOAT => + val length = parser.getValueAsString.getBytes("UTF-8").length + output.append(JsonLevel(currentPath.toArray, "DOUBLE", length, parser.getValueAsString)) + parser.nextToken() + case JsonToken.VALUE_TRUE | JsonToken.VALUE_FALSE => + val length = parser.getValueAsString.getBytes("UTF-8").length + output.append(JsonLevel(currentPath.toArray, "BOOLEAN", length, parser.getValueAsString)) + parser.nextToken() + case JsonToken.VALUE_NULL | null => + output.append(JsonLevel(currentPath.toArray, "VALUE_NULL", 4, "NULL")) + parser.nextToken() + case JsonToken.VALUE_STRING => + val length = parser.getValueAsString.getBytes("UTF-8").length + output.append(JsonLevel(currentPath.toArray, "STRING", length, parser.getValueAsString)) + parser.nextToken() + case other => + throw new IllegalStateException(s"DON'T KNOW HOW TO DEAL WITH $other") + } + } + + def jsonStatsUdf(json: String): Array[JsonLevel] = { + val output = new ArrayBuffer[JsonLevel]() + try { + val currentPath = new ArrayBuffer[JsonPathElement]() + if (json == null) { + output.append(JsonLevel(Array.empty, "NULL", 0, "")) + } else { + val parser = jsonFactory.createParser(json) + try { + parser.nextToken() + processNext(parser, currentPath, output) + } finally { + parser.close() + } + } + } catch { + case _: com.fasterxml.jackson.core.JsonParseException => + output.clear() + output.append(JsonLevel(Array.empty, "ERROR", json.getBytes("UTF-8").length, json)) + } + output.toArray + } + + private lazy val extractPaths = udf(json => jsonStatsUdf(json)) + + def anonymizeString(str: String, seed: Long): String = { + val length = str.length + val data = new Array[Byte](length) + val hash = XXH64.hashLong(str.hashCode, seed) + val r = new Random() + r.setSeed(hash) (0 until length).foreach { i => - if (i > 0) { - sb.append(",") + val tmp = r.nextInt(16) + data(i) = (tmp + 'A').toByte + } + new String(data) + } + + private lazy val anonPath = udf((str, seed) => anonymizeString(str, seed)) + + def anonymizeFingerPrint(df: DataFrame, anonSeed: Long): DataFrame = { + df.withColumn("tmp", transform(col("path"), + o => { + val name = o("name") + val isArray = o("is_array") + val anon = anonPath(name, lit(anonSeed)) + val newName = when(isArray, name).otherwise(anon).alias("name") + struct(newName, isArray) + })) + .drop("path").withColumnRenamed("tmp", "path") + .orderBy("path", "dt") + .selectExpr("path", "dt","c","mean_len","stddev_len","distinct","version") + } + + def fingerPrint(df: DataFrame, column: Column, anonymize: Option[Long] = None): DataFrame = { + val ret = df.select(extractPaths(column).alias("paths")) + .selectExpr("explode_outer(paths) as p") + .selectExpr("p.path as path", "p.data_type as dt", "p.length as len", "p.value as value") + .groupBy(col("path"), col("dt")).agg( + count(lit(1)).alias("c"), + avg(col("len")).alias("mean_len"), + coalesce(stddev(col("len")), lit(0.0)).alias("stddev_len"), + approx_count_distinct(col("value")).alias("distinct")) + .orderBy("path", "dt").withColumn("version", lit("0.1")) + .selectExpr("path", "dt","c","mean_len","stddev_len","distinct","version") + + anonymize.map { anonSeed => + anonymizeFingerPrint(ret, anonSeed) + }.getOrElse(ret) + } + + def apply(aggForColumn: DataFrame, genColumn: ColumnGen): Unit = + apply(aggForColumn, genColumn.dataGen) + + private val expectedSchema = StructType.fromDDL( + "path ARRAY>," + + "dt STRING," + + "c BIGINT," + + "mean_len DOUBLE," + + "stddev_len DOUBLE," + + "distinct BIGINT," + + "version STRING") + + def apply(aggForColumn: DataFrame, gen: DataGen): Unit = { + val aggData = aggForColumn.orderBy("path", "dt").collect() + val rootNode: JsonNode = new JsonNode() + assert(aggData.length > 0) + val schema = aggData.head.schema + assert(schema.length == expectedSchema.length) + schema.fields.zip(expectedSchema.fields).foreach { + case(found, expected) => + assert(found.name == expected.name) + // TODO we can worry about the exact types later if we need to + } + assert(aggData.head.getString(6) == "0.1") + aggData.foreach { row => + val fullPath = row.getAs[mutable.WrappedArray[Row]](0) + val parsedPath = fullPath.map(r => (r.getString(0), r.getBoolean(1))).toList + val dt = row.getString(1) + val count = row.getLong(2) + val meanLen = row.getDouble(3) + val stdLen = row.getDouble(4) + val dc = row.getLong(5) + + val stats = JsonNodeStats(count, meanLen, stdLen, dc) + var currentNode = rootNode + // Find everything up to the last path element + if (parsedPath.length > 1) { + parsedPath.slice(0, parsedPath.length - 1).foreach { + case (name, isArray) => + currentNode = currentNode.getChild(name, isArray) + } + } + + if (parsedPath.nonEmpty) { + // For the last path element (that is not the root element) we might need to add it + // as a child + val (name, isArray) = parsedPath.last + if (!currentNode.contains(name, isArray)) { + currentNode.addChild(name, isArray) + } + currentNode = currentNode.getChild(name, isArray) } - childType.appendRandomValue(sb, i, maxStringLength, maxArrayLength, maxObjectLength, - depth + 1, maxDepth, r) + currentNode.addChoice(dt, stats) } - sb.append("]") + + gen.setSubstringGen(cc => rootNode.makeGen(cc)) + rootNode.setStats(gen.substringGen, None) } } -object JSONObject extends JSONType { - override def appendRandomValue(sb: StringBuilder, - index: Int, - maxStringLength: Int, - maxArrayLength: Int, - maxObjectLength: Int, - depth: Int, - maxDepth: Int, - r: Random): Unit = { - val length = r.nextInt(maxObjectLength) + 1 - sb.append("{") - (0 until length).foreach { i => - if (i > 0) { - sb.append(",") + +case class JSONStringGenFunc(lengthGen: LengthGeneratorFunction = null, + mapping: LocationToSeedMapping = null) extends GeneratorFunction { + + override def apply(rowLoc: RowLocation): Any = { + val len = lengthGen(rowLoc) + val r = DataGen.getRandomFor(rowLoc, mapping) + val buffer = new Array[Byte](len) + var at = 0 + while (at < len) { + // Value range is 32 (Space) to 126 (~) + buffer(at) = (r.nextInt(126 - 31) + 32).toByte + at += 1 + } + val strVal = new String(buffer, 0, len) + .replace("\\", "\\\\") + .replace("\"", "\\\"") + .replace("\n", "\\n") + .replace("\r", "\\r") + .replace("\b", "\\b") + .replace("\f", "\\f") + '"' + strVal + '"' + } + + override def withLengthGeneratorFunction(lengthGen: LengthGeneratorFunction): JSONStringGenFunc = + JSONStringGenFunc(lengthGen, mapping) + + override def withLocationToSeedMapping(mapping: LocationToSeedMapping): JSONStringGenFunc = + JSONStringGenFunc(lengthGen, mapping) + + override def withValueRange(min: Any, max: Any): GeneratorFunction = + throw new IllegalArgumentException("value ranges are not supported for JSON") +} + +class JSONStringGen(conf: ColumnConf, + defaultValueRange: Option[(Any, Any)] = None) + extends SubstringDataGen(conf, defaultValueRange) { + + override protected def getValGen: GeneratorFunction = JSONStringGenFunc() + + override def children: Seq[(String, SubstringDataGen)] = Seq.empty +} + +case class JSONLongGenFunc(lengthGen: LengthGeneratorFunction = null, + mapping: LocationToSeedMapping = null) extends GeneratorFunction { + + override def apply(rowLoc: RowLocation): Any = { + val len = math.max(lengthGen(rowLoc), 1) // We need at least 1 long for a valid value + val r = DataGen.getRandomFor(rowLoc, mapping) + val buffer = new Array[Byte](len) + var at = 0 + while (at < len) { + if (at == 0) { + // No leading 0's + buffer(at) = (r.nextInt(9) + '1').toByte + } else { + buffer(at) = (r.nextInt(10) + '0').toByte } - sb.append("\"key_") - sb.append(i) - sb.append("_") - sb.append(depth ) - sb.append("\":") - val childType = JSONType.selectType(depth, maxDepth, r) - childType.appendRandomValue(sb, i, maxStringLength, maxArrayLength, maxObjectLength, - depth + 1, maxDepth, r) + at += 1 } - sb.append("}") + new String(buffer, 0, len) } + + override def withLengthGeneratorFunction(lengthGen: LengthGeneratorFunction): JSONLongGenFunc = + JSONLongGenFunc(lengthGen, mapping) + + override def withLocationToSeedMapping(mapping: LocationToSeedMapping): JSONLongGenFunc = + JSONLongGenFunc(lengthGen, mapping) + + override def withValueRange(min: Any, max: Any): GeneratorFunction = + throw new IllegalArgumentException("value ranges are not supported for JSON") } -case class JSONGenFunc( - maxStringLength: Int, - maxArrayLength: Int, - maxObjectLength: Int, - maxDepth: Int, - lengthGen: LengthGeneratorFunction = null, - mapping: LocationToSeedMapping = null) extends GeneratorFunction { +class JSONLongGen(conf: ColumnConf, + defaultValueRange: Option[(Any, Any)] = None) + extends SubstringDataGen(conf, defaultValueRange) { + + override protected def getValGen: GeneratorFunction = JSONLongGenFunc() + + override def children: Seq[(String, SubstringDataGen)] = Seq.empty +} + +case class JSONDoubleGenFunc(lengthGen: LengthGeneratorFunction = null, + mapping: LocationToSeedMapping = null) extends GeneratorFunction { override def apply(rowLoc: RowLocation): Any = { + val len = math.max(lengthGen(rowLoc), 3) // We have to have at least 3 chars NUM.NUM val r = DataGen.getRandomFor(rowLoc, mapping) - val sb = new StringBuilder() - JSONObject.appendRandomValue(sb, 0, maxStringLength, maxArrayLength, maxObjectLength, - 0, maxDepth, r) - // For now I am going to have some hard coded keys - UTF8String.fromString(sb.toString()) + val beforeLen = if (len == 3) { 1 } else { r.nextInt(len - 3) + 1 } + val buffer = new Array[Byte](len) + var at = 0 + while (at < len) { + if (at == 0) { + // No leading 0's + buffer(at) = (r.nextInt(9) + '1').toByte + } else if (at == beforeLen) { + buffer(at) = '.' + } else { + buffer(at) = (r.nextInt(10) + '0').toByte + } + at += 1 + } + UTF8String.fromBytes(buffer, 0, len) } - override def withLengthGeneratorFunction(lengthGen: LengthGeneratorFunction): GeneratorFunction = - JSONGenFunc(maxStringLength, maxArrayLength, maxObjectLength, maxDepth, lengthGen, mapping) + override def withLengthGeneratorFunction(lengthGen: LengthGeneratorFunction): JSONDoubleGenFunc = + JSONDoubleGenFunc(lengthGen, mapping) - override def withLocationToSeedMapping(mapping: LocationToSeedMapping): GeneratorFunction = - JSONGenFunc(maxStringLength, maxArrayLength, maxObjectLength, maxDepth, lengthGen, mapping) + override def withLocationToSeedMapping(mapping: LocationToSeedMapping): JSONDoubleGenFunc = + JSONDoubleGenFunc(lengthGen, mapping) override def withValueRange(min: Any, max: Any): GeneratorFunction = - throw new IllegalArgumentException("value ranges are not supported for strings") + throw new IllegalArgumentException("value ranges are not supported for JSON") +} + +class JSONDoubleGen(conf: ColumnConf, + defaultValueRange: Option[(Any, Any)] = None) + extends SubstringDataGen(conf, defaultValueRange) { + + override protected def getValGen: GeneratorFunction = JSONDoubleGenFunc() + + override def children: Seq[(String, SubstringDataGen)] = Seq.empty +} + +case class JSONBoolGenFunc(lengthGen: LengthGeneratorFunction = null, + mapping: LocationToSeedMapping = null) extends GeneratorFunction { + + override def apply(rowLoc: RowLocation): Any = { + val r = DataGen.getRandomFor(rowLoc, mapping) + val ret = if (r.nextBoolean()) "true" else "false" + UTF8String.fromString(ret) + } + + override def withLengthGeneratorFunction(lengthGen: LengthGeneratorFunction): JSONBoolGenFunc = + JSONBoolGenFunc(lengthGen, mapping) + + override def withLocationToSeedMapping(mapping: LocationToSeedMapping): JSONBoolGenFunc = + JSONBoolGenFunc(lengthGen, mapping) + + override def withValueRange(min: Any, max: Any): GeneratorFunction = + throw new IllegalArgumentException("value ranges are not supported for JSON") +} + +class JSONBoolGen(conf: ColumnConf, + defaultValueRange: Option[(Any, Any)] = None) + extends SubstringDataGen(conf, defaultValueRange) { + + override protected def getValGen: GeneratorFunction = JSONBoolGenFunc() + + override def children: Seq[(String, SubstringDataGen)] = Seq.empty +} + +case class JSONNullGenFunc(nullAsString: Boolean, + lengthGen: LengthGeneratorFunction = null, + mapping: LocationToSeedMapping = null) extends GeneratorFunction { + + override def apply(rowLoc: RowLocation): Any = + if (nullAsString) { + UTF8String.fromString("null") + } else { + null + } + + + override def withLengthGeneratorFunction(lengthGen: LengthGeneratorFunction): JSONNullGenFunc = + JSONNullGenFunc(nullAsString, lengthGen, mapping) + + override def withLocationToSeedMapping(mapping: LocationToSeedMapping): JSONNullGenFunc = + JSONNullGenFunc(nullAsString, lengthGen, mapping) + + override def withValueRange(min: Any, max: Any): GeneratorFunction = + throw new IllegalArgumentException("value ranges are not supported for JSON") +} + +class JSONNullGen(nullAsString: Boolean, + conf: ColumnConf, + defaultValueRange: Option[(Any, Any)] = None) + extends SubstringDataGen(conf, defaultValueRange) { + + override protected def getValGen: GeneratorFunction = JSONNullGenFunc(nullAsString) + + override def children: Seq[(String, SubstringDataGen)] = Seq.empty +} + +case class JSONErrorGenFunc(lengthGen: LengthGeneratorFunction = null, + mapping: LocationToSeedMapping = null) extends GeneratorFunction { + + override def apply(rowLoc: RowLocation): Any = { + val len = lengthGen(rowLoc) + val r = DataGen.getRandomFor(rowLoc, mapping) + val buffer = new Array[Byte](len) + var at = 0 + while (at < len) { + // Value range is 32 (Space) to 126 (~) + // But it is almost impossible to show up as valid JSON + buffer(at) = (r.nextInt(126 - 31) + 32).toByte + at += 1 + } + UTF8String.fromBytes(buffer, 0, len) + } + + override def withLengthGeneratorFunction(lengthGen: LengthGeneratorFunction): JSONErrorGenFunc = + JSONErrorGenFunc(lengthGen, mapping) + + override def withLocationToSeedMapping(mapping: LocationToSeedMapping): JSONErrorGenFunc = + JSONErrorGenFunc(lengthGen, mapping) + + override def withValueRange(min: Any, max: Any): GeneratorFunction = + throw new IllegalArgumentException("value ranges are not supported for JSON") +} + +class JSONErrorGen(conf: ColumnConf, + defaultValueRange: Option[(Any, Any)] = None) + extends SubstringDataGen(conf, defaultValueRange) { + + override protected def getValGen: GeneratorFunction = JSONErrorGenFunc() + + override def children: Seq[(String, SubstringDataGen)] = Seq.empty +} + +case class JSONArrayGenFunc(child: GeneratorFunction, + lengthGen: LengthGeneratorFunction = null, + mapping: LocationToSeedMapping = null) extends GeneratorFunction { + + override def apply(rowLoc: RowLocation): Any = { + val len = lengthGen(rowLoc) + val data = new Array[String](len) + val childRowLoc = rowLoc.withNewChild() + var i = 0 + while (i < len) { + childRowLoc.setLastChildIndex(i) + val v = child(childRowLoc) + if (v == null) { + // A null in an array must look like "null" + data(i) = "null" + } else { + data(i) = v.toString + } + i += 1 + } + val ret = data.mkString("[", ",", "]") + UTF8String.fromString(ret) + } + + override def withLengthGeneratorFunction(lengthGen: LengthGeneratorFunction): JSONArrayGenFunc = + JSONArrayGenFunc(child, lengthGen, mapping) + + override def withLocationToSeedMapping(mapping: LocationToSeedMapping): JSONArrayGenFunc = + JSONArrayGenFunc(child, lengthGen, mapping) + + override def withValueRange(min: Any, max: Any): GeneratorFunction = + throw new IllegalArgumentException("value ranges are not supported for JSON") +} + +class JSONArrayGen(child: SubstringDataGen, + conf: ColumnConf, + defaultValueRange: Option[(Any, Any)] = None) + extends SubstringDataGen(conf, defaultValueRange) { + + override def setCorrelatedKeyGroup(keyGroup: Long, + minSeed: Long, maxSeed: Long, + seedMapping: LocationToSeedMapping): SubstringDataGen = { + super.setCorrelatedKeyGroup(keyGroup, minSeed, maxSeed, seedMapping) + child.setCorrelatedKeyGroup(keyGroup, minSeed, maxSeed, seedMapping) + this + } + + override protected def getValGen: GeneratorFunction = JSONArrayGenFunc(child.getGen) + + override def get(name: String): Option[SubstringDataGen] = { + if ("data".equalsIgnoreCase(name) || "child".equalsIgnoreCase(name)) { + Some(child) + } else { + None + } + } + + override def children: Seq[(String, SubstringDataGen)] = Seq(("data", child)) +} + +case class JSONObjectGenFunc(childGens: Array[(String, GeneratorFunction)], + lengthGen: LengthGeneratorFunction = null, + mapping: LocationToSeedMapping = null) extends GeneratorFunction { + override def apply(rowLoc: RowLocation): Any = { + // TODO randomize the order of the children??? + // TODO duplicate child values??? + // The row location does not change for a struct/object + val data = childGens.map { + case (k, gen) => + val key = k.replace("\\", "\\\\") + .replace("\"", "\\\"") + .replace("\n", "\\n") + .replace("\r", "\\r") + .replace("\b", "\\b") + .replace("\f", "\\f") + val v = gen.apply(rowLoc) + if (v == null) { + "" + } else { + '"' + key + "\":" + v + } + } + val ret = data.filterNot(_.isEmpty).mkString("{",",","}") + UTF8String.fromString(ret) + } + + override def withLocationToSeedMapping(mapping: LocationToSeedMapping): JSONObjectGenFunc = + JSONObjectGenFunc(childGens, lengthGen, mapping) + + override def withLengthGeneratorFunction(lengthGen: LengthGeneratorFunction): JSONObjectGenFunc = + JSONObjectGenFunc(childGens, lengthGen, mapping) + + override def withValueRange(min: Any, max: Any): GeneratorFunction = + throw new IllegalArgumentException("value ranges are not supported for JSON") +} + +class JSONObjectGen(val children: Seq[(String, SubstringDataGen)], + conf: ColumnConf, + defaultValueRange: Option[(Any, Any)] = None) + extends SubstringDataGen(conf, defaultValueRange) { + + override def setCorrelatedKeyGroup(keyGroup: Long, + minSeed: Long, maxSeed: Long, + seedMapping: LocationToSeedMapping): SubstringDataGen = { + super.setCorrelatedKeyGroup(keyGroup, minSeed, maxSeed, seedMapping) + children.foreach { + case (_, gen) => + gen.setCorrelatedKeyGroup(keyGroup, minSeed, maxSeed, seedMapping) + } + this + } + + override def get(name: String): Option[SubstringDataGen] = + children.collectFirst { + case (childName, dataGen) if childName.equalsIgnoreCase(name) => dataGen + } + + override protected def getValGen: GeneratorFunction = { + val childGens = children.map(c => (c._1, c._2.getGen)).toArray + JSONObjectGenFunc(childGens) + } +} + +case class JSONChoiceGenFunc(choices: List[(Double, GeneratorFunction)], + lengthGen: LengthGeneratorFunction = null, + mapping: LocationToSeedMapping = null) extends GeneratorFunction { + override def apply(rowLoc: RowLocation): Any = { + val r = DataGen.getRandomFor(rowLoc, mapping) + val l = r.nextDouble() + var index = 0 + while (choices(index)._1 < l) { + index += 1 + } + val childRowLoc = rowLoc.withNewChild() + choices(index)._2(childRowLoc) + } + + override def withLengthGeneratorFunction(lengthGen: LengthGeneratorFunction): JSONChoiceGenFunc = + JSONChoiceGenFunc(choices, lengthGen, mapping) + + override def withLocationToSeedMapping(mapping: LocationToSeedMapping): JSONChoiceGenFunc = + JSONChoiceGenFunc(choices, lengthGen, mapping) + + override def withValueRange(min: Any, max: Any): GeneratorFunction = + throw new IllegalArgumentException("value ranges are not supported for JSON") +} + +class JSONChoiceGen(val choices: Seq[(Double, String, SubstringDataGen)], + conf: ColumnConf, + defaultValueRange: Option[(Any, Any)] = None) + extends SubstringDataGen(conf, defaultValueRange) { + + override val children: Seq[(String, SubstringDataGen)] = + choices.map { case (_, name, gen) => (name, gen) } + + override def setCorrelatedKeyGroup(keyGroup: Long, + minSeed: Long, maxSeed: Long, + seedMapping: LocationToSeedMapping): SubstringDataGen = { + super.setCorrelatedKeyGroup(keyGroup, minSeed, maxSeed, seedMapping) + children.foreach { + case (_, gen) => + gen.setCorrelatedKeyGroup(keyGroup, minSeed, maxSeed, seedMapping) + } + this + } + + override def get(name: String): Option[SubstringDataGen] = + children.collectFirst { + case (childName, dataGen) if childName.equalsIgnoreCase(name) => dataGen + } + + override protected def getValGen: GeneratorFunction = { + val childGens = choices.map(c => (c._1, c._3.getGen)).toList + JSONChoiceGenFunc(childGens) + } } case class ASCIIGenFunc( @@ -1672,14 +2451,46 @@ case class ASCIIGenFunc( throw new IllegalArgumentException("value ranges are not supported for strings") } -class StringGen(conf: ColumnConf, defaultValueRange: Option[(Any, Any)]) - extends DataGen(conf, defaultValueRange) { +/** + * This is here to wrap the substring gen function so that its length/settings + * are the ones used when generating a string, and not what was set for the string. + */ +case class SubstringGenFunc( + substringGen: GeneratorFunction, + lengthGen: LengthGeneratorFunction = null, + mapping: LocationToSeedMapping = null) extends GeneratorFunction { + + override def apply(rowLoc: RowLocation): Any = { + substringGen(rowLoc) + } + + // The length and location seed mapping are just ignored for this... + override def withLengthGeneratorFunction(lengthGen: LengthGeneratorFunction): GeneratorFunction = + this + + override def withLocationToSeedMapping(mapping: LocationToSeedMapping): GeneratorFunction = + this + + override def withValueRange(min: Any, max: Any): GeneratorFunction = + throw new IllegalArgumentException("value ranges are not supported for strings") +} + +class StringGen(conf: ColumnConf, + defaultValueRange: Option[(Any, Any)], + var substringDataGen: Option[SubstringDataGen] = None) + extends DataGen(conf, defaultValueRange) { override def dataType: DataType = StringType - override protected def getValGen: GeneratorFunction = ASCIIGenFunc() + override protected def getValGen: GeneratorFunction = + substringDataGen.map(s => SubstringGenFunc(s.getGen)).getOrElse(ASCIIGenFunc()) override def children: Seq[(String, DataGen)] = Seq.empty + + override def setSubstringGen(subgen: Option[SubstringDataGen]): Unit = + substringDataGen = subgen + + override def getSubstringGen: Option[SubstringDataGen] = substringDataGen } case class StructGenFunc(childGens: Array[GeneratorFunction]) extends GeneratorFunction { @@ -1854,7 +2665,6 @@ class MapGen(key: DataGen, override def children: Seq[(String, DataGen)] = Seq(("key", key), ("value", value)) } - object ColumnGen { private def genInternal(rowNumber: Column, dataType: DataType, @@ -1869,8 +2679,8 @@ object ColumnGen { */ class ColumnGen(val dataGen: DataGen) { def setCorrelatedKeyGroup(kg: Long, - minSeed: Long, maxSeed: Long, - seedMapping: LocationToSeedMapping): ColumnGen = { + minSeed: Long, maxSeed: Long, + seedMapping: LocationToSeedMapping): ColumnGen = { dataGen.setCorrelatedKeyGroup(kg, minSeed, maxSeed, seedMapping) this } @@ -1930,6 +2740,11 @@ class ColumnGen(val dataGen: DataGen) { this } + def setGaussianLength(mean: Double, stdDev: Double): ColumnGen = { + dataGen.setGaussianLength(mean, stdDev) + this + } + final def apply(name: String): DataGen = { get(name).getOrElse { throw new IllegalArgumentException(s"$name not a child of $this") @@ -1941,8 +2756,16 @@ class ColumnGen(val dataGen: DataGen) { def gen(rowNumber: Column): Column = { ColumnGen.genInternal(rowNumber, dataGen.dataType, dataGen.nullable, dataGen.getGen) } + + def getSubstring: Option[SubstringDataGen] = dataGen.getSubstringGen + + def substringGen: SubstringDataGen = dataGen.substringGen + + def setSubstringGen(f : ColumnConf => SubstringDataGen): Unit = + dataGen.setSubstringGen(f) } + sealed trait KeyGroupType /** @@ -2192,7 +3015,7 @@ object DBGen { numRows: Long, mapping: OrderedTypeMapping): Seq[(String, ColumnGen)] = { // a bit of a hack with the column num so that we update it before each time... - var conf = ColumnConf(ColumnLocation(tableId, -1), true, numRows) + var conf = ColumnConf(ColumnLocation(tableId, -1, 0), true, numRows) st.toArray.map { sf => if (!mapping.canMap(sf.dataType, mapping)) { throw new IllegalArgumentException(s"$sf is not supported at this time") From 73d76cfade9758f7f47357b59daa1dc5fe2743fa Mon Sep 17 00:00:00 2001 From: Feng Jiang <106386742+Feng-Jiang28@users.noreply.github.com> Date: Thu, 13 Jun 2024 13:23:38 +0800 Subject: [PATCH 40/68] Concat() Exception bug fix (#11039) * concat_null_bug_fix Signed-off-by: fejiang * concat_null_bug_fix Signed-off-by: fejiang * Setting modified Signed-off-by: fejiang * remove comment Signed-off-by: fejiang * concat considered as empty string Signed-off-by: fejiang --------- Signed-off-by: fejiang --- .../org/apache/spark/sql/rapids/collectionOperations.scala | 4 ++-- .../apache/spark/sql/rapids/utils/RapidsTestSettings.scala | 1 - 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/collectionOperations.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/collectionOperations.scala index 7f0a82517c3..41c2e5e3776 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/collectionOperations.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/collectionOperations.scala @@ -49,8 +49,8 @@ case class GpuConcat(children: Seq[Expression]) extends GpuComplexTypeMergingExp override def columnarEval(batch: ColumnarBatch): GpuColumnVector = { val res = dataType match { - // Explicitly return null for empty concat as Spark, since cuDF doesn't support empty concat. - case dt if children.isEmpty => GpuScalar.from(null, dt) + // in Spark concat() will be considered as an empty string here + case dt if children.isEmpty => GpuScalar("", dt) // For single column concat, we pass the result of child node to avoid extra cuDF call. case _ if children.length == 1 => children.head.columnarEval(batch) case StringType => stringConcat(batch) diff --git a/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/utils/RapidsTestSettings.scala b/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/utils/RapidsTestSettings.scala index ad93c4dd2e9..4cf155041d9 100644 --- a/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/utils/RapidsTestSettings.scala +++ b/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/utils/RapidsTestSettings.scala @@ -72,7 +72,6 @@ class RapidsTestSettings extends BackendTestSettings { enableSuite[RapidsMathFunctionsSuite] enableSuite[RapidsRegexpExpressionsSuite] enableSuite[RapidsStringExpressionsSuite] - .exclude("concat", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10775")) .exclude("string substring_index function", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10775")) .exclude("SPARK-22498: Concat should not generate codes beyond 64KB", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10775")) .exclude("SPARK-22549: ConcatWs should not generate codes beyond 64KB", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10775")) From 2bc5ab6ad633b3d4a436d38786009579ef7465da Mon Sep 17 00:00:00 2001 From: Tim Liu Date: Thu, 13 Jun 2024 18:10:26 +0800 Subject: [PATCH 41/68] Add spark343 shim for scala2.13 dist jar (#11052) We missed spark343 shim for the scala2.13 dist jar on branch-24.06. Add scala2.13 spark343 shim for v24.06.0 Signed-off-by: Tim Liu --- pom.xml | 1 + scala2.13/pom.xml | 1 + 2 files changed, 2 insertions(+) diff --git a/pom.xml b/pom.xml index 942905ea427..c16a64e1d65 100644 --- a/pom.xml +++ b/pom.xml @@ -886,6 +886,7 @@ 340, 341, 342, + 343, 350, 351 diff --git a/scala2.13/pom.xml b/scala2.13/pom.xml index 783aa38cf50..db00f77fe16 100644 --- a/scala2.13/pom.xml +++ b/scala2.13/pom.xml @@ -886,6 +886,7 @@ 340, 341, 342, + 343, 350, 351 From 4da4d4a6041077f1fd759de499de8c7f41f7af80 Mon Sep 17 00:00:00 2001 From: Jenkins Automation <70000568+nvauto@users.noreply.github.com> Date: Thu, 13 Jun 2024 18:25:01 +0800 Subject: [PATCH 42/68] Update latest changelog [skip ci] (#11056) Update change log with CLI: \n\n scripts/generate-changelog --token= --releases=24.04,24.06 Signed-off-by: jenkins Co-authored-by: jenkins --- CHANGELOG.md | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index d534d9398b6..788fed0a98f 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,5 +1,5 @@ # Change log -Generated on 2024-06-10 +Generated on 2024-06-13 ## Release 24.06 @@ -48,6 +48,8 @@ Generated on 2024-06-10 ### PRs ||| |:---|:---| +|[#11052](https://github.com/NVIDIA/spark-rapids/pull/11052)|Add spark343 shim for scala2.13 dist jar| +|[#10981](https://github.com/NVIDIA/spark-rapids/pull/10981)|Update latest changelog [skip ci]| |[#10984](https://github.com/NVIDIA/spark-rapids/pull/10984)|[DOC] Update docs for 24.06.0 release [skip ci]| |[#10974](https://github.com/NVIDIA/spark-rapids/pull/10974)|Update rapids JNI and private dependency to 24.06.0| |[#10947](https://github.com/NVIDIA/spark-rapids/pull/10947)|Prevent contains-PrefixRange optimization if not preceded by wildcards| From cfd8f0023806412b4b0526453cf4798b12fa66d9 Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Thu, 13 Jun 2024 09:07:32 -0500 Subject: [PATCH 43/68] Revert "Add in the ability to fingerprint JSON columns (#11002)" This reverts commit d9686d4dba4afd6e2c061fd2822d40d9a55d88d4. --- .../spark/sql/tests/datagen/bigDataGen.scala | 1181 +++-------------- 1 file changed, 179 insertions(+), 1002 deletions(-) diff --git a/datagen/src/main/scala/org/apache/spark/sql/tests/datagen/bigDataGen.scala b/datagen/src/main/scala/org/apache/spark/sql/tests/datagen/bigDataGen.scala index 14e0d4e0970..91335afe4e6 100644 --- a/datagen/src/main/scala/org/apache/spark/sql/tests/datagen/bigDataGen.scala +++ b/datagen/src/main/scala/org/apache/spark/sql/tests/datagen/bigDataGen.scala @@ -16,22 +16,21 @@ package org.apache.spark.sql.tests.datagen -import com.fasterxml.jackson.core.{JsonFactoryBuilder, JsonParser, JsonToken} -import com.fasterxml.jackson.core.json.JsonReadFeature import java.math.{BigDecimal => JavaBigDecimal} import java.sql.{Date, Timestamp} import java.time.{Duration, Instant, LocalDate, LocalDateTime} import java.util + import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import scala.math.BigDecimal.RoundingMode import scala.util.Random -import org.apache.spark.sql.{Column, DataFrame, Row, SparkSession} +import org.apache.spark.sql.{Column, DataFrame, SparkSession} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Expression, XXH64} import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, ArrayData, DateTimeUtils} -import org.apache.spark.sql.functions.{approx_count_distinct, avg, coalesce, col, count, lit, stddev, struct, transform, udf, when} +import org.apache.spark.sql.functions.col import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.random.XORShiftRandom @@ -80,28 +79,22 @@ class RowLocation(val rowNum: Long, val subRows: Array[Int] = null) { * hash. This makes the generated data correlated for all column/child columns. * @param tableNum a unique ID for the table this is a part of. * @param columnNum the location of the column in the data being generated - * @param substringNum the location of the substring column * @param correlatedKeyGroup the correlated key group this column is a part of, if any. */ -case class ColumnLocation(tableNum: Int, - columnNum: Int, - substringNum: Int, - correlatedKeyGroup: Option[Long] = None) { - def forNextColumn(): ColumnLocation = ColumnLocation(tableNum, columnNum + 1, 0) +case class ColumnLocation(tableNum: Int, columnNum: Int, correlatedKeyGroup: Option[Long] = None) { + def forNextColumn(): ColumnLocation = ColumnLocation(tableNum, columnNum + 1) - def forNextSubstring: ColumnLocation = ColumnLocation(tableNum, columnNum, substringNum + 1) /** * Create a new ColumnLocation that is specifically for a given key group */ def forCorrelatedKeyGroup(keyGroup: Long): ColumnLocation = - ColumnLocation(tableNum, columnNum, substringNum, Some(keyGroup)) + ColumnLocation(tableNum, columnNum, Some(keyGroup)) /** * Hash the location into a single long value. */ - lazy val hashLoc: Long = XXH64.hashLong(tableNum, - correlatedKeyGroup.getOrElse(XXH64.hashLong(columnNum, substringNum))) + lazy val hashLoc: Long = XXH64.hashLong(tableNum, correlatedKeyGroup.getOrElse(columnNum)) } /** @@ -122,9 +115,6 @@ case class ColumnConf(columnLoc: ColumnLocation, def forNextColumn(nullable: Boolean): ColumnConf = ColumnConf(columnLoc.forNextColumn(), nullable, numTableRows) - def forNextSubstring: ColumnConf = - ColumnConf(columnLoc.forNextSubstring, nullable = true, numTableRows) - /** * Create a new configuration based on this, but for a given correlated key group. */ @@ -313,23 +303,6 @@ case class VarLengthGeneratorFunction(minLength: Int, maxLength: Int) extends } } -case class StdDevLengthGen(mean: Double, - stdDev: Double, - mapping: LocationToSeedMapping = null) extends - LengthGeneratorFunction { - override def withLocationToSeedMapping(mapping: LocationToSeedMapping): LengthGeneratorFunction = - StdDevLengthGen(mean, stdDev, mapping) - - override def apply(rowLoc: RowLocation): Int = { - val r = DataGen.getRandomFor(rowLoc, mapping) - val g = r.nextGaussian() // g has a mean of 0 and a stddev of 1.0 - val adjusted = mean + (g * stdDev) - // If the range of seed is too small compared to the stddev and mean we will - // end up with an invalid distribution, but they asked for it. - math.max(0, math.round(adjusted).toInt) - } -} - /** * Generate nulls with a given probability. * @param prob 0.0 to 1.0 for how often nulls should appear in the output. @@ -589,8 +562,11 @@ case class DataGenExpr(child: Expression, } } -abstract class CommonDataGen( - var conf: ColumnConf, +/** + * Base class for generating a column/sub-column. This holds configuration for the column, + * and handles what is needed to convert it into GeneratorFunction + */ +abstract class DataGen(var conf: ColumnConf, defaultValueRange: Option[(Any, Any)], var seedMapping: LocationToSeedMapping = FlatDistribution(), var nullMapping: LocationToSeedMapping = FlatDistribution(), @@ -600,25 +576,26 @@ abstract class CommonDataGen( protected var valueRange: Option[(Any, Any)] = defaultValueRange /** - * Set a value range + * Set a value range for this data gen. */ - def setValueRange(min: Any, max: Any): CommonDataGen = { + def setValueRange(min: Any, max: Any): DataGen = { valueRange = Some((min, max)) this } /** - * Set a custom GeneratorFunction + * Set a custom GeneratorFunction to use for this column. */ - def setValueGen(f: GeneratorFunction): CommonDataGen = { + def setValueGen(f: GeneratorFunction): DataGen = { userProvidedValueGen = Some(f) this } /** - * Set a NullGeneratorFunction + * Set a NullGeneratorFunction for this column. This will not be used + * if the column is not nullable. */ - def setNullGen(f: NullGeneratorFunction): CommonDataGen = { + def setNullGen(f: NullGeneratorFunction): DataGen = { this.userProvidedNullGen = Some(f) this } @@ -627,12 +604,12 @@ abstract class CommonDataGen( * Set the probability of a null appearing in the output. The probability should be * 0.0 to 1.0. */ - def setNullProbability(probability: Double): CommonDataGen = { + def setNullProbability(probability: Double): DataGen = { this.userProvidedNullGen = Some(NullProbabilityGenerationFunction(probability)) this } - def setNullProbabilityRecursively(probability: Double): CommonDataGen = { + def setNullProbabilityRecursively(probability: Double): DataGen = { this.userProvidedNullGen = Some(NullProbabilityGenerationFunction(probability)) children.foreach { case (_, dataGen) => @@ -644,7 +621,7 @@ abstract class CommonDataGen( /** * Set a specific location to seed mapping for the value generation. */ - def setSeedMapping(seedMapping: LocationToSeedMapping): CommonDataGen = { + def setSeedMapping(seedMapping: LocationToSeedMapping): DataGen = { this.seedMapping = seedMapping this } @@ -652,7 +629,7 @@ abstract class CommonDataGen( /** * Set a specific location to seed mapping for the null generation. */ - def setNullMapping(nullMapping: LocationToSeedMapping): CommonDataGen = { + def setNullMapping(nullMapping: LocationToSeedMapping): DataGen = { this.nullMapping = nullMapping this } @@ -661,7 +638,7 @@ abstract class CommonDataGen( * Set a specific LengthGeneratorFunction to use. This will only be used if * the datatype needs a length. */ - def setLengthGen(lengthGen: LengthGeneratorFunction): CommonDataGen = { + def setLengthGen(lengthGen: LengthGeneratorFunction): DataGen = { this.lengthGen = lengthGen this } @@ -669,30 +646,25 @@ abstract class CommonDataGen( /** * Set the length generation to be a fixed length. */ - def setLength(len: Int): CommonDataGen = { + def setLength(len: Int): DataGen = { this.lengthGen = FixedLengthGeneratorFunction(len) this } - def setLength(minLen: Int, maxLen: Int): CommonDataGen = { + def setLength(minLen: Int, maxLen: Int) = { this.lengthGen = VarLengthGeneratorFunction(minLen, maxLen) this } - def setGaussianLength(mean: Double, stdDev: Double): CommonDataGen = { - this.lengthGen = StdDevLengthGen(mean, stdDev) - this - } - /** * Add this column to a specific correlated key group. This should not be * called directly by users. */ def setCorrelatedKeyGroup(keyGroup: Long, - minSeed: Long, maxSeed: Long, - seedMapping: LocationToSeedMapping): CommonDataGen = { + minSeed: Long, maxSeed: Long, + seedMapping: LocationToSeedMapping): DataGen = { conf = conf.forCorrelatedKeyGroup(keyGroup) - .forSeedRange(minSeed, maxSeed) + .forSeedRange(minSeed, maxSeed) this.seedMapping = seedMapping this } @@ -700,7 +672,7 @@ abstract class CommonDataGen( /** * Set a range of seed values that should be returned by the LocationToSeedMapping */ - def setSeedRange(min: Long, max: Long): CommonDataGen = { + def setSeedRange(min: Long, max: Long): DataGen = { conf = conf.forSeedRange(min, max) this } @@ -709,7 +681,7 @@ abstract class CommonDataGen( * Get the default value generator for this specific data gen. */ protected def getValGen: GeneratorFunction - def children: Seq[(String, CommonDataGen)] + def children: Seq[(String, DataGen)] /** * Get the final ready to use GeneratorFunction for the data generator. @@ -718,8 +690,8 @@ abstract class CommonDataGen( val sm = seedMapping.withColumnConf(conf) val lg = lengthGen.withLocationToSeedMapping(sm) var valGen = userProvidedValueGen.getOrElse(getValGen) - .withLocationToSeedMapping(sm) - .withLengthGeneratorFunction(lg) + .withLocationToSeedMapping(sm) + .withLengthGeneratorFunction(lg) valueRange.foreach { case (min, max) => valGen = valGen.withValueRange(min, max) @@ -728,75 +700,35 @@ abstract class CommonDataGen( val nullColConf = conf.forNulls val nm = nullMapping.withColumnConf(nullColConf) userProvidedNullGen.get - .withWrapped(valGen) - .withLocationToSeedMapping(nm) + .withWrapped(valGen) + .withLocationToSeedMapping(nm) } else { valGen } } + /** + * Get the data type for this column + */ + def dataType: DataType + /** * Is this column nullable or not. */ def nullable: Boolean = conf.nullable /** - * Get a child for a given name, if it has one. + * Get a child column for a given name, if it has one. */ - final def apply(name: String): CommonDataGen = { + final def apply(name: String): DataGen = { get(name).getOrElse{ throw new IllegalStateException(s"Could not find a child $name for $this") } } - def get(name: String): Option[CommonDataGen] = None -} - - -/** - * Base class for generating a column/sub-column. This holds configuration - * for the column, and handles what is needed to convert it into GeneratorFunction - */ -abstract class DataGen( - conf: ColumnConf, - defaultValueRange: Option[(Any, Any)], - seedMapping: LocationToSeedMapping = FlatDistribution(), - nullMapping: LocationToSeedMapping = FlatDistribution(), - lengthGen: LengthGeneratorFunction = FixedLengthGeneratorFunction(10)) extends - CommonDataGen(conf, defaultValueRange, seedMapping, nullMapping, lengthGen) { - - /** - * Get the data type for this column - */ - def dataType: DataType - - override def get(name: String): Option[DataGen] = None - - def getSubstringGen: Option[SubstringDataGen] = None - - def substringGen: SubstringDataGen = - getSubstringGen.getOrElse( - throw new IllegalArgumentException("substring data gen was not set")) - - def setSubstringGen(f : ColumnConf => SubstringDataGen): Unit = - setSubstringGen(Option(f(conf.forNextSubstring))) - - def setSubstringGen(subgen: Option[SubstringDataGen]): Unit = - throw new IllegalArgumentException("substring data gens can only be set for a STRING") + def get(name: String): Option[DataGen] = None } -/** - * Base class for generating a sub-string. This holds configuration - * for the substring, and handles what is needed to convert it into a GeneratorFunction - */ -abstract class SubstringDataGen( - conf: ColumnConf, - defaultValueRange: Option[(Any, Any)], - seedMapping: LocationToSeedMapping = FlatDistribution(), - nullMapping: LocationToSeedMapping = FlatDistribution(), - lengthGen: LengthGeneratorFunction = FixedLengthGeneratorFunction(10)) extends - CommonDataGen(conf, defaultValueRange, seedMapping, nullMapping, lengthGen) {} - /** * A special GeneratorFunction that just returns the computed seed. This is helpful for * debugging distributions or if you want long values without any abstraction in between. @@ -1562,866 +1494,155 @@ class FloatGen(conf: ColumnConf, defaultValueRange: Option[(Any, Any)]) override def children: Seq[(String, DataGen)] = Seq.empty } -case class JsonPathElement(name: String, is_array: Boolean) -case class JsonLevel(path: Array[JsonPathElement], data_type: String, length: Int, value: String) {} - -object JsonColumnStats { - private def printHelp(): Unit = { - println("JSON Fingerprinting Tool:") - println("PARAMS: ") - println(" is a path to a Spark dataframe to read in") - println(" is a path in a Spark file system to write out fingerprint data to.") - println() - println("OPTIONS:") - println(" --json= where is the name of a top level String column") - println(" --anon= where is a SEED used to anonymize the JSON keys ") - println(" and column names.") - println(" --input_format= where is parquet or ORC. Defaults to parquet.") - println(" --overwrite to enable overwriting the fingerprint output.") - println(" --debug to enable some debug information to be printed out") - println(" --help to print out this help message") - println() - } - - def main(args: Array[String]): Unit = { - var inputPath = Option.empty[String] - var outputPath = Option.empty[String] - val jsonColumns = ArrayBuffer.empty[String] - var anonSeed = Option.empty[Long] - var debug = false - var argsDone = false - var format = "parquet" - var overwrite = false - - args.foreach { - case a if !argsDone && a.startsWith("--json=") => - jsonColumns += a.substring("--json=".length) - case a if !argsDone && a.startsWith("--anon=") => - anonSeed = Some(a.substring("--anon=".length).toLong) - case a if !argsDone && a.startsWith("--input_format=") => - format = a.substring("--input_format=".length).toLowerCase(java.util.Locale.US) - case "--overwrite" if !argsDone => - overwrite = true - case "--debug" if !argsDone => - debug = true - case "--help" if !argsDone => - printHelp() - System.exit(0) - case "--" if !argsDone => - argsDone = true - case a if !argsDone && a.startsWith("--") => // "--" was covered above already - println(s"ERROR $a is not a supported argument") - printHelp() - System.exit(-1) - case a if inputPath.isEmpty => - inputPath = Some(a) - case a if outputPath.isEmpty => - outputPath = Some(a) - case a => - println(s"ERROR only two arguments are supported. Found $a") - printHelp() - System.exit(-1) - } - if (outputPath.isEmpty) { - println("ERROR both an inputPath and an outputPath are required") - printHelp() - System.exit(-1) - } - - val spark = SparkSession.builder.getOrCreate() - spark.sparkContext.setLogLevel("WARN") - - val df = spark.read.format(format).load(inputPath.get) - jsonColumns.foreach { column => - val fp = fingerPrint(df, df(column), anonSeed) - val name = anonSeed.map(s => anonymizeString(column, s)).getOrElse(column) - val fullOutPath = s"${outputPath.get}/$name" - var writer = fp.write - if (overwrite) { - writer = writer.mode("overwrite") - } - if (debug) { - anonSeed.foreach { s => - println(s"Keys and columns will be anonymized with seed $s") - } - println(s"Writing $column fingerprint to $fullOutPath") - spark.time(writer.parquet(fullOutPath)) - println(s"Wrote ${spark.read.parquet(fullOutPath).count} rows") - spark.read.parquet(fullOutPath).show() - } else { - writer.parquet(fullOutPath) - } - } - } - - case class JsonNodeStats(count: Long, meanLen: Double, stdDevLength: Double, dc: Long) - - class JsonNode() { - private val forDataType = - mutable.HashMap[String, (JsonNodeStats, mutable.HashMap[String, JsonNode])]() - - def getChild(name: String, isArray: Boolean): JsonNode = { - val dt = if (isArray) { "ARRAY" } else { "OBJECT" } - val typed = forDataType.getOrElse(dt, - throw new IllegalArgumentException(s"$dt is not a set data type yet.")) - typed._2.getOrElse(name, - throw new IllegalArgumentException(s"$name is not a child when the type is $dt")) - } - - def contains(name: String, isArray: Boolean): Boolean = { - val dt = if (isArray) { "ARRAY" } else { "OBJECT" } - forDataType.get(dt).exists { children => - children._2.contains(name) - } - } - - def addChild(name: String, isArray: Boolean): JsonNode = { - val dt = if (isArray) { "ARRAY" } else { "OBJECT" } - val found = forDataType.getOrElse(dt, - throw new IllegalArgumentException(s"$dt was not already added as a data type")) - if (found._2.contains(name)) { - throw new IllegalArgumentException(s"$dt already has a child named $name") - } - val node = new JsonNode() - found._2.put(name, node) - node - } - - def addChoice(dt: String, stats: JsonNodeStats): Unit = { - if (forDataType.contains(dt)) { - throw new IllegalArgumentException(s"$dt was already added as a data type") - } - forDataType.put(dt, (stats, new mutable.HashMap[String, JsonNode]())) - } - - override def toString: String = { - forDataType.toString() - } - - def totalCount: Long = { - forDataType.values.map{ case (stats, _) => stats.count}.sum - } - - private def makeNoChoiceGenRecursive(dt: String, - children: mutable.HashMap[String, JsonNode], - cc: ColumnConf): (SubstringDataGen, ColumnConf) = { - var c = cc - val ret = dt match { - case "LONG" => new JSONLongGen(c) - case "DOUBLE" => new JSONDoubleGen(c) - case "BOOLEAN" => new JSONBoolGen(c) - case "NULL" => new JSONNullGen(false, c) - case "VALUE_NULL" => new JSONNullGen(true, c) - case "ERROR" => new JSONErrorGen(c) - case "STRING" => new JSONStringGen(c) - case "ARRAY" => - val child = if (children.isEmpty) { - // A corner case, we will just make it a BOOL column and it will be ignored - val tmp = new JSONBoolGen(c) - c = c.forNextSubstring - tmp - } else { - val tmp = children.values.head.makeGenRecursive(c) - c = tmp._2 - tmp._1 - } - new JSONArrayGen(child, c) - case "OBJECT" => - val childGens = if (children.isEmpty) { - Seq.empty - } else { - children.toSeq.map { - case (k, node) => - val tmp = node.makeGenRecursive(c) - c = tmp._2 - (k, tmp._1) - } - } - new JSONObjectGen(childGens, c) - case other => - throw new IllegalArgumentException(s"$other is not a leaf node type") - } - (ret, c.forNextSubstring) - } - - private def makeGenRecursive(cc: ColumnConf): (SubstringDataGen, ColumnConf) = { - var c = cc - // We are going to recursively walk the tree for all of the values. - if (forDataType.size == 1) { - // We don't need a choice at all. This makes it simpler.. - val (dt, (_, children)) = forDataType.head - makeNoChoiceGenRecursive(dt, children, c) - } else { - val totalSum = forDataType.map(f => f._2._1.count).sum.toDouble - var runningSum = 0L - val allChoices = ArrayBuffer[(Double, String, SubstringDataGen)]() - forDataType.foreach { - case (dt, (stats, children)) => - val tmp = makeNoChoiceGenRecursive(dt, children, c) - c = tmp._2 - runningSum += stats.count - allChoices.append((runningSum/totalSum, dt, tmp._1)) - } - - val ret = new JSONChoiceGen(allChoices.toSeq, c) - (ret, c.forNextSubstring) - } - } - - def makeGen(cc: ColumnConf): SubstringDataGen = { - val (ret, _) = makeGenRecursive(cc) - ret - } - - def setStatsSingle(dg: CommonDataGen, - dt: String, - stats: JsonNodeStats, - nullPct: Double): Unit = { - - val includeLength = dt != "OBJECT" && dt != "BOOLEAN" && dt != "NULL" && dt != "VALUE_NULL" - val includeNullPct = nullPct > 0.0 - if (includeLength) { - dg.setGaussianLength(stats.meanLen, stats.stdDevLength) - } - if (includeNullPct) { - dg.setNullProbability(nullPct) - } - dg.setSeedRange(1, stats.dc) - } - - def setStats(dg: CommonDataGen, - parentCount: Option[Long]): Unit = { - // We are going to recursively walk the tree... - if (forDataType.size == 1) { - // We don't need a choice at all. This makes it simpler.. - val (dt, (stats, children)) = forDataType.head - val nullPct = parentCount.map { pc => - (pc - stats.count).toDouble/pc - }.getOrElse(0.0) - setStatsSingle(dg, dt, stats, nullPct) - val myCount = if (dt == "OBJECT") { - Some(totalCount) - } else { - None - } - children.foreach { - case (name, node) => - node.setStats(dg(name), myCount) - } - } else { - // We have choices to make between different types. - // The null percent cannot be calculated for each individual choice - // but is calculated on the group as a whole instead - parentCount.foreach { pc => - val tc = totalCount - val choiceNullPct = (pc - tc).toDouble / pc - if (choiceNullPct > 0.0) { - dg.setNullProbability(choiceNullPct) - } - } - forDataType.foreach { - case (dt, (stats, children)) => - // When there is a choice the name to access it is the data type - val choiceDg = dg(dt) - setStatsSingle(choiceDg, dt, stats, 0.0) - children.foreach { - case (name, node) => - val myCount = if (dt == "OBJECT") { - // Here we only want the count for the OBJECTs - Some(stats.count) - } else { - None - } - node.setStats(choiceDg(name), myCount) - } - } - } - } - } - - private lazy val jsonFactory = new JsonFactoryBuilder() - // The two options below enabled for Hive compatibility - .enable(JsonReadFeature.ALLOW_UNESCAPED_CONTROL_CHARS) - .enable(JsonReadFeature.ALLOW_SINGLE_QUOTES) - .build() - - private def processNext(parser: JsonParser, - currentPath: ArrayBuffer[JsonPathElement], - output: ArrayBuffer[JsonLevel]): Unit = { - parser.currentToken() match { - case JsonToken.START_OBJECT => - parser.nextToken() - while (parser.currentToken() != JsonToken.END_OBJECT) { - processNext(parser, currentPath, output) - } - output.append(JsonLevel(currentPath.toArray, "OBJECT", 0, "")) - parser.nextToken() - case JsonToken.START_ARRAY => - currentPath.append(JsonPathElement("data", is_array = true)) - parser.nextToken() - var length = 0 - while (parser.currentToken() != JsonToken.END_ARRAY) { - length += 1 - processNext(parser, currentPath, output) - } - currentPath.remove(currentPath.length - 1) - output.append(JsonLevel(currentPath.toArray, "ARRAY", length, "")) - parser.nextToken() - case JsonToken.FIELD_NAME => - currentPath.append(JsonPathElement(parser.getCurrentName, is_array = false)) - parser.nextToken() - processNext(parser, currentPath, output) - currentPath.remove(currentPath.length - 1) - case JsonToken.VALUE_NUMBER_INT => - val length = parser.getValueAsString.getBytes("UTF-8").length - output.append(JsonLevel(currentPath.toArray, "LONG", length, parser.getValueAsString)) - parser.nextToken() - case JsonToken.VALUE_NUMBER_FLOAT => - val length = parser.getValueAsString.getBytes("UTF-8").length - output.append(JsonLevel(currentPath.toArray, "DOUBLE", length, parser.getValueAsString)) - parser.nextToken() - case JsonToken.VALUE_TRUE | JsonToken.VALUE_FALSE => - val length = parser.getValueAsString.getBytes("UTF-8").length - output.append(JsonLevel(currentPath.toArray, "BOOLEAN", length, parser.getValueAsString)) - parser.nextToken() - case JsonToken.VALUE_NULL | null => - output.append(JsonLevel(currentPath.toArray, "VALUE_NULL", 4, "NULL")) - parser.nextToken() - case JsonToken.VALUE_STRING => - val length = parser.getValueAsString.getBytes("UTF-8").length - output.append(JsonLevel(currentPath.toArray, "STRING", length, parser.getValueAsString)) - parser.nextToken() - case other => - throw new IllegalStateException(s"DON'T KNOW HOW TO DEAL WITH $other") - } - } - - def jsonStatsUdf(json: String): Array[JsonLevel] = { - val output = new ArrayBuffer[JsonLevel]() - try { - val currentPath = new ArrayBuffer[JsonPathElement]() - if (json == null) { - output.append(JsonLevel(Array.empty, "NULL", 0, "")) - } else { - val parser = jsonFactory.createParser(json) - try { - parser.nextToken() - processNext(parser, currentPath, output) - } finally { - parser.close() - } - } - } catch { - case _: com.fasterxml.jackson.core.JsonParseException => - output.clear() - output.append(JsonLevel(Array.empty, "ERROR", json.getBytes("UTF-8").length, json)) - } - output.toArray - } - - private lazy val extractPaths = udf(json => jsonStatsUdf(json)) - - def anonymizeString(str: String, seed: Long): String = { - val length = str.length - val data = new Array[Byte](length) - val hash = XXH64.hashLong(str.hashCode, seed) - val r = new Random() - r.setSeed(hash) - (0 until length).foreach { i => - val tmp = r.nextInt(16) - data(i) = (tmp + 'A').toByte - } - new String(data) - } - - private lazy val anonPath = udf((str, seed) => anonymizeString(str, seed)) - - def anonymizeFingerPrint(df: DataFrame, anonSeed: Long): DataFrame = { - df.withColumn("tmp", transform(col("path"), - o => { - val name = o("name") - val isArray = o("is_array") - val anon = anonPath(name, lit(anonSeed)) - val newName = when(isArray, name).otherwise(anon).alias("name") - struct(newName, isArray) - })) - .drop("path").withColumnRenamed("tmp", "path") - .orderBy("path", "dt") - .selectExpr("path", "dt","c","mean_len","stddev_len","distinct","version") - } - - def fingerPrint(df: DataFrame, column: Column, anonymize: Option[Long] = None): DataFrame = { - val ret = df.select(extractPaths(column).alias("paths")) - .selectExpr("explode_outer(paths) as p") - .selectExpr("p.path as path", "p.data_type as dt", "p.length as len", "p.value as value") - .groupBy(col("path"), col("dt")).agg( - count(lit(1)).alias("c"), - avg(col("len")).alias("mean_len"), - coalesce(stddev(col("len")), lit(0.0)).alias("stddev_len"), - approx_count_distinct(col("value")).alias("distinct")) - .orderBy("path", "dt").withColumn("version", lit("0.1")) - .selectExpr("path", "dt","c","mean_len","stddev_len","distinct","version") - - anonymize.map { anonSeed => - anonymizeFingerPrint(ret, anonSeed) - }.getOrElse(ret) - } - - def apply(aggForColumn: DataFrame, genColumn: ColumnGen): Unit = - apply(aggForColumn, genColumn.dataGen) - - private val expectedSchema = StructType.fromDDL( - "path ARRAY>," + - "dt STRING," + - "c BIGINT," + - "mean_len DOUBLE," + - "stddev_len DOUBLE," + - "distinct BIGINT," + - "version STRING") - - def apply(aggForColumn: DataFrame, gen: DataGen): Unit = { - val aggData = aggForColumn.orderBy("path", "dt").collect() - val rootNode: JsonNode = new JsonNode() - assert(aggData.length > 0) - val schema = aggData.head.schema - assert(schema.length == expectedSchema.length) - schema.fields.zip(expectedSchema.fields).foreach { - case(found, expected) => - assert(found.name == expected.name) - // TODO we can worry about the exact types later if we need to - } - assert(aggData.head.getString(6) == "0.1") - aggData.foreach { row => - val fullPath = row.getAs[mutable.WrappedArray[Row]](0) - val parsedPath = fullPath.map(r => (r.getString(0), r.getBoolean(1))).toList - val dt = row.getString(1) - val count = row.getLong(2) - val meanLen = row.getDouble(3) - val stdLen = row.getDouble(4) - val dc = row.getLong(5) - - val stats = JsonNodeStats(count, meanLen, stdLen, dc) - var currentNode = rootNode - // Find everything up to the last path element - if (parsedPath.length > 1) { - parsedPath.slice(0, parsedPath.length - 1).foreach { - case (name, isArray) => - currentNode = currentNode.getChild(name, isArray) - } - } - - if (parsedPath.nonEmpty) { - // For the last path element (that is not the root element) we might need to add it - // as a child - val (name, isArray) = parsedPath.last - if (!currentNode.contains(name, isArray)) { - currentNode.addChild(name, isArray) - } - currentNode = currentNode.getChild(name, isArray) - } - currentNode.addChoice(dt, stats) - } - - gen.setSubstringGen(cc => rootNode.makeGen(cc)) - rootNode.setStats(gen.substringGen, None) - } -} - - -case class JSONStringGenFunc(lengthGen: LengthGeneratorFunction = null, - mapping: LocationToSeedMapping = null) extends GeneratorFunction { - - override def apply(rowLoc: RowLocation): Any = { - val len = lengthGen(rowLoc) - val r = DataGen.getRandomFor(rowLoc, mapping) - val buffer = new Array[Byte](len) - var at = 0 - while (at < len) { - // Value range is 32 (Space) to 126 (~) - buffer(at) = (r.nextInt(126 - 31) + 32).toByte - at += 1 - } - val strVal = new String(buffer, 0, len) - .replace("\\", "\\\\") - .replace("\"", "\\\"") - .replace("\n", "\\n") - .replace("\r", "\\r") - .replace("\b", "\\b") - .replace("\f", "\\f") - '"' + strVal + '"' - } - - override def withLengthGeneratorFunction(lengthGen: LengthGeneratorFunction): JSONStringGenFunc = - JSONStringGenFunc(lengthGen, mapping) - - override def withLocationToSeedMapping(mapping: LocationToSeedMapping): JSONStringGenFunc = - JSONStringGenFunc(lengthGen, mapping) - - override def withValueRange(min: Any, max: Any): GeneratorFunction = - throw new IllegalArgumentException("value ranges are not supported for JSON") -} - -class JSONStringGen(conf: ColumnConf, - defaultValueRange: Option[(Any, Any)] = None) - extends SubstringDataGen(conf, defaultValueRange) { - - override protected def getValGen: GeneratorFunction = JSONStringGenFunc() - - override def children: Seq[(String, SubstringDataGen)] = Seq.empty -} - -case class JSONLongGenFunc(lengthGen: LengthGeneratorFunction = null, - mapping: LocationToSeedMapping = null) extends GeneratorFunction { - - override def apply(rowLoc: RowLocation): Any = { - val len = math.max(lengthGen(rowLoc), 1) // We need at least 1 long for a valid value - val r = DataGen.getRandomFor(rowLoc, mapping) - val buffer = new Array[Byte](len) - var at = 0 - while (at < len) { - if (at == 0) { - // No leading 0's - buffer(at) = (r.nextInt(9) + '1').toByte - } else { - buffer(at) = (r.nextInt(10) + '0').toByte - } - at += 1 - } - new String(buffer, 0, len) - } - - override def withLengthGeneratorFunction(lengthGen: LengthGeneratorFunction): JSONLongGenFunc = - JSONLongGenFunc(lengthGen, mapping) - - override def withLocationToSeedMapping(mapping: LocationToSeedMapping): JSONLongGenFunc = - JSONLongGenFunc(lengthGen, mapping) - - override def withValueRange(min: Any, max: Any): GeneratorFunction = - throw new IllegalArgumentException("value ranges are not supported for JSON") -} - -class JSONLongGen(conf: ColumnConf, - defaultValueRange: Option[(Any, Any)] = None) - extends SubstringDataGen(conf, defaultValueRange) { - - override protected def getValGen: GeneratorFunction = JSONLongGenFunc() - - override def children: Seq[(String, SubstringDataGen)] = Seq.empty -} - -case class JSONDoubleGenFunc(lengthGen: LengthGeneratorFunction = null, - mapping: LocationToSeedMapping = null) extends GeneratorFunction { - - override def apply(rowLoc: RowLocation): Any = { - val len = math.max(lengthGen(rowLoc), 3) // We have to have at least 3 chars NUM.NUM - val r = DataGen.getRandomFor(rowLoc, mapping) - val beforeLen = if (len == 3) { 1 } else { r.nextInt(len - 3) + 1 } - val buffer = new Array[Byte](len) - var at = 0 - while (at < len) { - if (at == 0) { - // No leading 0's - buffer(at) = (r.nextInt(9) + '1').toByte - } else if (at == beforeLen) { - buffer(at) = '.' - } else { - buffer(at) = (r.nextInt(10) + '0').toByte - } - at += 1 - } - UTF8String.fromBytes(buffer, 0, len) - } - - override def withLengthGeneratorFunction(lengthGen: LengthGeneratorFunction): JSONDoubleGenFunc = - JSONDoubleGenFunc(lengthGen, mapping) - - override def withLocationToSeedMapping(mapping: LocationToSeedMapping): JSONDoubleGenFunc = - JSONDoubleGenFunc(lengthGen, mapping) - - override def withValueRange(min: Any, max: Any): GeneratorFunction = - throw new IllegalArgumentException("value ranges are not supported for JSON") -} - -class JSONDoubleGen(conf: ColumnConf, - defaultValueRange: Option[(Any, Any)] = None) - extends SubstringDataGen(conf, defaultValueRange) { - - override protected def getValGen: GeneratorFunction = JSONDoubleGenFunc() - - override def children: Seq[(String, SubstringDataGen)] = Seq.empty +trait JSONType { + def appendRandomValue(sb: StringBuilder, + index: Int, + maxStringLength: Int, + maxArrayLength: Int, + maxObjectLength: Int, + depth: Int, + maxDepth: Int, + r: Random): Unit } -case class JSONBoolGenFunc(lengthGen: LengthGeneratorFunction = null, - mapping: LocationToSeedMapping = null) extends GeneratorFunction { - - override def apply(rowLoc: RowLocation): Any = { - val r = DataGen.getRandomFor(rowLoc, mapping) - val ret = if (r.nextBoolean()) "true" else "false" - UTF8String.fromString(ret) - } - - override def withLengthGeneratorFunction(lengthGen: LengthGeneratorFunction): JSONBoolGenFunc = - JSONBoolGenFunc(lengthGen, mapping) - - override def withLocationToSeedMapping(mapping: LocationToSeedMapping): JSONBoolGenFunc = - JSONBoolGenFunc(lengthGen, mapping) - - override def withValueRange(min: Any, max: Any): GeneratorFunction = - throw new IllegalArgumentException("value ranges are not supported for JSON") -} - -class JSONBoolGen(conf: ColumnConf, - defaultValueRange: Option[(Any, Any)] = None) - extends SubstringDataGen(conf, defaultValueRange) { - - override protected def getValGen: GeneratorFunction = JSONBoolGenFunc() - - override def children: Seq[(String, SubstringDataGen)] = Seq.empty -} - -case class JSONNullGenFunc(nullAsString: Boolean, - lengthGen: LengthGeneratorFunction = null, - mapping: LocationToSeedMapping = null) extends GeneratorFunction { - - override def apply(rowLoc: RowLocation): Any = - if (nullAsString) { - UTF8String.fromString("null") +object JSONType { + def selectType(depth: Int, + maxDepth: Int, + r: Random): JSONType = { + val toSelectFrom = if (depth < maxDepth) { + Seq(QuotedJSONString, JSONLong, JSONDouble, JSONArray, JSONObject) } else { - null - } - - - override def withLengthGeneratorFunction(lengthGen: LengthGeneratorFunction): JSONNullGenFunc = - JSONNullGenFunc(nullAsString, lengthGen, mapping) - - override def withLocationToSeedMapping(mapping: LocationToSeedMapping): JSONNullGenFunc = - JSONNullGenFunc(nullAsString, lengthGen, mapping) - - override def withValueRange(min: Any, max: Any): GeneratorFunction = - throw new IllegalArgumentException("value ranges are not supported for JSON") -} - -class JSONNullGen(nullAsString: Boolean, - conf: ColumnConf, - defaultValueRange: Option[(Any, Any)] = None) - extends SubstringDataGen(conf, defaultValueRange) { - - override protected def getValGen: GeneratorFunction = JSONNullGenFunc(nullAsString) - - override def children: Seq[(String, SubstringDataGen)] = Seq.empty -} - -case class JSONErrorGenFunc(lengthGen: LengthGeneratorFunction = null, - mapping: LocationToSeedMapping = null) extends GeneratorFunction { - - override def apply(rowLoc: RowLocation): Any = { - val len = lengthGen(rowLoc) - val r = DataGen.getRandomFor(rowLoc, mapping) - val buffer = new Array[Byte](len) - var at = 0 - while (at < len) { - // Value range is 32 (Space) to 126 (~) - // But it is almost impossible to show up as valid JSON - buffer(at) = (r.nextInt(126 - 31) + 32).toByte - at += 1 - } - UTF8String.fromBytes(buffer, 0, len) - } - - override def withLengthGeneratorFunction(lengthGen: LengthGeneratorFunction): JSONErrorGenFunc = - JSONErrorGenFunc(lengthGen, mapping) - - override def withLocationToSeedMapping(mapping: LocationToSeedMapping): JSONErrorGenFunc = - JSONErrorGenFunc(lengthGen, mapping) - - override def withValueRange(min: Any, max: Any): GeneratorFunction = - throw new IllegalArgumentException("value ranges are not supported for JSON") -} - -class JSONErrorGen(conf: ColumnConf, - defaultValueRange: Option[(Any, Any)] = None) - extends SubstringDataGen(conf, defaultValueRange) { - - override protected def getValGen: GeneratorFunction = JSONErrorGenFunc() - - override def children: Seq[(String, SubstringDataGen)] = Seq.empty -} - -case class JSONArrayGenFunc(child: GeneratorFunction, - lengthGen: LengthGeneratorFunction = null, - mapping: LocationToSeedMapping = null) extends GeneratorFunction { - - override def apply(rowLoc: RowLocation): Any = { - val len = lengthGen(rowLoc) - val data = new Array[String](len) - val childRowLoc = rowLoc.withNewChild() - var i = 0 - while (i < len) { - childRowLoc.setLastChildIndex(i) - val v = child(childRowLoc) - if (v == null) { - // A null in an array must look like "null" - data(i) = "null" - } else { - data(i) = v.toString + Seq(QuotedJSONString, JSONLong, JSONDouble) + } + val index = r.nextInt(toSelectFrom.length) + toSelectFrom(index) + } +} + +object QuotedJSONString extends JSONType { + override def appendRandomValue(sb: StringBuilder, + index: Int, + maxStringLength: Int, + maxArrayLength: Int, + maxObjectLength: Int, + depth: Int, + maxDepth: Int, + r: Random): Unit = { + val strValue = r.nextString(r.nextInt(maxStringLength + 1)) + .replace("\\", "\\\\") + .replace("\"", "\\\"") + .replace("\n", "\\n") + .replace("\r", "\\r") + .replace("\b", "\\b") + .replace("\f", "\\f") + sb.append('"') + sb.append(strValue) + sb.append('"') + } +} + +object JSONLong extends JSONType { + override def appendRandomValue(sb: StringBuilder, + index: Int, + maxStringLength: Int, + maxArrayLength: Int, + maxObjectLength: Int, + depth: Int, + maxDepth: Int, + r: Random): Unit = { + sb.append(r.nextLong()) + } +} + +object JSONDouble extends JSONType { + override def appendRandomValue(sb: StringBuilder, + index: Int, + maxStringLength: Int, + maxArrayLength: Int, + maxObjectLength: Int, + depth: Int, + maxDepth: Int, + r: Random): Unit = { + sb.append(r.nextDouble() * 4096.0) + } +} + +object JSONArray extends JSONType { + override def appendRandomValue(sb: StringBuilder, + index: Int, + maxStringLength: Int, + maxArrayLength: Int, + maxObjectLength: Int, + depth: Int, + maxDepth: Int, + r: Random): Unit = { + val childType = JSONType.selectType(depth, maxDepth, r) + val length = r.nextInt(maxArrayLength + 1) + sb.append("[") + (0 until length).foreach { i => + if (i > 0) { + sb.append(",") } - i += 1 + childType.appendRandomValue(sb, i, maxStringLength, maxArrayLength, maxObjectLength, + depth + 1, maxDepth, r) } - val ret = data.mkString("[", ",", "]") - UTF8String.fromString(ret) + sb.append("]") } - - override def withLengthGeneratorFunction(lengthGen: LengthGeneratorFunction): JSONArrayGenFunc = - JSONArrayGenFunc(child, lengthGen, mapping) - - override def withLocationToSeedMapping(mapping: LocationToSeedMapping): JSONArrayGenFunc = - JSONArrayGenFunc(child, lengthGen, mapping) - - override def withValueRange(min: Any, max: Any): GeneratorFunction = - throw new IllegalArgumentException("value ranges are not supported for JSON") } -class JSONArrayGen(child: SubstringDataGen, - conf: ColumnConf, - defaultValueRange: Option[(Any, Any)] = None) - extends SubstringDataGen(conf, defaultValueRange) { - - override def setCorrelatedKeyGroup(keyGroup: Long, - minSeed: Long, maxSeed: Long, - seedMapping: LocationToSeedMapping): SubstringDataGen = { - super.setCorrelatedKeyGroup(keyGroup, minSeed, maxSeed, seedMapping) - child.setCorrelatedKeyGroup(keyGroup, minSeed, maxSeed, seedMapping) - this - } - - override protected def getValGen: GeneratorFunction = JSONArrayGenFunc(child.getGen) - - override def get(name: String): Option[SubstringDataGen] = { - if ("data".equalsIgnoreCase(name) || "child".equalsIgnoreCase(name)) { - Some(child) - } else { - None - } - } - - override def children: Seq[(String, SubstringDataGen)] = Seq(("data", child)) -} - -case class JSONObjectGenFunc(childGens: Array[(String, GeneratorFunction)], - lengthGen: LengthGeneratorFunction = null, - mapping: LocationToSeedMapping = null) extends GeneratorFunction { - override def apply(rowLoc: RowLocation): Any = { - // TODO randomize the order of the children??? - // TODO duplicate child values??? - // The row location does not change for a struct/object - val data = childGens.map { - case (k, gen) => - val key = k.replace("\\", "\\\\") - .replace("\"", "\\\"") - .replace("\n", "\\n") - .replace("\r", "\\r") - .replace("\b", "\\b") - .replace("\f", "\\f") - val v = gen.apply(rowLoc) - if (v == null) { - "" - } else { - '"' + key + "\":" + v - } +object JSONObject extends JSONType { + override def appendRandomValue(sb: StringBuilder, + index: Int, + maxStringLength: Int, + maxArrayLength: Int, + maxObjectLength: Int, + depth: Int, + maxDepth: Int, + r: Random): Unit = { + val length = r.nextInt(maxObjectLength) + 1 + sb.append("{") + (0 until length).foreach { i => + if (i > 0) { + sb.append(",") + } + sb.append("\"key_") + sb.append(i) + sb.append("_") + sb.append(depth ) + sb.append("\":") + val childType = JSONType.selectType(depth, maxDepth, r) + childType.appendRandomValue(sb, i, maxStringLength, maxArrayLength, maxObjectLength, + depth + 1, maxDepth, r) } - val ret = data.filterNot(_.isEmpty).mkString("{",",","}") - UTF8String.fromString(ret) + sb.append("}") } - - override def withLocationToSeedMapping(mapping: LocationToSeedMapping): JSONObjectGenFunc = - JSONObjectGenFunc(childGens, lengthGen, mapping) - - override def withLengthGeneratorFunction(lengthGen: LengthGeneratorFunction): JSONObjectGenFunc = - JSONObjectGenFunc(childGens, lengthGen, mapping) - - override def withValueRange(min: Any, max: Any): GeneratorFunction = - throw new IllegalArgumentException("value ranges are not supported for JSON") } -class JSONObjectGen(val children: Seq[(String, SubstringDataGen)], - conf: ColumnConf, - defaultValueRange: Option[(Any, Any)] = None) - extends SubstringDataGen(conf, defaultValueRange) { - - override def setCorrelatedKeyGroup(keyGroup: Long, - minSeed: Long, maxSeed: Long, - seedMapping: LocationToSeedMapping): SubstringDataGen = { - super.setCorrelatedKeyGroup(keyGroup, minSeed, maxSeed, seedMapping) - children.foreach { - case (_, gen) => - gen.setCorrelatedKeyGroup(keyGroup, minSeed, maxSeed, seedMapping) - } - this - } - - override def get(name: String): Option[SubstringDataGen] = - children.collectFirst { - case (childName, dataGen) if childName.equalsIgnoreCase(name) => dataGen - } - - override protected def getValGen: GeneratorFunction = { - val childGens = children.map(c => (c._1, c._2.getGen)).toArray - JSONObjectGenFunc(childGens) - } -} +case class JSONGenFunc( + maxStringLength: Int, + maxArrayLength: Int, + maxObjectLength: Int, + maxDepth: Int, + lengthGen: LengthGeneratorFunction = null, + mapping: LocationToSeedMapping = null) extends GeneratorFunction { -case class JSONChoiceGenFunc(choices: List[(Double, GeneratorFunction)], - lengthGen: LengthGeneratorFunction = null, - mapping: LocationToSeedMapping = null) extends GeneratorFunction { override def apply(rowLoc: RowLocation): Any = { val r = DataGen.getRandomFor(rowLoc, mapping) - val l = r.nextDouble() - var index = 0 - while (choices(index)._1 < l) { - index += 1 - } - val childRowLoc = rowLoc.withNewChild() - choices(index)._2(childRowLoc) + val sb = new StringBuilder() + JSONObject.appendRandomValue(sb, 0, maxStringLength, maxArrayLength, maxObjectLength, + 0, maxDepth, r) + // For now I am going to have some hard coded keys + UTF8String.fromString(sb.toString()) } - override def withLengthGeneratorFunction(lengthGen: LengthGeneratorFunction): JSONChoiceGenFunc = - JSONChoiceGenFunc(choices, lengthGen, mapping) + override def withLengthGeneratorFunction(lengthGen: LengthGeneratorFunction): GeneratorFunction = + JSONGenFunc(maxStringLength, maxArrayLength, maxObjectLength, maxDepth, lengthGen, mapping) - override def withLocationToSeedMapping(mapping: LocationToSeedMapping): JSONChoiceGenFunc = - JSONChoiceGenFunc(choices, lengthGen, mapping) + override def withLocationToSeedMapping(mapping: LocationToSeedMapping): GeneratorFunction = + JSONGenFunc(maxStringLength, maxArrayLength, maxObjectLength, maxDepth, lengthGen, mapping) override def withValueRange(min: Any, max: Any): GeneratorFunction = - throw new IllegalArgumentException("value ranges are not supported for JSON") -} - -class JSONChoiceGen(val choices: Seq[(Double, String, SubstringDataGen)], - conf: ColumnConf, - defaultValueRange: Option[(Any, Any)] = None) - extends SubstringDataGen(conf, defaultValueRange) { - - override val children: Seq[(String, SubstringDataGen)] = - choices.map { case (_, name, gen) => (name, gen) } - - override def setCorrelatedKeyGroup(keyGroup: Long, - minSeed: Long, maxSeed: Long, - seedMapping: LocationToSeedMapping): SubstringDataGen = { - super.setCorrelatedKeyGroup(keyGroup, minSeed, maxSeed, seedMapping) - children.foreach { - case (_, gen) => - gen.setCorrelatedKeyGroup(keyGroup, minSeed, maxSeed, seedMapping) - } - this - } - - override def get(name: String): Option[SubstringDataGen] = - children.collectFirst { - case (childName, dataGen) if childName.equalsIgnoreCase(name) => dataGen - } - - override protected def getValGen: GeneratorFunction = { - val childGens = choices.map(c => (c._1, c._3.getGen)).toList - JSONChoiceGenFunc(childGens) - } + throw new IllegalArgumentException("value ranges are not supported for strings") } case class ASCIIGenFunc( @@ -2451,46 +1672,14 @@ case class ASCIIGenFunc( throw new IllegalArgumentException("value ranges are not supported for strings") } -/** - * This is here to wrap the substring gen function so that its length/settings - * are the ones used when generating a string, and not what was set for the string. - */ -case class SubstringGenFunc( - substringGen: GeneratorFunction, - lengthGen: LengthGeneratorFunction = null, - mapping: LocationToSeedMapping = null) extends GeneratorFunction { - - override def apply(rowLoc: RowLocation): Any = { - substringGen(rowLoc) - } - - // The length and location seed mapping are just ignored for this... - override def withLengthGeneratorFunction(lengthGen: LengthGeneratorFunction): GeneratorFunction = - this - - override def withLocationToSeedMapping(mapping: LocationToSeedMapping): GeneratorFunction = - this - - override def withValueRange(min: Any, max: Any): GeneratorFunction = - throw new IllegalArgumentException("value ranges are not supported for strings") -} - -class StringGen(conf: ColumnConf, - defaultValueRange: Option[(Any, Any)], - var substringDataGen: Option[SubstringDataGen] = None) - extends DataGen(conf, defaultValueRange) { +class StringGen(conf: ColumnConf, defaultValueRange: Option[(Any, Any)]) + extends DataGen(conf, defaultValueRange) { override def dataType: DataType = StringType - override protected def getValGen: GeneratorFunction = - substringDataGen.map(s => SubstringGenFunc(s.getGen)).getOrElse(ASCIIGenFunc()) + override protected def getValGen: GeneratorFunction = ASCIIGenFunc() override def children: Seq[(String, DataGen)] = Seq.empty - - override def setSubstringGen(subgen: Option[SubstringDataGen]): Unit = - substringDataGen = subgen - - override def getSubstringGen: Option[SubstringDataGen] = substringDataGen } case class StructGenFunc(childGens: Array[GeneratorFunction]) extends GeneratorFunction { @@ -2665,6 +1854,7 @@ class MapGen(key: DataGen, override def children: Seq[(String, DataGen)] = Seq(("key", key), ("value", value)) } + object ColumnGen { private def genInternal(rowNumber: Column, dataType: DataType, @@ -2679,8 +1869,8 @@ object ColumnGen { */ class ColumnGen(val dataGen: DataGen) { def setCorrelatedKeyGroup(kg: Long, - minSeed: Long, maxSeed: Long, - seedMapping: LocationToSeedMapping): ColumnGen = { + minSeed: Long, maxSeed: Long, + seedMapping: LocationToSeedMapping): ColumnGen = { dataGen.setCorrelatedKeyGroup(kg, minSeed, maxSeed, seedMapping) this } @@ -2740,11 +1930,6 @@ class ColumnGen(val dataGen: DataGen) { this } - def setGaussianLength(mean: Double, stdDev: Double): ColumnGen = { - dataGen.setGaussianLength(mean, stdDev) - this - } - final def apply(name: String): DataGen = { get(name).getOrElse { throw new IllegalArgumentException(s"$name not a child of $this") @@ -2756,16 +1941,8 @@ class ColumnGen(val dataGen: DataGen) { def gen(rowNumber: Column): Column = { ColumnGen.genInternal(rowNumber, dataGen.dataType, dataGen.nullable, dataGen.getGen) } - - def getSubstring: Option[SubstringDataGen] = dataGen.getSubstringGen - - def substringGen: SubstringDataGen = dataGen.substringGen - - def setSubstringGen(f : ColumnConf => SubstringDataGen): Unit = - dataGen.setSubstringGen(f) } - sealed trait KeyGroupType /** @@ -3015,7 +2192,7 @@ object DBGen { numRows: Long, mapping: OrderedTypeMapping): Seq[(String, ColumnGen)] = { // a bit of a hack with the column num so that we update it before each time... - var conf = ColumnConf(ColumnLocation(tableId, -1, 0), true, numRows) + var conf = ColumnConf(ColumnLocation(tableId, -1), true, numRows) st.toArray.map { sf => if (!mapping.canMap(sf.dataType, mapping)) { throw new IllegalArgumentException(s"$sf is not supported at this time") From 531a9f5e6501baeda67bd12fa223726d5e8c3572 Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Thu, 13 Jun 2024 14:22:50 -0500 Subject: [PATCH 44/68] Add in the ability to fingerprint JSON columns [databricks] (#11060) Also fixed issue with databricks dependency not being what we said it was. Signed-off-by: Robert (Bobby) Evans --- .../spark/sql/tests/datagen/bigDataGen.scala | 1181 ++++++++++++++--- jenkins/databricks/install_deps.py | 4 +- scala2.13/shim-deps/databricks/pom.xml | 8 +- shim-deps/databricks/pom.xml | 8 +- 4 files changed, 1019 insertions(+), 182 deletions(-) diff --git a/datagen/src/main/scala/org/apache/spark/sql/tests/datagen/bigDataGen.scala b/datagen/src/main/scala/org/apache/spark/sql/tests/datagen/bigDataGen.scala index 91335afe4e6..14e0d4e0970 100644 --- a/datagen/src/main/scala/org/apache/spark/sql/tests/datagen/bigDataGen.scala +++ b/datagen/src/main/scala/org/apache/spark/sql/tests/datagen/bigDataGen.scala @@ -16,21 +16,22 @@ package org.apache.spark.sql.tests.datagen +import com.fasterxml.jackson.core.{JsonFactoryBuilder, JsonParser, JsonToken} +import com.fasterxml.jackson.core.json.JsonReadFeature import java.math.{BigDecimal => JavaBigDecimal} import java.sql.{Date, Timestamp} import java.time.{Duration, Instant, LocalDate, LocalDateTime} import java.util - import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import scala.math.BigDecimal.RoundingMode import scala.util.Random -import org.apache.spark.sql.{Column, DataFrame, SparkSession} +import org.apache.spark.sql.{Column, DataFrame, Row, SparkSession} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Expression, XXH64} import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, ArrayData, DateTimeUtils} -import org.apache.spark.sql.functions.col +import org.apache.spark.sql.functions.{approx_count_distinct, avg, coalesce, col, count, lit, stddev, struct, transform, udf, when} import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.random.XORShiftRandom @@ -79,22 +80,28 @@ class RowLocation(val rowNum: Long, val subRows: Array[Int] = null) { * hash. This makes the generated data correlated for all column/child columns. * @param tableNum a unique ID for the table this is a part of. * @param columnNum the location of the column in the data being generated + * @param substringNum the location of the substring column * @param correlatedKeyGroup the correlated key group this column is a part of, if any. */ -case class ColumnLocation(tableNum: Int, columnNum: Int, correlatedKeyGroup: Option[Long] = None) { - def forNextColumn(): ColumnLocation = ColumnLocation(tableNum, columnNum + 1) +case class ColumnLocation(tableNum: Int, + columnNum: Int, + substringNum: Int, + correlatedKeyGroup: Option[Long] = None) { + def forNextColumn(): ColumnLocation = ColumnLocation(tableNum, columnNum + 1, 0) + def forNextSubstring: ColumnLocation = ColumnLocation(tableNum, columnNum, substringNum + 1) /** * Create a new ColumnLocation that is specifically for a given key group */ def forCorrelatedKeyGroup(keyGroup: Long): ColumnLocation = - ColumnLocation(tableNum, columnNum, Some(keyGroup)) + ColumnLocation(tableNum, columnNum, substringNum, Some(keyGroup)) /** * Hash the location into a single long value. */ - lazy val hashLoc: Long = XXH64.hashLong(tableNum, correlatedKeyGroup.getOrElse(columnNum)) + lazy val hashLoc: Long = XXH64.hashLong(tableNum, + correlatedKeyGroup.getOrElse(XXH64.hashLong(columnNum, substringNum))) } /** @@ -115,6 +122,9 @@ case class ColumnConf(columnLoc: ColumnLocation, def forNextColumn(nullable: Boolean): ColumnConf = ColumnConf(columnLoc.forNextColumn(), nullable, numTableRows) + def forNextSubstring: ColumnConf = + ColumnConf(columnLoc.forNextSubstring, nullable = true, numTableRows) + /** * Create a new configuration based on this, but for a given correlated key group. */ @@ -303,6 +313,23 @@ case class VarLengthGeneratorFunction(minLength: Int, maxLength: Int) extends } } +case class StdDevLengthGen(mean: Double, + stdDev: Double, + mapping: LocationToSeedMapping = null) extends + LengthGeneratorFunction { + override def withLocationToSeedMapping(mapping: LocationToSeedMapping): LengthGeneratorFunction = + StdDevLengthGen(mean, stdDev, mapping) + + override def apply(rowLoc: RowLocation): Int = { + val r = DataGen.getRandomFor(rowLoc, mapping) + val g = r.nextGaussian() // g has a mean of 0 and a stddev of 1.0 + val adjusted = mean + (g * stdDev) + // If the range of seed is too small compared to the stddev and mean we will + // end up with an invalid distribution, but they asked for it. + math.max(0, math.round(adjusted).toInt) + } +} + /** * Generate nulls with a given probability. * @param prob 0.0 to 1.0 for how often nulls should appear in the output. @@ -562,11 +589,8 @@ case class DataGenExpr(child: Expression, } } -/** - * Base class for generating a column/sub-column. This holds configuration for the column, - * and handles what is needed to convert it into GeneratorFunction - */ -abstract class DataGen(var conf: ColumnConf, +abstract class CommonDataGen( + var conf: ColumnConf, defaultValueRange: Option[(Any, Any)], var seedMapping: LocationToSeedMapping = FlatDistribution(), var nullMapping: LocationToSeedMapping = FlatDistribution(), @@ -576,26 +600,25 @@ abstract class DataGen(var conf: ColumnConf, protected var valueRange: Option[(Any, Any)] = defaultValueRange /** - * Set a value range for this data gen. + * Set a value range */ - def setValueRange(min: Any, max: Any): DataGen = { + def setValueRange(min: Any, max: Any): CommonDataGen = { valueRange = Some((min, max)) this } /** - * Set a custom GeneratorFunction to use for this column. + * Set a custom GeneratorFunction */ - def setValueGen(f: GeneratorFunction): DataGen = { + def setValueGen(f: GeneratorFunction): CommonDataGen = { userProvidedValueGen = Some(f) this } /** - * Set a NullGeneratorFunction for this column. This will not be used - * if the column is not nullable. + * Set a NullGeneratorFunction */ - def setNullGen(f: NullGeneratorFunction): DataGen = { + def setNullGen(f: NullGeneratorFunction): CommonDataGen = { this.userProvidedNullGen = Some(f) this } @@ -604,12 +627,12 @@ abstract class DataGen(var conf: ColumnConf, * Set the probability of a null appearing in the output. The probability should be * 0.0 to 1.0. */ - def setNullProbability(probability: Double): DataGen = { + def setNullProbability(probability: Double): CommonDataGen = { this.userProvidedNullGen = Some(NullProbabilityGenerationFunction(probability)) this } - def setNullProbabilityRecursively(probability: Double): DataGen = { + def setNullProbabilityRecursively(probability: Double): CommonDataGen = { this.userProvidedNullGen = Some(NullProbabilityGenerationFunction(probability)) children.foreach { case (_, dataGen) => @@ -621,7 +644,7 @@ abstract class DataGen(var conf: ColumnConf, /** * Set a specific location to seed mapping for the value generation. */ - def setSeedMapping(seedMapping: LocationToSeedMapping): DataGen = { + def setSeedMapping(seedMapping: LocationToSeedMapping): CommonDataGen = { this.seedMapping = seedMapping this } @@ -629,7 +652,7 @@ abstract class DataGen(var conf: ColumnConf, /** * Set a specific location to seed mapping for the null generation. */ - def setNullMapping(nullMapping: LocationToSeedMapping): DataGen = { + def setNullMapping(nullMapping: LocationToSeedMapping): CommonDataGen = { this.nullMapping = nullMapping this } @@ -638,7 +661,7 @@ abstract class DataGen(var conf: ColumnConf, * Set a specific LengthGeneratorFunction to use. This will only be used if * the datatype needs a length. */ - def setLengthGen(lengthGen: LengthGeneratorFunction): DataGen = { + def setLengthGen(lengthGen: LengthGeneratorFunction): CommonDataGen = { this.lengthGen = lengthGen this } @@ -646,25 +669,30 @@ abstract class DataGen(var conf: ColumnConf, /** * Set the length generation to be a fixed length. */ - def setLength(len: Int): DataGen = { + def setLength(len: Int): CommonDataGen = { this.lengthGen = FixedLengthGeneratorFunction(len) this } - def setLength(minLen: Int, maxLen: Int) = { + def setLength(minLen: Int, maxLen: Int): CommonDataGen = { this.lengthGen = VarLengthGeneratorFunction(minLen, maxLen) this } + def setGaussianLength(mean: Double, stdDev: Double): CommonDataGen = { + this.lengthGen = StdDevLengthGen(mean, stdDev) + this + } + /** * Add this column to a specific correlated key group. This should not be * called directly by users. */ def setCorrelatedKeyGroup(keyGroup: Long, - minSeed: Long, maxSeed: Long, - seedMapping: LocationToSeedMapping): DataGen = { + minSeed: Long, maxSeed: Long, + seedMapping: LocationToSeedMapping): CommonDataGen = { conf = conf.forCorrelatedKeyGroup(keyGroup) - .forSeedRange(minSeed, maxSeed) + .forSeedRange(minSeed, maxSeed) this.seedMapping = seedMapping this } @@ -672,7 +700,7 @@ abstract class DataGen(var conf: ColumnConf, /** * Set a range of seed values that should be returned by the LocationToSeedMapping */ - def setSeedRange(min: Long, max: Long): DataGen = { + def setSeedRange(min: Long, max: Long): CommonDataGen = { conf = conf.forSeedRange(min, max) this } @@ -681,7 +709,7 @@ abstract class DataGen(var conf: ColumnConf, * Get the default value generator for this specific data gen. */ protected def getValGen: GeneratorFunction - def children: Seq[(String, DataGen)] + def children: Seq[(String, CommonDataGen)] /** * Get the final ready to use GeneratorFunction for the data generator. @@ -690,8 +718,8 @@ abstract class DataGen(var conf: ColumnConf, val sm = seedMapping.withColumnConf(conf) val lg = lengthGen.withLocationToSeedMapping(sm) var valGen = userProvidedValueGen.getOrElse(getValGen) - .withLocationToSeedMapping(sm) - .withLengthGeneratorFunction(lg) + .withLocationToSeedMapping(sm) + .withLengthGeneratorFunction(lg) valueRange.foreach { case (min, max) => valGen = valGen.withValueRange(min, max) @@ -700,35 +728,75 @@ abstract class DataGen(var conf: ColumnConf, val nullColConf = conf.forNulls val nm = nullMapping.withColumnConf(nullColConf) userProvidedNullGen.get - .withWrapped(valGen) - .withLocationToSeedMapping(nm) + .withWrapped(valGen) + .withLocationToSeedMapping(nm) } else { valGen } } - /** - * Get the data type for this column - */ - def dataType: DataType - /** * Is this column nullable or not. */ def nullable: Boolean = conf.nullable /** - * Get a child column for a given name, if it has one. + * Get a child for a given name, if it has one. */ - final def apply(name: String): DataGen = { + final def apply(name: String): CommonDataGen = { get(name).getOrElse{ throw new IllegalStateException(s"Could not find a child $name for $this") } } - def get(name: String): Option[DataGen] = None + def get(name: String): Option[CommonDataGen] = None +} + + +/** + * Base class for generating a column/sub-column. This holds configuration + * for the column, and handles what is needed to convert it into GeneratorFunction + */ +abstract class DataGen( + conf: ColumnConf, + defaultValueRange: Option[(Any, Any)], + seedMapping: LocationToSeedMapping = FlatDistribution(), + nullMapping: LocationToSeedMapping = FlatDistribution(), + lengthGen: LengthGeneratorFunction = FixedLengthGeneratorFunction(10)) extends + CommonDataGen(conf, defaultValueRange, seedMapping, nullMapping, lengthGen) { + + /** + * Get the data type for this column + */ + def dataType: DataType + + override def get(name: String): Option[DataGen] = None + + def getSubstringGen: Option[SubstringDataGen] = None + + def substringGen: SubstringDataGen = + getSubstringGen.getOrElse( + throw new IllegalArgumentException("substring data gen was not set")) + + def setSubstringGen(f : ColumnConf => SubstringDataGen): Unit = + setSubstringGen(Option(f(conf.forNextSubstring))) + + def setSubstringGen(subgen: Option[SubstringDataGen]): Unit = + throw new IllegalArgumentException("substring data gens can only be set for a STRING") } +/** + * Base class for generating a sub-string. This holds configuration + * for the substring, and handles what is needed to convert it into a GeneratorFunction + */ +abstract class SubstringDataGen( + conf: ColumnConf, + defaultValueRange: Option[(Any, Any)], + seedMapping: LocationToSeedMapping = FlatDistribution(), + nullMapping: LocationToSeedMapping = FlatDistribution(), + lengthGen: LengthGeneratorFunction = FixedLengthGeneratorFunction(10)) extends + CommonDataGen(conf, defaultValueRange, seedMapping, nullMapping, lengthGen) {} + /** * A special GeneratorFunction that just returns the computed seed. This is helpful for * debugging distributions or if you want long values without any abstraction in between. @@ -1494,155 +1562,866 @@ class FloatGen(conf: ColumnConf, defaultValueRange: Option[(Any, Any)]) override def children: Seq[(String, DataGen)] = Seq.empty } -trait JSONType { - def appendRandomValue(sb: StringBuilder, - index: Int, - maxStringLength: Int, - maxArrayLength: Int, - maxObjectLength: Int, - depth: Int, - maxDepth: Int, - r: Random): Unit -} +case class JsonPathElement(name: String, is_array: Boolean) +case class JsonLevel(path: Array[JsonPathElement], data_type: String, length: Int, value: String) {} + +object JsonColumnStats { + private def printHelp(): Unit = { + println("JSON Fingerprinting Tool:") + println("PARAMS: ") + println(" is a path to a Spark dataframe to read in") + println(" is a path in a Spark file system to write out fingerprint data to.") + println() + println("OPTIONS:") + println(" --json= where is the name of a top level String column") + println(" --anon= where is a SEED used to anonymize the JSON keys ") + println(" and column names.") + println(" --input_format= where is parquet or ORC. Defaults to parquet.") + println(" --overwrite to enable overwriting the fingerprint output.") + println(" --debug to enable some debug information to be printed out") + println(" --help to print out this help message") + println() + } + + def main(args: Array[String]): Unit = { + var inputPath = Option.empty[String] + var outputPath = Option.empty[String] + val jsonColumns = ArrayBuffer.empty[String] + var anonSeed = Option.empty[Long] + var debug = false + var argsDone = false + var format = "parquet" + var overwrite = false + + args.foreach { + case a if !argsDone && a.startsWith("--json=") => + jsonColumns += a.substring("--json=".length) + case a if !argsDone && a.startsWith("--anon=") => + anonSeed = Some(a.substring("--anon=".length).toLong) + case a if !argsDone && a.startsWith("--input_format=") => + format = a.substring("--input_format=".length).toLowerCase(java.util.Locale.US) + case "--overwrite" if !argsDone => + overwrite = true + case "--debug" if !argsDone => + debug = true + case "--help" if !argsDone => + printHelp() + System.exit(0) + case "--" if !argsDone => + argsDone = true + case a if !argsDone && a.startsWith("--") => // "--" was covered above already + println(s"ERROR $a is not a supported argument") + printHelp() + System.exit(-1) + case a if inputPath.isEmpty => + inputPath = Some(a) + case a if outputPath.isEmpty => + outputPath = Some(a) + case a => + println(s"ERROR only two arguments are supported. Found $a") + printHelp() + System.exit(-1) + } + if (outputPath.isEmpty) { + println("ERROR both an inputPath and an outputPath are required") + printHelp() + System.exit(-1) + } + + val spark = SparkSession.builder.getOrCreate() + spark.sparkContext.setLogLevel("WARN") + + val df = spark.read.format(format).load(inputPath.get) + jsonColumns.foreach { column => + val fp = fingerPrint(df, df(column), anonSeed) + val name = anonSeed.map(s => anonymizeString(column, s)).getOrElse(column) + val fullOutPath = s"${outputPath.get}/$name" + var writer = fp.write + if (overwrite) { + writer = writer.mode("overwrite") + } + if (debug) { + anonSeed.foreach { s => + println(s"Keys and columns will be anonymized with seed $s") + } + println(s"Writing $column fingerprint to $fullOutPath") + spark.time(writer.parquet(fullOutPath)) + println(s"Wrote ${spark.read.parquet(fullOutPath).count} rows") + spark.read.parquet(fullOutPath).show() + } else { + writer.parquet(fullOutPath) + } + } + } -object JSONType { - def selectType(depth: Int, - maxDepth: Int, - r: Random): JSONType = { - val toSelectFrom = if (depth < maxDepth) { - Seq(QuotedJSONString, JSONLong, JSONDouble, JSONArray, JSONObject) - } else { - Seq(QuotedJSONString, JSONLong, JSONDouble) - } - val index = r.nextInt(toSelectFrom.length) - toSelectFrom(index) - } -} - -object QuotedJSONString extends JSONType { - override def appendRandomValue(sb: StringBuilder, - index: Int, - maxStringLength: Int, - maxArrayLength: Int, - maxObjectLength: Int, - depth: Int, - maxDepth: Int, - r: Random): Unit = { - val strValue = r.nextString(r.nextInt(maxStringLength + 1)) - .replace("\\", "\\\\") - .replace("\"", "\\\"") - .replace("\n", "\\n") - .replace("\r", "\\r") - .replace("\b", "\\b") - .replace("\f", "\\f") - sb.append('"') - sb.append(strValue) - sb.append('"') - } -} - -object JSONLong extends JSONType { - override def appendRandomValue(sb: StringBuilder, - index: Int, - maxStringLength: Int, - maxArrayLength: Int, - maxObjectLength: Int, - depth: Int, - maxDepth: Int, - r: Random): Unit = { - sb.append(r.nextLong()) - } -} - -object JSONDouble extends JSONType { - override def appendRandomValue(sb: StringBuilder, - index: Int, - maxStringLength: Int, - maxArrayLength: Int, - maxObjectLength: Int, - depth: Int, - maxDepth: Int, - r: Random): Unit = { - sb.append(r.nextDouble() * 4096.0) - } -} - -object JSONArray extends JSONType { - override def appendRandomValue(sb: StringBuilder, - index: Int, - maxStringLength: Int, - maxArrayLength: Int, - maxObjectLength: Int, - depth: Int, - maxDepth: Int, - r: Random): Unit = { - val childType = JSONType.selectType(depth, maxDepth, r) - val length = r.nextInt(maxArrayLength + 1) - sb.append("[") + case class JsonNodeStats(count: Long, meanLen: Double, stdDevLength: Double, dc: Long) + + class JsonNode() { + private val forDataType = + mutable.HashMap[String, (JsonNodeStats, mutable.HashMap[String, JsonNode])]() + + def getChild(name: String, isArray: Boolean): JsonNode = { + val dt = if (isArray) { "ARRAY" } else { "OBJECT" } + val typed = forDataType.getOrElse(dt, + throw new IllegalArgumentException(s"$dt is not a set data type yet.")) + typed._2.getOrElse(name, + throw new IllegalArgumentException(s"$name is not a child when the type is $dt")) + } + + def contains(name: String, isArray: Boolean): Boolean = { + val dt = if (isArray) { "ARRAY" } else { "OBJECT" } + forDataType.get(dt).exists { children => + children._2.contains(name) + } + } + + def addChild(name: String, isArray: Boolean): JsonNode = { + val dt = if (isArray) { "ARRAY" } else { "OBJECT" } + val found = forDataType.getOrElse(dt, + throw new IllegalArgumentException(s"$dt was not already added as a data type")) + if (found._2.contains(name)) { + throw new IllegalArgumentException(s"$dt already has a child named $name") + } + val node = new JsonNode() + found._2.put(name, node) + node + } + + def addChoice(dt: String, stats: JsonNodeStats): Unit = { + if (forDataType.contains(dt)) { + throw new IllegalArgumentException(s"$dt was already added as a data type") + } + forDataType.put(dt, (stats, new mutable.HashMap[String, JsonNode]())) + } + + override def toString: String = { + forDataType.toString() + } + + def totalCount: Long = { + forDataType.values.map{ case (stats, _) => stats.count}.sum + } + + private def makeNoChoiceGenRecursive(dt: String, + children: mutable.HashMap[String, JsonNode], + cc: ColumnConf): (SubstringDataGen, ColumnConf) = { + var c = cc + val ret = dt match { + case "LONG" => new JSONLongGen(c) + case "DOUBLE" => new JSONDoubleGen(c) + case "BOOLEAN" => new JSONBoolGen(c) + case "NULL" => new JSONNullGen(false, c) + case "VALUE_NULL" => new JSONNullGen(true, c) + case "ERROR" => new JSONErrorGen(c) + case "STRING" => new JSONStringGen(c) + case "ARRAY" => + val child = if (children.isEmpty) { + // A corner case, we will just make it a BOOL column and it will be ignored + val tmp = new JSONBoolGen(c) + c = c.forNextSubstring + tmp + } else { + val tmp = children.values.head.makeGenRecursive(c) + c = tmp._2 + tmp._1 + } + new JSONArrayGen(child, c) + case "OBJECT" => + val childGens = if (children.isEmpty) { + Seq.empty + } else { + children.toSeq.map { + case (k, node) => + val tmp = node.makeGenRecursive(c) + c = tmp._2 + (k, tmp._1) + } + } + new JSONObjectGen(childGens, c) + case other => + throw new IllegalArgumentException(s"$other is not a leaf node type") + } + (ret, c.forNextSubstring) + } + + private def makeGenRecursive(cc: ColumnConf): (SubstringDataGen, ColumnConf) = { + var c = cc + // We are going to recursively walk the tree for all of the values. + if (forDataType.size == 1) { + // We don't need a choice at all. This makes it simpler.. + val (dt, (_, children)) = forDataType.head + makeNoChoiceGenRecursive(dt, children, c) + } else { + val totalSum = forDataType.map(f => f._2._1.count).sum.toDouble + var runningSum = 0L + val allChoices = ArrayBuffer[(Double, String, SubstringDataGen)]() + forDataType.foreach { + case (dt, (stats, children)) => + val tmp = makeNoChoiceGenRecursive(dt, children, c) + c = tmp._2 + runningSum += stats.count + allChoices.append((runningSum/totalSum, dt, tmp._1)) + } + + val ret = new JSONChoiceGen(allChoices.toSeq, c) + (ret, c.forNextSubstring) + } + } + + def makeGen(cc: ColumnConf): SubstringDataGen = { + val (ret, _) = makeGenRecursive(cc) + ret + } + + def setStatsSingle(dg: CommonDataGen, + dt: String, + stats: JsonNodeStats, + nullPct: Double): Unit = { + + val includeLength = dt != "OBJECT" && dt != "BOOLEAN" && dt != "NULL" && dt != "VALUE_NULL" + val includeNullPct = nullPct > 0.0 + if (includeLength) { + dg.setGaussianLength(stats.meanLen, stats.stdDevLength) + } + if (includeNullPct) { + dg.setNullProbability(nullPct) + } + dg.setSeedRange(1, stats.dc) + } + + def setStats(dg: CommonDataGen, + parentCount: Option[Long]): Unit = { + // We are going to recursively walk the tree... + if (forDataType.size == 1) { + // We don't need a choice at all. This makes it simpler.. + val (dt, (stats, children)) = forDataType.head + val nullPct = parentCount.map { pc => + (pc - stats.count).toDouble/pc + }.getOrElse(0.0) + setStatsSingle(dg, dt, stats, nullPct) + val myCount = if (dt == "OBJECT") { + Some(totalCount) + } else { + None + } + children.foreach { + case (name, node) => + node.setStats(dg(name), myCount) + } + } else { + // We have choices to make between different types. + // The null percent cannot be calculated for each individual choice + // but is calculated on the group as a whole instead + parentCount.foreach { pc => + val tc = totalCount + val choiceNullPct = (pc - tc).toDouble / pc + if (choiceNullPct > 0.0) { + dg.setNullProbability(choiceNullPct) + } + } + forDataType.foreach { + case (dt, (stats, children)) => + // When there is a choice the name to access it is the data type + val choiceDg = dg(dt) + setStatsSingle(choiceDg, dt, stats, 0.0) + children.foreach { + case (name, node) => + val myCount = if (dt == "OBJECT") { + // Here we only want the count for the OBJECTs + Some(stats.count) + } else { + None + } + node.setStats(choiceDg(name), myCount) + } + } + } + } + } + + private lazy val jsonFactory = new JsonFactoryBuilder() + // The two options below enabled for Hive compatibility + .enable(JsonReadFeature.ALLOW_UNESCAPED_CONTROL_CHARS) + .enable(JsonReadFeature.ALLOW_SINGLE_QUOTES) + .build() + + private def processNext(parser: JsonParser, + currentPath: ArrayBuffer[JsonPathElement], + output: ArrayBuffer[JsonLevel]): Unit = { + parser.currentToken() match { + case JsonToken.START_OBJECT => + parser.nextToken() + while (parser.currentToken() != JsonToken.END_OBJECT) { + processNext(parser, currentPath, output) + } + output.append(JsonLevel(currentPath.toArray, "OBJECT", 0, "")) + parser.nextToken() + case JsonToken.START_ARRAY => + currentPath.append(JsonPathElement("data", is_array = true)) + parser.nextToken() + var length = 0 + while (parser.currentToken() != JsonToken.END_ARRAY) { + length += 1 + processNext(parser, currentPath, output) + } + currentPath.remove(currentPath.length - 1) + output.append(JsonLevel(currentPath.toArray, "ARRAY", length, "")) + parser.nextToken() + case JsonToken.FIELD_NAME => + currentPath.append(JsonPathElement(parser.getCurrentName, is_array = false)) + parser.nextToken() + processNext(parser, currentPath, output) + currentPath.remove(currentPath.length - 1) + case JsonToken.VALUE_NUMBER_INT => + val length = parser.getValueAsString.getBytes("UTF-8").length + output.append(JsonLevel(currentPath.toArray, "LONG", length, parser.getValueAsString)) + parser.nextToken() + case JsonToken.VALUE_NUMBER_FLOAT => + val length = parser.getValueAsString.getBytes("UTF-8").length + output.append(JsonLevel(currentPath.toArray, "DOUBLE", length, parser.getValueAsString)) + parser.nextToken() + case JsonToken.VALUE_TRUE | JsonToken.VALUE_FALSE => + val length = parser.getValueAsString.getBytes("UTF-8").length + output.append(JsonLevel(currentPath.toArray, "BOOLEAN", length, parser.getValueAsString)) + parser.nextToken() + case JsonToken.VALUE_NULL | null => + output.append(JsonLevel(currentPath.toArray, "VALUE_NULL", 4, "NULL")) + parser.nextToken() + case JsonToken.VALUE_STRING => + val length = parser.getValueAsString.getBytes("UTF-8").length + output.append(JsonLevel(currentPath.toArray, "STRING", length, parser.getValueAsString)) + parser.nextToken() + case other => + throw new IllegalStateException(s"DON'T KNOW HOW TO DEAL WITH $other") + } + } + + def jsonStatsUdf(json: String): Array[JsonLevel] = { + val output = new ArrayBuffer[JsonLevel]() + try { + val currentPath = new ArrayBuffer[JsonPathElement]() + if (json == null) { + output.append(JsonLevel(Array.empty, "NULL", 0, "")) + } else { + val parser = jsonFactory.createParser(json) + try { + parser.nextToken() + processNext(parser, currentPath, output) + } finally { + parser.close() + } + } + } catch { + case _: com.fasterxml.jackson.core.JsonParseException => + output.clear() + output.append(JsonLevel(Array.empty, "ERROR", json.getBytes("UTF-8").length, json)) + } + output.toArray + } + + private lazy val extractPaths = udf(json => jsonStatsUdf(json)) + + def anonymizeString(str: String, seed: Long): String = { + val length = str.length + val data = new Array[Byte](length) + val hash = XXH64.hashLong(str.hashCode, seed) + val r = new Random() + r.setSeed(hash) (0 until length).foreach { i => - if (i > 0) { - sb.append(",") + val tmp = r.nextInt(16) + data(i) = (tmp + 'A').toByte + } + new String(data) + } + + private lazy val anonPath = udf((str, seed) => anonymizeString(str, seed)) + + def anonymizeFingerPrint(df: DataFrame, anonSeed: Long): DataFrame = { + df.withColumn("tmp", transform(col("path"), + o => { + val name = o("name") + val isArray = o("is_array") + val anon = anonPath(name, lit(anonSeed)) + val newName = when(isArray, name).otherwise(anon).alias("name") + struct(newName, isArray) + })) + .drop("path").withColumnRenamed("tmp", "path") + .orderBy("path", "dt") + .selectExpr("path", "dt","c","mean_len","stddev_len","distinct","version") + } + + def fingerPrint(df: DataFrame, column: Column, anonymize: Option[Long] = None): DataFrame = { + val ret = df.select(extractPaths(column).alias("paths")) + .selectExpr("explode_outer(paths) as p") + .selectExpr("p.path as path", "p.data_type as dt", "p.length as len", "p.value as value") + .groupBy(col("path"), col("dt")).agg( + count(lit(1)).alias("c"), + avg(col("len")).alias("mean_len"), + coalesce(stddev(col("len")), lit(0.0)).alias("stddev_len"), + approx_count_distinct(col("value")).alias("distinct")) + .orderBy("path", "dt").withColumn("version", lit("0.1")) + .selectExpr("path", "dt","c","mean_len","stddev_len","distinct","version") + + anonymize.map { anonSeed => + anonymizeFingerPrint(ret, anonSeed) + }.getOrElse(ret) + } + + def apply(aggForColumn: DataFrame, genColumn: ColumnGen): Unit = + apply(aggForColumn, genColumn.dataGen) + + private val expectedSchema = StructType.fromDDL( + "path ARRAY>," + + "dt STRING," + + "c BIGINT," + + "mean_len DOUBLE," + + "stddev_len DOUBLE," + + "distinct BIGINT," + + "version STRING") + + def apply(aggForColumn: DataFrame, gen: DataGen): Unit = { + val aggData = aggForColumn.orderBy("path", "dt").collect() + val rootNode: JsonNode = new JsonNode() + assert(aggData.length > 0) + val schema = aggData.head.schema + assert(schema.length == expectedSchema.length) + schema.fields.zip(expectedSchema.fields).foreach { + case(found, expected) => + assert(found.name == expected.name) + // TODO we can worry about the exact types later if we need to + } + assert(aggData.head.getString(6) == "0.1") + aggData.foreach { row => + val fullPath = row.getAs[mutable.WrappedArray[Row]](0) + val parsedPath = fullPath.map(r => (r.getString(0), r.getBoolean(1))).toList + val dt = row.getString(1) + val count = row.getLong(2) + val meanLen = row.getDouble(3) + val stdLen = row.getDouble(4) + val dc = row.getLong(5) + + val stats = JsonNodeStats(count, meanLen, stdLen, dc) + var currentNode = rootNode + // Find everything up to the last path element + if (parsedPath.length > 1) { + parsedPath.slice(0, parsedPath.length - 1).foreach { + case (name, isArray) => + currentNode = currentNode.getChild(name, isArray) + } + } + + if (parsedPath.nonEmpty) { + // For the last path element (that is not the root element) we might need to add it + // as a child + val (name, isArray) = parsedPath.last + if (!currentNode.contains(name, isArray)) { + currentNode.addChild(name, isArray) + } + currentNode = currentNode.getChild(name, isArray) } - childType.appendRandomValue(sb, i, maxStringLength, maxArrayLength, maxObjectLength, - depth + 1, maxDepth, r) + currentNode.addChoice(dt, stats) } - sb.append("]") + + gen.setSubstringGen(cc => rootNode.makeGen(cc)) + rootNode.setStats(gen.substringGen, None) } } -object JSONObject extends JSONType { - override def appendRandomValue(sb: StringBuilder, - index: Int, - maxStringLength: Int, - maxArrayLength: Int, - maxObjectLength: Int, - depth: Int, - maxDepth: Int, - r: Random): Unit = { - val length = r.nextInt(maxObjectLength) + 1 - sb.append("{") - (0 until length).foreach { i => - if (i > 0) { - sb.append(",") + +case class JSONStringGenFunc(lengthGen: LengthGeneratorFunction = null, + mapping: LocationToSeedMapping = null) extends GeneratorFunction { + + override def apply(rowLoc: RowLocation): Any = { + val len = lengthGen(rowLoc) + val r = DataGen.getRandomFor(rowLoc, mapping) + val buffer = new Array[Byte](len) + var at = 0 + while (at < len) { + // Value range is 32 (Space) to 126 (~) + buffer(at) = (r.nextInt(126 - 31) + 32).toByte + at += 1 + } + val strVal = new String(buffer, 0, len) + .replace("\\", "\\\\") + .replace("\"", "\\\"") + .replace("\n", "\\n") + .replace("\r", "\\r") + .replace("\b", "\\b") + .replace("\f", "\\f") + '"' + strVal + '"' + } + + override def withLengthGeneratorFunction(lengthGen: LengthGeneratorFunction): JSONStringGenFunc = + JSONStringGenFunc(lengthGen, mapping) + + override def withLocationToSeedMapping(mapping: LocationToSeedMapping): JSONStringGenFunc = + JSONStringGenFunc(lengthGen, mapping) + + override def withValueRange(min: Any, max: Any): GeneratorFunction = + throw new IllegalArgumentException("value ranges are not supported for JSON") +} + +class JSONStringGen(conf: ColumnConf, + defaultValueRange: Option[(Any, Any)] = None) + extends SubstringDataGen(conf, defaultValueRange) { + + override protected def getValGen: GeneratorFunction = JSONStringGenFunc() + + override def children: Seq[(String, SubstringDataGen)] = Seq.empty +} + +case class JSONLongGenFunc(lengthGen: LengthGeneratorFunction = null, + mapping: LocationToSeedMapping = null) extends GeneratorFunction { + + override def apply(rowLoc: RowLocation): Any = { + val len = math.max(lengthGen(rowLoc), 1) // We need at least 1 long for a valid value + val r = DataGen.getRandomFor(rowLoc, mapping) + val buffer = new Array[Byte](len) + var at = 0 + while (at < len) { + if (at == 0) { + // No leading 0's + buffer(at) = (r.nextInt(9) + '1').toByte + } else { + buffer(at) = (r.nextInt(10) + '0').toByte } - sb.append("\"key_") - sb.append(i) - sb.append("_") - sb.append(depth ) - sb.append("\":") - val childType = JSONType.selectType(depth, maxDepth, r) - childType.appendRandomValue(sb, i, maxStringLength, maxArrayLength, maxObjectLength, - depth + 1, maxDepth, r) + at += 1 } - sb.append("}") + new String(buffer, 0, len) } + + override def withLengthGeneratorFunction(lengthGen: LengthGeneratorFunction): JSONLongGenFunc = + JSONLongGenFunc(lengthGen, mapping) + + override def withLocationToSeedMapping(mapping: LocationToSeedMapping): JSONLongGenFunc = + JSONLongGenFunc(lengthGen, mapping) + + override def withValueRange(min: Any, max: Any): GeneratorFunction = + throw new IllegalArgumentException("value ranges are not supported for JSON") } -case class JSONGenFunc( - maxStringLength: Int, - maxArrayLength: Int, - maxObjectLength: Int, - maxDepth: Int, - lengthGen: LengthGeneratorFunction = null, - mapping: LocationToSeedMapping = null) extends GeneratorFunction { +class JSONLongGen(conf: ColumnConf, + defaultValueRange: Option[(Any, Any)] = None) + extends SubstringDataGen(conf, defaultValueRange) { + + override protected def getValGen: GeneratorFunction = JSONLongGenFunc() + + override def children: Seq[(String, SubstringDataGen)] = Seq.empty +} + +case class JSONDoubleGenFunc(lengthGen: LengthGeneratorFunction = null, + mapping: LocationToSeedMapping = null) extends GeneratorFunction { override def apply(rowLoc: RowLocation): Any = { + val len = math.max(lengthGen(rowLoc), 3) // We have to have at least 3 chars NUM.NUM val r = DataGen.getRandomFor(rowLoc, mapping) - val sb = new StringBuilder() - JSONObject.appendRandomValue(sb, 0, maxStringLength, maxArrayLength, maxObjectLength, - 0, maxDepth, r) - // For now I am going to have some hard coded keys - UTF8String.fromString(sb.toString()) + val beforeLen = if (len == 3) { 1 } else { r.nextInt(len - 3) + 1 } + val buffer = new Array[Byte](len) + var at = 0 + while (at < len) { + if (at == 0) { + // No leading 0's + buffer(at) = (r.nextInt(9) + '1').toByte + } else if (at == beforeLen) { + buffer(at) = '.' + } else { + buffer(at) = (r.nextInt(10) + '0').toByte + } + at += 1 + } + UTF8String.fromBytes(buffer, 0, len) } - override def withLengthGeneratorFunction(lengthGen: LengthGeneratorFunction): GeneratorFunction = - JSONGenFunc(maxStringLength, maxArrayLength, maxObjectLength, maxDepth, lengthGen, mapping) + override def withLengthGeneratorFunction(lengthGen: LengthGeneratorFunction): JSONDoubleGenFunc = + JSONDoubleGenFunc(lengthGen, mapping) - override def withLocationToSeedMapping(mapping: LocationToSeedMapping): GeneratorFunction = - JSONGenFunc(maxStringLength, maxArrayLength, maxObjectLength, maxDepth, lengthGen, mapping) + override def withLocationToSeedMapping(mapping: LocationToSeedMapping): JSONDoubleGenFunc = + JSONDoubleGenFunc(lengthGen, mapping) override def withValueRange(min: Any, max: Any): GeneratorFunction = - throw new IllegalArgumentException("value ranges are not supported for strings") + throw new IllegalArgumentException("value ranges are not supported for JSON") +} + +class JSONDoubleGen(conf: ColumnConf, + defaultValueRange: Option[(Any, Any)] = None) + extends SubstringDataGen(conf, defaultValueRange) { + + override protected def getValGen: GeneratorFunction = JSONDoubleGenFunc() + + override def children: Seq[(String, SubstringDataGen)] = Seq.empty +} + +case class JSONBoolGenFunc(lengthGen: LengthGeneratorFunction = null, + mapping: LocationToSeedMapping = null) extends GeneratorFunction { + + override def apply(rowLoc: RowLocation): Any = { + val r = DataGen.getRandomFor(rowLoc, mapping) + val ret = if (r.nextBoolean()) "true" else "false" + UTF8String.fromString(ret) + } + + override def withLengthGeneratorFunction(lengthGen: LengthGeneratorFunction): JSONBoolGenFunc = + JSONBoolGenFunc(lengthGen, mapping) + + override def withLocationToSeedMapping(mapping: LocationToSeedMapping): JSONBoolGenFunc = + JSONBoolGenFunc(lengthGen, mapping) + + override def withValueRange(min: Any, max: Any): GeneratorFunction = + throw new IllegalArgumentException("value ranges are not supported for JSON") +} + +class JSONBoolGen(conf: ColumnConf, + defaultValueRange: Option[(Any, Any)] = None) + extends SubstringDataGen(conf, defaultValueRange) { + + override protected def getValGen: GeneratorFunction = JSONBoolGenFunc() + + override def children: Seq[(String, SubstringDataGen)] = Seq.empty +} + +case class JSONNullGenFunc(nullAsString: Boolean, + lengthGen: LengthGeneratorFunction = null, + mapping: LocationToSeedMapping = null) extends GeneratorFunction { + + override def apply(rowLoc: RowLocation): Any = + if (nullAsString) { + UTF8String.fromString("null") + } else { + null + } + + + override def withLengthGeneratorFunction(lengthGen: LengthGeneratorFunction): JSONNullGenFunc = + JSONNullGenFunc(nullAsString, lengthGen, mapping) + + override def withLocationToSeedMapping(mapping: LocationToSeedMapping): JSONNullGenFunc = + JSONNullGenFunc(nullAsString, lengthGen, mapping) + + override def withValueRange(min: Any, max: Any): GeneratorFunction = + throw new IllegalArgumentException("value ranges are not supported for JSON") +} + +class JSONNullGen(nullAsString: Boolean, + conf: ColumnConf, + defaultValueRange: Option[(Any, Any)] = None) + extends SubstringDataGen(conf, defaultValueRange) { + + override protected def getValGen: GeneratorFunction = JSONNullGenFunc(nullAsString) + + override def children: Seq[(String, SubstringDataGen)] = Seq.empty +} + +case class JSONErrorGenFunc(lengthGen: LengthGeneratorFunction = null, + mapping: LocationToSeedMapping = null) extends GeneratorFunction { + + override def apply(rowLoc: RowLocation): Any = { + val len = lengthGen(rowLoc) + val r = DataGen.getRandomFor(rowLoc, mapping) + val buffer = new Array[Byte](len) + var at = 0 + while (at < len) { + // Value range is 32 (Space) to 126 (~) + // But it is almost impossible to show up as valid JSON + buffer(at) = (r.nextInt(126 - 31) + 32).toByte + at += 1 + } + UTF8String.fromBytes(buffer, 0, len) + } + + override def withLengthGeneratorFunction(lengthGen: LengthGeneratorFunction): JSONErrorGenFunc = + JSONErrorGenFunc(lengthGen, mapping) + + override def withLocationToSeedMapping(mapping: LocationToSeedMapping): JSONErrorGenFunc = + JSONErrorGenFunc(lengthGen, mapping) + + override def withValueRange(min: Any, max: Any): GeneratorFunction = + throw new IllegalArgumentException("value ranges are not supported for JSON") +} + +class JSONErrorGen(conf: ColumnConf, + defaultValueRange: Option[(Any, Any)] = None) + extends SubstringDataGen(conf, defaultValueRange) { + + override protected def getValGen: GeneratorFunction = JSONErrorGenFunc() + + override def children: Seq[(String, SubstringDataGen)] = Seq.empty +} + +case class JSONArrayGenFunc(child: GeneratorFunction, + lengthGen: LengthGeneratorFunction = null, + mapping: LocationToSeedMapping = null) extends GeneratorFunction { + + override def apply(rowLoc: RowLocation): Any = { + val len = lengthGen(rowLoc) + val data = new Array[String](len) + val childRowLoc = rowLoc.withNewChild() + var i = 0 + while (i < len) { + childRowLoc.setLastChildIndex(i) + val v = child(childRowLoc) + if (v == null) { + // A null in an array must look like "null" + data(i) = "null" + } else { + data(i) = v.toString + } + i += 1 + } + val ret = data.mkString("[", ",", "]") + UTF8String.fromString(ret) + } + + override def withLengthGeneratorFunction(lengthGen: LengthGeneratorFunction): JSONArrayGenFunc = + JSONArrayGenFunc(child, lengthGen, mapping) + + override def withLocationToSeedMapping(mapping: LocationToSeedMapping): JSONArrayGenFunc = + JSONArrayGenFunc(child, lengthGen, mapping) + + override def withValueRange(min: Any, max: Any): GeneratorFunction = + throw new IllegalArgumentException("value ranges are not supported for JSON") +} + +class JSONArrayGen(child: SubstringDataGen, + conf: ColumnConf, + defaultValueRange: Option[(Any, Any)] = None) + extends SubstringDataGen(conf, defaultValueRange) { + + override def setCorrelatedKeyGroup(keyGroup: Long, + minSeed: Long, maxSeed: Long, + seedMapping: LocationToSeedMapping): SubstringDataGen = { + super.setCorrelatedKeyGroup(keyGroup, minSeed, maxSeed, seedMapping) + child.setCorrelatedKeyGroup(keyGroup, minSeed, maxSeed, seedMapping) + this + } + + override protected def getValGen: GeneratorFunction = JSONArrayGenFunc(child.getGen) + + override def get(name: String): Option[SubstringDataGen] = { + if ("data".equalsIgnoreCase(name) || "child".equalsIgnoreCase(name)) { + Some(child) + } else { + None + } + } + + override def children: Seq[(String, SubstringDataGen)] = Seq(("data", child)) +} + +case class JSONObjectGenFunc(childGens: Array[(String, GeneratorFunction)], + lengthGen: LengthGeneratorFunction = null, + mapping: LocationToSeedMapping = null) extends GeneratorFunction { + override def apply(rowLoc: RowLocation): Any = { + // TODO randomize the order of the children??? + // TODO duplicate child values??? + // The row location does not change for a struct/object + val data = childGens.map { + case (k, gen) => + val key = k.replace("\\", "\\\\") + .replace("\"", "\\\"") + .replace("\n", "\\n") + .replace("\r", "\\r") + .replace("\b", "\\b") + .replace("\f", "\\f") + val v = gen.apply(rowLoc) + if (v == null) { + "" + } else { + '"' + key + "\":" + v + } + } + val ret = data.filterNot(_.isEmpty).mkString("{",",","}") + UTF8String.fromString(ret) + } + + override def withLocationToSeedMapping(mapping: LocationToSeedMapping): JSONObjectGenFunc = + JSONObjectGenFunc(childGens, lengthGen, mapping) + + override def withLengthGeneratorFunction(lengthGen: LengthGeneratorFunction): JSONObjectGenFunc = + JSONObjectGenFunc(childGens, lengthGen, mapping) + + override def withValueRange(min: Any, max: Any): GeneratorFunction = + throw new IllegalArgumentException("value ranges are not supported for JSON") +} + +class JSONObjectGen(val children: Seq[(String, SubstringDataGen)], + conf: ColumnConf, + defaultValueRange: Option[(Any, Any)] = None) + extends SubstringDataGen(conf, defaultValueRange) { + + override def setCorrelatedKeyGroup(keyGroup: Long, + minSeed: Long, maxSeed: Long, + seedMapping: LocationToSeedMapping): SubstringDataGen = { + super.setCorrelatedKeyGroup(keyGroup, minSeed, maxSeed, seedMapping) + children.foreach { + case (_, gen) => + gen.setCorrelatedKeyGroup(keyGroup, minSeed, maxSeed, seedMapping) + } + this + } + + override def get(name: String): Option[SubstringDataGen] = + children.collectFirst { + case (childName, dataGen) if childName.equalsIgnoreCase(name) => dataGen + } + + override protected def getValGen: GeneratorFunction = { + val childGens = children.map(c => (c._1, c._2.getGen)).toArray + JSONObjectGenFunc(childGens) + } +} + +case class JSONChoiceGenFunc(choices: List[(Double, GeneratorFunction)], + lengthGen: LengthGeneratorFunction = null, + mapping: LocationToSeedMapping = null) extends GeneratorFunction { + override def apply(rowLoc: RowLocation): Any = { + val r = DataGen.getRandomFor(rowLoc, mapping) + val l = r.nextDouble() + var index = 0 + while (choices(index)._1 < l) { + index += 1 + } + val childRowLoc = rowLoc.withNewChild() + choices(index)._2(childRowLoc) + } + + override def withLengthGeneratorFunction(lengthGen: LengthGeneratorFunction): JSONChoiceGenFunc = + JSONChoiceGenFunc(choices, lengthGen, mapping) + + override def withLocationToSeedMapping(mapping: LocationToSeedMapping): JSONChoiceGenFunc = + JSONChoiceGenFunc(choices, lengthGen, mapping) + + override def withValueRange(min: Any, max: Any): GeneratorFunction = + throw new IllegalArgumentException("value ranges are not supported for JSON") +} + +class JSONChoiceGen(val choices: Seq[(Double, String, SubstringDataGen)], + conf: ColumnConf, + defaultValueRange: Option[(Any, Any)] = None) + extends SubstringDataGen(conf, defaultValueRange) { + + override val children: Seq[(String, SubstringDataGen)] = + choices.map { case (_, name, gen) => (name, gen) } + + override def setCorrelatedKeyGroup(keyGroup: Long, + minSeed: Long, maxSeed: Long, + seedMapping: LocationToSeedMapping): SubstringDataGen = { + super.setCorrelatedKeyGroup(keyGroup, minSeed, maxSeed, seedMapping) + children.foreach { + case (_, gen) => + gen.setCorrelatedKeyGroup(keyGroup, minSeed, maxSeed, seedMapping) + } + this + } + + override def get(name: String): Option[SubstringDataGen] = + children.collectFirst { + case (childName, dataGen) if childName.equalsIgnoreCase(name) => dataGen + } + + override protected def getValGen: GeneratorFunction = { + val childGens = choices.map(c => (c._1, c._3.getGen)).toList + JSONChoiceGenFunc(childGens) + } } case class ASCIIGenFunc( @@ -1672,14 +2451,46 @@ case class ASCIIGenFunc( throw new IllegalArgumentException("value ranges are not supported for strings") } -class StringGen(conf: ColumnConf, defaultValueRange: Option[(Any, Any)]) - extends DataGen(conf, defaultValueRange) { +/** + * This is here to wrap the substring gen function so that its length/settings + * are the ones used when generating a string, and not what was set for the string. + */ +case class SubstringGenFunc( + substringGen: GeneratorFunction, + lengthGen: LengthGeneratorFunction = null, + mapping: LocationToSeedMapping = null) extends GeneratorFunction { + + override def apply(rowLoc: RowLocation): Any = { + substringGen(rowLoc) + } + + // The length and location seed mapping are just ignored for this... + override def withLengthGeneratorFunction(lengthGen: LengthGeneratorFunction): GeneratorFunction = + this + + override def withLocationToSeedMapping(mapping: LocationToSeedMapping): GeneratorFunction = + this + + override def withValueRange(min: Any, max: Any): GeneratorFunction = + throw new IllegalArgumentException("value ranges are not supported for strings") +} + +class StringGen(conf: ColumnConf, + defaultValueRange: Option[(Any, Any)], + var substringDataGen: Option[SubstringDataGen] = None) + extends DataGen(conf, defaultValueRange) { override def dataType: DataType = StringType - override protected def getValGen: GeneratorFunction = ASCIIGenFunc() + override protected def getValGen: GeneratorFunction = + substringDataGen.map(s => SubstringGenFunc(s.getGen)).getOrElse(ASCIIGenFunc()) override def children: Seq[(String, DataGen)] = Seq.empty + + override def setSubstringGen(subgen: Option[SubstringDataGen]): Unit = + substringDataGen = subgen + + override def getSubstringGen: Option[SubstringDataGen] = substringDataGen } case class StructGenFunc(childGens: Array[GeneratorFunction]) extends GeneratorFunction { @@ -1854,7 +2665,6 @@ class MapGen(key: DataGen, override def children: Seq[(String, DataGen)] = Seq(("key", key), ("value", value)) } - object ColumnGen { private def genInternal(rowNumber: Column, dataType: DataType, @@ -1869,8 +2679,8 @@ object ColumnGen { */ class ColumnGen(val dataGen: DataGen) { def setCorrelatedKeyGroup(kg: Long, - minSeed: Long, maxSeed: Long, - seedMapping: LocationToSeedMapping): ColumnGen = { + minSeed: Long, maxSeed: Long, + seedMapping: LocationToSeedMapping): ColumnGen = { dataGen.setCorrelatedKeyGroup(kg, minSeed, maxSeed, seedMapping) this } @@ -1930,6 +2740,11 @@ class ColumnGen(val dataGen: DataGen) { this } + def setGaussianLength(mean: Double, stdDev: Double): ColumnGen = { + dataGen.setGaussianLength(mean, stdDev) + this + } + final def apply(name: String): DataGen = { get(name).getOrElse { throw new IllegalArgumentException(s"$name not a child of $this") @@ -1941,8 +2756,16 @@ class ColumnGen(val dataGen: DataGen) { def gen(rowNumber: Column): Column = { ColumnGen.genInternal(rowNumber, dataGen.dataType, dataGen.nullable, dataGen.getGen) } + + def getSubstring: Option[SubstringDataGen] = dataGen.getSubstringGen + + def substringGen: SubstringDataGen = dataGen.substringGen + + def setSubstringGen(f : ColumnConf => SubstringDataGen): Unit = + dataGen.setSubstringGen(f) } + sealed trait KeyGroupType /** @@ -2192,7 +3015,7 @@ object DBGen { numRows: Long, mapping: OrderedTypeMapping): Seq[(String, ColumnGen)] = { // a bit of a hack with the column num so that we update it before each time... - var conf = ColumnConf(ColumnLocation(tableId, -1), true, numRows) + var conf = ColumnConf(ColumnLocation(tableId, -1, 0), true, numRows) st.toArray.map { sf => if (!mapping.canMap(sf.dataType, mapping)) { throw new IllegalArgumentException(s"$sf is not supported at this time") diff --git a/jenkins/databricks/install_deps.py b/jenkins/databricks/install_deps.py index be5cb9bc040..8d21a4f9556 100644 --- a/jenkins/databricks/install_deps.py +++ b/jenkins/databricks/install_deps.py @@ -115,8 +115,10 @@ def define_deps(spark_version, scala_version): f'{prefix_ws_sp_mvn_hadoop}--org.json4s--json4s-jackson_{scala_version}--org.json4s__json4s-jackson_{scala_version}__*.jar'), Artifact('org.javaassist', 'javaassist', f'{prefix_ws_sp_mvn_hadoop}--org.javassist--javassist--org.javassist__javassist__*.jar'), - Artifact('com.fasterxml.jackson.core', 'jackson-core', + Artifact('com.fasterxml.jackson.core', 'jackson-databind', f'{prefix_ws_sp_mvn_hadoop}--com.fasterxml.jackson.core--jackson-databind--com.fasterxml.jackson.core__jackson-databind__*.jar'), + Artifact('com.fasterxml.jackson.core', 'jackson-core', + f'{prefix_ws_sp_mvn_hadoop}--com.fasterxml.jackson.core--jackson-core--com.fasterxml.jackson.core__jackson-core__*.jar'), Artifact('com.fasterxml.jackson.core', 'jackson-annotations', f'{prefix_ws_sp_mvn_hadoop}--com.fasterxml.jackson.core--jackson-annotations--com.fasterxml.jackson.core__jackson-annotations__*.jar'), Artifact('org.apache.spark', f'spark-avro_{scala_version}', diff --git a/scala2.13/shim-deps/databricks/pom.xml b/scala2.13/shim-deps/databricks/pom.xml index b342f381c71..a0459901079 100644 --- a/scala2.13/shim-deps/databricks/pom.xml +++ b/scala2.13/shim-deps/databricks/pom.xml @@ -105,6 +105,12 @@ ${spark.version} compile + + com.fasterxml.jackson.core + jackson-databind + ${spark.version} + compile + com.fasterxml.jackson.core jackson-annotations @@ -286,4 +292,4 @@ compile - \ No newline at end of file + diff --git a/shim-deps/databricks/pom.xml b/shim-deps/databricks/pom.xml index bef8a90d227..22842b0f7c0 100644 --- a/shim-deps/databricks/pom.xml +++ b/shim-deps/databricks/pom.xml @@ -105,6 +105,12 @@ ${spark.version} compile + + com.fasterxml.jackson.core + jackson-databind + ${spark.version} + compile + com.fasterxml.jackson.core jackson-annotations @@ -286,4 +292,4 @@ compile - \ No newline at end of file + From eb1549c5764e2ceaaac1d9f7f885c0d6c11ae92c Mon Sep 17 00:00:00 2001 From: Raza Jafri Date: Thu, 13 Jun 2024 14:23:57 -0700 Subject: [PATCH 45/68] `binary-dedupe` changes for Spark 4.0.0 [databricks] (#10993) * Binary dedupe changes for Spark 4.0.0 Signed-off-by: Raza Jafri * updated comments * Changed the URL for the common classes among shims * renamed spark34-common to spark-shared and renamed relevant variables * addressed review comments * renamed variable from common to shared --------- Signed-off-by: Raza Jafri --- dist/maven-antrun/build-parallel-worlds.xml | 4 +- dist/scripts/binary-dedupe.sh | 58 +++++++++---------- .../com/nvidia/spark/rapids/ShimLoader.scala | 10 ++-- 3 files changed, 36 insertions(+), 36 deletions(-) diff --git a/dist/maven-antrun/build-parallel-worlds.xml b/dist/maven-antrun/build-parallel-worlds.xml index 524b15addf9..07838616340 100644 --- a/dist/maven-antrun/build-parallel-worlds.xml +++ b/dist/maven-antrun/build-parallel-worlds.xml @@ -1,6 +1,6 @@ - diff --git a/dist/scripts/binary-dedupe.sh b/dist/scripts/binary-dedupe.sh index 183e86b1524..356b0b4dbae 100755 --- a/dist/scripts/binary-dedupe.sh +++ b/dist/scripts/binary-dedupe.sh @@ -1,6 +1,6 @@ #!/bin/bash -# Copyright (c) 2021-2023, NVIDIA CORPORATION. +# Copyright (c) 2021-2024, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -34,10 +34,10 @@ case "$OSTYPE" in esac STEP=0 -export SPARK3XX_COMMON_TXT="$PWD/spark3xx-common.txt" -export SPARK3XX_COMMON_COPY_LIST="$PWD/spark-common-copy-list.txt" +export SPARK_SHARED_TXT="$PWD/spark-shared.txt" +export SPARK_SHARED_COPY_LIST="$PWD/spark-shared-copy-list.txt" export DELETE_DUPLICATES_TXT="$PWD/delete-duplicates.txt" -export SPARK3XX_COMMON_DIR="$PWD/spark3xx-common" +export SPARK_SHARED_DIR="$PWD/spark-shared" # This script de-duplicates .class files at the binary level. # We could also diff classes using scalap / javap outputs. @@ -47,17 +47,17 @@ export SPARK3XX_COMMON_DIR="$PWD/spark3xx-common" # The following pipeline determines identical classes across shims in this build. # - checksum all class files -# - move the varying-prefix spark3xy to the left so it can be easily skipped for uniq and sort +# - move the varying-prefix sparkxyz to the left so it can be easily skipped for uniq and sort # - sort by path, secondary sort by checksum, print one line per group # - produce uniq count for paths # - filter the paths with count=1, the class files without diverging checksums -# - put the path starting with /spark3xy back together for the final list +# - put the path starting with /sparkxyz back together for the final list echo "Retrieving class files hashing to a single value ..." echo "$((++STEP))/ SHA1 of all non-META files > tmp-sha1-files.txt" -find ./parallel-world/spark3* -name META-INF -prune -o \( -type f -print \) | \ - xargs $SHASUM > tmp-sha1-files.txt +find ./parallel-world/spark[34]* -name META-INF -prune -o -name webapps -prune -o \( -type f -print0 \) | \ + xargs --null $SHASUM > tmp-sha1-files.txt echo "$((++STEP))/ make shim column 1 > tmp-shim-sha-package-files.txt" < tmp-sha1-files.txt awk -F/ '$1=$1' | \ @@ -68,10 +68,10 @@ echo "$((++STEP))/ sort by path, sha1; output first from each group > tmp-count- sort -k3 -k2,2 -u tmp-shim-sha-package-files.txt | \ uniq -f 2 -c > tmp-count-shim-sha-package-files.txt -echo "$((++STEP))/ files with unique sha1 > $SPARK3XX_COMMON_TXT" +echo "$((++STEP))/ files with unique sha1 > $SPARK_SHARED_TXT" grep '^\s\+1 .*' tmp-count-shim-sha-package-files.txt | \ awk '{$1=""; $3=""; print $0 }' | \ - tr -s ' ' | sed 's/\ /\//g' > "$SPARK3XX_COMMON_TXT" + tr -s ' ' | sed 's/\ /\//g' > "$SPARK_SHARED_TXT" function retain_single_copy() { set -e @@ -93,10 +93,10 @@ function retain_single_copy() { package_class="${package_class_with_spaces// //}" # get the reference copy out of the way - echo "$package_class" >> "from-$shim-to-spark3xx-common.txt" + echo "$package_class" >> "from-$shim-to-spark-shared.txt" # expanding directories separately because full path # glob is broken for class file name including the "$" character - for pw in ./parallel-world/spark3* ; do + for pw in ./parallel-world/spark[34]* ; do delete_path="$pw/$package_class" [[ -f "$delete_path" ]] && echo "$delete_path" || true done >> "$DELETE_DUPLICATES_TXT" || exit 255 @@ -106,26 +106,26 @@ function retain_single_copy() { # standalone debugging # truncate incremental files : > "$DELETE_DUPLICATES_TXT" -rm -f from-spark3*-to-spark3xx-common.txt -rm -rf "$SPARK3XX_COMMON_DIR" -mkdir -p "$SPARK3XX_COMMON_DIR" +rm -f from-spark[34]*-to-spark-shared.txt +rm -rf "$SPARK_SHARED_DIR" +mkdir -p "$SPARK_SHARED_DIR" -echo "$((++STEP))/ retaining a single copy of spark3xx-common classes" +echo "$((++STEP))/ retaining a single copy of spark-shared classes" while read spark_common_class; do retain_single_copy "$spark_common_class" -done < "$SPARK3XX_COMMON_TXT" +done < "$SPARK_SHARED_TXT" -echo "$((++STEP))/ rsyncing common classes to $SPARK3XX_COMMON_DIR" -for copy_list in from-spark3*-to-spark3xx-common.txt; do +echo "$((++STEP))/ rsyncing common classes to $SPARK_SHARED_DIR" +for copy_list in from-spark[34]*-to-spark-shared.txt; do echo Initializing rsync of "$copy_list" IFS='-' <<< "$copy_list" read -ra copy_list_parts # declare -p copy_list_parts shim="${copy_list_parts[1]}" # use rsync to reduce process forking - rsync --files-from="$copy_list" ./parallel-world/"$shim" "$SPARK3XX_COMMON_DIR" + rsync --files-from="$copy_list" ./parallel-world/"$shim" "$SPARK_SHARED_DIR" done -mv "$SPARK3XX_COMMON_DIR" parallel-world/ +mv "$SPARK_SHARED_DIR" parallel-world/ # TODO further dedupe by FEATURE version lines: # spark30x-common @@ -137,9 +137,9 @@ mv "$SPARK3XX_COMMON_DIR" parallel-world/ # # At this point the duplicate classes have not been removed from version-specific jar # locations such as parallel-world/spark312. -# For each unshimmed class file look for all of its copies inside /spark3* and +# For each unshimmed class file look for all of its copies inside /spark[34]* and # and count the number of distinct checksums. There are two representative cases -# 1) The class is contributed to the unshimmed location via the unshimmed-from-each-spark3xx list. These are classes +# 1) The class is contributed to the unshimmed location via the unshimmed-from-each-spark34 list. These are classes # carrying the shim classifier in their package name such as # com.nvidia.spark.rapids.spark312.RapidsShuffleManager. They are unique by construction, # and will have zero copies in any non-spark312 shims. Although such classes are currently excluded from @@ -157,25 +157,25 @@ mv "$SPARK3XX_COMMON_DIR" parallel-world/ # Determine the list of unshimmed class files UNSHIMMED_LIST_TXT=unshimmed-result.txt echo "$((++STEP))/ creating sorted list of unshimmed classes > $UNSHIMMED_LIST_TXT" -find ./parallel-world -name '*.class' -not -path './parallel-world/spark3*' | \ +find ./parallel-world -name '*.class' -not -path './parallel-world/spark[34-]*' | \ cut -d/ -f 3- | sort > "$UNSHIMMED_LIST_TXT" function verify_same_sha_for_unshimmed() { set -e class_file="$1" - # the raw spark3xx-common.txt file list contains all single-sha1 classes + # the raw spark-shared.txt file list contains all single-sha1 classes # including the ones that are unshimmed. Instead of expensively recomputing # sha1 look up if there is an entry with the unshimmed class as a suffix class_file_quoted=$(printf '%q' "$class_file") - # TODO currently RapidsShuffleManager is "removed" from /spark3* by construction in + # TODO currently RapidsShuffleManager is "removed" from /spark* by construction in # dist pom.xml via ant. We could delegate this logic to this script # and make both simmpler - if [[ ! "$class_file_quoted" =~ (com/nvidia/spark/rapids/spark3.*/.*ShuffleManager.class|org/apache/spark/sql/rapids/shims/spark3.*/ProxyRapidsShuffleInternalManager.class) ]]; then + if [[ ! "$class_file_quoted" =~ (com/nvidia/spark/rapids/spark[34].*/.*ShuffleManager.class|org/apache/spark/sql/rapids/shims/spark[34].*/ProxyRapidsShuffleInternalManager.class) ]]; then - if ! grep -q "/spark.\+/$class_file_quoted" "$SPARK3XX_COMMON_TXT"; then + if ! grep -q "/spark.\+/$class_file_quoted" "$SPARK_SHARED_TXT"; then echo >&2 "$class_file is not bitwise-identical across shims" exit 255 fi @@ -192,7 +192,7 @@ done < "$UNSHIMMED_LIST_TXT" echo "$((++STEP))/ removing duplicates of unshimmed classes" while read unshimmed_class; do - for pw in ./parallel-world/spark3* ; do + for pw in ./parallel-world/spark[34]* ; do unshimmed_path="$pw/$unshimmed_class" [[ -f "$unshimmed_path" ]] && echo "$unshimmed_path" || true done >> "$DELETE_DUPLICATES_TXT" diff --git a/sql-plugin-api/src/main/scala/com/nvidia/spark/rapids/ShimLoader.scala b/sql-plugin-api/src/main/scala/com/nvidia/spark/rapids/ShimLoader.scala index 36abc75ba87..2d7a51c4e43 100644 --- a/sql-plugin-api/src/main/scala/com/nvidia/spark/rapids/ShimLoader.scala +++ b/sql-plugin-api/src/main/scala/com/nvidia/spark/rapids/ShimLoader.scala @@ -40,19 +40,19 @@ import org.apache.spark.util.MutableURLClassLoader "parallel worlds" in the JDK's com.sun.istack.internal.tools.ParallelWorldClassLoader parlance 1. a few publicly documented classes in the conventional layout at the top 2. a large fraction of classes whose bytecode is identical under all supported Spark versions - in spark3xx-common + in spark-shared 3. a smaller fraction of classes that differ under one of the supported Spark versions com/nvidia/spark/SQLPlugin.class - spark3xx-common/com/nvidia/spark/rapids/CastExprMeta.class + spark-shared/com/nvidia/spark/rapids/CastExprMeta.class spark311/org/apache/spark/sql/rapids/GpuUnaryMinus.class spark320/org/apache/spark/sql/rapids/GpuUnaryMinus.class Each shim can see a consistent parallel world without conflicts by referencing only one conflicting directory. E.g., Spark 3.2.0 Shim will use - jar:file:/home/spark/rapids-4-spark_2.12-24.08.0.jar!/spark3xx-common/ + jar:file:/home/spark/rapids-4-spark_2.12-24.08.0.jar!/spark-shared/ jar:file:/home/spark/rapids-4-spark_2.12-24.08.0.jar!/spark320/ Spark 3.1.1 will use - jar:file:/home/spark/rapids-4-spark_2.12-24.08.0.jar!/spark3xx-common/ + jar:file:/home/spark/rapids-4-spark_2.12-24.08.0.jar!/spark-shared/ jar:file:/home/spark/rapids-4-spark_2.12-24.08.0.jar!/spark311/ Using these Jar URL's allows referencing different bytecode produced from identical sources by incompatible Scala / Spark dependencies. @@ -67,7 +67,7 @@ object ShimLoader extends Logging { new URL(rootUrlStr) } - private val shimCommonURL = new URL(s"${shimRootURL.toString}spark3xx-common/") + private val shimCommonURL = new URL(s"${shimRootURL.toString}spark-shared/") @volatile private var shimProviderClass: String = _ @volatile private var shimProvider: SparkShimServiceProvider = _ @volatile private var shimURL: URL = _ From 356d5a1a5021c421cef7fbb73734527ba0ac1857 Mon Sep 17 00:00:00 2001 From: Renjie Liu Date: Fri, 14 Jun 2024 08:45:28 +0800 Subject: [PATCH 46/68] [FEA] Increase parallelism of deltalake test on databricks (#11051) --- integration_tests/run_pyspark_from_build.sh | 6 ++++++ jenkins/databricks/test.sh | 5 +---- 2 files changed, 7 insertions(+), 4 deletions(-) diff --git a/integration_tests/run_pyspark_from_build.sh b/integration_tests/run_pyspark_from_build.sh index dec93e6f22a..8b10b3debac 100755 --- a/integration_tests/run_pyspark_from_build.sh +++ b/integration_tests/run_pyspark_from_build.sh @@ -245,6 +245,12 @@ else DRIVER_EXTRA_JAVA_OPTIONS="-ea -Duser.timezone=$TZ -Ddelta.log.cacheSize=$deltaCacheSize" export PYSP_TEST_spark_driver_extraJavaOptions="$DRIVER_EXTRA_JAVA_OPTIONS $COVERAGE_SUBMIT_FLAGS" export PYSP_TEST_spark_executor_extraJavaOptions="-ea -Duser.timezone=$TZ" + + # Set driver memory to speed up tests such as deltalake + if [[ -n "${DRIVER_MEMORY}" ]]; then + export PYSP_TEST_spark_driver_memory="${DRIVER_MEMORY}" + fi + export PYSP_TEST_spark_ui_showConsoleProgress='false' export PYSP_TEST_spark_sql_session_timeZone=$TZ export PYSP_TEST_spark_sql_shuffle_partitions='4' diff --git a/jenkins/databricks/test.sh b/jenkins/databricks/test.sh index f71f69844f7..c966d5a92f7 100755 --- a/jenkins/databricks/test.sh +++ b/jenkins/databricks/test.sh @@ -66,9 +66,6 @@ TEST_MODE=${TEST_MODE:-'DEFAULT'} # --packages in distributed setups, should be fixed by # https://github.com/NVIDIA/spark-rapids/pull/5646 -# Increase driver memory as Delta Lake tests can slowdown with default 1G (possibly due to caching?) -DELTA_LAKE_CONFS="--driver-memory 2g" - # Enable event log for qualification & profiling tools testing export PYSP_TEST_spark_eventLog_enabled=true mkdir -p /tmp/spark-events @@ -105,7 +102,7 @@ if [[ "$(pwd)" == "$SOURCE_PATH" ]]; then if [[ "$TEST_MODE" == "DEFAULT" || $TEST_MODE == "CI_PART2" || "$TEST_MODE" == "DELTA_LAKE_ONLY" ]]; then ## Run Delta Lake tests - SPARK_SUBMIT_FLAGS="$SPARK_CONF $DELTA_LAKE_CONFS" TEST_PARALLEL=1 \ + DRIVER_MEMORY="4g" \ bash integration_tests/run_pyspark_from_build.sh --runtime_env="databricks" -m "delta_lake" --delta_lake --test_type=$TEST_TYPE fi From 599ae17e02c8b88c63fe0133caf19570f332092e Mon Sep 17 00:00:00 2001 From: "Hongbin Ma (Mahone)" Date: Fri, 14 Jun 2024 09:27:03 +0800 Subject: [PATCH 47/68] fix flaky array_item test failures (#11054) * fix flaky array_item test failures Signed-off-by: Hongbin Ma (Mahone) * fix indent Signed-off-by: Hongbin Ma (Mahone) * fix whitespace Signed-off-by: Hongbin Ma (Mahone) --------- Signed-off-by: Hongbin Ma (Mahone) --- integration_tests/src/main/python/data_gen.py | 23 ++++++++++++------- .../src/main/python/parquet_write_test.py | 6 ++++- 2 files changed, 20 insertions(+), 9 deletions(-) diff --git a/integration_tests/src/main/python/data_gen.py b/integration_tests/src/main/python/data_gen.py index 2e6c36b77d9..fb1627af75b 100644 --- a/integration_tests/src/main/python/data_gen.py +++ b/integration_tests/src/main/python/data_gen.py @@ -159,7 +159,8 @@ def __repr__(self): return super().__repr__() + '(' + str(self._child_gen) + ')' def _cache_repr(self): - return super()._cache_repr() + '(' + self._child_gen._cache_repr() + ')' + return (super()._cache_repr() + '(' + self._child_gen._cache_repr() + + ',' + str(self._func.__code__) + ')' ) def start(self, rand): self._child_gen.start(rand) @@ -667,7 +668,10 @@ def __repr__(self): return super().__repr__() + '(' + str(self._child_gen) + ')' def _cache_repr(self): - return super()._cache_repr() + '(' + self._child_gen._cache_repr() + ')' + return (super()._cache_repr() + '(' + self._child_gen._cache_repr() + + ',' + str(self._min_length) + ',' + str(self._max_length) + ',' + + str(self.all_null) + ',' + str(self.convert_to_tuple) + ')') + def start(self, rand): self._child_gen.start(rand) @@ -701,7 +705,8 @@ def __repr__(self): return super().__repr__() + '(' + str(self._key_gen) + ',' + str(self._value_gen) + ')' def _cache_repr(self): - return super()._cache_repr() + '(' + self._key_gen._cache_repr() + ',' + self._value_gen._cache_repr() + ')' + return (super()._cache_repr() + '(' + self._key_gen._cache_repr() + ',' + self._value_gen._cache_repr() + + ',' + str(self._min_length) + ',' + str(self._max_length) + ')') def start(self, rand): self._key_gen.start(rand) @@ -769,12 +774,13 @@ def __init__(self, min_value=MIN_DAY_TIME_INTERVAL, max_value=MAX_DAY_TIME_INTER self._min_micros = (math.floor(min_value.total_seconds()) * 1000000) + min_value.microseconds self._max_micros = (math.floor(max_value.total_seconds()) * 1000000) + max_value.microseconds fields = ["day", "hour", "minute", "second"] - start_index = fields.index(start_field) - end_index = fields.index(end_field) - if start_index > end_index: + self._start_index = fields.index(start_field) + self._end_index = fields.index(end_field) + if self._start_index > self._end_index: raise RuntimeError('Start field {}, end field {}, valid fields is {}, start field index should <= end ' 'field index'.format(start_field, end_field, fields)) - super().__init__(DayTimeIntervalType(start_index, end_index), nullable=nullable, special_cases=special_cases) + super().__init__(DayTimeIntervalType(self._start_index, self._end_index), nullable=nullable, + special_cases=special_cases) def _gen_random(self, rand): micros = rand.randint(self._min_micros, self._max_micros) @@ -784,7 +790,8 @@ def _gen_random(self, rand): return timedelta(microseconds=micros) def _cache_repr(self): - return super()._cache_repr() + '(' + str(self._min_micros) + ',' + str(self._max_micros) + ')' + return (super()._cache_repr() + '(' + str(self._min_micros) + ',' + str(self._max_micros) + + ',' + str(self._start_index) + ',' + str(self._end_index) + ')') def start(self, rand): self._start(rand, lambda: self._gen_random(rand)) diff --git a/integration_tests/src/main/python/parquet_write_test.py b/integration_tests/src/main/python/parquet_write_test.py index 99a2d4241e8..38dab9e84a4 100644 --- a/integration_tests/src/main/python/parquet_write_test.py +++ b/integration_tests/src/main/python/parquet_write_test.py @@ -1,4 +1,4 @@ -# Copyright (c) 2020-2023, NVIDIA CORPORATION. +# Copyright (c) 2020-2024, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -224,6 +224,10 @@ def test_all_null_int96(spark_tmp_path): class AllNullTimestampGen(TimestampGen): def start(self, rand): self._start(rand, lambda : None) + + def _cache_repr(self): + return super()._cache_repr() + '(all_nulls)' + data_path = spark_tmp_path + '/PARQUET_DATA' confs = copy_and_update(writer_confs, {'spark.sql.parquet.outputTimestampType': 'INT96'}) assert_gpu_and_cpu_writes_are_equal_collect( From 2f3c0c276dba3e8870f3e3bfa9287e644809f4f6 Mon Sep 17 00:00:00 2001 From: Tim Liu Date: Fri, 14 Jun 2024 19:47:56 +0800 Subject: [PATCH 48/68] Calculate parallelism to speed up pre-merge CI (#11046) * Calculate parallelism to speed up pre-merge CI Calculate parallelism based on GPU memory to speed up pre-merge CI with appropriate amount of parallelism. But when TEST_PARALLEL > 8 and as it increases, the integration tests running speed will become slower and slower, so we limit TEST_PARALLEL <= 8. Based on this change, and ran pre-merge CI on powerful nodes, we observed the pre-merge CI 1 hour less than on common nodes. 16 CPU/128G Mem/24G GPU : [2hours] VS 8 CPU/64G Mem/16G GPU : [3hours] Note: currently we only have 3 fixed powerful nodes for the pre-merge CI job, so only 1 pre-merge CI be speeded up at the same time Signed-off-by: Tim Liu * Add a variable to set maximum test parallelism for the integration tests Signed-off-by: Tim Liu * Fix typo Signed-off-by: Tim Liu --------- Signed-off-by: Tim Liu --- integration_tests/run_pyspark_from_build.sh | 5 +++++ jenkins/spark-premerge-build.sh | 4 +--- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/integration_tests/run_pyspark_from_build.sh b/integration_tests/run_pyspark_from_build.sh index 8b10b3debac..18c26aa26e7 100755 --- a/integration_tests/run_pyspark_from_build.sh +++ b/integration_tests/run_pyspark_from_build.sh @@ -171,11 +171,16 @@ else TEST_TYPE_PARAM="--test_type $TEST_TYPE" fi + # We found that when parallelism > 8, as it increases, the test speed will become slower and slower. So we set the default maximum parallelism to 8. + # Note that MAX_PARALLEL varies with the hardware, OS, and test case. Please overwrite it with an appropriate value if needed. + MAX_PARALLEL=${MAX_PARALLEL:-8} if [[ ${TEST_PARALLEL} -lt 2 ]]; then # With xdist 0 and 1 are the same parallelism but # 0 is more efficient TEST_PARALLEL_OPTS=() + elif [[ ${TEST_PARALLEL} -gt ${MAX_PARALLEL} ]]; then + TEST_PARALLEL_OPTS=("-n" "$MAX_PARALLEL") else TEST_PARALLEL_OPTS=("-n" "$TEST_PARALLEL") fi diff --git a/jenkins/spark-premerge-build.sh b/jenkins/spark-premerge-build.sh index 883b3f3acfc..697722c0138 100755 --- a/jenkins/spark-premerge-build.sh +++ b/jenkins/spark-premerge-build.sh @@ -78,7 +78,7 @@ mvn_verify() { # Here run Python integration tests tagged with 'premerge_ci_1' only, that would help balance test duration and memory # consumption from two k8s pods running in parallel, which executes 'mvn_verify()' and 'ci_2()' respectively. $MVN_CMD -B $MVN_URM_MIRROR $PREMERGE_PROFILES clean verify -Dpytest.TEST_TAGS="premerge_ci_1" \ - -Dpytest.TEST_TYPE="pre-commit" -Dpytest.TEST_PARALLEL=4 -Dcuda.version=$CLASSIFIER + -Dpytest.TEST_TYPE="pre-commit" -Dcuda.version=$CLASSIFIER # The jacoco coverage should have been collected, but because of how the shade plugin # works and jacoco we need to clean some things up so jacoco will only report for the @@ -162,7 +162,6 @@ ci_2() { $MVN_CMD -U -B $MVN_URM_MIRROR clean package $MVN_BUILD_ARGS -DskipTests=true export TEST_TAGS="not premerge_ci_1" export TEST_TYPE="pre-commit" - export TEST_PARALLEL=5 # Download a Scala 2.12 build of spark prepare_spark $SPARK_VER 2.12 @@ -206,7 +205,6 @@ ci_scala213() { cd .. # Run integration tests in the project root dir to leverage test cases and resource files export TEST_TAGS="not premerge_ci_1" export TEST_TYPE="pre-commit" - export TEST_PARALLEL=5 # SPARK_HOME (and related) must be set to a Spark built with Scala 2.13 SPARK_HOME=$SPARK_HOME PYTHONPATH=$PYTHONPATH \ ./integration_tests/run_pyspark_from_build.sh From 6eb854dc82a4ec0e96b97871a97c6b9ed7471722 Mon Sep 17 00:00:00 2001 From: Peixin Date: Mon, 17 Jun 2024 18:12:49 +0800 Subject: [PATCH 49/68] WAR numpy2 failed fastparquet compatibility issue (#11072) Signed-off-by: Peixin Li --- .../src/main/python/fastparquet_compatibility_test.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/integration_tests/src/main/python/fastparquet_compatibility_test.py b/integration_tests/src/main/python/fastparquet_compatibility_test.py index 53a99d32bd2..4b0fc2827f4 100644 --- a/integration_tests/src/main/python/fastparquet_compatibility_test.py +++ b/integration_tests/src/main/python/fastparquet_compatibility_test.py @@ -30,6 +30,8 @@ def fastparquet_unavailable(): return False except ImportError: return True + except ValueError: # TODO: remove when https://github.com/NVIDIA/spark-rapids/issues/11070 is fixed + return True rebase_write_corrected_conf = { From 0952dea254df6fc4f1f01a9e0e8ac50f97285233 Mon Sep 17 00:00:00 2001 From: Haoyang Li Date: Tue, 18 Jun 2024 14:24:16 +0800 Subject: [PATCH 50/68] Fallback non-UTC TimeZoneAwareExpression with zoneId [databricks] (#10996) * Fallback non-UTC TimeZoneAwareExpression with zoneId instead of timeZone config Signed-off-by: Haoyang Li * clean up Signed-off-by: Haoyang Li --------- Signed-off-by: Haoyang Li --- .../main/scala/com/nvidia/spark/rapids/RapidsMeta.scala | 8 ++++---- .../spark/sql/rapids/utils/RapidsTestSettings.scala | 2 -- 2 files changed, 4 insertions(+), 6 deletions(-) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala index a876ea6c9e0..984892cd787 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019-2023, NVIDIA CORPORATION. + * Copyright (c) 2019-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -1123,7 +1123,7 @@ abstract class BaseExprMeta[INPUT <: Expression]( if (!needTimeZoneCheck) return // Level 2 check - if (!isTimeZoneSupported) return checkUTCTimezone(this) + if (!isTimeZoneSupported) return checkUTCTimezone(this, getZoneId()) // Level 3 check val zoneId = getZoneId() @@ -1203,8 +1203,8 @@ abstract class BaseExprMeta[INPUT <: Expression]( * * @param meta to check whether it's UTC */ - def checkUTCTimezone(meta: RapidsMeta[_, _, _]): Unit = { - if (!GpuOverrides.isUTCTimezone()) { + def checkUTCTimezone(meta: RapidsMeta[_, _, _], zoneId: ZoneId): Unit = { + if (!GpuOverrides.isUTCTimezone(zoneId)) { meta.willNotWorkOnGpu( TimeZoneDB.nonUTCTimezoneNotSupportedStr(meta.wrapped.getClass.toString)) } diff --git a/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/utils/RapidsTestSettings.scala b/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/utils/RapidsTestSettings.scala index 4cf155041d9..63649376829 100644 --- a/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/utils/RapidsTestSettings.scala +++ b/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/utils/RapidsTestSettings.scala @@ -48,11 +48,9 @@ class RapidsTestSettings extends BackendTestSettings { .exclude("from_json - input=empty array, schema=struct, output=single row with null", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10849")) .exclude("from_json - input=empty object, schema=struct, output=single row with null", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10849")) .exclude("SPARK-20549: from_json bad UTF-8", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10849")) - .exclude("from_json with timestamp", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10849")) .exclude("to_json - array", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10849")) .exclude("to_json - array with single empty row", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10849")) .exclude("to_json - empty array", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10849")) - .exclude("to_json with timestamp", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10849")) .exclude("SPARK-21513: to_json support map[string, struct] to json", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10849")) .exclude("SPARK-21513: to_json support map[struct, struct] to json", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10849")) .exclude("SPARK-21513: to_json support map[string, integer] to json", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10849")) From 7bac3a6439c10efb1961d3c4ba028128d9dca249 Mon Sep 17 00:00:00 2001 From: Renjie Liu Date: Wed, 19 Jun 2024 09:44:48 +0800 Subject: [PATCH 51/68] [FEA] Introduce low shuffle merge. (#10979) * feat: Introduce low shuffle merge. Signed-off-by: liurenjie1024 * fix * Test databricks parallel * Test more databricks parallel * Fix comments * Config && scala 2.13 * Revert * Fix comments * scala 2.13 * Revert unnecessary changes * Revert "Revert unnecessary changes" This reverts commit 9fa4cf268cc3fce4d2732e04cb33eb53e4859c99. * restore change --------- Signed-off-by: liurenjie1024 --- aggregator/pom.xml | 4 + .../GpuDeltaParquetFileFormatUtils.scala | 160 +++ .../nvidia/spark/rapids/delta/deltaUDFs.scala | 83 +- .../delta/delta24x/Delta24xProvider.scala | 5 +- .../GpuDelta24xParquetFileFormat.scala | 61 +- .../delta/delta24x/MergeIntoCommandMeta.scala | 58 +- .../delta24x/GpuLowShuffleMergeCommand.scala | 1084 +++++++++++++++++ .../rapids/GpuLowShuffleMergeCommand.scala | 1083 ++++++++++++++++ .../delta/GpuDeltaParquetFileFormat.scala | 63 +- .../shims/MergeIntoCommandMetaShim.scala | 101 +- .../advanced_configs.md | 6 + .../delta_lake_low_shuffle_merge_test.py | 165 +++ .../main/python/delta_lake_merge_common.py | 155 +++ .../src/main/python/delta_lake_merge_test.py | 127 +- pom.xml | 10 + scala2.13/aggregator/pom.xml | 4 + scala2.13/pom.xml | 10 + scala2.13/sql-plugin/pom.xml | 4 + sql-plugin/pom.xml | 4 + .../com/nvidia/spark/rapids/RapidsConf.scala | 28 + 20 files changed, 3061 insertions(+), 154 deletions(-) create mode 100644 delta-lake/common/src/main/scala/com/nvidia/spark/rapids/delta/GpuDeltaParquetFileFormatUtils.scala create mode 100644 delta-lake/delta-24x/src/main/scala/org/apache/spark/sql/delta/rapids/delta24x/GpuLowShuffleMergeCommand.scala create mode 100644 delta-lake/delta-spark341db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuLowShuffleMergeCommand.scala create mode 100644 integration_tests/src/main/python/delta_lake_low_shuffle_merge_test.py create mode 100644 integration_tests/src/main/python/delta_lake_merge_common.py diff --git a/aggregator/pom.xml b/aggregator/pom.xml index 22bfe11105e..8cf881419c9 100644 --- a/aggregator/pom.xml +++ b/aggregator/pom.xml @@ -94,6 +94,10 @@ com.google.flatbuffers ${rapids.shade.package}.com.google.flatbuffers + + org.roaringbitmap + ${rapids.shade.package}.org.roaringbitmap + diff --git a/delta-lake/common/src/main/scala/com/nvidia/spark/rapids/delta/GpuDeltaParquetFileFormatUtils.scala b/delta-lake/common/src/main/scala/com/nvidia/spark/rapids/delta/GpuDeltaParquetFileFormatUtils.scala new file mode 100644 index 00000000000..101a82da830 --- /dev/null +++ b/delta-lake/common/src/main/scala/com/nvidia/spark/rapids/delta/GpuDeltaParquetFileFormatUtils.scala @@ -0,0 +1,160 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed 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 com.nvidia.spark.rapids.delta + +import ai.rapids.cudf.{ColumnVector => CudfColumnVector, Scalar, Table} +import com.nvidia.spark.rapids.Arm.{closeOnExcept, withResource} +import com.nvidia.spark.rapids.GpuColumnVector +import org.roaringbitmap.longlong.{PeekableLongIterator, Roaring64Bitmap} + +import org.apache.spark.sql.types.{BooleanType, LongType, StringType, StructField, StructType} +import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} + + +object GpuDeltaParquetFileFormatUtils { + /** + * Row number of the row in the file. When used with [[FILE_PATH_COL]] together, it can be used + * as unique id of a row in file. Currently to correctly calculate this, the caller needs to + * set both [[isSplitable]] to false, and [[RapidsConf.PARQUET_READER_TYPE]] to "PERFILE". + */ + val METADATA_ROW_IDX_COL: String = "__metadata_row_index" + val METADATA_ROW_IDX_FIELD: StructField = StructField(METADATA_ROW_IDX_COL, LongType, + nullable = false) + + val METADATA_ROW_DEL_COL: String = "__metadata_row_del" + val METADATA_ROW_DEL_FIELD: StructField = StructField(METADATA_ROW_DEL_COL, BooleanType, + nullable = false) + + + /** + * File path of the file that the row came from. + */ + val FILE_PATH_COL: String = "_metadata_file_path" + val FILE_PATH_FIELD: StructField = StructField(FILE_PATH_COL, StringType, nullable = false) + + /** + * Add a metadata column to the iterator. Currently only support [[METADATA_ROW_IDX_COL]]. + */ + def addMetadataColumnToIterator( + schema: StructType, + delVector: Option[Roaring64Bitmap], + input: Iterator[ColumnarBatch], + maxBatchSize: Int): Iterator[ColumnarBatch] = { + val metadataRowIndexCol = schema.fieldNames.indexOf(METADATA_ROW_IDX_COL) + val delRowIdx = schema.fieldNames.indexOf(METADATA_ROW_DEL_COL) + if (metadataRowIndexCol == -1 && delRowIdx == -1) { + return input + } + var rowIndex = 0L + input.map { batch => + withResource(batch) { _ => + val rowIdxCol = if (metadataRowIndexCol == -1) { + None + } else { + Some(metadataRowIndexCol) + } + + val delRowIdx2 = if (delRowIdx == -1) { + None + } else { + Some(delRowIdx) + } + val newBatch = addMetadataColumns(rowIdxCol, delRowIdx2, delVector,maxBatchSize, + rowIndex, batch) + rowIndex += batch.numRows() + newBatch + } + } + } + + private def addMetadataColumns( + rowIdxPos: Option[Int], + delRowIdx: Option[Int], + delVec: Option[Roaring64Bitmap], + maxBatchSize: Int, + rowIdxStart: Long, + batch: ColumnarBatch): ColumnarBatch = { + val rowIdxCol = rowIdxPos.map { _ => + withResource(Scalar.fromLong(rowIdxStart)) { start => + GpuColumnVector.from(CudfColumnVector.sequence(start, batch.numRows()), + METADATA_ROW_IDX_FIELD.dataType) + } + } + + closeOnExcept(rowIdxCol) { rowIdxCol => + + val delVecCol = delVec.map { delVec => + withResource(Scalar.fromBool(false)) { s => + withResource(CudfColumnVector.fromScalar(s, batch.numRows())) { c => + var table = new Table(c) + val posIter = new RoaringBitmapIterator( + delVec.getLongIteratorFrom(rowIdxStart), + rowIdxStart, + rowIdxStart + batch.numRows(), + ).grouped(Math.min(maxBatchSize, batch.numRows())) + + for (posChunk <- posIter) { + withResource(CudfColumnVector.fromLongs(posChunk: _*)) { poses => + withResource(Scalar.fromBool(true)) { s => + table = withResource(table) { _ => + Table.scatter(Array(s), poses, table) + } + } + } + } + + withResource(table) { _ => + GpuColumnVector.from(table.getColumn(0).incRefCount(), + METADATA_ROW_DEL_FIELD.dataType) + } + } + } + } + + closeOnExcept(delVecCol) { delVecCol => + // Replace row_idx column + val columns = new Array[ColumnVector](batch.numCols()) + for (i <- 0 until batch.numCols()) { + if (rowIdxPos.contains(i)) { + columns(i) = rowIdxCol.get + } else if (delRowIdx.contains(i)) { + columns(i) = delVecCol.get + } else { + columns(i) = batch.column(i) match { + case gpuCol: GpuColumnVector => gpuCol.incRefCount() + case col => col + } + } + } + + new ColumnarBatch(columns, batch.numRows()) + } + } + } +} + +class RoaringBitmapIterator(val inner: PeekableLongIterator, val start: Long, val end: Long) + extends Iterator[Long] { + + override def hasNext: Boolean = { + inner.hasNext && inner.peekNext() < end + } + + override def next(): Long = { + inner.next() - start + } +} diff --git a/delta-lake/common/src/main/scala/com/nvidia/spark/rapids/delta/deltaUDFs.scala b/delta-lake/common/src/main/scala/com/nvidia/spark/rapids/delta/deltaUDFs.scala index 6b2c63407d7..9893545a4ad 100644 --- a/delta-lake/common/src/main/scala/com/nvidia/spark/rapids/delta/deltaUDFs.scala +++ b/delta-lake/common/src/main/scala/com/nvidia/spark/rapids/delta/deltaUDFs.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023, NVIDIA CORPORATION. + * Copyright (c) 2023-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -16,12 +16,19 @@ package com.nvidia.spark.rapids.delta +import java.io.{ByteArrayInputStream, ByteArrayOutputStream, DataInputStream, DataOutputStream} + import ai.rapids.cudf.{ColumnVector, Scalar, Table} import ai.rapids.cudf.Table.DuplicateKeepOption import com.nvidia.spark.RapidsUDF import com.nvidia.spark.rapids.Arm.withResource +import org.roaringbitmap.longlong.Roaring64Bitmap +import org.apache.spark.sql.Encoder +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.execution.metric.SQLMetric +import org.apache.spark.sql.expressions.Aggregator +import org.apache.spark.sql.types.{BinaryType, DataType, SQLUserDefinedType, UserDefinedType} import org.apache.spark.util.AccumulatorV2 class GpuDeltaRecordTouchedFileNameUDF(accum: AccumulatorV2[String, java.util.Set[String]]) @@ -73,3 +80,77 @@ class GpuDeltaMetricUpdateUDF(metric: SQLMetric) } } } + +class GpuDeltaNoopUDF extends Function1[Boolean, Boolean] with RapidsUDF with Serializable { + override def apply(v1: Boolean): Boolean = v1 + + override def evaluateColumnar(numRows: Int, args: ColumnVector*): ColumnVector = { + require(args.length == 1) + args(0).incRefCount() + } +} + +@SQLUserDefinedType(udt = classOf[RoaringBitmapUDT]) +case class RoaringBitmapWrapper(inner: Roaring64Bitmap) { + def serializeToBytes(): Array[Byte] = { + withResource(new ByteArrayOutputStream()) { bout => + withResource(new DataOutputStream(bout)) { dao => + inner.serialize(dao) + } + bout.toByteArray + } + } +} + +object RoaringBitmapWrapper { + def deserializeFromBytes(bytes: Array[Byte]): RoaringBitmapWrapper = { + withResource(new ByteArrayInputStream(bytes)) { bin => + withResource(new DataInputStream(bin)) { din => + val ret = RoaringBitmapWrapper(new Roaring64Bitmap) + ret.inner.deserialize(din) + ret + } + } + } +} + +class RoaringBitmapUDT extends UserDefinedType[RoaringBitmapWrapper] { + + override def sqlType: DataType = BinaryType + + override def serialize(obj: RoaringBitmapWrapper): Any = { + obj.serializeToBytes() + } + + override def deserialize(datum: Any): RoaringBitmapWrapper = { + datum match { + case b: Array[Byte] => RoaringBitmapWrapper.deserializeFromBytes(b) + case t => throw new IllegalArgumentException(s"t: ${t.getClass}") + } + } + + override def userClass: Class[RoaringBitmapWrapper] = classOf[RoaringBitmapWrapper] + + override def typeName: String = "RoaringBitmap" +} + +object RoaringBitmapUDAF extends Aggregator[Long, RoaringBitmapWrapper, RoaringBitmapWrapper] { + override def zero: RoaringBitmapWrapper = RoaringBitmapWrapper(new Roaring64Bitmap()) + + override def reduce(b: RoaringBitmapWrapper, a: Long): RoaringBitmapWrapper = { + b.inner.addLong(a) + b + } + + override def merge(b1: RoaringBitmapWrapper, b2: RoaringBitmapWrapper): RoaringBitmapWrapper = { + val ret = b1.inner.clone() + ret.or(b2.inner) + RoaringBitmapWrapper(ret) + } + + override def finish(reduction: RoaringBitmapWrapper): RoaringBitmapWrapper = reduction + + override def bufferEncoder: Encoder[RoaringBitmapWrapper] = ExpressionEncoder() + + override def outputEncoder: Encoder[RoaringBitmapWrapper] = ExpressionEncoder() +} diff --git a/delta-lake/delta-24x/src/main/scala/com/nvidia/spark/rapids/delta/delta24x/Delta24xProvider.scala b/delta-lake/delta-24x/src/main/scala/com/nvidia/spark/rapids/delta/delta24x/Delta24xProvider.scala index d3f952b856c..f90f31300e5 100644 --- a/delta-lake/delta-24x/src/main/scala/com/nvidia/spark/rapids/delta/delta24x/Delta24xProvider.scala +++ b/delta-lake/delta-24x/src/main/scala/com/nvidia/spark/rapids/delta/delta24x/Delta24xProvider.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023, NVIDIA CORPORATION. + * Copyright (c) 2023-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -74,7 +74,8 @@ object Delta24xProvider extends DeltaIOProvider { override def getReadFileFormat(format: FileFormat): FileFormat = { val cpuFormat = format.asInstanceOf[DeltaParquetFileFormat] - GpuDelta24xParquetFileFormat(cpuFormat.metadata, cpuFormat.isSplittable) + GpuDelta24xParquetFileFormat(cpuFormat.metadata, cpuFormat.isSplittable, + cpuFormat.disablePushDowns, cpuFormat.broadcastDvMap) } override def convertToGpu( diff --git a/delta-lake/delta-24x/src/main/scala/com/nvidia/spark/rapids/delta/delta24x/GpuDelta24xParquetFileFormat.scala b/delta-lake/delta-24x/src/main/scala/com/nvidia/spark/rapids/delta/delta24x/GpuDelta24xParquetFileFormat.scala index 709df7e9416..ef579d78e6f 100644 --- a/delta-lake/delta-24x/src/main/scala/com/nvidia/spark/rapids/delta/delta24x/GpuDelta24xParquetFileFormat.scala +++ b/delta-lake/delta-24x/src/main/scala/com/nvidia/spark/rapids/delta/delta24x/GpuDelta24xParquetFileFormat.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023, NVIDIA CORPORATION. + * Copyright (c) 2023-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -16,18 +16,32 @@ package com.nvidia.spark.rapids.delta.delta24x -import com.nvidia.spark.rapids.delta.GpuDeltaParquetFileFormat +import java.net.URI + +import com.nvidia.spark.rapids.{GpuMetric, RapidsConf} +import com.nvidia.spark.rapids.delta.{GpuDeltaParquetFileFormat, RoaringBitmapWrapper} +import com.nvidia.spark.rapids.delta.GpuDeltaParquetFileFormatUtils.addMetadataColumnToIterator +import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path +import org.apache.spark.broadcast.Broadcast import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.delta.{DeltaColumnMappingMode, IdMapping} +import org.apache.spark.sql.delta.DeltaParquetFileFormat.DeletionVectorDescriptorWithFilterType import org.apache.spark.sql.delta.actions.Metadata +import org.apache.spark.sql.execution.datasources.PartitionedFile import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.vectorized.ColumnarBatch case class GpuDelta24xParquetFileFormat( metadata: Metadata, - isSplittable: Boolean) extends GpuDeltaParquetFileFormat { + isSplittable: Boolean, + disablePushDown: Boolean, + broadcastDvMap: Option[Broadcast[Map[URI, DeletionVectorDescriptorWithFilterType]]]) + extends GpuDeltaParquetFileFormat { override val columnMappingMode: DeltaColumnMappingMode = metadata.columnMappingMode override val referenceSchema: StructType = metadata.schema @@ -46,6 +60,47 @@ case class GpuDelta24xParquetFileFormat( options: Map[String, String], path: Path): Boolean = isSplittable + override def buildReaderWithPartitionValuesAndMetrics( + sparkSession: SparkSession, + dataSchema: StructType, + partitionSchema: StructType, + requiredSchema: StructType, + filters: Seq[Filter], + options: Map[String, String], + hadoopConf: Configuration, + metrics: Map[String, GpuMetric], + alluxioPathReplacementMap: Option[Map[String, String]]) + : PartitionedFile => Iterator[InternalRow] = { + + + val dataReader = super.buildReaderWithPartitionValuesAndMetrics( + sparkSession, + dataSchema, + partitionSchema, + requiredSchema, + if (disablePushDown) Seq.empty else filters, + options, + hadoopConf, + metrics, + alluxioPathReplacementMap) + + val delVecs = broadcastDvMap + val maxDelVecScatterBatchSize = RapidsConf + .DELTA_LOW_SHUFFLE_MERGE_SCATTER_DEL_VECTOR_BATCH_SIZE + .get(sparkSession.sessionState.conf) + + (file: PartitionedFile) => { + val input = dataReader(file) + val dv = delVecs.flatMap(_.value.get(new URI(file.filePath.toString()))) + .map(dv => RoaringBitmapWrapper.deserializeFromBytes(dv.descriptor.inlineData).inner) + addMetadataColumnToIterator(prepareSchema(requiredSchema), + dv, + input.asInstanceOf[Iterator[ColumnarBatch]], + maxDelVecScatterBatchSize) + .asInstanceOf[Iterator[InternalRow]] + } + } + /** * We sometimes need to replace FileFormat within LogicalPlans, so we have to override * `equals` to ensure file format changes are captured diff --git a/delta-lake/delta-24x/src/main/scala/com/nvidia/spark/rapids/delta/delta24x/MergeIntoCommandMeta.scala b/delta-lake/delta-24x/src/main/scala/com/nvidia/spark/rapids/delta/delta24x/MergeIntoCommandMeta.scala index 4b4dfb624b5..8ce813ef011 100644 --- a/delta-lake/delta-24x/src/main/scala/com/nvidia/spark/rapids/delta/delta24x/MergeIntoCommandMeta.scala +++ b/delta-lake/delta-24x/src/main/scala/com/nvidia/spark/rapids/delta/delta24x/MergeIntoCommandMeta.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023, NVIDIA CORPORATION. + * Copyright (c) 2023-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -16,13 +16,14 @@ package com.nvidia.spark.rapids.delta.delta24x -import com.nvidia.spark.rapids.{DataFromReplacementRule, RapidsConf, RapidsMeta, RunnableCommandMeta} +import com.nvidia.spark.rapids.{DataFromReplacementRule, RapidsConf, RapidsMeta, RapidsReaderType, RunnableCommandMeta} import com.nvidia.spark.rapids.delta.RapidsDeltaUtils +import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession import org.apache.spark.sql.delta.commands.MergeIntoCommand import org.apache.spark.sql.delta.rapids.GpuDeltaLog -import org.apache.spark.sql.delta.rapids.delta24x.GpuMergeIntoCommand +import org.apache.spark.sql.delta.rapids.delta24x.{GpuLowShuffleMergeCommand, GpuMergeIntoCommand} import org.apache.spark.sql.execution.command.RunnableCommand class MergeIntoCommandMeta( @@ -30,12 +31,12 @@ class MergeIntoCommandMeta( conf: RapidsConf, parent: Option[RapidsMeta[_, _, _]], rule: DataFromReplacementRule) - extends RunnableCommandMeta[MergeIntoCommand](mergeCmd, conf, parent, rule) { + extends RunnableCommandMeta[MergeIntoCommand](mergeCmd, conf, parent, rule) with Logging { override def tagSelfForGpu(): Unit = { if (!conf.isDeltaWriteEnabled) { willNotWorkOnGpu("Delta Lake output acceleration has been disabled. To enable set " + - s"${RapidsConf.ENABLE_DELTA_WRITE} to true") + s"${RapidsConf.ENABLE_DELTA_WRITE} to true") } if (mergeCmd.notMatchedBySourceClauses.nonEmpty) { // https://github.com/NVIDIA/spark-rapids/issues/8415 @@ -48,14 +49,43 @@ class MergeIntoCommandMeta( } override def convertToGpu(): RunnableCommand = { - GpuMergeIntoCommand( - mergeCmd.source, - mergeCmd.target, - new GpuDeltaLog(mergeCmd.targetFileIndex.deltaLog, conf), - mergeCmd.condition, - mergeCmd.matchedClauses, - mergeCmd.notMatchedClauses, - mergeCmd.notMatchedBySourceClauses, - mergeCmd.migratedSchema)(conf) + // TODO: Currently we only support low shuffler merge only when parquet per file read is enabled + // due to the limitation of implementing row index metadata column. + if (conf.isDeltaLowShuffleMergeEnabled) { + if (conf.isParquetPerFileReadEnabled) { + GpuLowShuffleMergeCommand( + mergeCmd.source, + mergeCmd.target, + new GpuDeltaLog(mergeCmd.targetFileIndex.deltaLog, conf), + mergeCmd.condition, + mergeCmd.matchedClauses, + mergeCmd.notMatchedClauses, + mergeCmd.notMatchedBySourceClauses, + mergeCmd.migratedSchema)(conf) + } else { + logWarning(s"""Low shuffle merge disabled since ${RapidsConf.PARQUET_READER_TYPE} is + not set to ${RapidsReaderType.PERFILE}. Falling back to classic merge.""") + GpuMergeIntoCommand( + mergeCmd.source, + mergeCmd.target, + new GpuDeltaLog(mergeCmd.targetFileIndex.deltaLog, conf), + mergeCmd.condition, + mergeCmd.matchedClauses, + mergeCmd.notMatchedClauses, + mergeCmd.notMatchedBySourceClauses, + mergeCmd.migratedSchema)(conf) + } + } else { + GpuMergeIntoCommand( + mergeCmd.source, + mergeCmd.target, + new GpuDeltaLog(mergeCmd.targetFileIndex.deltaLog, conf), + mergeCmd.condition, + mergeCmd.matchedClauses, + mergeCmd.notMatchedClauses, + mergeCmd.notMatchedBySourceClauses, + mergeCmd.migratedSchema)(conf) + } } + } diff --git a/delta-lake/delta-24x/src/main/scala/org/apache/spark/sql/delta/rapids/delta24x/GpuLowShuffleMergeCommand.scala b/delta-lake/delta-24x/src/main/scala/org/apache/spark/sql/delta/rapids/delta24x/GpuLowShuffleMergeCommand.scala new file mode 100644 index 00000000000..9c27d28ebd3 --- /dev/null +++ b/delta-lake/delta-24x/src/main/scala/org/apache/spark/sql/delta/rapids/delta24x/GpuLowShuffleMergeCommand.scala @@ -0,0 +1,1084 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * This file was derived from MergeIntoCommand.scala + * in the Delta Lake project at https://github.com/delta-io/delta. + * + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed 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.delta.rapids.delta24x + +import java.net.URI +import java.util.concurrent.TimeUnit + +import scala.collection.mutable + +import com.nvidia.spark.rapids.{GpuOverrides, RapidsConf, SparkPlanMeta} +import com.nvidia.spark.rapids.RapidsConf.DELTA_LOW_SHUFFLE_MERGE_DEL_VECTOR_BROADCAST_THRESHOLD +import com.nvidia.spark.rapids.delta._ +import com.nvidia.spark.rapids.delta.GpuDeltaParquetFileFormatUtils._ +import com.nvidia.spark.rapids.shims.FileSourceScanExecMeta +import org.roaringbitmap.longlong.Roaring64Bitmap + +import org.apache.spark.SparkContext +import org.apache.spark.internal.Logging +import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute +import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeReference, CaseWhen, Expression, Literal, NamedExpression, PredicateHelper} +import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral +import org.apache.spark.sql.catalyst.plans.logical.{DeltaMergeAction, DeltaMergeIntoClause, DeltaMergeIntoMatchedClause, DeltaMergeIntoMatchedDeleteClause, DeltaMergeIntoMatchedUpdateClause, DeltaMergeIntoNotMatchedBySourceClause, DeltaMergeIntoNotMatchedBySourceDeleteClause, DeltaMergeIntoNotMatchedBySourceUpdateClause, DeltaMergeIntoNotMatchedClause, DeltaMergeIntoNotMatchedInsertClause, LogicalPlan, Project} +import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap +import org.apache.spark.sql.delta.{DeltaErrors, DeltaLog, DeltaOperations, DeltaParquetFileFormat, DeltaTableUtils, DeltaUDF, NoMapping, OptimisticTransaction, RowIndexFilterType} +import org.apache.spark.sql.delta.DeltaOperations.MergePredicate +import org.apache.spark.sql.delta.DeltaParquetFileFormat.DeletionVectorDescriptorWithFilterType +import org.apache.spark.sql.delta.actions.{AddCDCFile, AddFile, DeletionVectorDescriptor, FileAction} +import org.apache.spark.sql.delta.commands.DeltaCommand +import org.apache.spark.sql.delta.rapids.{GpuDeltaLog, GpuOptimisticTransactionBase} +import org.apache.spark.sql.delta.rapids.delta24x.MergeExecutor.{toDeletionVector, totalBytesAndDistinctPartitionValues, INCR_METRICS_COL, INCR_METRICS_FIELD, ROW_DROPPED_COL, ROW_DROPPED_FIELD, SOURCE_ROW_PRESENT_COL, SOURCE_ROW_PRESENT_FIELD, TARGET_ROW_PRESENT_COL, TARGET_ROW_PRESENT_FIELD} +import org.apache.spark.sql.delta.schema.ImplicitMetadataOperation +import org.apache.spark.sql.delta.sources.DeltaSQLConf +import org.apache.spark.sql.delta.util.{AnalysisHelper, DeltaFileOperations} +import org.apache.spark.sql.execution.{SparkPlan, SQLExecution} +import org.apache.spark.sql.execution.command.LeafRunnableCommand +import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation} +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types.{BooleanType, LongType, StringType, StructField, StructType} + +/** + * GPU version of Delta Lake's low shuffle merge implementation. + * + * Performs a merge of a source query/table into a Delta table. + * + * Issues an error message when the ON search_condition of the MERGE statement can match + * a single row from the target table with multiple rows of the source table-reference. + * Different from the original implementation, it optimized writing touched unmodified target files. + * + * Algorithm: + * + * Phase 1: Find the input files in target that are touched by the rows that satisfy + * the condition and verify that no two source rows match with the same target row. + * This is implemented as an inner-join using the given condition. See [[findTouchedFiles]] + * for more details. + * + * Phase 2: Read the touched files again and write new files with updated and/or inserted rows + * without copying unmodified rows. + * + * Phase 3: Read the touched files again and write new files with unmodified rows in target table, + * trying to keep its original order and avoid shuffle as much as possible. + * + * Phase 4: Use the Delta protocol to atomically remove the touched files and add the new files. + * + * @param source Source data to merge from + * @param target Target table to merge into + * @param gpuDeltaLog Delta log to use + * @param condition Condition for a source row to match with a target row + * @param matchedClauses All info related to matched clauses. + * @param notMatchedClauses All info related to not matched clause. + * @param migratedSchema The final schema of the target - may be changed by schema evolution. + */ +case class GpuLowShuffleMergeCommand( + @transient source: LogicalPlan, + @transient target: LogicalPlan, + @transient gpuDeltaLog: GpuDeltaLog, + condition: Expression, + matchedClauses: Seq[DeltaMergeIntoMatchedClause], + notMatchedClauses: Seq[DeltaMergeIntoNotMatchedClause], + notMatchedBySourceClauses: Seq[DeltaMergeIntoNotMatchedBySourceClause], + migratedSchema: Option[StructType])( + @transient val rapidsConf: RapidsConf) + extends LeafRunnableCommand + with DeltaCommand with PredicateHelper with AnalysisHelper with ImplicitMetadataOperation { + + import SQLMetrics._ + + override val otherCopyArgs: Seq[AnyRef] = Seq(rapidsConf) + + override val canMergeSchema: Boolean = conf.getConf(DeltaSQLConf.DELTA_SCHEMA_AUTO_MIGRATE) + override val canOverwriteSchema: Boolean = false + + override val output: Seq[Attribute] = Seq( + AttributeReference("num_affected_rows", LongType)(), + AttributeReference("num_updated_rows", LongType)(), + AttributeReference("num_deleted_rows", LongType)(), + AttributeReference("num_inserted_rows", LongType)()) + + @transient private lazy val sc: SparkContext = SparkContext.getOrCreate() + @transient private[delta] lazy val targetDeltaLog: DeltaLog = gpuDeltaLog.deltaLog + + override lazy val metrics = Map[String, SQLMetric]( + "numSourceRows" -> createMetric(sc, "number of source rows"), + "numSourceRowsInSecondScan" -> + createMetric(sc, "number of source rows (during repeated scan)"), + "numTargetRowsCopied" -> createMetric(sc, "number of target rows rewritten unmodified"), + "numTargetRowsInserted" -> createMetric(sc, "number of inserted rows"), + "numTargetRowsUpdated" -> createMetric(sc, "number of updated rows"), + "numTargetRowsDeleted" -> createMetric(sc, "number of deleted rows"), + "numTargetRowsMatchedUpdated" -> createMetric(sc, "number of target rows updated when matched"), + "numTargetRowsMatchedDeleted" -> createMetric(sc, "number of target rows deleted when matched"), + "numTargetRowsNotMatchedBySourceUpdated" -> createMetric(sc, + "number of target rows updated when not matched by source"), + "numTargetRowsNotMatchedBySourceDeleted" -> createMetric(sc, + "number of target rows deleted when not matched by source"), + "numTargetFilesBeforeSkipping" -> createMetric(sc, "number of target files before skipping"), + "numTargetFilesAfterSkipping" -> createMetric(sc, "number of target files after skipping"), + "numTargetFilesRemoved" -> createMetric(sc, "number of files removed to target"), + "numTargetFilesAdded" -> createMetric(sc, "number of files added to target"), + "numTargetChangeFilesAdded" -> + createMetric(sc, "number of change data capture files generated"), + "numTargetChangeFileBytes" -> + createMetric(sc, "total size of change data capture files generated"), + "numTargetBytesBeforeSkipping" -> createMetric(sc, "number of target bytes before skipping"), + "numTargetBytesAfterSkipping" -> createMetric(sc, "number of target bytes after skipping"), + "numTargetBytesRemoved" -> createMetric(sc, "number of target bytes removed"), + "numTargetBytesAdded" -> createMetric(sc, "number of target bytes added"), + "numTargetPartitionsAfterSkipping" -> + createMetric(sc, "number of target partitions after skipping"), + "numTargetPartitionsRemovedFrom" -> + createMetric(sc, "number of target partitions from which files were removed"), + "numTargetPartitionsAddedTo" -> + createMetric(sc, "number of target partitions to which files were added"), + "executionTimeMs" -> + createMetric(sc, "time taken to execute the entire operation"), + "scanTimeMs" -> + createMetric(sc, "time taken to scan the files for matches"), + "rewriteTimeMs" -> + createMetric(sc, "time taken to rewrite the matched files")) + + /** Whether this merge statement has only a single insert (NOT MATCHED) clause. */ + protected def isSingleInsertOnly: Boolean = matchedClauses.isEmpty && + notMatchedClauses.length == 1 + + override def run(spark: SparkSession): Seq[Row] = { + recordDeltaOperation(targetDeltaLog, "delta.dml.lowshufflemerge") { + val startTime = System.nanoTime() + val result = gpuDeltaLog.withNewTransaction { deltaTxn => + if (target.schema.size != deltaTxn.metadata.schema.size) { + throw DeltaErrors.schemaChangedSinceAnalysis( + atAnalysis = target.schema, latestSchema = deltaTxn.metadata.schema) + } + + if (canMergeSchema) { + updateMetadata( + spark, deltaTxn, migratedSchema.getOrElse(target.schema), + deltaTxn.metadata.partitionColumns, deltaTxn.metadata.configuration, + isOverwriteMode = false, rearrangeOnly = false) + } + + + val (executor, fallback) = { + val context = MergeExecutorContext(this, spark, deltaTxn, rapidsConf) + if (isSingleInsertOnly && spark.conf.get(DeltaSQLConf.MERGE_INSERT_ONLY_ENABLED)) { + (new InsertOnlyMergeExecutor(context), false) + } else { + val executor = new LowShuffleMergeExecutor(context) + (executor, executor.shouldFallback()) + } + } + + if (fallback) { + None + } else { + Some(runLowShuffleMerge(spark, startTime, deltaTxn, executor)) + } + } + + result match { + case Some(row) => row + case None => + // We should rollback to normal gpu + new GpuMergeIntoCommand(source, target, gpuDeltaLog, condition, matchedClauses, + notMatchedClauses, notMatchedBySourceClauses, migratedSchema)(rapidsConf) + .run(spark) + } + } + } + + + private def runLowShuffleMerge( + spark: SparkSession, + startTime: Long, + deltaTxn: GpuOptimisticTransactionBase, + mergeExecutor: MergeExecutor): Seq[Row] = { + val deltaActions = mergeExecutor.execute() + // Metrics should be recorded before commit (where they are written to delta logs). + metrics("executionTimeMs").set((System.nanoTime() - startTime) / 1000 / 1000) + deltaTxn.registerSQLMetrics(spark, metrics) + + // This is a best-effort sanity check. + if (metrics("numSourceRowsInSecondScan").value >= 0 && + metrics("numSourceRows").value != metrics("numSourceRowsInSecondScan").value) { + log.warn(s"Merge source has ${metrics("numSourceRows").value} rows in initial scan but " + + s"${metrics("numSourceRowsInSecondScan").value} rows in second scan") + if (conf.getConf(DeltaSQLConf.MERGE_FAIL_IF_SOURCE_CHANGED)) { + throw DeltaErrors.sourceNotDeterministicInMergeException(spark) + } + } + + deltaTxn.commit( + deltaActions, + DeltaOperations.Merge( + Option(condition), + matchedClauses.map(DeltaOperations.MergePredicate(_)), + notMatchedClauses.map(DeltaOperations.MergePredicate(_)), + // We do not support notMatchedBySourcePredicates yet and fall back to CPU + // See https://github.com/NVIDIA/spark-rapids/issues/8415 + notMatchedBySourcePredicates = Seq.empty[MergePredicate] + )) + + // Record metrics + val stats = GpuMergeStats.fromMergeSQLMetrics( + metrics, + condition, + matchedClauses, + notMatchedClauses, + notMatchedBySourceClauses, + deltaTxn.metadata.partitionColumns.nonEmpty) + recordDeltaEvent(targetDeltaLog, "delta.dml.merge.stats", data = stats) + + + spark.sharedState.cacheManager.recacheByPlan(spark, target) + + // This is needed to make the SQL metrics visible in the Spark UI. Also this needs + // to be outside the recordMergeOperation because this method will update some metric. + val executionId = spark.sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) + SQLMetrics.postDriverMetricUpdates(spark.sparkContext, executionId, metrics.values.toSeq) + Seq(Row(metrics("numTargetRowsUpdated").value + metrics("numTargetRowsDeleted").value + + metrics("numTargetRowsInserted").value, metrics("numTargetRowsUpdated").value, + metrics("numTargetRowsDeleted").value, metrics("numTargetRowsInserted").value)) + } + + /** + * Execute the given `thunk` and return its result while recording the time taken to do it. + * + * @param sqlMetricName name of SQL metric to update with the time taken by the thunk + * @param thunk the code to execute + */ + private[delta] def recordMergeOperation[A](sqlMetricName: String)(thunk: => A): A = { + val startTimeNs = System.nanoTime() + val r = thunk + val timeTakenMs = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTimeNs) + if (sqlMetricName != null && timeTakenMs > 0) { + metrics(sqlMetricName) += timeTakenMs + } + r + } + + /** Expressions to increment SQL metrics */ + private[delta] def makeMetricUpdateUDF(name: String, deterministic: Boolean = false) + : Expression = { + // only capture the needed metric in a local variable + val metric = metrics(name) + var u = DeltaUDF.boolean(new GpuDeltaMetricUpdateUDF(metric)) + if (!deterministic) { + u = u.asNondeterministic() + } + u.apply().expr + } +} + +/** + * Context merge execution. + */ +case class MergeExecutorContext(cmd: GpuLowShuffleMergeCommand, + spark: SparkSession, + deltaTxn: OptimisticTransaction, + rapidsConf: RapidsConf) + +trait MergeExecutor extends AnalysisHelper with PredicateHelper with Logging { + + val context: MergeExecutorContext + + + /** + * Map to get target output attributes by name. + * The case sensitivity of the map is set accordingly to Spark configuration. + */ + @transient private lazy val targetOutputAttributesMap: Map[String, Attribute] = { + val attrMap: Map[String, Attribute] = context.cmd.target + .outputSet.view + .map(attr => attr.name -> attr).toMap + if (context.cmd.conf.caseSensitiveAnalysis) { + attrMap + } else { + CaseInsensitiveMap(attrMap) + } + } + + def execute(): Seq[FileAction] + + protected def targetOutputCols: Seq[NamedExpression] = { + context.deltaTxn.metadata.schema.map { col => + targetOutputAttributesMap + .get(col.name) + .map { a => + AttributeReference(col.name, col.dataType, col.nullable)(a.exprId) + } + .getOrElse(Alias(Literal(null), col.name)()) + } + } + + /** + * Build a DataFrame using the given `files` that has the same output columns (exprIds) + * as the `target` logical plan, so that existing update/insert expressions can be applied + * on this new plan. + */ + protected def buildTargetDFWithFiles(files: Seq[AddFile]): DataFrame = { + val targetOutputColsMap = { + val colsMap: Map[String, NamedExpression] = targetOutputCols.view + .map(col => col.name -> col).toMap + if (context.cmd.conf.caseSensitiveAnalysis) { + colsMap + } else { + CaseInsensitiveMap(colsMap) + } + } + + val plan = { + // We have to do surgery to use the attributes from `targetOutputCols` to scan the table. + // In cases of schema evolution, they may not be the same type as the original attributes. + val original = + context.deltaTxn.deltaLog.createDataFrame(context.deltaTxn.snapshot, files) + .queryExecution + .analyzed + val transformed = original.transform { + case LogicalRelation(base, _, catalogTbl, isStreaming) => + LogicalRelation( + base, + // We can ignore the new columns which aren't yet AttributeReferences. + targetOutputCols.collect { case a: AttributeReference => a }, + catalogTbl, + isStreaming) + } + + // In case of schema evolution & column mapping, we would also need to rebuild the file + // format because under column mapping, the reference schema within DeltaParquetFileFormat + // that is used to populate metadata needs to be updated + if (context.deltaTxn.metadata.columnMappingMode != NoMapping) { + val updatedFileFormat = context.deltaTxn.deltaLog.fileFormat( + context.deltaTxn.deltaLog.unsafeVolatileSnapshot.protocol, context.deltaTxn.metadata) + DeltaTableUtils.replaceFileFormat(transformed, updatedFileFormat) + } else { + transformed + } + } + + // For each plan output column, find the corresponding target output column (by name) and + // create an alias + val aliases = plan.output.map { + case newAttrib: AttributeReference => + val existingTargetAttrib = targetOutputColsMap.getOrElse(newAttrib.name, + throw new AnalysisException( + s"Could not find ${newAttrib.name} among the existing target output " + + targetOutputCols.mkString(","))).asInstanceOf[AttributeReference] + + if (existingTargetAttrib.exprId == newAttrib.exprId) { + // It's not valid to alias an expression to its own exprId (this is considered a + // non-unique exprId by the analyzer), so we just use the attribute directly. + newAttrib + } else { + Alias(newAttrib, existingTargetAttrib.name)(exprId = existingTargetAttrib.exprId) + } + } + + Dataset.ofRows(context.spark, Project(aliases, plan)) + } + + + /** + * Repartitions the output DataFrame by the partition columns if table is partitioned + * and `merge.repartitionBeforeWrite.enabled` is set to true. + */ + protected def repartitionIfNeeded(df: DataFrame): DataFrame = { + val partitionColumns = context.deltaTxn.metadata.partitionColumns + // TODO: We should remove this method and use optimized write instead, see + // https://github.com/NVIDIA/spark-rapids/issues/10417 + if (partitionColumns.nonEmpty && context.spark.conf.get(DeltaSQLConf + .MERGE_REPARTITION_BEFORE_WRITE)) { + df.repartition(partitionColumns.map(col): _*) + } else { + df + } + } + + protected def sourceDF: DataFrame = { + // UDF to increment metrics + val incrSourceRowCountExpr = context.cmd.makeMetricUpdateUDF("numSourceRows") + Dataset.ofRows(context.spark, context.cmd.source) + .filter(new Column(incrSourceRowCountExpr)) + } + + /** Whether this merge statement has no insert (NOT MATCHED) clause. */ + protected def hasNoInserts: Boolean = context.cmd.notMatchedClauses.isEmpty + + +} + +/** + * This is an optimization of the case when there is no update clause for the merge. + * We perform an left anti join on the source data to find the rows to be inserted. + * + * This will currently only optimize for the case when there is a _single_ notMatchedClause. + */ +class InsertOnlyMergeExecutor(override val context: MergeExecutorContext) extends MergeExecutor { + override def execute(): Seq[FileAction] = { + context.cmd.recordMergeOperation(sqlMetricName = "rewriteTimeMs") { + + // UDFs to update metrics + val incrSourceRowCountExpr = context.cmd.makeMetricUpdateUDF("numSourceRows") + val incrInsertedCountExpr = context.cmd.makeMetricUpdateUDF("numTargetRowsInserted") + + val outputColNames = targetOutputCols.map(_.name) + // we use head here since we know there is only a single notMatchedClause + val outputExprs = context.cmd.notMatchedClauses.head.resolvedActions.map(_.expr) + val outputCols = outputExprs.zip(outputColNames).map { case (expr, name) => + new Column(Alias(expr, name)()) + } + + // source DataFrame + val sourceDF = Dataset.ofRows(context.spark, context.cmd.source) + .filter(new Column(incrSourceRowCountExpr)) + .filter(new Column(context.cmd.notMatchedClauses.head.condition + .getOrElse(Literal.TrueLiteral))) + + // Skip data based on the merge condition + val conjunctivePredicates = splitConjunctivePredicates(context.cmd.condition) + val targetOnlyPredicates = + conjunctivePredicates.filter(_.references.subsetOf(context.cmd.target.outputSet)) + val dataSkippedFiles = context.deltaTxn.filterFiles(targetOnlyPredicates) + + // target DataFrame + val targetDF = buildTargetDFWithFiles(dataSkippedFiles) + + val insertDf = sourceDF.join(targetDF, new Column(context.cmd.condition), "leftanti") + .select(outputCols: _*) + .filter(new Column(incrInsertedCountExpr)) + + val newFiles = context.deltaTxn + .writeFiles(repartitionIfNeeded(insertDf, + )) + + // Update metrics + context.cmd.metrics("numTargetFilesBeforeSkipping") += context.deltaTxn.snapshot.numOfFiles + context.cmd.metrics("numTargetBytesBeforeSkipping") += context.deltaTxn.snapshot.sizeInBytes + val (afterSkippingBytes, afterSkippingPartitions) = + totalBytesAndDistinctPartitionValues(dataSkippedFiles) + context.cmd.metrics("numTargetFilesAfterSkipping") += dataSkippedFiles.size + context.cmd.metrics("numTargetBytesAfterSkipping") += afterSkippingBytes + context.cmd.metrics("numTargetPartitionsAfterSkipping") += afterSkippingPartitions + context.cmd.metrics("numTargetFilesRemoved") += 0 + context.cmd.metrics("numTargetBytesRemoved") += 0 + context.cmd.metrics("numTargetPartitionsRemovedFrom") += 0 + val (addedBytes, addedPartitions) = totalBytesAndDistinctPartitionValues(newFiles) + context.cmd.metrics("numTargetFilesAdded") += newFiles.count(_.isInstanceOf[AddFile]) + context.cmd.metrics("numTargetBytesAdded") += addedBytes + context.cmd.metrics("numTargetPartitionsAddedTo") += addedPartitions + newFiles + } + } +} + + +/** + * This is an optimized algorithm for merge statement, where we avoid shuffling the unmodified + * target data. + * + * The algorithm is as follows: + * 1. Find touched target files in the target table by joining the source and target data, with + * collecting joined row identifiers as (`__metadata_file_path`, `__metadata_row_idx`) pairs. + * 2. Read the touched files again and write new files with updated and/or inserted rows + * without coping unmodified data from target table, but filtering target table with collected + * rows mentioned above. + * 3. Read the touched files again, filtering unmodified rows with collected row identifiers + * collected in first step, and saving them without shuffle. + */ +class LowShuffleMergeExecutor(override val context: MergeExecutorContext) extends MergeExecutor { + + // We over-count numTargetRowsDeleted when there are multiple matches; + // this is the amount of the overcount, so we can subtract it to get a correct final metric. + private var multipleMatchDeleteOnlyOvercount: Option[Long] = None + + // UDFs to update metrics + private val incrSourceRowCountExpr: Expression = context.cmd. + makeMetricUpdateUDF("numSourceRowsInSecondScan") + private val incrUpdatedCountExpr: Expression = context.cmd + .makeMetricUpdateUDF("numTargetRowsUpdated") + private val incrUpdatedMatchedCountExpr: Expression = context.cmd + .makeMetricUpdateUDF("numTargetRowsMatchedUpdated") + private val incrUpdatedNotMatchedBySourceCountExpr: Expression = context.cmd + .makeMetricUpdateUDF("numTargetRowsNotMatchedBySourceUpdated") + private val incrInsertedCountExpr: Expression = context.cmd + .makeMetricUpdateUDF("numTargetRowsInserted") + private val incrDeletedCountExpr: Expression = context.cmd + .makeMetricUpdateUDF("numTargetRowsDeleted") + private val incrDeletedMatchedCountExpr: Expression = context.cmd + .makeMetricUpdateUDF("numTargetRowsMatchedDeleted") + private val incrDeletedNotMatchedBySourceCountExpr: Expression = context.cmd + .makeMetricUpdateUDF("numTargetRowsNotMatchedBySourceDeleted") + + private def updateOutput(resolvedActions: Seq[DeltaMergeAction], incrExpr: Expression) + : Seq[Expression] = { + resolvedActions.map(_.expr) :+ + Literal.FalseLiteral :+ + UnresolvedAttribute(TARGET_ROW_PRESENT_COL) :+ + UnresolvedAttribute(SOURCE_ROW_PRESENT_COL) :+ + incrExpr + } + + private def deleteOutput(incrExpr: Expression): Seq[Expression] = { + targetOutputCols :+ + TrueLiteral :+ + UnresolvedAttribute(TARGET_ROW_PRESENT_COL) :+ + UnresolvedAttribute(SOURCE_ROW_PRESENT_COL) :+ + incrExpr + } + + private def insertOutput(resolvedActions: Seq[DeltaMergeAction], incrExpr: Expression) + : Seq[Expression] = { + resolvedActions.map(_.expr) :+ + Literal.FalseLiteral :+ + UnresolvedAttribute(TARGET_ROW_PRESENT_COL) :+ + UnresolvedAttribute(SOURCE_ROW_PRESENT_COL) :+ + incrExpr + } + + private def clauseOutput(clause: DeltaMergeIntoClause): Seq[Expression] = clause match { + case u: DeltaMergeIntoMatchedUpdateClause => + updateOutput(u.resolvedActions, And(incrUpdatedCountExpr, incrUpdatedMatchedCountExpr)) + case _: DeltaMergeIntoMatchedDeleteClause => + deleteOutput(And(incrDeletedCountExpr, incrDeletedMatchedCountExpr)) + case i: DeltaMergeIntoNotMatchedInsertClause => + insertOutput(i.resolvedActions, incrInsertedCountExpr) + case u: DeltaMergeIntoNotMatchedBySourceUpdateClause => + updateOutput(u.resolvedActions, + And(incrUpdatedCountExpr, incrUpdatedNotMatchedBySourceCountExpr)) + case _: DeltaMergeIntoNotMatchedBySourceDeleteClause => + deleteOutput(And(incrDeletedCountExpr, incrDeletedNotMatchedBySourceCountExpr)) + } + + private def clauseCondition(clause: DeltaMergeIntoClause): Expression = { + // if condition is None, then expression always evaluates to true + clause.condition.getOrElse(TrueLiteral) + } + + /** + * Though low shuffle merge algorithm performs better than traditional merge algorithm in some + * cases, there are some case we should fallback to traditional merge executor: + * + * 1. Low shuffle merge algorithm requires generating metadata columns such as + * [[METADATA_ROW_IDX_COL]], [[METADATA_ROW_DEL_COL]], which only implemented on + * [[org.apache.spark.sql.rapids.GpuFileSourceScanExec]]. That means we need to fallback to + * this normal executor when [[org.apache.spark.sql.rapids.GpuFileSourceScanExec]] is disabled + * for some reason. + * 2. Low shuffle merge algorithm currently needs to broadcast deletion vector, which may + * introduce extra overhead. It maybe better to fallback to this algorithm when the changeset + * it too large. + */ + private[delta] def shouldFallback(): Boolean = { + // Trying to detect if we can execute finding touched files. + val touchFilePlanOverrideSucceed = verifyGpuPlan(planForFindingTouchedFiles()) { planMeta => + def check(meta: SparkPlanMeta[SparkPlan]): Boolean = { + meta match { + case scan if scan.isInstanceOf[FileSourceScanExecMeta] => scan + .asInstanceOf[FileSourceScanExecMeta] + .wrapped + .schema + .fieldNames + .contains(METADATA_ROW_IDX_COL) && scan.canThisBeReplaced + case m => m.childPlans.exists(check) + } + } + + check(planMeta) + } + if (!touchFilePlanOverrideSucceed) { + logWarning("Unable to override file scan for low shuffle merge for finding touched files " + + "plan, fallback to tradition merge.") + return true + } + + // Trying to detect if we can execute the merge plan. + val mergePlanOverrideSucceed = verifyGpuPlan(planForMergeExecution(touchedFiles)) { planMeta => + var overrideCount = 0 + def count(meta: SparkPlanMeta[SparkPlan]): Unit = { + meta match { + case scan if scan.isInstanceOf[FileSourceScanExecMeta] => + if (scan.asInstanceOf[FileSourceScanExecMeta] + .wrapped.schema.fieldNames.contains(METADATA_ROW_DEL_COL) && scan.canThisBeReplaced) { + overrideCount += 1 + } + case m => m.childPlans.foreach(count) + } + } + + count(planMeta) + overrideCount == 2 + } + + if (!mergePlanOverrideSucceed) { + logWarning("Unable to override file scan for low shuffle merge for merge plan, fallback to " + + "tradition merge.") + return true + } + + val deletionVectorSize = touchedFiles.values.map(_._1.serializedSizeInBytes()).sum + val maxDelVectorSize = context.rapidsConf + .get(DELTA_LOW_SHUFFLE_MERGE_DEL_VECTOR_BROADCAST_THRESHOLD) + if (deletionVectorSize > maxDelVectorSize) { + logWarning( + s"""Low shuffle merge can't be executed because broadcast deletion vector count + |$deletionVectorSize is large than max value $maxDelVectorSize """.stripMargin) + return true + } + + false + } + + private def verifyGpuPlan(input: DataFrame)(checkPlanMeta: SparkPlanMeta[SparkPlan] => Boolean) + : Boolean = { + val overridePlan = GpuOverrides.wrapAndTagPlan(input.queryExecution.sparkPlan, + context.rapidsConf) + checkPlanMeta(overridePlan) + } + + override def execute(): Seq[FileAction] = { + val newFiles = context.cmd.withStatusCode("DELTA", + s"Rewriting ${touchedFiles.size} files and saving modified data") { + val df = planForMergeExecution(touchedFiles) + context.deltaTxn.writeFiles(df) + } + + // Update metrics + val (addedBytes, addedPartitions) = totalBytesAndDistinctPartitionValues(newFiles) + context.cmd.metrics("numTargetFilesAdded") += newFiles.count(_.isInstanceOf[AddFile]) + context.cmd.metrics("numTargetChangeFilesAdded") += newFiles.count(_.isInstanceOf[AddCDCFile]) + context.cmd.metrics("numTargetChangeFileBytes") += newFiles.collect { + case f: AddCDCFile => f.size + } + .sum + context.cmd.metrics("numTargetBytesAdded") += addedBytes + context.cmd.metrics("numTargetPartitionsAddedTo") += addedPartitions + + if (multipleMatchDeleteOnlyOvercount.isDefined) { + // Compensate for counting duplicates during the query. + val actualRowsDeleted = + context.cmd.metrics("numTargetRowsDeleted").value - multipleMatchDeleteOnlyOvercount.get + assert(actualRowsDeleted >= 0) + context.cmd.metrics("numTargetRowsDeleted").set(actualRowsDeleted) + } + + touchedFiles.values.map(_._2).map(_.remove).toSeq ++ newFiles + } + + private lazy val dataSkippedFiles: Seq[AddFile] = { + // Skip data based on the merge condition + val targetOnlyPredicates = splitConjunctivePredicates(context.cmd.condition) + .filter(_.references.subsetOf(context.cmd.target.outputSet)) + context.deltaTxn.filterFiles(targetOnlyPredicates) + } + + private lazy val dataSkippedTargetDF: DataFrame = { + addRowIndexMetaColumn(buildTargetDFWithFiles(dataSkippedFiles)) + } + + private lazy val touchedFiles: Map[String, (Roaring64Bitmap, AddFile)] = this.findTouchedFiles() + + private def planForFindingTouchedFiles(): DataFrame = { + + // Apply inner join to between source and target using the merge condition to find matches + // In addition, we attach two columns + // - METADATA_ROW_IDX column to identify target row in file + // - FILE_PATH_COL the target file name the row is from to later identify the files touched + // by matched rows + val targetDF = dataSkippedTargetDF.withColumn(FILE_PATH_COL, input_file_name()) + + sourceDF.join(targetDF, new Column(context.cmd.condition), "inner") + } + + private def planForMergeExecution(touchedFiles: Map[String, (Roaring64Bitmap, AddFile)]) + : DataFrame = { + getModifiedDF(touchedFiles).unionAll(getUnmodifiedDF(touchedFiles)) + } + + /** + * Find the target table files that contain the rows that satisfy the merge condition. This is + * implemented as an inner-join between the source query/table and the target table using + * the merge condition. + */ + private def findTouchedFiles(): Map[String, (Roaring64Bitmap, AddFile)] = + context.cmd.recordMergeOperation(sqlMetricName = "scanTimeMs") { + context.spark.udf.register("row_index_set", udaf(RoaringBitmapUDAF)) + // Process the matches from the inner join to record touched files and find multiple matches + val collectTouchedFiles = planForFindingTouchedFiles() + .select(col(FILE_PATH_COL), col(METADATA_ROW_IDX_COL)) + .groupBy(FILE_PATH_COL) + .agg( + expr(s"row_index_set($METADATA_ROW_IDX_COL) as row_idxes"), + count("*").as("count")) + .collect().map(row => { + val filename = row.getAs[String](FILE_PATH_COL) + val rowIdxSet = row.getAs[RoaringBitmapWrapper]("row_idxes").inner + val count = row.getAs[Long]("count") + (filename, (rowIdxSet, count)) + }) + .toMap + + val duplicateCount = { + val distinctMatchedRowCounts = collectTouchedFiles.values + .map(_._1.getLongCardinality).sum + val allMatchedRowCounts = collectTouchedFiles.values.map(_._2).sum + allMatchedRowCounts - distinctMatchedRowCounts + } + + val hasMultipleMatches = duplicateCount > 0 + + // Throw error if multiple matches are ambiguous or cannot be computed correctly. + val canBeComputedUnambiguously = { + // Multiple matches are not ambiguous when there is only one unconditional delete as + // all the matched row pairs in the 2nd join in `writeAllChanges` will get deleted. + val isUnconditionalDelete = context.cmd.matchedClauses.headOption match { + case Some(DeltaMergeIntoMatchedDeleteClause(None)) => true + case _ => false + } + context.cmd.matchedClauses.size == 1 && isUnconditionalDelete + } + + if (hasMultipleMatches && !canBeComputedUnambiguously) { + throw DeltaErrors.multipleSourceRowMatchingTargetRowInMergeException(context.spark) + } + + if (hasMultipleMatches) { + // This is only allowed for delete-only queries. + // This query will count the duplicates for numTargetRowsDeleted in Job 2, + // because we count matches after the join and not just the target rows. + // We have to compensate for this by subtracting the duplicates later, + // so we need to record them here. + multipleMatchDeleteOnlyOvercount = Some(duplicateCount) + } + + // Get the AddFiles using the touched file names. + val touchedFileNames = collectTouchedFiles.keys.toSeq + + val nameToAddFileMap = context.cmd.generateCandidateFileMap( + context.cmd.targetDeltaLog.dataPath, + dataSkippedFiles) + + val touchedAddFiles = touchedFileNames.map(f => + context.cmd.getTouchedFile(context.cmd.targetDeltaLog.dataPath, f, nameToAddFileMap)) + .map(f => (DeltaFileOperations + .absolutePath(context.cmd.targetDeltaLog.dataPath.toString, f.path) + .toString, f)).toMap + + // When the target table is empty, and the optimizer optimized away the join entirely + // numSourceRows will be incorrectly 0. + // We need to scan the source table once to get the correct + // metric here. + if (context.cmd.metrics("numSourceRows").value == 0 && + (dataSkippedFiles.isEmpty || dataSkippedTargetDF.take(1).isEmpty)) { + val numSourceRows = sourceDF.count() + context.cmd.metrics("numSourceRows").set(numSourceRows) + } + + // Update metrics + context.cmd.metrics("numTargetFilesBeforeSkipping") += context.deltaTxn.snapshot.numOfFiles + context.cmd.metrics("numTargetBytesBeforeSkipping") += context.deltaTxn.snapshot.sizeInBytes + val (afterSkippingBytes, afterSkippingPartitions) = + totalBytesAndDistinctPartitionValues(dataSkippedFiles) + context.cmd.metrics("numTargetFilesAfterSkipping") += dataSkippedFiles.size + context.cmd.metrics("numTargetBytesAfterSkipping") += afterSkippingBytes + context.cmd.metrics("numTargetPartitionsAfterSkipping") += afterSkippingPartitions + val (removedBytes, removedPartitions) = + totalBytesAndDistinctPartitionValues(touchedAddFiles.values.toSeq) + context.cmd.metrics("numTargetFilesRemoved") += touchedAddFiles.size + context.cmd.metrics("numTargetBytesRemoved") += removedBytes + context.cmd.metrics("numTargetPartitionsRemovedFrom") += removedPartitions + + collectTouchedFiles.map(kv => (kv._1, (kv._2._1, touchedAddFiles(kv._1)))) + } + + + /** + * Modify original data frame to insert + * [[GpuDeltaParquetFileFormatUtils.METADATA_ROW_IDX_COL]]. + */ + private def addRowIndexMetaColumn(baseDF: DataFrame): DataFrame = { + val rowIdxAttr = AttributeReference( + METADATA_ROW_IDX_COL, + METADATA_ROW_IDX_FIELD.dataType, + METADATA_ROW_IDX_FIELD.nullable)() + + val newPlan = baseDF.queryExecution.analyzed.transformUp { + case r@LogicalRelation(fs: HadoopFsRelation, _, _, _) => + val newSchema = StructType(fs.dataSchema.fields).add(METADATA_ROW_IDX_FIELD) + + // This is required to ensure that row index is correctly calculated. + val newFileFormat = fs.fileFormat.asInstanceOf[DeltaParquetFileFormat] + .copy(isSplittable = false, disablePushDowns = true) + + val newFs = fs.copy(dataSchema = newSchema, fileFormat = newFileFormat)(context.spark) + + val newOutput = r.output :+ rowIdxAttr + r.copy(relation = newFs, output = newOutput) + case p@Project(projectList, _) => + val newProjectList = projectList :+ rowIdxAttr + p.copy(projectList = newProjectList) + } + + Dataset.ofRows(context.spark, newPlan) + } + + /** + * The result is scanning target table with touched files, and added an extra + * [[METADATA_ROW_DEL_COL]] to indicate whether filtered by joining with source table in first + * step. + */ + private def getTouchedTargetDF(touchedFiles: Map[String, (Roaring64Bitmap, AddFile)]) + : DataFrame = { + // Generate a new target dataframe that has same output attributes exprIds as the target plan. + // This allows us to apply the existing resolved update/insert expressions. + val baseTargetDF = buildTargetDFWithFiles(touchedFiles.values.map(_._2).toSeq) + + val newPlan = { + val rowDelAttr = AttributeReference( + METADATA_ROW_DEL_COL, + METADATA_ROW_DEL_FIELD.dataType, + METADATA_ROW_DEL_FIELD.nullable)() + + baseTargetDF.queryExecution.analyzed.transformUp { + case r@LogicalRelation(fs: HadoopFsRelation, _, _, _) => + val newSchema = StructType(fs.dataSchema.fields).add(METADATA_ROW_DEL_FIELD) + + // This is required to ensure that row index is correctly calculated. + val newFileFormat = { + val oldFormat = fs.fileFormat.asInstanceOf[DeltaParquetFileFormat] + val dvs = touchedFiles.map(kv => (new URI(kv._1), + DeletionVectorDescriptorWithFilterType(toDeletionVector(kv._2._1), + RowIndexFilterType.UNKNOWN))) + val broadcastDVs = context.spark.sparkContext.broadcast(dvs) + + oldFormat.copy(isSplittable = false, + broadcastDvMap = Some(broadcastDVs), + disablePushDowns = true) + } + + val newFs = fs.copy(dataSchema = newSchema, fileFormat = newFileFormat)(context.spark) + + val newOutput = r.output :+ rowDelAttr + r.copy(relation = newFs, output = newOutput) + case p@Project(projectList, _) => + val newProjectList = projectList :+ rowDelAttr + p.copy(projectList = newProjectList) + } + } + + val df = Dataset.ofRows(context.spark, newPlan) + .withColumn(TARGET_ROW_PRESENT_COL, lit(true)) + + df + } + + /** + * Generate a plan by calculating modified rows. It's computed by joining source and target + * tables, where target table has been filtered by (`__metadata_file_name`, + * `__metadata_row_idx`) pairs collected in first step. + * + * Schema of `modifiedDF`: + * + * targetSchema + ROW_DROPPED_COL + TARGET_ROW_PRESENT_COL + + * SOURCE_ROW_PRESENT_COL + INCR_METRICS_COL + * INCR_METRICS_COL + * + * It consists of several parts: + * + * 1. Unmatched source rows which are inserted + * 2. Unmatched source rows which are deleted + * 3. Target rows which are updated + * 4. Target rows which are deleted + */ + private def getModifiedDF(touchedFiles: Map[String, (Roaring64Bitmap, AddFile)]): DataFrame = { + val sourceDF = this.sourceDF + .withColumn(SOURCE_ROW_PRESENT_COL, new Column(incrSourceRowCountExpr)) + + val targetDF = getTouchedTargetDF(touchedFiles) + + val joinedDF = { + val joinType = if (hasNoInserts && + context.spark.conf.get(DeltaSQLConf.MERGE_MATCHED_ONLY_ENABLED)) { + "inner" + } else { + "leftOuter" + } + val matchedTargetDF = targetDF.filter(METADATA_ROW_DEL_COL) + .drop(METADATA_ROW_DEL_COL) + + sourceDF.join(matchedTargetDF, new Column(context.cmd.condition), joinType) + } + + val modifiedRowsSchema = context.deltaTxn.metadata.schema + .add(ROW_DROPPED_FIELD) + .add(TARGET_ROW_PRESENT_FIELD.copy(nullable = true)) + .add(SOURCE_ROW_PRESENT_FIELD.copy(nullable = true)) + .add(INCR_METRICS_FIELD) + + // Here we generate a case when statement to handle all cases: + // CASE + // WHEN + // CASE WHEN + // + // WHEN + // + // ELSE + // + // WHEN + // CASE WHEN + // + // WHEN + // + // ELSE + // + // END + + val notMatchedConditions = context.cmd.notMatchedClauses.map(clauseCondition) + val notMatchedExpr = { + val deletedNotMatchedRow = { + targetOutputCols :+ + Literal.TrueLiteral :+ + Literal.FalseLiteral :+ + Literal(null) :+ + Literal.TrueLiteral + } + if (context.cmd.notMatchedClauses.isEmpty) { + // If there no `WHEN NOT MATCHED` clause, we should just delete not matched row + deletedNotMatchedRow + } else { + val notMatchedOutputs = context.cmd.notMatchedClauses.map(clauseOutput) + modifiedRowsSchema.zipWithIndex.map { + case (_, idx) => + CaseWhen(notMatchedConditions.zip(notMatchedOutputs.map(_(idx))), + deletedNotMatchedRow(idx)) + } + } + } + + val matchedConditions = context.cmd.matchedClauses.map(clauseCondition) + val matchedOutputs = context.cmd.matchedClauses.map(clauseOutput) + val matchedExprs = { + val notMatchedRow = { + targetOutputCols :+ + Literal.FalseLiteral :+ + Literal.TrueLiteral :+ + Literal(null) :+ + Literal.TrueLiteral + } + if (context.cmd.matchedClauses.isEmpty) { + // If there is not matched clause, this is insert only, we should delete this row. + notMatchedRow + } else { + modifiedRowsSchema.zipWithIndex.map { + case (_, idx) => + CaseWhen(matchedConditions.zip(matchedOutputs.map(_(idx))), + notMatchedRow(idx)) + } + } + } + + val sourceRowHasNoMatch = col(TARGET_ROW_PRESENT_COL).isNull.expr + + val modifiedCols = modifiedRowsSchema.zipWithIndex.map { case (col, idx) => + val caseWhen = CaseWhen( + Seq(sourceRowHasNoMatch -> notMatchedExpr(idx)), + matchedExprs(idx)) + Column(Alias(caseWhen, col.name)()) + } + + val modifiedDF = { + + // Make this a udf to avoid catalyst to be too aggressive to even remove the join! + val noopRowDroppedCol = udf(new GpuDeltaNoopUDF()).apply(!col(ROW_DROPPED_COL)) + + val modifiedDF = joinedDF.select(modifiedCols: _*) + // This will not filter anything since they always return true, but we need to avoid + // catalyst from optimizing these udf + .filter(noopRowDroppedCol && col(INCR_METRICS_COL)) + .drop(ROW_DROPPED_COL, INCR_METRICS_COL, TARGET_ROW_PRESENT_COL, SOURCE_ROW_PRESENT_COL) + + repartitionIfNeeded(modifiedDF) + } + + modifiedDF + } + + private def getUnmodifiedDF(touchedFiles: Map[String, (Roaring64Bitmap, AddFile)]): DataFrame = { + getTouchedTargetDF(touchedFiles) + .filter(!col(METADATA_ROW_DEL_COL)) + .drop(TARGET_ROW_PRESENT_COL, METADATA_ROW_DEL_COL) + } +} + + +object MergeExecutor { + + /** + * Spark UI will track all normal accumulators along with Spark tasks to show them on Web UI. + * However, the accumulator used by `MergeIntoCommand` can store a very large value since it + * tracks all files that need to be rewritten. We should ask Spark UI to not remember it, + * otherwise, the UI data may consume lots of memory. Hence, we use the prefix `internal.metrics.` + * to make this accumulator become an internal accumulator, so that it will not be tracked by + * Spark UI. + */ + val TOUCHED_FILES_ACCUM_NAME = "internal.metrics.MergeIntoDelta.touchedFiles" + + val ROW_ID_COL = "_row_id_" + val FILE_PATH_COL: String = GpuDeltaParquetFileFormatUtils.FILE_PATH_COL + val SOURCE_ROW_PRESENT_COL: String = "_source_row_present_" + val SOURCE_ROW_PRESENT_FIELD: StructField = StructField(SOURCE_ROW_PRESENT_COL, BooleanType, + nullable = false) + val TARGET_ROW_PRESENT_COL: String = "_target_row_present_" + val TARGET_ROW_PRESENT_FIELD: StructField = StructField(TARGET_ROW_PRESENT_COL, BooleanType, + nullable = false) + val ROW_DROPPED_COL: String = GpuDeltaMergeConstants.ROW_DROPPED_COL + val ROW_DROPPED_FIELD: StructField = StructField(ROW_DROPPED_COL, BooleanType, nullable = false) + val INCR_METRICS_COL: String = "_incr_metrics_" + val INCR_METRICS_FIELD: StructField = StructField(INCR_METRICS_COL, BooleanType, nullable = false) + val INCR_ROW_COUNT_COL: String = "_incr_row_count_" + + // Some Delta versions use Literal(null) which translates to a literal of NullType instead + // of the Literal(null, StringType) which is needed, so using a fixed version here + // rather than the version from Delta Lake. + val CDC_TYPE_NOT_CDC_LITERAL: Literal = Literal(null, StringType) + + private[delta] def toDeletionVector(bitmap: Roaring64Bitmap): DeletionVectorDescriptor = { + DeletionVectorDescriptor.inlineInLog(RoaringBitmapWrapper(bitmap).serializeToBytes(), + bitmap.getLongCardinality) + } + + /** Count the number of distinct partition values among the AddFiles in the given set. */ + private[delta] def totalBytesAndDistinctPartitionValues(files: Seq[FileAction]): (Long, Int) = { + val distinctValues = new mutable.HashSet[Map[String, String]]() + var bytes = 0L + val iter = files.collect { case a: AddFile => a }.iterator + while (iter.hasNext) { + val file = iter.next() + distinctValues += file.partitionValues + bytes += file.size + } + // If the only distinct value map is an empty map, then it must be an unpartitioned table. + // Return 0 in that case. + val numDistinctValues = + if (distinctValues.size == 1 && distinctValues.head.isEmpty) 0 else distinctValues.size + (bytes, numDistinctValues) + } +} \ No newline at end of file diff --git a/delta-lake/delta-spark341db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuLowShuffleMergeCommand.scala b/delta-lake/delta-spark341db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuLowShuffleMergeCommand.scala new file mode 100644 index 00000000000..fddebda33bd --- /dev/null +++ b/delta-lake/delta-spark341db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuLowShuffleMergeCommand.scala @@ -0,0 +1,1083 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * This file was derived from MergeIntoCommand.scala + * in the Delta Lake project at https://github.com/delta-io/delta. + * + * Copyright (2021) The Delta Lake Project Authors. + * + * Licensed 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 com.databricks.sql.transaction.tahoe.rapids + +import java.net.URI +import java.util.concurrent.TimeUnit + +import scala.collection.mutable + +import com.databricks.sql.io.RowIndexFilterType +import com.databricks.sql.transaction.tahoe._ +import com.databricks.sql.transaction.tahoe.DeltaOperations.MergePredicate +import com.databricks.sql.transaction.tahoe.DeltaParquetFileFormat.DeletionVectorDescriptorWithFilterType +import com.databricks.sql.transaction.tahoe.actions.{AddCDCFile, AddFile, DeletionVectorDescriptor, FileAction} +import com.databricks.sql.transaction.tahoe.commands.DeltaCommand +import com.databricks.sql.transaction.tahoe.rapids.MergeExecutor.{toDeletionVector, totalBytesAndDistinctPartitionValues, FILE_PATH_COL, INCR_METRICS_COL, INCR_METRICS_FIELD, ROW_DROPPED_COL, ROW_DROPPED_FIELD, SOURCE_ROW_PRESENT_COL, SOURCE_ROW_PRESENT_FIELD, TARGET_ROW_PRESENT_COL, TARGET_ROW_PRESENT_FIELD} +import com.databricks.sql.transaction.tahoe.schema.ImplicitMetadataOperation +import com.databricks.sql.transaction.tahoe.sources.DeltaSQLConf +import com.databricks.sql.transaction.tahoe.util.{AnalysisHelper, DeltaFileOperations} +import com.nvidia.spark.rapids.{GpuOverrides, RapidsConf, SparkPlanMeta} +import com.nvidia.spark.rapids.RapidsConf.DELTA_LOW_SHUFFLE_MERGE_DEL_VECTOR_BROADCAST_THRESHOLD +import com.nvidia.spark.rapids.delta._ +import com.nvidia.spark.rapids.delta.GpuDeltaParquetFileFormatUtils.{METADATA_ROW_DEL_COL, METADATA_ROW_DEL_FIELD, METADATA_ROW_IDX_COL, METADATA_ROW_IDX_FIELD} +import com.nvidia.spark.rapids.shims.FileSourceScanExecMeta +import org.roaringbitmap.longlong.Roaring64Bitmap + +import org.apache.spark.SparkContext +import org.apache.spark.internal.Logging +import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute +import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeReference, CaseWhen, Expression, Literal, NamedExpression, PredicateHelper} +import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral +import org.apache.spark.sql.catalyst.plans.logical.{DeltaMergeAction, DeltaMergeIntoClause, DeltaMergeIntoMatchedClause, DeltaMergeIntoMatchedDeleteClause, DeltaMergeIntoMatchedUpdateClause, DeltaMergeIntoNotMatchedBySourceClause, DeltaMergeIntoNotMatchedBySourceDeleteClause, DeltaMergeIntoNotMatchedBySourceUpdateClause, DeltaMergeIntoNotMatchedClause, DeltaMergeIntoNotMatchedInsertClause, LogicalPlan, Project} +import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap +import org.apache.spark.sql.execution.{SparkPlan, SQLExecution} +import org.apache.spark.sql.execution.command.LeafRunnableCommand +import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation} +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types.{BooleanType, LongType, StringType, StructField, StructType} + +/** + * GPU version of Delta Lake's low shuffle merge implementation. + * + * Performs a merge of a source query/table into a Delta table. + * + * Issues an error message when the ON search_condition of the MERGE statement can match + * a single row from the target table with multiple rows of the source table-reference. + * Different from the original implementation, it optimized writing touched unmodified target files. + * + * Algorithm: + * + * Phase 1: Find the input files in target that are touched by the rows that satisfy + * the condition and verify that no two source rows match with the same target row. + * This is implemented as an inner-join using the given condition. See [[findTouchedFiles]] + * for more details. + * + * Phase 2: Read the touched files again and write new files with updated and/or inserted rows + * without copying unmodified rows. + * + * Phase 3: Read the touched files again and write new files with unmodified rows in target table, + * trying to keep its original order and avoid shuffle as much as possible. + * + * Phase 4: Use the Delta protocol to atomically remove the touched files and add the new files. + * + * @param source Source data to merge from + * @param target Target table to merge into + * @param gpuDeltaLog Delta log to use + * @param condition Condition for a source row to match with a target row + * @param matchedClauses All info related to matched clauses. + * @param notMatchedClauses All info related to not matched clause. + * @param migratedSchema The final schema of the target - may be changed by schema evolution. + */ +case class GpuLowShuffleMergeCommand( + @transient source: LogicalPlan, + @transient target: LogicalPlan, + @transient gpuDeltaLog: GpuDeltaLog, + condition: Expression, + matchedClauses: Seq[DeltaMergeIntoMatchedClause], + notMatchedClauses: Seq[DeltaMergeIntoNotMatchedClause], + notMatchedBySourceClauses: Seq[DeltaMergeIntoNotMatchedBySourceClause], + migratedSchema: Option[StructType])( + @transient val rapidsConf: RapidsConf) + extends LeafRunnableCommand + with DeltaCommand with PredicateHelper with AnalysisHelper with ImplicitMetadataOperation { + + import SQLMetrics._ + + override val otherCopyArgs: Seq[AnyRef] = Seq(rapidsConf) + + override val canMergeSchema: Boolean = conf.getConf(DeltaSQLConf.DELTA_SCHEMA_AUTO_MIGRATE) + override val canOverwriteSchema: Boolean = false + + override val output: Seq[Attribute] = Seq( + AttributeReference("num_affected_rows", LongType)(), + AttributeReference("num_updated_rows", LongType)(), + AttributeReference("num_deleted_rows", LongType)(), + AttributeReference("num_inserted_rows", LongType)()) + + @transient private lazy val sc: SparkContext = SparkContext.getOrCreate() + @transient lazy val targetDeltaLog: DeltaLog = gpuDeltaLog.deltaLog + + override lazy val metrics = Map[String, SQLMetric]( + "numSourceRows" -> createMetric(sc, "number of source rows"), + "numSourceRowsInSecondScan" -> + createMetric(sc, "number of source rows (during repeated scan)"), + "numTargetRowsCopied" -> createMetric(sc, "number of target rows rewritten unmodified"), + "numTargetRowsInserted" -> createMetric(sc, "number of inserted rows"), + "numTargetRowsUpdated" -> createMetric(sc, "number of updated rows"), + "numTargetRowsDeleted" -> createMetric(sc, "number of deleted rows"), + "numTargetRowsMatchedUpdated" -> createMetric(sc, "number of target rows updated when matched"), + "numTargetRowsMatchedDeleted" -> createMetric(sc, "number of target rows deleted when matched"), + "numTargetRowsNotMatchedBySourceUpdated" -> createMetric(sc, + "number of target rows updated when not matched by source"), + "numTargetRowsNotMatchedBySourceDeleted" -> createMetric(sc, + "number of target rows deleted when not matched by source"), + "numTargetFilesBeforeSkipping" -> createMetric(sc, "number of target files before skipping"), + "numTargetFilesAfterSkipping" -> createMetric(sc, "number of target files after skipping"), + "numTargetFilesRemoved" -> createMetric(sc, "number of files removed to target"), + "numTargetFilesAdded" -> createMetric(sc, "number of files added to target"), + "numTargetChangeFilesAdded" -> + createMetric(sc, "number of change data capture files generated"), + "numTargetChangeFileBytes" -> + createMetric(sc, "total size of change data capture files generated"), + "numTargetBytesBeforeSkipping" -> createMetric(sc, "number of target bytes before skipping"), + "numTargetBytesAfterSkipping" -> createMetric(sc, "number of target bytes after skipping"), + "numTargetBytesRemoved" -> createMetric(sc, "number of target bytes removed"), + "numTargetBytesAdded" -> createMetric(sc, "number of target bytes added"), + "numTargetPartitionsAfterSkipping" -> + createMetric(sc, "number of target partitions after skipping"), + "numTargetPartitionsRemovedFrom" -> + createMetric(sc, "number of target partitions from which files were removed"), + "numTargetPartitionsAddedTo" -> + createMetric(sc, "number of target partitions to which files were added"), + "executionTimeMs" -> + createMetric(sc, "time taken to execute the entire operation"), + "scanTimeMs" -> + createMetric(sc, "time taken to scan the files for matches"), + "rewriteTimeMs" -> + createMetric(sc, "time taken to rewrite the matched files")) + + /** Whether this merge statement has only a single insert (NOT MATCHED) clause. */ + protected def isSingleInsertOnly: Boolean = matchedClauses.isEmpty && + notMatchedClauses.length == 1 + + override def run(spark: SparkSession): Seq[Row] = { + recordDeltaOperation(targetDeltaLog, "delta.dml.lowshufflemerge") { + val startTime = System.nanoTime() + val result = gpuDeltaLog.withNewTransaction { deltaTxn => + if (target.schema.size != deltaTxn.metadata.schema.size) { + throw DeltaErrors.schemaChangedSinceAnalysis( + atAnalysis = target.schema, latestSchema = deltaTxn.metadata.schema) + } + + if (canMergeSchema) { + updateMetadata( + spark, deltaTxn, migratedSchema.getOrElse(target.schema), + deltaTxn.metadata.partitionColumns, deltaTxn.metadata.configuration, + isOverwriteMode = false, rearrangeOnly = false) + } + + + val (executor, fallback) = { + val context = MergeExecutorContext(this, spark, deltaTxn, rapidsConf) + if (isSingleInsertOnly && spark.conf.get(DeltaSQLConf.MERGE_INSERT_ONLY_ENABLED)) { + (new InsertOnlyMergeExecutor(context), false) + } else { + val executor = new LowShuffleMergeExecutor(context) + (executor, executor.shouldFallback()) + } + } + + if (fallback) { + None + } else { + Some(runLowShuffleMerge(spark, startTime, deltaTxn, executor)) + } + } + + result match { + case Some(row) => row + case None => + // We should rollback to normal gpu + new GpuMergeIntoCommand(source, target, gpuDeltaLog, condition, matchedClauses, + notMatchedClauses, notMatchedBySourceClauses, migratedSchema)(rapidsConf) + .run(spark) + } + } + } + + + private def runLowShuffleMerge( + spark: SparkSession, + startTime: Long, + deltaTxn: GpuOptimisticTransactionBase, + mergeExecutor: MergeExecutor): Seq[Row] = { + val deltaActions = mergeExecutor.execute() + // Metrics should be recorded before commit (where they are written to delta logs). + metrics("executionTimeMs").set((System.nanoTime() - startTime) / 1000 / 1000) + deltaTxn.registerSQLMetrics(spark, metrics) + + // This is a best-effort sanity check. + if (metrics("numSourceRowsInSecondScan").value >= 0 && + metrics("numSourceRows").value != metrics("numSourceRowsInSecondScan").value) { + log.warn(s"Merge source has ${metrics("numSourceRows").value} rows in initial scan but " + + s"${metrics("numSourceRowsInSecondScan").value} rows in second scan") + if (conf.getConf(DeltaSQLConf.MERGE_FAIL_IF_SOURCE_CHANGED)) { + throw DeltaErrors.sourceNotDeterministicInMergeException(spark) + } + } + + deltaTxn.commit( + deltaActions, + DeltaOperations.Merge( + Option(condition), + matchedClauses.map(DeltaOperations.MergePredicate(_)), + notMatchedClauses.map(DeltaOperations.MergePredicate(_)), + // We do not support notMatchedBySourcePredicates yet and fall back to CPU + // See https://github.com/NVIDIA/spark-rapids/issues/8415 + notMatchedBySourcePredicates = Seq.empty[MergePredicate] + )) + + // Record metrics + val stats = GpuMergeStats.fromMergeSQLMetrics( + metrics, + condition, + matchedClauses, + notMatchedClauses, + deltaTxn.metadata.partitionColumns.nonEmpty) + recordDeltaEvent(targetDeltaLog, "delta.dml.merge.stats", data = stats) + + + spark.sharedState.cacheManager.recacheByPlan(spark, target) + + // This is needed to make the SQL metrics visible in the Spark UI. Also this needs + // to be outside the recordMergeOperation because this method will update some metric. + val executionId = spark.sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) + SQLMetrics.postDriverMetricUpdates(spark.sparkContext, executionId, metrics.values.toSeq) + Seq(Row(metrics("numTargetRowsUpdated").value + metrics("numTargetRowsDeleted").value + + metrics("numTargetRowsInserted").value, metrics("numTargetRowsUpdated").value, + metrics("numTargetRowsDeleted").value, metrics("numTargetRowsInserted").value)) + } + + /** + * Execute the given `thunk` and return its result while recording the time taken to do it. + * + * @param sqlMetricName name of SQL metric to update with the time taken by the thunk + * @param thunk the code to execute + */ + def recordMergeOperation[A](sqlMetricName: String)(thunk: => A): A = { + val startTimeNs = System.nanoTime() + val r = thunk + val timeTakenMs = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTimeNs) + if (sqlMetricName != null && timeTakenMs > 0) { + metrics(sqlMetricName) += timeTakenMs + } + r + } + + /** Expressions to increment SQL metrics */ + def makeMetricUpdateUDF(name: String, deterministic: Boolean = false) + : Expression = { + // only capture the needed metric in a local variable + val metric = metrics(name) + var u = DeltaUDF.boolean(new GpuDeltaMetricUpdateUDF(metric)) + if (!deterministic) { + u = u.asNondeterministic() + } + u.apply().expr + } +} + +/** + * Context merge execution. + */ +case class MergeExecutorContext(cmd: GpuLowShuffleMergeCommand, + spark: SparkSession, + deltaTxn: OptimisticTransaction, + rapidsConf: RapidsConf) + +trait MergeExecutor extends AnalysisHelper with PredicateHelper with Logging { + + val context: MergeExecutorContext + + + /** + * Map to get target output attributes by name. + * The case sensitivity of the map is set accordingly to Spark configuration. + */ + @transient private lazy val targetOutputAttributesMap: Map[String, Attribute] = { + val attrMap: Map[String, Attribute] = context.cmd.target + .outputSet.view + .map(attr => attr.name -> attr).toMap + if (context.cmd.conf.caseSensitiveAnalysis) { + attrMap + } else { + CaseInsensitiveMap(attrMap) + } + } + + def execute(): Seq[FileAction] + + protected def targetOutputCols: Seq[NamedExpression] = { + context.deltaTxn.metadata.schema.map { col => + targetOutputAttributesMap + .get(col.name) + .map { a => + AttributeReference(col.name, col.dataType, col.nullable)(a.exprId) + } + .getOrElse(Alias(Literal(null), col.name)()) + } + } + + /** + * Build a DataFrame using the given `files` that has the same output columns (exprIds) + * as the `target` logical plan, so that existing update/insert expressions can be applied + * on this new plan. + */ + protected def buildTargetDFWithFiles(files: Seq[AddFile]): DataFrame = { + val targetOutputColsMap = { + val colsMap: Map[String, NamedExpression] = targetOutputCols.view + .map(col => col.name -> col).toMap + if (context.cmd.conf.caseSensitiveAnalysis) { + colsMap + } else { + CaseInsensitiveMap(colsMap) + } + } + + val plan = { + // We have to do surgery to use the attributes from `targetOutputCols` to scan the table. + // In cases of schema evolution, they may not be the same type as the original attributes. + val original = + context.deltaTxn.deltaLog.createDataFrame(context.deltaTxn.snapshot, files) + .queryExecution + .analyzed + val transformed = original.transform { + case LogicalRelation(base, _, catalogTbl, isStreaming) => + LogicalRelation( + base, + // We can ignore the new columns which aren't yet AttributeReferences. + targetOutputCols.collect { case a: AttributeReference => a }, + catalogTbl, + isStreaming) + } + + // In case of schema evolution & column mapping, we would also need to rebuild the file + // format because under column mapping, the reference schema within DeltaParquetFileFormat + // that is used to populate metadata needs to be updated + if (context.deltaTxn.metadata.columnMappingMode != NoMapping) { + val updatedFileFormat = context.deltaTxn.deltaLog.fileFormat( + context.deltaTxn.deltaLog.unsafeVolatileSnapshot.protocol, context.deltaTxn.metadata) + DeltaTableUtils.replaceFileFormat(transformed, updatedFileFormat) + } else { + transformed + } + } + + // For each plan output column, find the corresponding target output column (by name) and + // create an alias + val aliases = plan.output.map { + case newAttrib: AttributeReference => + val existingTargetAttrib = targetOutputColsMap.getOrElse(newAttrib.name, + throw new AnalysisException( + s"Could not find ${newAttrib.name} among the existing target output " + + targetOutputCols.mkString(","))).asInstanceOf[AttributeReference] + + if (existingTargetAttrib.exprId == newAttrib.exprId) { + // It's not valid to alias an expression to its own exprId (this is considered a + // non-unique exprId by the analyzer), so we just use the attribute directly. + newAttrib + } else { + Alias(newAttrib, existingTargetAttrib.name)(exprId = existingTargetAttrib.exprId) + } + } + + Dataset.ofRows(context.spark, Project(aliases, plan)) + } + + + /** + * Repartitions the output DataFrame by the partition columns if table is partitioned + * and `merge.repartitionBeforeWrite.enabled` is set to true. + */ + protected def repartitionIfNeeded(df: DataFrame): DataFrame = { + val partitionColumns = context.deltaTxn.metadata.partitionColumns + // TODO: We should remove this method and use optimized write instead, see + // https://github.com/NVIDIA/spark-rapids/issues/10417 + if (partitionColumns.nonEmpty && context.spark.conf.get(DeltaSQLConf + .MERGE_REPARTITION_BEFORE_WRITE)) { + df.repartition(partitionColumns.map(col): _*) + } else { + df + } + } + + protected def sourceDF: DataFrame = { + // UDF to increment metrics + val incrSourceRowCountExpr = context.cmd.makeMetricUpdateUDF("numSourceRows") + Dataset.ofRows(context.spark, context.cmd.source) + .filter(new Column(incrSourceRowCountExpr)) + } + + /** Whether this merge statement has no insert (NOT MATCHED) clause. */ + protected def hasNoInserts: Boolean = context.cmd.notMatchedClauses.isEmpty + + +} + +/** + * This is an optimization of the case when there is no update clause for the merge. + * We perform an left anti join on the source data to find the rows to be inserted. + * + * This will currently only optimize for the case when there is a _single_ notMatchedClause. + */ +class InsertOnlyMergeExecutor(override val context: MergeExecutorContext) extends MergeExecutor { + override def execute(): Seq[FileAction] = { + context.cmd.recordMergeOperation(sqlMetricName = "rewriteTimeMs") { + + // UDFs to update metrics + val incrSourceRowCountExpr = context.cmd.makeMetricUpdateUDF("numSourceRows") + val incrInsertedCountExpr = context.cmd.makeMetricUpdateUDF("numTargetRowsInserted") + + val outputColNames = targetOutputCols.map(_.name) + // we use head here since we know there is only a single notMatchedClause + val outputExprs = context.cmd.notMatchedClauses.head.resolvedActions.map(_.expr) + val outputCols = outputExprs.zip(outputColNames).map { case (expr, name) => + new Column(Alias(expr, name)()) + } + + // source DataFrame + val sourceDF = Dataset.ofRows(context.spark, context.cmd.source) + .filter(new Column(incrSourceRowCountExpr)) + .filter(new Column(context.cmd.notMatchedClauses.head.condition + .getOrElse(Literal.TrueLiteral))) + + // Skip data based on the merge condition + val conjunctivePredicates = splitConjunctivePredicates(context.cmd.condition) + val targetOnlyPredicates = + conjunctivePredicates.filter(_.references.subsetOf(context.cmd.target.outputSet)) + val dataSkippedFiles = context.deltaTxn.filterFiles(targetOnlyPredicates) + + // target DataFrame + val targetDF = buildTargetDFWithFiles(dataSkippedFiles) + + val insertDf = sourceDF.join(targetDF, new Column(context.cmd.condition), "leftanti") + .select(outputCols: _*) + .filter(new Column(incrInsertedCountExpr)) + + val newFiles = context.deltaTxn + .writeFiles(repartitionIfNeeded(insertDf, + )) + + // Update metrics + context.cmd.metrics("numTargetFilesBeforeSkipping") += context.deltaTxn.snapshot.numOfFiles + context.cmd.metrics("numTargetBytesBeforeSkipping") += context.deltaTxn.snapshot.sizeInBytes + val (afterSkippingBytes, afterSkippingPartitions) = + totalBytesAndDistinctPartitionValues(dataSkippedFiles) + context.cmd.metrics("numTargetFilesAfterSkipping") += dataSkippedFiles.size + context.cmd.metrics("numTargetBytesAfterSkipping") += afterSkippingBytes + context.cmd.metrics("numTargetPartitionsAfterSkipping") += afterSkippingPartitions + context.cmd.metrics("numTargetFilesRemoved") += 0 + context.cmd.metrics("numTargetBytesRemoved") += 0 + context.cmd.metrics("numTargetPartitionsRemovedFrom") += 0 + val (addedBytes, addedPartitions) = totalBytesAndDistinctPartitionValues(newFiles) + context.cmd.metrics("numTargetFilesAdded") += newFiles.count(_.isInstanceOf[AddFile]) + context.cmd.metrics("numTargetBytesAdded") += addedBytes + context.cmd.metrics("numTargetPartitionsAddedTo") += addedPartitions + newFiles + } + } +} + + +/** + * This is an optimized algorithm for merge statement, where we avoid shuffling the unmodified + * target data. + * + * The algorithm is as follows: + * 1. Find touched target files in the target table by joining the source and target data, with + * collecting joined row identifiers as (`__metadata_file_path`, `__metadata_row_idx`) pairs. + * 2. Read the touched files again and write new files with updated and/or inserted rows + * without coping unmodified data from target table, but filtering target table with collected + * rows mentioned above. + * 3. Read the touched files again, filtering unmodified rows with collected row identifiers + * collected in first step, and saving them without shuffle. + */ +class LowShuffleMergeExecutor(override val context: MergeExecutorContext) extends MergeExecutor { + + // We over-count numTargetRowsDeleted when there are multiple matches; + // this is the amount of the overcount, so we can subtract it to get a correct final metric. + private var multipleMatchDeleteOnlyOvercount: Option[Long] = None + + // UDFs to update metrics + private val incrSourceRowCountExpr: Expression = context.cmd. + makeMetricUpdateUDF("numSourceRowsInSecondScan") + private val incrUpdatedCountExpr: Expression = context.cmd + .makeMetricUpdateUDF("numTargetRowsUpdated") + private val incrUpdatedMatchedCountExpr: Expression = context.cmd + .makeMetricUpdateUDF("numTargetRowsMatchedUpdated") + private val incrUpdatedNotMatchedBySourceCountExpr: Expression = context.cmd + .makeMetricUpdateUDF("numTargetRowsNotMatchedBySourceUpdated") + private val incrInsertedCountExpr: Expression = context.cmd + .makeMetricUpdateUDF("numTargetRowsInserted") + private val incrDeletedCountExpr: Expression = context.cmd + .makeMetricUpdateUDF("numTargetRowsDeleted") + private val incrDeletedMatchedCountExpr: Expression = context.cmd + .makeMetricUpdateUDF("numTargetRowsMatchedDeleted") + private val incrDeletedNotMatchedBySourceCountExpr: Expression = context.cmd + .makeMetricUpdateUDF("numTargetRowsNotMatchedBySourceDeleted") + + private def updateOutput(resolvedActions: Seq[DeltaMergeAction], incrExpr: Expression) + : Seq[Expression] = { + resolvedActions.map(_.expr) :+ + Literal.FalseLiteral :+ + UnresolvedAttribute(TARGET_ROW_PRESENT_COL) :+ + UnresolvedAttribute(SOURCE_ROW_PRESENT_COL) :+ + incrExpr + } + + private def deleteOutput(incrExpr: Expression): Seq[Expression] = { + targetOutputCols :+ + TrueLiteral :+ + UnresolvedAttribute(TARGET_ROW_PRESENT_COL) :+ + UnresolvedAttribute(SOURCE_ROW_PRESENT_COL) :+ + incrExpr + } + + private def insertOutput(resolvedActions: Seq[DeltaMergeAction], incrExpr: Expression) + : Seq[Expression] = { + resolvedActions.map(_.expr) :+ + Literal.FalseLiteral :+ + UnresolvedAttribute(TARGET_ROW_PRESENT_COL) :+ + UnresolvedAttribute(SOURCE_ROW_PRESENT_COL) :+ + incrExpr + } + + private def clauseOutput(clause: DeltaMergeIntoClause): Seq[Expression] = clause match { + case u: DeltaMergeIntoMatchedUpdateClause => + updateOutput(u.resolvedActions, And(incrUpdatedCountExpr, incrUpdatedMatchedCountExpr)) + case _: DeltaMergeIntoMatchedDeleteClause => + deleteOutput(And(incrDeletedCountExpr, incrDeletedMatchedCountExpr)) + case i: DeltaMergeIntoNotMatchedInsertClause => + insertOutput(i.resolvedActions, incrInsertedCountExpr) + case u: DeltaMergeIntoNotMatchedBySourceUpdateClause => + updateOutput(u.resolvedActions, + And(incrUpdatedCountExpr, incrUpdatedNotMatchedBySourceCountExpr)) + case _: DeltaMergeIntoNotMatchedBySourceDeleteClause => + deleteOutput(And(incrDeletedCountExpr, incrDeletedNotMatchedBySourceCountExpr)) + } + + private def clauseCondition(clause: DeltaMergeIntoClause): Expression = { + // if condition is None, then expression always evaluates to true + clause.condition.getOrElse(TrueLiteral) + } + + /** + * Though low shuffle merge algorithm performs better than traditional merge algorithm in some + * cases, there are some case we should fallback to traditional merge executor: + * + * 1. Low shuffle merge algorithm requires generating metadata columns such as + * [[METADATA_ROW_IDX_COL]], [[METADATA_ROW_DEL_COL]], which only implemented on + * [[org.apache.spark.sql.rapids.GpuFileSourceScanExec]]. That means we need to fallback to + * this normal executor when [[org.apache.spark.sql.rapids.GpuFileSourceScanExec]] is disabled + * for some reason. + * 2. Low shuffle merge algorithm currently needs to broadcast deletion vector, which may + * introduce extra overhead. It maybe better to fallback to this algorithm when the changeset + * it too large. + */ + def shouldFallback(): Boolean = { + // Trying to detect if we can execute finding touched files. + val touchFilePlanOverrideSucceed = verifyGpuPlan(planForFindingTouchedFiles()) { planMeta => + def check(meta: SparkPlanMeta[SparkPlan]): Boolean = { + meta match { + case scan if scan.isInstanceOf[FileSourceScanExecMeta] => scan + .asInstanceOf[FileSourceScanExecMeta] + .wrapped + .schema + .fieldNames + .contains(METADATA_ROW_IDX_COL) && scan.canThisBeReplaced + case m => m.childPlans.exists(check) + } + } + + check(planMeta) + } + if (!touchFilePlanOverrideSucceed) { + logWarning("Unable to override file scan for low shuffle merge for finding touched files " + + "plan, fallback to tradition merge.") + return true + } + + // Trying to detect if we can execute the merge plan. + val mergePlanOverrideSucceed = verifyGpuPlan(planForMergeExecution(touchedFiles)) { planMeta => + var overrideCount = 0 + def count(meta: SparkPlanMeta[SparkPlan]): Unit = { + meta match { + case scan if scan.isInstanceOf[FileSourceScanExecMeta] => + if (scan.asInstanceOf[FileSourceScanExecMeta] + .wrapped.schema.fieldNames.contains(METADATA_ROW_DEL_COL) && scan.canThisBeReplaced) { + overrideCount += 1 + } + case m => m.childPlans.foreach(count) + } + } + + count(planMeta) + overrideCount == 2 + } + + if (!mergePlanOverrideSucceed) { + logWarning("Unable to override file scan for low shuffle merge for merge plan, fallback to " + + "tradition merge.") + return true + } + + val deletionVectorSize = touchedFiles.values.map(_._1.serializedSizeInBytes()).sum + val maxDelVectorSize = context.rapidsConf + .get(DELTA_LOW_SHUFFLE_MERGE_DEL_VECTOR_BROADCAST_THRESHOLD) + if (deletionVectorSize > maxDelVectorSize) { + logWarning( + s"""Low shuffle merge can't be executed because broadcast deletion vector count + |$deletionVectorSize is large than max value $maxDelVectorSize """.stripMargin) + return true + } + + false + } + + private def verifyGpuPlan(input: DataFrame)(checkPlanMeta: SparkPlanMeta[SparkPlan] => Boolean) + : Boolean = { + val overridePlan = GpuOverrides.wrapAndTagPlan(input.queryExecution.sparkPlan, + context.rapidsConf) + checkPlanMeta(overridePlan) + } + + override def execute(): Seq[FileAction] = { + val newFiles = context.cmd.withStatusCode("DELTA", + s"Rewriting ${touchedFiles.size} files and saving modified data") { + val df = planForMergeExecution(touchedFiles) + context.deltaTxn.writeFiles(df) + } + + // Update metrics + val (addedBytes, addedPartitions) = totalBytesAndDistinctPartitionValues(newFiles) + context.cmd.metrics("numTargetFilesAdded") += newFiles.count(_.isInstanceOf[AddFile]) + context.cmd.metrics("numTargetChangeFilesAdded") += newFiles.count(_.isInstanceOf[AddCDCFile]) + context.cmd.metrics("numTargetChangeFileBytes") += newFiles.collect { + case f: AddCDCFile => f.size + } + .sum + context.cmd.metrics("numTargetBytesAdded") += addedBytes + context.cmd.metrics("numTargetPartitionsAddedTo") += addedPartitions + + if (multipleMatchDeleteOnlyOvercount.isDefined) { + // Compensate for counting duplicates during the query. + val actualRowsDeleted = + context.cmd.metrics("numTargetRowsDeleted").value - multipleMatchDeleteOnlyOvercount.get + assert(actualRowsDeleted >= 0) + context.cmd.metrics("numTargetRowsDeleted").set(actualRowsDeleted) + } + + touchedFiles.values.map(_._2).map(_.remove).toSeq ++ newFiles + } + + private lazy val dataSkippedFiles: Seq[AddFile] = { + // Skip data based on the merge condition + val targetOnlyPredicates = splitConjunctivePredicates(context.cmd.condition) + .filter(_.references.subsetOf(context.cmd.target.outputSet)) + context.deltaTxn.filterFiles(targetOnlyPredicates) + } + + private lazy val dataSkippedTargetDF: DataFrame = { + addRowIndexMetaColumn(buildTargetDFWithFiles(dataSkippedFiles)) + } + + private lazy val touchedFiles: Map[String, (Roaring64Bitmap, AddFile)] = this.findTouchedFiles() + + private def planForFindingTouchedFiles(): DataFrame = { + + // Apply inner join to between source and target using the merge condition to find matches + // In addition, we attach two columns + // - METADATA_ROW_IDX column to identify target row in file + // - FILE_PATH_COL the target file name the row is from to later identify the files touched + // by matched rows + val targetDF = dataSkippedTargetDF.withColumn(FILE_PATH_COL, input_file_name()) + + sourceDF.join(targetDF, new Column(context.cmd.condition), "inner") + } + + private def planForMergeExecution(touchedFiles: Map[String, (Roaring64Bitmap, AddFile)]) + : DataFrame = { + getModifiedDF(touchedFiles).unionAll(getUnmodifiedDF(touchedFiles)) + } + + /** + * Find the target table files that contain the rows that satisfy the merge condition. This is + * implemented as an inner-join between the source query/table and the target table using + * the merge condition. + */ + private def findTouchedFiles(): Map[String, (Roaring64Bitmap, AddFile)] = + context.cmd.recordMergeOperation(sqlMetricName = "scanTimeMs") { + context.spark.udf.register("row_index_set", udaf(RoaringBitmapUDAF)) + // Process the matches from the inner join to record touched files and find multiple matches + val collectTouchedFiles = planForFindingTouchedFiles() + .select(col(FILE_PATH_COL), col(METADATA_ROW_IDX_COL)) + .groupBy(FILE_PATH_COL) + .agg( + expr(s"row_index_set($METADATA_ROW_IDX_COL) as row_idxes"), + count("*").as("count")) + .collect().map(row => { + val filename = row.getAs[String](FILE_PATH_COL) + val rowIdxSet = row.getAs[RoaringBitmapWrapper]("row_idxes").inner + val count = row.getAs[Long]("count") + (filename, (rowIdxSet, count)) + }) + .toMap + + val duplicateCount = { + val distinctMatchedRowCounts = collectTouchedFiles.values + .map(_._1.getLongCardinality).sum + val allMatchedRowCounts = collectTouchedFiles.values.map(_._2).sum + allMatchedRowCounts - distinctMatchedRowCounts + } + + val hasMultipleMatches = duplicateCount > 0 + + // Throw error if multiple matches are ambiguous or cannot be computed correctly. + val canBeComputedUnambiguously = { + // Multiple matches are not ambiguous when there is only one unconditional delete as + // all the matched row pairs in the 2nd join in `writeAllChanges` will get deleted. + val isUnconditionalDelete = context.cmd.matchedClauses.headOption match { + case Some(DeltaMergeIntoMatchedDeleteClause(None)) => true + case _ => false + } + context.cmd.matchedClauses.size == 1 && isUnconditionalDelete + } + + if (hasMultipleMatches && !canBeComputedUnambiguously) { + throw DeltaErrors.multipleSourceRowMatchingTargetRowInMergeException(context.spark) + } + + if (hasMultipleMatches) { + // This is only allowed for delete-only queries. + // This query will count the duplicates for numTargetRowsDeleted in Job 2, + // because we count matches after the join and not just the target rows. + // We have to compensate for this by subtracting the duplicates later, + // so we need to record them here. + multipleMatchDeleteOnlyOvercount = Some(duplicateCount) + } + + // Get the AddFiles using the touched file names. + val touchedFileNames = collectTouchedFiles.keys.toSeq + + val nameToAddFileMap = context.cmd.generateCandidateFileMap( + context.cmd.targetDeltaLog.dataPath, + dataSkippedFiles) + + val touchedAddFiles = touchedFileNames.map(f => + context.cmd.getTouchedFile(context.cmd.targetDeltaLog.dataPath, f, nameToAddFileMap)) + .map(f => (DeltaFileOperations + .absolutePath(context.cmd.targetDeltaLog.dataPath.toString, f.path) + .toString, f)).toMap + + // When the target table is empty, and the optimizer optimized away the join entirely + // numSourceRows will be incorrectly 0. + // We need to scan the source table once to get the correct + // metric here. + if (context.cmd.metrics("numSourceRows").value == 0 && + (dataSkippedFiles.isEmpty || dataSkippedTargetDF.take(1).isEmpty)) { + val numSourceRows = sourceDF.count() + context.cmd.metrics("numSourceRows").set(numSourceRows) + } + + // Update metrics + context.cmd.metrics("numTargetFilesBeforeSkipping") += context.deltaTxn.snapshot.numOfFiles + context.cmd.metrics("numTargetBytesBeforeSkipping") += context.deltaTxn.snapshot.sizeInBytes + val (afterSkippingBytes, afterSkippingPartitions) = + totalBytesAndDistinctPartitionValues(dataSkippedFiles) + context.cmd.metrics("numTargetFilesAfterSkipping") += dataSkippedFiles.size + context.cmd.metrics("numTargetBytesAfterSkipping") += afterSkippingBytes + context.cmd.metrics("numTargetPartitionsAfterSkipping") += afterSkippingPartitions + val (removedBytes, removedPartitions) = + totalBytesAndDistinctPartitionValues(touchedAddFiles.values.toSeq) + context.cmd.metrics("numTargetFilesRemoved") += touchedAddFiles.size + context.cmd.metrics("numTargetBytesRemoved") += removedBytes + context.cmd.metrics("numTargetPartitionsRemovedFrom") += removedPartitions + + collectTouchedFiles.map(kv => (kv._1, (kv._2._1, touchedAddFiles(kv._1)))) + } + + + /** + * Modify original data frame to insert + * [[GpuDeltaParquetFileFormatUtils.METADATA_ROW_IDX_COL]]. + */ + private def addRowIndexMetaColumn(baseDF: DataFrame): DataFrame = { + val rowIdxAttr = AttributeReference( + METADATA_ROW_IDX_COL, + METADATA_ROW_IDX_FIELD.dataType, + METADATA_ROW_IDX_FIELD.nullable)() + + val newPlan = baseDF.queryExecution.analyzed.transformUp { + case r@LogicalRelation(fs: HadoopFsRelation, _, _, _) => + val newSchema = StructType(fs.dataSchema.fields).add(METADATA_ROW_IDX_FIELD) + + // This is required to ensure that row index is correctly calculated. + val newFileFormat = fs.fileFormat.asInstanceOf[DeltaParquetFileFormat] + .copy(isSplittable = false, disablePushDowns = true) + + val newFs = fs.copy(dataSchema = newSchema, fileFormat = newFileFormat)(context.spark) + + val newOutput = r.output :+ rowIdxAttr + r.copy(relation = newFs, output = newOutput) + case p@Project(projectList, _) => + val newProjectList = projectList :+ rowIdxAttr + p.copy(projectList = newProjectList) + } + + Dataset.ofRows(context.spark, newPlan) + } + + /** + * The result is scanning target table with touched files, and added an extra + * [[METADATA_ROW_DEL_COL]] to indicate whether filtered by joining with source table in first + * step. + */ + private def getTouchedTargetDF(touchedFiles: Map[String, (Roaring64Bitmap, AddFile)]) + : DataFrame = { + // Generate a new target dataframe that has same output attributes exprIds as the target plan. + // This allows us to apply the existing resolved update/insert expressions. + val baseTargetDF = buildTargetDFWithFiles(touchedFiles.values.map(_._2).toSeq) + + val newPlan = { + val rowDelAttr = AttributeReference( + METADATA_ROW_DEL_COL, + METADATA_ROW_DEL_FIELD.dataType, + METADATA_ROW_DEL_FIELD.nullable)() + + baseTargetDF.queryExecution.analyzed.transformUp { + case r@LogicalRelation(fs: HadoopFsRelation, _, _, _) => + val newSchema = StructType(fs.dataSchema.fields).add(METADATA_ROW_DEL_FIELD) + + // This is required to ensure that row index is correctly calculated. + val newFileFormat = { + val oldFormat = fs.fileFormat.asInstanceOf[DeltaParquetFileFormat] + val dvs = touchedFiles.map(kv => (new URI(kv._1), + DeletionVectorDescriptorWithFilterType(toDeletionVector(kv._2._1), + RowIndexFilterType.UNKNOWN))) + val broadcastDVs = context.spark.sparkContext.broadcast(dvs) + + oldFormat.copy(isSplittable = false, + broadcastDvMap = Some(broadcastDVs), + disablePushDowns = true) + } + + val newFs = fs.copy(dataSchema = newSchema, fileFormat = newFileFormat)(context.spark) + + val newOutput = r.output :+ rowDelAttr + r.copy(relation = newFs, output = newOutput) + case p@Project(projectList, _) => + val newProjectList = projectList :+ rowDelAttr + p.copy(projectList = newProjectList) + } + } + + val df = Dataset.ofRows(context.spark, newPlan) + .withColumn(TARGET_ROW_PRESENT_COL, lit(true)) + + df + } + + /** + * Generate a plan by calculating modified rows. It's computed by joining source and target + * tables, where target table has been filtered by (`__metadata_file_name`, + * `__metadata_row_idx`) pairs collected in first step. + * + * Schema of `modifiedDF`: + * + * targetSchema + ROW_DROPPED_COL + TARGET_ROW_PRESENT_COL + + * SOURCE_ROW_PRESENT_COL + INCR_METRICS_COL + * INCR_METRICS_COL + * + * It consists of several parts: + * + * 1. Unmatched source rows which are inserted + * 2. Unmatched source rows which are deleted + * 3. Target rows which are updated + * 4. Target rows which are deleted + */ + private def getModifiedDF(touchedFiles: Map[String, (Roaring64Bitmap, AddFile)]): DataFrame = { + val sourceDF = this.sourceDF + .withColumn(SOURCE_ROW_PRESENT_COL, new Column(incrSourceRowCountExpr)) + + val targetDF = getTouchedTargetDF(touchedFiles) + + val joinedDF = { + val joinType = if (hasNoInserts && + context.spark.conf.get(DeltaSQLConf.MERGE_MATCHED_ONLY_ENABLED)) { + "inner" + } else { + "leftOuter" + } + val matchedTargetDF = targetDF.filter(METADATA_ROW_DEL_COL) + .drop(METADATA_ROW_DEL_COL) + + sourceDF.join(matchedTargetDF, new Column(context.cmd.condition), joinType) + } + + val modifiedRowsSchema = context.deltaTxn.metadata.schema + .add(ROW_DROPPED_FIELD) + .add(TARGET_ROW_PRESENT_FIELD.copy(nullable = true)) + .add(SOURCE_ROW_PRESENT_FIELD.copy(nullable = true)) + .add(INCR_METRICS_FIELD) + + // Here we generate a case when statement to handle all cases: + // CASE + // WHEN + // CASE WHEN + // + // WHEN + // + // ELSE + // + // WHEN + // CASE WHEN + // + // WHEN + // + // ELSE + // + // END + + val notMatchedConditions = context.cmd.notMatchedClauses.map(clauseCondition) + val notMatchedExpr = { + val deletedNotMatchedRow = { + targetOutputCols :+ + Literal.TrueLiteral :+ + Literal.FalseLiteral :+ + Literal(null) :+ + Literal.TrueLiteral + } + if (context.cmd.notMatchedClauses.isEmpty) { + // If there no `WHEN NOT MATCHED` clause, we should just delete not matched row + deletedNotMatchedRow + } else { + val notMatchedOutputs = context.cmd.notMatchedClauses.map(clauseOutput) + modifiedRowsSchema.zipWithIndex.map { + case (_, idx) => + CaseWhen(notMatchedConditions.zip(notMatchedOutputs.map(_(idx))), + deletedNotMatchedRow(idx)) + } + } + } + + val matchedConditions = context.cmd.matchedClauses.map(clauseCondition) + val matchedOutputs = context.cmd.matchedClauses.map(clauseOutput) + val matchedExprs = { + val notMatchedRow = { + targetOutputCols :+ + Literal.FalseLiteral :+ + Literal.TrueLiteral :+ + Literal(null) :+ + Literal.TrueLiteral + } + if (context.cmd.matchedClauses.isEmpty) { + // If there is not matched clause, this is insert only, we should delete this row. + notMatchedRow + } else { + modifiedRowsSchema.zipWithIndex.map { + case (_, idx) => + CaseWhen(matchedConditions.zip(matchedOutputs.map(_(idx))), + notMatchedRow(idx)) + } + } + } + + val sourceRowHasNoMatch = col(TARGET_ROW_PRESENT_COL).isNull.expr + + val modifiedCols = modifiedRowsSchema.zipWithIndex.map { case (col, idx) => + val caseWhen = CaseWhen( + Seq(sourceRowHasNoMatch -> notMatchedExpr(idx)), + matchedExprs(idx)) + new Column(Alias(caseWhen, col.name)()) + } + + val modifiedDF = { + + // Make this a udf to avoid catalyst to be too aggressive to even remove the join! + val noopRowDroppedCol = udf(new GpuDeltaNoopUDF()).apply(!col(ROW_DROPPED_COL)) + + val modifiedDF = joinedDF.select(modifiedCols: _*) + // This will not filter anything since they always return true, but we need to avoid + // catalyst from optimizing these udf + .filter(noopRowDroppedCol && col(INCR_METRICS_COL)) + .drop(ROW_DROPPED_COL, INCR_METRICS_COL, TARGET_ROW_PRESENT_COL, SOURCE_ROW_PRESENT_COL) + + repartitionIfNeeded(modifiedDF) + } + + modifiedDF + } + + private def getUnmodifiedDF(touchedFiles: Map[String, (Roaring64Bitmap, AddFile)]): DataFrame = { + getTouchedTargetDF(touchedFiles) + .filter(!col(METADATA_ROW_DEL_COL)) + .drop(TARGET_ROW_PRESENT_COL, METADATA_ROW_DEL_COL) + } +} + + +object MergeExecutor { + + /** + * Spark UI will track all normal accumulators along with Spark tasks to show them on Web UI. + * However, the accumulator used by `MergeIntoCommand` can store a very large value since it + * tracks all files that need to be rewritten. We should ask Spark UI to not remember it, + * otherwise, the UI data may consume lots of memory. Hence, we use the prefix `internal.metrics.` + * to make this accumulator become an internal accumulator, so that it will not be tracked by + * Spark UI. + */ + val TOUCHED_FILES_ACCUM_NAME = "internal.metrics.MergeIntoDelta.touchedFiles" + + val ROW_ID_COL = "_row_id_" + val FILE_PATH_COL: String = GpuDeltaParquetFileFormatUtils.FILE_PATH_COL + val SOURCE_ROW_PRESENT_COL: String = "_source_row_present_" + val SOURCE_ROW_PRESENT_FIELD: StructField = StructField(SOURCE_ROW_PRESENT_COL, BooleanType, + nullable = false) + val TARGET_ROW_PRESENT_COL: String = "_target_row_present_" + val TARGET_ROW_PRESENT_FIELD: StructField = StructField(TARGET_ROW_PRESENT_COL, BooleanType, + nullable = false) + val ROW_DROPPED_COL: String = GpuDeltaMergeConstants.ROW_DROPPED_COL + val ROW_DROPPED_FIELD: StructField = StructField(ROW_DROPPED_COL, BooleanType, nullable = false) + val INCR_METRICS_COL: String = "_incr_metrics_" + val INCR_METRICS_FIELD: StructField = StructField(INCR_METRICS_COL, BooleanType, nullable = false) + val INCR_ROW_COUNT_COL: String = "_incr_row_count_" + + // Some Delta versions use Literal(null) which translates to a literal of NullType instead + // of the Literal(null, StringType) which is needed, so using a fixed version here + // rather than the version from Delta Lake. + val CDC_TYPE_NOT_CDC_LITERAL: Literal = Literal(null, StringType) + + def toDeletionVector(bitmap: Roaring64Bitmap): DeletionVectorDescriptor = { + DeletionVectorDescriptor.inlineInLog(RoaringBitmapWrapper(bitmap).serializeToBytes(), + bitmap.getLongCardinality) + } + + /** Count the number of distinct partition values among the AddFiles in the given set. */ + def totalBytesAndDistinctPartitionValues(files: Seq[FileAction]): (Long, Int) = { + val distinctValues = new mutable.HashSet[Map[String, String]]() + var bytes = 0L + val iter = files.collect { case a: AddFile => a }.iterator + while (iter.hasNext) { + val file = iter.next() + distinctValues += file.partitionValues + bytes += file.size + } + // If the only distinct value map is an empty map, then it must be an unpartitioned table. + // Return 0 in that case. + val numDistinctValues = + if (distinctValues.size == 1 && distinctValues.head.isEmpty) 0 else distinctValues.size + (bytes, numDistinctValues) + } +} \ No newline at end of file diff --git a/delta-lake/delta-spark341db/src/main/scala/com/nvidia/spark/rapids/delta/GpuDeltaParquetFileFormat.scala b/delta-lake/delta-spark341db/src/main/scala/com/nvidia/spark/rapids/delta/GpuDeltaParquetFileFormat.scala index 969d005b573..604ed826397 100644 --- a/delta-lake/delta-spark341db/src/main/scala/com/nvidia/spark/rapids/delta/GpuDeltaParquetFileFormat.scala +++ b/delta-lake/delta-spark341db/src/main/scala/com/nvidia/spark/rapids/delta/GpuDeltaParquetFileFormat.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023, NVIDIA CORPORATION. + * Copyright (c) 2023-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -16,20 +16,32 @@ package com.nvidia.spark.rapids.delta +import java.net.URI + import com.databricks.sql.transaction.tahoe.{DeltaColumnMappingMode, DeltaParquetFileFormat, IdMapping} -import com.databricks.sql.transaction.tahoe.DeltaParquetFileFormat.IS_ROW_DELETED_COLUMN_NAME -import com.nvidia.spark.rapids.SparkPlanMeta +import com.databricks.sql.transaction.tahoe.DeltaParquetFileFormat.{DeletionVectorDescriptorWithFilterType, IS_ROW_DELETED_COLUMN_NAME} +import com.nvidia.spark.rapids.{GpuMetric, RapidsConf, SparkPlanMeta} +import com.nvidia.spark.rapids.delta.GpuDeltaParquetFileFormatUtils.addMetadataColumnToIterator +import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path +import org.apache.spark.broadcast.Broadcast import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.FileSourceScanExec +import org.apache.spark.sql.execution.datasources.PartitionedFile import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.vectorized.ColumnarBatch case class GpuDeltaParquetFileFormat( override val columnMappingMode: DeltaColumnMappingMode, override val referenceSchema: StructType, - isSplittable: Boolean) extends GpuDeltaParquetFileFormatBase { + isSplittable: Boolean, + disablePushDown: Boolean, + broadcastDvMap: Option[Broadcast[Map[URI, DeletionVectorDescriptorWithFilterType]]] +) extends GpuDeltaParquetFileFormatBase { if (columnMappingMode == IdMapping) { val requiredReadConf = SQLConf.PARQUET_FIELD_ID_READ_ENABLED @@ -44,6 +56,46 @@ case class GpuDeltaParquetFileFormat( sparkSession: SparkSession, options: Map[String, String], path: Path): Boolean = isSplittable + + override def buildReaderWithPartitionValuesAndMetrics( + sparkSession: SparkSession, + dataSchema: StructType, + partitionSchema: StructType, + requiredSchema: StructType, + filters: Seq[Filter], + options: Map[String, String], + hadoopConf: Configuration, + metrics: Map[String, GpuMetric], + alluxioPathReplacementMap: Option[Map[String, String]]) + : PartitionedFile => Iterator[InternalRow] = { + + val dataReader = super.buildReaderWithPartitionValuesAndMetrics( + sparkSession, + dataSchema, + partitionSchema, + requiredSchema, + filters, + options, + hadoopConf, + metrics, + alluxioPathReplacementMap) + + val delVecs = broadcastDvMap + val maxDelVecScatterBatchSize = RapidsConf + .DELTA_LOW_SHUFFLE_MERGE_SCATTER_DEL_VECTOR_BATCH_SIZE + .get(sparkSession.sessionState.conf) + + (file: PartitionedFile) => { + val input = dataReader(file) + val dv = delVecs.flatMap(_.value.get(new URI(file.filePath.toString()))) + .map(dv => RoaringBitmapWrapper.deserializeFromBytes(dv.descriptor.inlineData).inner) + addMetadataColumnToIterator(prepareSchema(requiredSchema), + dv, + input.asInstanceOf[Iterator[ColumnarBatch]], + maxDelVecScatterBatchSize + ).asInstanceOf[Iterator[InternalRow]] + } + } } object GpuDeltaParquetFileFormat { @@ -60,6 +112,7 @@ object GpuDeltaParquetFileFormat { } def convertToGpu(fmt: DeltaParquetFileFormat): GpuDeltaParquetFileFormat = { - GpuDeltaParquetFileFormat(fmt.columnMappingMode, fmt.referenceSchema, fmt.isSplittable) + GpuDeltaParquetFileFormat(fmt.columnMappingMode, fmt.referenceSchema, fmt.isSplittable, + fmt.disablePushDowns, fmt.broadcastDvMap) } } diff --git a/delta-lake/delta-spark341db/src/main/scala/com/nvidia/spark/rapids/delta/shims/MergeIntoCommandMetaShim.scala b/delta-lake/delta-spark341db/src/main/scala/com/nvidia/spark/rapids/delta/shims/MergeIntoCommandMetaShim.scala index 8e13a9e4b5a..5a2b4e7b52e 100644 --- a/delta-lake/delta-spark341db/src/main/scala/com/nvidia/spark/rapids/delta/shims/MergeIntoCommandMetaShim.scala +++ b/delta-lake/delta-spark341db/src/main/scala/com/nvidia/spark/rapids/delta/shims/MergeIntoCommandMetaShim.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023, NVIDIA CORPORATION. + * Copyright (c) 2023-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,13 +17,14 @@ package com.nvidia.spark.rapids.delta.shims import com.databricks.sql.transaction.tahoe.commands.{MergeIntoCommand, MergeIntoCommandEdge} -import com.databricks.sql.transaction.tahoe.rapids.{GpuDeltaLog, GpuMergeIntoCommand} -import com.nvidia.spark.rapids.RapidsConf +import com.databricks.sql.transaction.tahoe.rapids.{GpuDeltaLog, GpuLowShuffleMergeCommand, GpuMergeIntoCommand} +import com.nvidia.spark.rapids.{RapidsConf, RapidsReaderType} import com.nvidia.spark.rapids.delta.{MergeIntoCommandEdgeMeta, MergeIntoCommandMeta} +import org.apache.spark.internal.Logging import org.apache.spark.sql.execution.command.RunnableCommand -object MergeIntoCommandMetaShim { +object MergeIntoCommandMetaShim extends Logging { def tagForGpu(meta: MergeIntoCommandMeta, mergeCmd: MergeIntoCommand): Unit = { // see https://github.com/NVIDIA/spark-rapids/issues/8415 for more information if (mergeCmd.notMatchedBySourceClauses.nonEmpty) { @@ -39,26 +40,82 @@ object MergeIntoCommandMetaShim { } def convertToGpu(mergeCmd: MergeIntoCommand, conf: RapidsConf): RunnableCommand = { - GpuMergeIntoCommand( - mergeCmd.source, - mergeCmd.target, - new GpuDeltaLog(mergeCmd.targetFileIndex.deltaLog, conf), - mergeCmd.condition, - mergeCmd.matchedClauses, - mergeCmd.notMatchedClauses, - mergeCmd.notMatchedBySourceClauses, - mergeCmd.migratedSchema)(conf) + // TODO: Currently we only support low shuffler merge only when parquet per file read is enabled + // due to the limitation of implementing row index metadata column. + if (conf.isDeltaLowShuffleMergeEnabled) { + if (conf.isParquetPerFileReadEnabled) { + GpuLowShuffleMergeCommand( + mergeCmd.source, + mergeCmd.target, + new GpuDeltaLog(mergeCmd.targetFileIndex.deltaLog, conf), + mergeCmd.condition, + mergeCmd.matchedClauses, + mergeCmd.notMatchedClauses, + mergeCmd.notMatchedBySourceClauses, + mergeCmd.migratedSchema)(conf) + } else { + logWarning(s"""Low shuffle merge disabled since ${RapidsConf.PARQUET_READER_TYPE} is + not set to ${RapidsReaderType.PERFILE}. Falling back to classic merge.""") + GpuMergeIntoCommand( + mergeCmd.source, + mergeCmd.target, + new GpuDeltaLog(mergeCmd.targetFileIndex.deltaLog, conf), + mergeCmd.condition, + mergeCmd.matchedClauses, + mergeCmd.notMatchedClauses, + mergeCmd.notMatchedBySourceClauses, + mergeCmd.migratedSchema)(conf) + } + } else { + GpuMergeIntoCommand( + mergeCmd.source, + mergeCmd.target, + new GpuDeltaLog(mergeCmd.targetFileIndex.deltaLog, conf), + mergeCmd.condition, + mergeCmd.matchedClauses, + mergeCmd.notMatchedClauses, + mergeCmd.notMatchedBySourceClauses, + mergeCmd.migratedSchema)(conf) + } } def convertToGpu(mergeCmd: MergeIntoCommandEdge, conf: RapidsConf): RunnableCommand = { - GpuMergeIntoCommand( - mergeCmd.source, - mergeCmd.target, - new GpuDeltaLog(mergeCmd.targetFileIndex.deltaLog, conf), - mergeCmd.condition, - mergeCmd.matchedClauses, - mergeCmd.notMatchedClauses, - mergeCmd.notMatchedBySourceClauses, - mergeCmd.migratedSchema)(conf) + // TODO: Currently we only support low shuffler merge only when parquet per file read is enabled + // due to the limitation of implementing row index metadata column. + if (conf.isDeltaLowShuffleMergeEnabled) { + if (conf.isParquetPerFileReadEnabled) { + GpuLowShuffleMergeCommand( + mergeCmd.source, + mergeCmd.target, + new GpuDeltaLog(mergeCmd.targetFileIndex.deltaLog, conf), + mergeCmd.condition, + mergeCmd.matchedClauses, + mergeCmd.notMatchedClauses, + mergeCmd.notMatchedBySourceClauses, + mergeCmd.migratedSchema)(conf) + } else { + logWarning(s"""Low shuffle merge is still disable since ${RapidsConf.PARQUET_READER_TYPE} is + not set to ${RapidsReaderType.PERFILE}. Falling back to classic merge.""") + GpuMergeIntoCommand( + mergeCmd.source, + mergeCmd.target, + new GpuDeltaLog(mergeCmd.targetFileIndex.deltaLog, conf), + mergeCmd.condition, + mergeCmd.matchedClauses, + mergeCmd.notMatchedClauses, + mergeCmd.notMatchedBySourceClauses, + mergeCmd.migratedSchema)(conf) + } + } else { + GpuMergeIntoCommand( + mergeCmd.source, + mergeCmd.target, + new GpuDeltaLog(mergeCmd.targetFileIndex.deltaLog, conf), + mergeCmd.condition, + mergeCmd.matchedClauses, + mergeCmd.notMatchedClauses, + mergeCmd.notMatchedBySourceClauses, + mergeCmd.migratedSchema)(conf) + } } } diff --git a/docs/additional-functionality/advanced_configs.md b/docs/additional-functionality/advanced_configs.md index 3231b7b3069..941ab4046e6 100644 --- a/docs/additional-functionality/advanced_configs.md +++ b/docs/additional-functionality/advanced_configs.md @@ -73,6 +73,12 @@ Name | Description | Default Value | Applicable at spark.rapids.sql.csv.read.double.enabled|CSV reading is not 100% compatible when reading doubles.|true|Runtime spark.rapids.sql.csv.read.float.enabled|CSV reading is not 100% compatible when reading floats.|true|Runtime spark.rapids.sql.decimalOverflowGuarantees|FOR TESTING ONLY. DO NOT USE IN PRODUCTION. Please see the decimal section of the compatibility documents for more information on this config.|true|Runtime +spark.rapids.sql.delta.lowShuffleMerge.deletionVector.broadcast.threshold|Currently we need to broadcast deletion vector to all executors to perform low shuffle merge. When we detect the deletion vector broadcast size is larger than this value, we will fallback to normal shuffle merge.|20971520|Runtime +spark.rapids.sql.delta.lowShuffleMerge.enabled|Option to turn on the low shuffle merge for Delta Lake. Currently there are some limitations for this feature: +1. We only support Databricks Runtime 13.3 and Deltalake 2.4. +2. The file scan mode must be set to PERFILE +3. The deletion vector size must be smaller than spark.rapids.sql.delta.lowShuffleMerge.deletionVector.broadcast.threshold +|false|Runtime spark.rapids.sql.detectDeltaCheckpointQueries|Queries against Delta Lake _delta_log checkpoint Parquet files are not efficient on the GPU. When this option is enabled, the plugin will attempt to detect these queries and fall back to the CPU.|true|Runtime spark.rapids.sql.detectDeltaLogQueries|Queries against Delta Lake _delta_log JSON files are not efficient on the GPU. When this option is enabled, the plugin will attempt to detect these queries and fall back to the CPU.|true|Runtime spark.rapids.sql.fast.sample|Option to turn on fast sample. If enable it is inconsistent with CPU sample because of GPU sample algorithm is inconsistent with CPU.|false|Runtime diff --git a/integration_tests/src/main/python/delta_lake_low_shuffle_merge_test.py b/integration_tests/src/main/python/delta_lake_low_shuffle_merge_test.py new file mode 100644 index 00000000000..6935ee13751 --- /dev/null +++ b/integration_tests/src/main/python/delta_lake_low_shuffle_merge_test.py @@ -0,0 +1,165 @@ +# Copyright (c) 2024, NVIDIA CORPORATION. +# +# Licensed 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. + +import pyspark.sql.functions as f +import pytest + +from delta_lake_merge_common import * +from marks import * +from pyspark.sql.types import * +from spark_session import is_databricks133_or_later, spark_version + +delta_merge_enabled_conf = copy_and_update(delta_writes_enabled_conf, + {"spark.rapids.sql.command.MergeIntoCommand": "true", + "spark.rapids.sql.command.MergeIntoCommandEdge": "true", + "spark.rapids.sql.delta.lowShuffleMerge.enabled": "true", + "spark.rapids.sql.format.parquet.reader.type": "PERFILE"}) + +@allow_non_gpu("ColumnarToRowExec", *delta_meta_allow) +@delta_lake +@ignore_order +@pytest.mark.skipif(not ((is_databricks_runtime() and is_databricks133_or_later()) or + (not is_databricks_runtime() and spark_version().startswith("3.4"))), + reason="Delta Lake Low Shuffle Merge only supports Databricks 13.3 or OSS " + "delta 2.4") +@pytest.mark.parametrize("use_cdf", [True, False], ids=idfn) +@pytest.mark.parametrize("num_slices", num_slices_to_test, ids=idfn) +def test_delta_low_shuffle_merge_when_gpu_file_scan_override_failed(spark_tmp_path, + spark_tmp_table_factory, + use_cdf, num_slices): + # Need to eliminate duplicate keys in the source table otherwise update semantics are ambiguous + src_table_func = lambda spark: two_col_df(spark, int_gen, string_gen, num_slices=num_slices).groupBy("a").agg(f.max("b").alias("b")) + dest_table_func = lambda spark: two_col_df(spark, int_gen, string_gen, seed=1, num_slices=num_slices) + merge_sql = "MERGE INTO {dest_table} USING {src_table} ON {dest_table}.a == {src_table}.a" \ + " WHEN MATCHED THEN UPDATE SET * WHEN NOT MATCHED THEN INSERT *" + + conf = copy_and_update(delta_merge_enabled_conf, + { + "spark.rapids.sql.exec.FileSourceScanExec": "false", + # Disable auto broadcast join due to this issue: + # https://github.com/NVIDIA/spark-rapids/issues/10973 + "spark.sql.autoBroadcastJoinThreshold": "-1" + }) + assert_delta_sql_merge_collect(spark_tmp_path, spark_tmp_table_factory, use_cdf, + src_table_func, dest_table_func, merge_sql, False, conf=conf) + + + +@allow_non_gpu(*delta_meta_allow) +@delta_lake +@ignore_order +@pytest.mark.skipif(not ((is_databricks_runtime() and is_databricks133_or_later()) or + (not is_databricks_runtime() and spark_version().startswith("3.4"))), + reason="Delta Lake Low Shuffle Merge only supports Databricks 13.3 or OSS " + "delta 2.4") +@pytest.mark.parametrize("table_ranges", [(range(20), range(10)), # partial insert of source + (range(5), range(5)), # no-op insert + (range(10), range(20, 30)) # full insert of source + ], ids=idfn) +@pytest.mark.parametrize("use_cdf", [True, False], ids=idfn) +@pytest.mark.parametrize("partition_columns", [None, ["a"], ["b"], ["a", "b"]], ids=idfn) +@pytest.mark.parametrize("num_slices", num_slices_to_test, ids=idfn) +def test_delta_merge_not_match_insert_only(spark_tmp_path, spark_tmp_table_factory, table_ranges, + use_cdf, partition_columns, num_slices): + do_test_delta_merge_not_match_insert_only(spark_tmp_path, spark_tmp_table_factory, + table_ranges, use_cdf, partition_columns, + num_slices, False, delta_merge_enabled_conf) + +@allow_non_gpu(*delta_meta_allow) +@delta_lake +@ignore_order +@pytest.mark.skipif(not ((is_databricks_runtime() and is_databricks133_or_later()) or + (not is_databricks_runtime() and spark_version().startswith("3.4"))), + reason="Delta Lake Low Shuffle Merge only supports Databricks 13.3 or OSS " + "delta 2.4") +@pytest.mark.parametrize("table_ranges", [(range(10), range(20)), # partial delete of target + (range(5), range(5)), # full delete of target + (range(10), range(20, 30)) # no-op delete + ], ids=idfn) +@pytest.mark.parametrize("use_cdf", [True, False], ids=idfn) +@pytest.mark.parametrize("partition_columns", [None, ["a"], ["b"], ["a", "b"]], ids=idfn) +@pytest.mark.parametrize("num_slices", num_slices_to_test, ids=idfn) +def test_delta_merge_match_delete_only(spark_tmp_path, spark_tmp_table_factory, table_ranges, + use_cdf, partition_columns, num_slices): + do_test_delta_merge_match_delete_only(spark_tmp_path, spark_tmp_table_factory, table_ranges, + use_cdf, partition_columns, num_slices, False, + delta_merge_enabled_conf) + +@allow_non_gpu(*delta_meta_allow) +@delta_lake +@ignore_order +@pytest.mark.skipif(not ((is_databricks_runtime() and is_databricks133_or_later()) or + (not is_databricks_runtime() and spark_version().startswith("3.4"))), + reason="Delta Lake Low Shuffle Merge only supports Databricks 13.3 or OSS " + "delta 2.4") +@pytest.mark.parametrize("use_cdf", [True, False], ids=idfn) +@pytest.mark.parametrize("num_slices", num_slices_to_test, ids=idfn) +def test_delta_merge_standard_upsert(spark_tmp_path, spark_tmp_table_factory, use_cdf, num_slices): + do_test_delta_merge_standard_upsert(spark_tmp_path, spark_tmp_table_factory, use_cdf, + num_slices, False, delta_merge_enabled_conf) + +@allow_non_gpu(*delta_meta_allow) +@delta_lake +@ignore_order +@pytest.mark.skipif(not ((is_databricks_runtime() and is_databricks133_or_later()) or + (not is_databricks_runtime() and spark_version().startswith("3.4"))), + reason="Delta Lake Low Shuffle Merge only supports Databricks 13.3 or OSS " + "delta 2.4") +@pytest.mark.parametrize("use_cdf", [True, False], ids=idfn) +@pytest.mark.parametrize("merge_sql", [ + "MERGE INTO {dest_table} d USING {src_table} s ON d.a == s.a" \ + " WHEN MATCHED AND s.b > 'q' THEN UPDATE SET d.a = s.a / 2, d.b = s.b" \ + " WHEN NOT MATCHED THEN INSERT *", + "MERGE INTO {dest_table} d USING {src_table} s ON d.a == s.a" \ + " WHEN NOT MATCHED AND s.b > 'q' THEN INSERT *", + "MERGE INTO {dest_table} d USING {src_table} s ON d.a == s.a" \ + " WHEN MATCHED AND s.b > 'a' AND s.b < 'g' THEN UPDATE SET d.a = s.a / 2, d.b = s.b" \ + " WHEN MATCHED AND s.b > 'g' AND s.b < 'z' THEN UPDATE SET d.a = s.a / 4, d.b = concat('extra', s.b)" \ + " WHEN NOT MATCHED AND s.b > 'b' AND s.b < 'f' THEN INSERT *" \ + " WHEN NOT MATCHED AND s.b > 'f' AND s.b < 'z' THEN INSERT (b) VALUES ('not here')" ], ids=idfn) +@pytest.mark.parametrize("num_slices", num_slices_to_test, ids=idfn) +def test_delta_merge_upsert_with_condition(spark_tmp_path, spark_tmp_table_factory, use_cdf, merge_sql, num_slices): + do_test_delta_merge_upsert_with_condition(spark_tmp_path, spark_tmp_table_factory, use_cdf, + merge_sql, num_slices, False, + delta_merge_enabled_conf) + +@allow_non_gpu(*delta_meta_allow) +@delta_lake +@ignore_order +@pytest.mark.skipif(not ((is_databricks_runtime() and is_databricks133_or_later()) or + (not is_databricks_runtime() and spark_version().startswith("3.4"))), + reason="Delta Lake Low Shuffle Merge only supports Databricks 13.3 or OSS " + "delta 2.4") +@pytest.mark.parametrize("use_cdf", [True, False], ids=idfn) +@pytest.mark.parametrize("num_slices", num_slices_to_test, ids=idfn) +def test_delta_merge_upsert_with_unmatchable_match_condition(spark_tmp_path, spark_tmp_table_factory, use_cdf, num_slices): + do_test_delta_merge_upsert_with_unmatchable_match_condition(spark_tmp_path, + spark_tmp_table_factory, + use_cdf, + num_slices, + False, + delta_merge_enabled_conf) + +@allow_non_gpu(*delta_meta_allow) +@delta_lake +@ignore_order +@pytest.mark.skipif(not ((is_databricks_runtime() and is_databricks133_or_later()) or + (not is_databricks_runtime() and spark_version().startswith("3.4"))), + reason="Delta Lake Low Shuffle Merge only supports Databricks 13.3 or OSS " + "delta 2.4") +@pytest.mark.parametrize("use_cdf", [True, False], ids=idfn) +def test_delta_merge_update_with_aggregation(spark_tmp_path, spark_tmp_table_factory, use_cdf): + do_test_delta_merge_update_with_aggregation(spark_tmp_path, spark_tmp_table_factory, use_cdf, + delta_merge_enabled_conf) + diff --git a/integration_tests/src/main/python/delta_lake_merge_common.py b/integration_tests/src/main/python/delta_lake_merge_common.py new file mode 100644 index 00000000000..e6e9676625d --- /dev/null +++ b/integration_tests/src/main/python/delta_lake_merge_common.py @@ -0,0 +1,155 @@ +# Copyright (c) 2024, NVIDIA CORPORATION. +# +# Licensed 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. + +import pyspark.sql.functions as f +import string +from pyspark.sql.types import * + +from asserts import * +from data_gen import * +from delta_lake_utils import * +from spark_session import is_databricks_runtime + +# Databricks changes the number of files being written, so we cannot compare logs +num_slices_to_test = [10] if is_databricks_runtime() else [1, 10] + + +def make_df(spark, gen, num_slices): + return three_col_df(spark, gen, SetValuesGen(StringType(), string.ascii_lowercase), + SetValuesGen(StringType(), string.ascii_uppercase), num_slices=num_slices) + + +def delta_sql_merge_test(spark_tmp_path, spark_tmp_table_factory, use_cdf, + src_table_func, dest_table_func, merge_sql, check_func, + partition_columns=None): + data_path = spark_tmp_path + "/DELTA_DATA" + src_table = spark_tmp_table_factory.get() + + def setup_tables(spark): + setup_delta_dest_tables(spark, data_path, dest_table_func, use_cdf, partition_columns) + src_table_func(spark).createOrReplaceTempView(src_table) + + def do_merge(spark, path): + dest_table = spark_tmp_table_factory.get() + read_delta_path(spark, path).createOrReplaceTempView(dest_table) + return spark.sql(merge_sql.format(src_table=src_table, dest_table=dest_table)).collect() + with_cpu_session(setup_tables) + check_func(data_path, do_merge) + + +def assert_delta_sql_merge_collect(spark_tmp_path, spark_tmp_table_factory, use_cdf, + src_table_func, dest_table_func, merge_sql, + compare_logs, partition_columns=None, conf=None): + assert conf is not None, "conf must be set" + + def read_data(spark, path): + read_func = read_delta_path_with_cdf if use_cdf else read_delta_path + df = read_func(spark, path) + return df.sort(df.columns) + + def checker(data_path, do_merge): + cpu_path = data_path + "/CPU" + gpu_path = data_path + "/GPU" + # compare resulting dataframe from the merge operation (some older Spark versions return empty here) + cpu_result = with_cpu_session(lambda spark: do_merge(spark, cpu_path), conf=conf) + gpu_result = with_gpu_session(lambda spark: do_merge(spark, gpu_path), conf=conf) + assert_equal(cpu_result, gpu_result) + # compare merged table data results, read both via CPU to make sure GPU write can be read by CPU + cpu_result = with_cpu_session(lambda spark: read_data(spark, cpu_path).collect(), conf=conf) + gpu_result = with_cpu_session(lambda spark: read_data(spark, gpu_path).collect(), conf=conf) + assert_equal(cpu_result, gpu_result) + # Using partition columns involves sorting, and there's no guarantees on the task + # partitioning due to random sampling. + if compare_logs and not partition_columns: + with_cpu_session(lambda spark: assert_gpu_and_cpu_delta_logs_equivalent(spark, data_path)) + delta_sql_merge_test(spark_tmp_path, spark_tmp_table_factory, use_cdf, + src_table_func, dest_table_func, merge_sql, checker, partition_columns) + + +def do_test_delta_merge_not_match_insert_only(spark_tmp_path, spark_tmp_table_factory, table_ranges, + use_cdf, partition_columns, num_slices, compare_logs, + conf): + src_range, dest_range = table_ranges + src_table_func = lambda spark: make_df(spark, SetValuesGen(IntegerType(), src_range), num_slices) + dest_table_func = lambda spark: make_df(spark, SetValuesGen(IntegerType(), dest_range), num_slices) + merge_sql = "MERGE INTO {dest_table} USING {src_table} ON {dest_table}.a == {src_table}.a" \ + " WHEN NOT MATCHED THEN INSERT *" + assert_delta_sql_merge_collect(spark_tmp_path, spark_tmp_table_factory, use_cdf, + src_table_func, dest_table_func, merge_sql, compare_logs, + partition_columns, conf=conf) + + +def do_test_delta_merge_match_delete_only(spark_tmp_path, spark_tmp_table_factory, table_ranges, + use_cdf, partition_columns, num_slices, compare_logs, + conf): + src_range, dest_range = table_ranges + src_table_func = lambda spark: make_df(spark, SetValuesGen(IntegerType(), src_range), num_slices) + dest_table_func = lambda spark: make_df(spark, SetValuesGen(IntegerType(), dest_range), num_slices) + merge_sql = "MERGE INTO {dest_table} USING {src_table} ON {dest_table}.a == {src_table}.a" \ + " WHEN MATCHED THEN DELETE" + assert_delta_sql_merge_collect(spark_tmp_path, spark_tmp_table_factory, use_cdf, + src_table_func, dest_table_func, merge_sql, compare_logs, + partition_columns, conf=conf) + + +def do_test_delta_merge_standard_upsert(spark_tmp_path, spark_tmp_table_factory, use_cdf, + num_slices, compare_logs, conf): + # Need to eliminate duplicate keys in the source table otherwise update semantics are ambiguous + src_table_func = lambda spark: two_col_df(spark, int_gen, string_gen, num_slices=num_slices).groupBy("a").agg(f.max("b").alias("b")) + dest_table_func = lambda spark: two_col_df(spark, int_gen, string_gen, seed=1, num_slices=num_slices) + merge_sql = "MERGE INTO {dest_table} USING {src_table} ON {dest_table}.a == {src_table}.a" \ + " WHEN MATCHED THEN UPDATE SET * WHEN NOT MATCHED THEN INSERT *" + assert_delta_sql_merge_collect(spark_tmp_path, spark_tmp_table_factory, use_cdf, + src_table_func, dest_table_func, merge_sql, compare_logs, + conf=conf) + + +def do_test_delta_merge_upsert_with_condition(spark_tmp_path, spark_tmp_table_factory, use_cdf, + merge_sql, num_slices, compare_logs, conf): + # Need to eliminate duplicate keys in the source table otherwise update semantics are ambiguous + src_table_func = lambda spark: two_col_df(spark, int_gen, string_gen, num_slices=num_slices).groupBy("a").agg(f.max("b").alias("b")) + dest_table_func = lambda spark: two_col_df(spark, int_gen, string_gen, seed=1, num_slices=num_slices) + assert_delta_sql_merge_collect(spark_tmp_path, spark_tmp_table_factory, use_cdf, + src_table_func, dest_table_func, merge_sql, compare_logs, + conf=conf) + + +def do_test_delta_merge_upsert_with_unmatchable_match_condition(spark_tmp_path, + spark_tmp_table_factory, use_cdf, + num_slices, compare_logs, conf): + # Need to eliminate duplicate keys in the source table otherwise update semantics are ambiguous + src_table_func = lambda spark: two_col_df(spark, int_gen, string_gen, num_slices=num_slices).groupBy("a").agg(f.max("b").alias("b")) + dest_table_func = lambda spark: two_col_df(spark, SetValuesGen(IntegerType(), range(100)), string_gen, seed=1, num_slices=num_slices) + merge_sql = "MERGE INTO {dest_table} USING {src_table} ON {dest_table}.a == {src_table}.a" \ + " WHEN MATCHED AND {dest_table}.a > 100 THEN UPDATE SET *" + assert_delta_sql_merge_collect(spark_tmp_path, spark_tmp_table_factory, use_cdf, + src_table_func, dest_table_func, merge_sql, compare_logs, + conf=conf) + + +def do_test_delta_merge_update_with_aggregation(spark_tmp_path, spark_tmp_table_factory, use_cdf, + conf): + # Need to eliminate duplicate keys in the source table otherwise update semantics are ambiguous + src_table_func = lambda spark: spark.range(10).withColumn("x", f.col("id") + 1) \ + .select(f.col("id"), (f.col("x") + 1).alias("x")) \ + .drop_duplicates(["id"]) \ + .limit(10) + dest_table_func = lambda spark: spark.range(5).withColumn("x", f.col("id") + 1) + merge_sql = "MERGE INTO {dest_table} USING {src_table} ON {dest_table}.id == {src_table}.id" \ + " WHEN MATCHED THEN UPDATE SET {dest_table}.x = {src_table}.x + 2" \ + " WHEN NOT MATCHED AND {src_table}.x < 7 THEN INSERT *" + + assert_delta_sql_merge_collect(spark_tmp_path, spark_tmp_table_factory, use_cdf, + src_table_func, dest_table_func, merge_sql, + compare_logs=False, conf=conf) diff --git a/integration_tests/src/main/python/delta_lake_merge_test.py b/integration_tests/src/main/python/delta_lake_merge_test.py index 0880db16434..5c3bb915ddb 100644 --- a/integration_tests/src/main/python/delta_lake_merge_test.py +++ b/integration_tests/src/main/python/delta_lake_merge_test.py @@ -14,66 +14,17 @@ import pyspark.sql.functions as f import pytest -import string -from asserts import * -from data_gen import * -from delta_lake_utils import * +from delta_lake_merge_common import * from marks import * from pyspark.sql.types import * from spark_session import is_before_spark_320, is_databricks_runtime, spark_version -# Databricks changes the number of files being written, so we cannot compare logs -num_slices_to_test = [10] if is_databricks_runtime() else [1, 10] delta_merge_enabled_conf = copy_and_update(delta_writes_enabled_conf, {"spark.rapids.sql.command.MergeIntoCommand": "true", "spark.rapids.sql.command.MergeIntoCommandEdge": "true"}) -def make_df(spark, gen, num_slices): - return three_col_df(spark, gen, SetValuesGen(StringType(), string.ascii_lowercase), - SetValuesGen(StringType(), string.ascii_uppercase), num_slices=num_slices) - -def delta_sql_merge_test(spark_tmp_path, spark_tmp_table_factory, use_cdf, - src_table_func, dest_table_func, merge_sql, check_func, - partition_columns=None): - data_path = spark_tmp_path + "/DELTA_DATA" - src_table = spark_tmp_table_factory.get() - def setup_tables(spark): - setup_delta_dest_tables(spark, data_path, dest_table_func, use_cdf, partition_columns) - src_table_func(spark).createOrReplaceTempView(src_table) - def do_merge(spark, path): - dest_table = spark_tmp_table_factory.get() - read_delta_path(spark, path).createOrReplaceTempView(dest_table) - return spark.sql(merge_sql.format(src_table=src_table, dest_table=dest_table)).collect() - with_cpu_session(setup_tables) - check_func(data_path, do_merge) - -def assert_delta_sql_merge_collect(spark_tmp_path, spark_tmp_table_factory, use_cdf, - src_table_func, dest_table_func, merge_sql, - compare_logs, partition_columns=None, - conf=delta_merge_enabled_conf): - def read_data(spark, path): - read_func = read_delta_path_with_cdf if use_cdf else read_delta_path - df = read_func(spark, path) - return df.sort(df.columns) - def checker(data_path, do_merge): - cpu_path = data_path + "/CPU" - gpu_path = data_path + "/GPU" - # compare resulting dataframe from the merge operation (some older Spark versions return empty here) - cpu_result = with_cpu_session(lambda spark: do_merge(spark, cpu_path), conf=conf) - gpu_result = with_gpu_session(lambda spark: do_merge(spark, gpu_path), conf=conf) - assert_equal(cpu_result, gpu_result) - # compare merged table data results, read both via CPU to make sure GPU write can be read by CPU - cpu_result = with_cpu_session(lambda spark: read_data(spark, cpu_path).collect(), conf=conf) - gpu_result = with_cpu_session(lambda spark: read_data(spark, gpu_path).collect(), conf=conf) - assert_equal(cpu_result, gpu_result) - # Using partition columns involves sorting, and there's no guarantees on the task - # partitioning due to random sampling. - if compare_logs and not partition_columns: - with_cpu_session(lambda spark: assert_gpu_and_cpu_delta_logs_equivalent(spark, data_path)) - delta_sql_merge_test(spark_tmp_path, spark_tmp_table_factory, use_cdf, - src_table_func, dest_table_func, merge_sql, checker, partition_columns) @allow_non_gpu(delta_write_fallback_allow, *delta_meta_allow) @delta_lake @@ -162,16 +113,9 @@ def test_delta_merge_partial_fallback_via_conf(spark_tmp_path, spark_tmp_table_f @pytest.mark.parametrize("num_slices", num_slices_to_test, ids=idfn) def test_delta_merge_not_match_insert_only(spark_tmp_path, spark_tmp_table_factory, table_ranges, use_cdf, partition_columns, num_slices): - src_range, dest_range = table_ranges - src_table_func = lambda spark: make_df(spark, SetValuesGen(IntegerType(), src_range), num_slices) - dest_table_func = lambda spark: make_df(spark, SetValuesGen(IntegerType(), dest_range), num_slices) - merge_sql = "MERGE INTO {dest_table} USING {src_table} ON {dest_table}.a == {src_table}.a" \ - " WHEN NOT MATCHED THEN INSERT *" - # Non-deterministic input for each task means we can only reliably compare record counts when using only one task - compare_logs = num_slices == 1 - assert_delta_sql_merge_collect(spark_tmp_path, spark_tmp_table_factory, use_cdf, - src_table_func, dest_table_func, merge_sql, compare_logs, - partition_columns) + do_test_delta_merge_not_match_insert_only(spark_tmp_path, spark_tmp_table_factory, + table_ranges, use_cdf, partition_columns, + num_slices, num_slices == 1, delta_merge_enabled_conf) @allow_non_gpu(*delta_meta_allow) @delta_lake @@ -186,16 +130,9 @@ def test_delta_merge_not_match_insert_only(spark_tmp_path, spark_tmp_table_facto @pytest.mark.parametrize("num_slices", num_slices_to_test, ids=idfn) def test_delta_merge_match_delete_only(spark_tmp_path, spark_tmp_table_factory, table_ranges, use_cdf, partition_columns, num_slices): - src_range, dest_range = table_ranges - src_table_func = lambda spark: make_df(spark, SetValuesGen(IntegerType(), src_range), num_slices) - dest_table_func = lambda spark: make_df(spark, SetValuesGen(IntegerType(), dest_range), num_slices) - merge_sql = "MERGE INTO {dest_table} USING {src_table} ON {dest_table}.a == {src_table}.a" \ - " WHEN MATCHED THEN DELETE" - # Non-deterministic input for each task means we can only reliably compare record counts when using only one task - compare_logs = num_slices == 1 - assert_delta_sql_merge_collect(spark_tmp_path, spark_tmp_table_factory, use_cdf, - src_table_func, dest_table_func, merge_sql, compare_logs, - partition_columns) + do_test_delta_merge_match_delete_only(spark_tmp_path, spark_tmp_table_factory, table_ranges, + use_cdf, partition_columns, num_slices, num_slices == 1, + delta_merge_enabled_conf) @allow_non_gpu(*delta_meta_allow) @delta_lake @@ -204,15 +141,9 @@ def test_delta_merge_match_delete_only(spark_tmp_path, spark_tmp_table_factory, @pytest.mark.parametrize("use_cdf", [True, False], ids=idfn) @pytest.mark.parametrize("num_slices", num_slices_to_test, ids=idfn) def test_delta_merge_standard_upsert(spark_tmp_path, spark_tmp_table_factory, use_cdf, num_slices): - # Need to eliminate duplicate keys in the source table otherwise update semantics are ambiguous - src_table_func = lambda spark: two_col_df(spark, int_gen, string_gen, num_slices=num_slices).groupBy("a").agg(f.max("b").alias("b")) - dest_table_func = lambda spark: two_col_df(spark, int_gen, string_gen, seed=1, num_slices=num_slices) - merge_sql = "MERGE INTO {dest_table} USING {src_table} ON {dest_table}.a == {src_table}.a" \ - " WHEN MATCHED THEN UPDATE SET * WHEN NOT MATCHED THEN INSERT *" - # Non-deterministic input for each task means we can only reliably compare record counts when using only one task - compare_logs = num_slices == 1 - assert_delta_sql_merge_collect(spark_tmp_path, spark_tmp_table_factory, use_cdf, - src_table_func, dest_table_func, merge_sql, compare_logs) + do_test_delta_merge_standard_upsert(spark_tmp_path, spark_tmp_table_factory, use_cdf, + num_slices, num_slices == 1, delta_merge_enabled_conf) + @allow_non_gpu(*delta_meta_allow) @delta_lake @@ -232,13 +163,10 @@ def test_delta_merge_standard_upsert(spark_tmp_path, spark_tmp_table_factory, us " WHEN NOT MATCHED AND s.b > 'f' AND s.b < 'z' THEN INSERT (b) VALUES ('not here')" ], ids=idfn) @pytest.mark.parametrize("num_slices", num_slices_to_test, ids=idfn) def test_delta_merge_upsert_with_condition(spark_tmp_path, spark_tmp_table_factory, use_cdf, merge_sql, num_slices): - # Need to eliminate duplicate keys in the source table otherwise update semantics are ambiguous - src_table_func = lambda spark: two_col_df(spark, int_gen, string_gen, num_slices=num_slices).groupBy("a").agg(f.max("b").alias("b")) - dest_table_func = lambda spark: two_col_df(spark, int_gen, string_gen, seed=1, num_slices=num_slices) - # Non-deterministic input for each task means we can only reliably compare record counts when using only one task - compare_logs = num_slices == 1 - assert_delta_sql_merge_collect(spark_tmp_path, spark_tmp_table_factory, use_cdf, - src_table_func, dest_table_func, merge_sql, compare_logs) + do_test_delta_merge_upsert_with_condition(spark_tmp_path, spark_tmp_table_factory, use_cdf, + merge_sql, num_slices, num_slices == 1, + delta_merge_enabled_conf) + @allow_non_gpu(*delta_meta_allow) @delta_lake @@ -247,15 +175,10 @@ def test_delta_merge_upsert_with_condition(spark_tmp_path, spark_tmp_table_facto @pytest.mark.parametrize("use_cdf", [True, False], ids=idfn) @pytest.mark.parametrize("num_slices", num_slices_to_test, ids=idfn) def test_delta_merge_upsert_with_unmatchable_match_condition(spark_tmp_path, spark_tmp_table_factory, use_cdf, num_slices): - # Need to eliminate duplicate keys in the source table otherwise update semantics are ambiguous - src_table_func = lambda spark: two_col_df(spark, int_gen, string_gen, num_slices=num_slices).groupBy("a").agg(f.max("b").alias("b")) - dest_table_func = lambda spark: two_col_df(spark, SetValuesGen(IntegerType(), range(100)), string_gen, seed=1, num_slices=num_slices) - merge_sql = "MERGE INTO {dest_table} USING {src_table} ON {dest_table}.a == {src_table}.a" \ - " WHEN MATCHED AND {dest_table}.a > 100 THEN UPDATE SET *" - # Non-deterministic input for each task means we can only reliably compare record counts when using only one task - compare_logs = num_slices == 1 - assert_delta_sql_merge_collect(spark_tmp_path, spark_tmp_table_factory, use_cdf, - src_table_func, dest_table_func, merge_sql, compare_logs) + do_test_delta_merge_upsert_with_unmatchable_match_condition(spark_tmp_path, + spark_tmp_table_factory, use_cdf, + num_slices, num_slices == 1, + delta_merge_enabled_conf) @allow_non_gpu(*delta_meta_allow) @delta_lake @@ -263,18 +186,8 @@ def test_delta_merge_upsert_with_unmatchable_match_condition(spark_tmp_path, spa @pytest.mark.skipif(is_before_spark_320(), reason="Delta Lake writes are not supported before Spark 3.2.x") @pytest.mark.parametrize("use_cdf", [True, False], ids=idfn) def test_delta_merge_update_with_aggregation(spark_tmp_path, spark_tmp_table_factory, use_cdf): - # Need to eliminate duplicate keys in the source table otherwise update semantics are ambiguous - src_table_func = lambda spark: spark.range(10).withColumn("x", f.col("id") + 1)\ - .select(f.col("id"), (f.col("x") + 1).alias("x"))\ - .drop_duplicates(["id"])\ - .limit(10) - dest_table_func = lambda spark: spark.range(5).withColumn("x", f.col("id") + 1) - merge_sql = "MERGE INTO {dest_table} USING {src_table} ON {dest_table}.id == {src_table}.id" \ - " WHEN MATCHED THEN UPDATE SET {dest_table}.x = {src_table}.x + 2" \ - " WHEN NOT MATCHED AND {src_table}.x < 7 THEN INSERT *" - - assert_delta_sql_merge_collect(spark_tmp_path, spark_tmp_table_factory, use_cdf, - src_table_func, dest_table_func, merge_sql, compare_logs=False) + do_test_delta_merge_update_with_aggregation(spark_tmp_path, spark_tmp_table_factory, use_cdf, + delta_merge_enabled_conf) @allow_non_gpu(*delta_meta_allow) @delta_lake diff --git a/pom.xml b/pom.xml index 06947857521..3ff87c3cb97 100644 --- a/pom.xml +++ b/pom.xml @@ -733,6 +733,7 @@ --> -Xlint:all,-serial,-path,-try,-processing|-Werror 1.16.0 + 1.0.6 ${ucx.baseVersion} true @@ -1016,6 +1017,15 @@ ${alluxio.client.version} provided + + + org.roaringbitmap + RoaringBitmap + ${roaringbitmap.version} + compile + org.scalatest scalatest_${scala.binary.version} diff --git a/scala2.13/aggregator/pom.xml b/scala2.13/aggregator/pom.xml index 1d70c76f037..198b62d5fa6 100644 --- a/scala2.13/aggregator/pom.xml +++ b/scala2.13/aggregator/pom.xml @@ -94,6 +94,10 @@ com.google.flatbuffers ${rapids.shade.package}.com.google.flatbuffers + + org.roaringbitmap + ${rapids.shade.package}.org.roaringbitmap + diff --git a/scala2.13/pom.xml b/scala2.13/pom.xml index cbc4aecbd26..e32a64f0529 100644 --- a/scala2.13/pom.xml +++ b/scala2.13/pom.xml @@ -733,6 +733,7 @@ --> -Xlint:all,-serial,-path,-try,-processing|-Werror 1.16.0 + 1.0.6 ${ucx.baseVersion} true @@ -1016,6 +1017,15 @@ ${alluxio.client.version} provided + + + org.roaringbitmap + RoaringBitmap + ${roaringbitmap.version} + compile + org.scalatest scalatest_${scala.binary.version} diff --git a/scala2.13/sql-plugin/pom.xml b/scala2.13/sql-plugin/pom.xml index df3532a3592..eb6f240a3f6 100644 --- a/scala2.13/sql-plugin/pom.xml +++ b/scala2.13/sql-plugin/pom.xml @@ -97,6 +97,10 @@ org.alluxio alluxio-shaded-client + + org.roaringbitmap + RoaringBitmap + org.mockito mockito-core diff --git a/sql-plugin/pom.xml b/sql-plugin/pom.xml index 961e6f08372..08657a9d40b 100644 --- a/sql-plugin/pom.xml +++ b/sql-plugin/pom.xml @@ -97,6 +97,10 @@ org.alluxio alluxio-shaded-client + + org.roaringbitmap + RoaringBitmap + org.mockito mockito-core diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala index 8ea1641fb4a..5203e926efa 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala @@ -2274,6 +2274,32 @@ val SHUFFLE_COMPRESSION_LZ4_CHUNK_SIZE = conf("spark.rapids.shuffle.compression. .integerConf .createWithDefault(1024) + val DELTA_LOW_SHUFFLE_MERGE_SCATTER_DEL_VECTOR_BATCH_SIZE = + conf("spark.rapids.sql.delta.lowShuffleMerge.deletion.scatter.max.size") + .doc("Option to set max batch size when scattering deletion vector") + .internal() + .integerConf + .createWithDefault(32 * 1024) + + val DELTA_LOW_SHUFFLE_MERGE_DEL_VECTOR_BROADCAST_THRESHOLD = + conf("spark.rapids.sql.delta.lowShuffleMerge.deletionVector.broadcast.threshold") + .doc("Currently we need to broadcast deletion vector to all executors to perform low " + + "shuffle merge. When we detect the deletion vector broadcast size is larger than this " + + "value, we will fallback to normal shuffle merge.") + .bytesConf(ByteUnit.BYTE) + .createWithDefault(20 * 1024 * 1024) + + val ENABLE_DELTA_LOW_SHUFFLE_MERGE = + conf("spark.rapids.sql.delta.lowShuffleMerge.enabled") + .doc("Option to turn on the low shuffle merge for Delta Lake. Currently there are some " + + "limitations for this feature: \n" + + "1. We only support Databricks Runtime 13.3 and Deltalake 2.4. \n" + + s"2. The file scan mode must be set to ${RapidsReaderType.PERFILE} \n" + + "3. The deletion vector size must be smaller than " + + s"${DELTA_LOW_SHUFFLE_MERGE_DEL_VECTOR_BROADCAST_THRESHOLD.key} \n") + .booleanConf + .createWithDefault(false) + private def printSectionHeader(category: String): Unit = println(s"\n### $category") @@ -3083,6 +3109,8 @@ class RapidsConf(conf: Map[String, String]) extends Logging { lazy val testGetJsonObjectSaveRows: Int = get(TEST_GET_JSON_OBJECT_SAVE_ROWS) + lazy val isDeltaLowShuffleMergeEnabled: Boolean = get(ENABLE_DELTA_LOW_SHUFFLE_MERGE) + private val optimizerDefaults = Map( // this is not accurate because CPU projections do have a cost due to appending values // to each row that is produced, but this needs to be a really small number because From 4b449034f2a0105c687646176590b349f9901ea7 Mon Sep 17 00:00:00 2001 From: Liangcai Li Date: Mon, 24 Jun 2024 09:32:03 +0800 Subject: [PATCH 52/68] Support bucketing write for GPU (#10957) This PR adds the GPU support for the bucketing write. - React the code of the dynamic partition single writer and concurrent writer to try to reuse the code as much as possible, and then add in the bucketing write logic for both of them. - Update the bucket check during the plan overriding for the write commands, including InsertIntoHadoopFsRelationCommand, CreateDataSourceTableAsSelectCommand, InsertIntoHiveTable, CreateHiveTableAsSelectCommand. - From 330, Spark also supports HiveHash to generate the bucket IDs, in addition to Murmur3Hash. So the shim object GpuBucketingUtils is introduced to handle the shim things. - This change also adds two functions (tagForHiveBucketingWrite and tagForBucketing) to do the overriding check for the two hashing functions separately. And the Hive write nodes will fall back to CPU when HiveHash is chosen, because HiveHash is not supported on GPU. --------- Signed-off-by: Firestarman --- integration_tests/src/main/python/asserts.py | 6 +- .../src/main/python/orc_write_test.py | 48 +- .../src/main/python/parquet_write_test.py | 79 +- .../rapids/GpuHashPartitioningBase.scala | 8 +- .../nvidia/spark/rapids/GpuOverrides.scala | 7 +- .../sql/hive/rapids/GpuHiveFileFormat.scala | 6 +- .../sql/rapids/GpuFileFormatDataWriter.scala | 1112 +++++++---------- ...aSourceTableAsSelectCommandMetaShims.scala | 8 +- ...dCreateHiveTableAsSelectCommandShims.scala | 5 +- .../shims/spark311/GpuBucketingUtils.scala | 77 ++ .../GpuCreateHiveTableAsSelectCommand.scala | 9 +- .../rapids/shims/GpuInsertIntoHiveTable.scala | 5 +- .../sql/rapids/GpuFileFormatWriter.scala | 15 +- .../shims/spark330/GpuBucketingUtils.scala | 88 ++ ...aSourceTableAsSelectCommandMetaShims.scala | 12 +- .../rapids/shims/GpuInsertIntoHiveTable.scala | 7 +- ...dCreateHiveTableAsSelectCommandShims.scala | 6 +- .../sql/rapids/GpuFileFormatWriter.scala | 15 +- .../rapids/GpuFileFormatDataWriterSuite.scala | 132 +- 19 files changed, 896 insertions(+), 749 deletions(-) create mode 100644 sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/spark311/GpuBucketingUtils.scala create mode 100644 sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/spark330/GpuBucketingUtils.scala diff --git a/integration_tests/src/main/python/asserts.py b/integration_tests/src/main/python/asserts.py index 32416612d26..b861e89b726 100644 --- a/integration_tests/src/main/python/asserts.py +++ b/integration_tests/src/main/python/asserts.py @@ -1,4 +1,4 @@ -# Copyright (c) 2020-2023, NVIDIA CORPORATION. +# Copyright (c) 2020-2024, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -101,6 +101,10 @@ def _assert_equal(cpu, gpu, float_check, path): else: assert False, "Found unexpected type {} at {}".format(t, path) +def assert_equal_with_local_sort(cpu, gpu): + _sort_locally(cpu, gpu) + assert_equal(cpu, gpu) + def assert_equal(cpu, gpu): """Verify that the result from the CPU and the GPU are equal""" try: diff --git a/integration_tests/src/main/python/orc_write_test.py b/integration_tests/src/main/python/orc_write_test.py index 8d3013cbe8b..5b5c7b786b6 100644 --- a/integration_tests/src/main/python/orc_write_test.py +++ b/integration_tests/src/main/python/orc_write_test.py @@ -1,4 +1,4 @@ -# Copyright (c) 2020-2023, NVIDIA CORPORATION. +# Copyright (c) 2020-2024, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -209,7 +209,7 @@ def test_write_sql_save_table(spark_tmp_path, orc_gens, ts_type, orc_impl, spark @pytest.mark.parametrize('codec', ['zlib', 'lzo']) def test_orc_write_compression_fallback(spark_tmp_path, codec, spark_tmp_table_factory): gen = TimestampGen() - data_path = spark_tmp_path + '/PARQUET_DATA' + data_path = spark_tmp_path + '/ORC_DATA' all_confs={'spark.sql.orc.compression.codec': codec, 'spark.rapids.sql.format.orc.write.enabled': True} assert_gpu_fallback_write( lambda spark, path: unary_op_df(spark, gen).coalesce(1).write.format("orc").mode('overwrite').option("path", path).saveAsTable(spark_tmp_table_factory.get()), @@ -218,17 +218,45 @@ def test_orc_write_compression_fallback(spark_tmp_path, codec, spark_tmp_table_f 'DataWritingCommandExec', conf=all_confs) -@ignore_order -@allow_non_gpu('DataWritingCommandExec,ExecutedCommandExec,WriteFilesExec') -def test_buckets_write_fallback(spark_tmp_path, spark_tmp_table_factory): +@ignore_order(local=True) +def test_buckets_write_round_trip(spark_tmp_path, spark_tmp_table_factory): data_path = spark_tmp_path + '/ORC_DATA' + gen_list = [["id", int_gen], ["data", long_gen]] + assert_gpu_and_cpu_writes_are_equal_collect( + lambda spark, path: gen_df(spark, gen_list).selectExpr("id % 100 as b_id", "data").write + .bucketBy(4, "b_id").format('orc').mode('overwrite').option("path", path) + .saveAsTable(spark_tmp_table_factory.get()), + lambda spark, path: spark.read.orc(path), + data_path, + conf={'spark.rapids.sql.format.orc.write.enabled': True}) + +@ignore_order(local=True) +@allow_non_gpu('DataWritingCommandExec,ExecutedCommandExec,WriteFilesExec, SortExec') +def test_buckets_write_fallback_unsupported_types(spark_tmp_path, spark_tmp_table_factory): + data_path = spark_tmp_path + '/ORC_DATA' + gen_list = [["id", binary_gen], ["data", long_gen]] assert_gpu_fallback_write( - lambda spark, path: spark.range(10e4).write.bucketBy(4, "id").sortBy("id").format('orc').mode('overwrite').option("path", path).saveAsTable(spark_tmp_table_factory.get()), - lambda spark, path: spark.read.orc(path), - data_path, - 'DataWritingCommandExec', - conf = {'spark.rapids.sql.format.orc.write.enabled': True}) + lambda spark, path: gen_df(spark, gen_list).selectExpr("id as b_id", "data").write + .bucketBy(4, "b_id").format('orc').mode('overwrite').option("path", path) + .saveAsTable(spark_tmp_table_factory.get()), + lambda spark, path: spark.read.orc(path), + data_path, + 'DataWritingCommandExec', + conf={'spark.rapids.sql.format.orc.write.enabled': True}) +@ignore_order(local=True) +def test_partitions_and_buckets_write_round_trip(spark_tmp_path, spark_tmp_table_factory): + data_path = spark_tmp_path + '/ORC_DATA' + gen_list = [["id", int_gen], ["data", long_gen]] + assert_gpu_and_cpu_writes_are_equal_collect( + lambda spark, path: gen_df(spark, gen_list) + .selectExpr("id % 5 as b_id", "id % 10 as p_id", "data").write + .partitionBy("p_id") + .bucketBy(4, "b_id").format('orc').mode('overwrite').option("path", path) + .saveAsTable(spark_tmp_table_factory.get()), + lambda spark, path: spark.read.orc(path), + data_path, + conf={'spark.rapids.sql.format.orc.write.enabled': True}) @ignore_order @allow_non_gpu('DataWritingCommandExec,ExecutedCommandExec,WriteFilesExec') diff --git a/integration_tests/src/main/python/parquet_write_test.py b/integration_tests/src/main/python/parquet_write_test.py index 38dab9e84a4..805a0b8137c 100644 --- a/integration_tests/src/main/python/parquet_write_test.py +++ b/integration_tests/src/main/python/parquet_write_test.py @@ -409,16 +409,81 @@ def test_parquet_writeLegacyFormat_fallback(spark_tmp_path, spark_tmp_table_fact 'DataWritingCommandExec', conf=all_confs) -@ignore_order -@allow_non_gpu('DataWritingCommandExec,ExecutedCommandExec,WriteFilesExec') -def test_buckets_write_fallback(spark_tmp_path, spark_tmp_table_factory): +@ignore_order(local=True) +def test_buckets_write_round_trip(spark_tmp_path, spark_tmp_table_factory): + data_path = spark_tmp_path + '/PARQUET_DATA' + gen_list = [["id", int_gen], ["data", long_gen]] + assert_gpu_and_cpu_writes_are_equal_collect( + lambda spark, path: gen_df(spark, gen_list).selectExpr("id % 100 as b_id", "data").write + .bucketBy(4, "b_id").format('parquet').mode('overwrite').option("path", path) + .saveAsTable(spark_tmp_table_factory.get()), + lambda spark, path: spark.read.parquet(path), + data_path, + conf=writer_confs) + + +def test_buckets_write_correctness(spark_tmp_path, spark_tmp_table_factory): + cpu_path = spark_tmp_path + '/PARQUET_DATA/CPU' + gpu_path = spark_tmp_path + '/PARQUET_DATA/GPU' + gen_list = [["id", int_gen], ["data", long_gen]] + num_buckets = 4 + + def do_bucketing_write(spark, path): + df = gen_df(spark, gen_list).selectExpr("id % 100 as b_id", "data") + df.write.bucketBy(num_buckets, "b_id").format('parquet').mode('overwrite') \ + .option("path", path).saveAsTable(spark_tmp_table_factory.get()) + + def read_single_bucket(path, bucket_id): + # Bucket Id string format: f"_$id%05d" + ".c$fileCounter%03d". + # fileCounter is always 0 in this test. For example '_00002.c000' is for + # bucket id being 2. + # We leverage this bucket segment in the file path to filter rows belong + # to a bucket. + bucket_segment = '_' + "{}".format(bucket_id).rjust(5, '0') + '.c000' + return with_cpu_session( + lambda spark: spark.read.parquet(path) + .withColumn('file_name', f.input_file_name()) + .filter(f.col('file_name').contains(bucket_segment)) + .selectExpr('b_id', 'data') # need to drop the file_name column for comparison. + .collect()) + + with_cpu_session(lambda spark: do_bucketing_write(spark, cpu_path), writer_confs) + with_gpu_session(lambda spark: do_bucketing_write(spark, gpu_path), writer_confs) + cur_bucket_id = 0 + while cur_bucket_id < num_buckets: + # Verify the result bucket by bucket + ret_cpu = read_single_bucket(cpu_path, cur_bucket_id) + ret_gpu = read_single_bucket(gpu_path, cur_bucket_id) + assert_equal_with_local_sort(ret_cpu, ret_gpu) + cur_bucket_id += 1 + +@ignore_order(local=True) +@allow_non_gpu('DataWritingCommandExec,ExecutedCommandExec,WriteFilesExec, SortExec') +def test_buckets_write_fallback_unsupported_types(spark_tmp_path, spark_tmp_table_factory): data_path = spark_tmp_path + '/PARQUET_DATA' + gen_list = [["id", binary_gen], ["data", long_gen]] assert_gpu_fallback_write( - lambda spark, path: spark.range(10e4).write.bucketBy(4, "id").sortBy("id").format('parquet').mode('overwrite').option("path", path).saveAsTable(spark_tmp_table_factory.get()), - lambda spark, path: spark.read.parquet(path), - data_path, - 'DataWritingCommandExec') + lambda spark, path: gen_df(spark, gen_list).selectExpr("id as b_id", "data").write + .bucketBy(4, "b_id").format('parquet').mode('overwrite').option("path", path) + .saveAsTable(spark_tmp_table_factory.get()), + lambda spark, path: spark.read.parquet(path), + data_path, + 'DataWritingCommandExec', + conf=writer_confs) +@ignore_order(local=True) +def test_partitions_and_buckets_write_round_trip(spark_tmp_path, spark_tmp_table_factory): + data_path = spark_tmp_path + '/PARQUET_DATA' + gen_list = [["id", int_gen], ["data", long_gen]] + assert_gpu_and_cpu_writes_are_equal_collect( + lambda spark, path: gen_df(spark, gen_list) + .selectExpr("id % 5 as b_id", "id % 10 as p_id", "data").write + .partitionBy("p_id") + .bucketBy(4, "b_id").format('parquet').mode('overwrite').option("path", path) + .saveAsTable(spark_tmp_table_factory.get()), + lambda spark, path: spark.read.parquet(path), + data_path, + conf=writer_confs) @ignore_order @allow_non_gpu('DataWritingCommandExec,ExecutedCommandExec,WriteFilesExec') diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuHashPartitioningBase.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuHashPartitioningBase.scala index b17b2782e90..baa009d0669 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuHashPartitioningBase.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuHashPartitioningBase.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020-2023, NVIDIA CORPORATION. + * Copyright (c) 2020-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -21,7 +21,7 @@ import com.nvidia.spark.rapids.Arm.withResource import com.nvidia.spark.rapids.shims.ShimExpression import org.apache.spark.sql.catalyst.expressions.Expression -import org.apache.spark.sql.rapids.GpuMurmur3Hash +import org.apache.spark.sql.rapids.{GpuMurmur3Hash, GpuPmod} import org.apache.spark.sql.types.{DataType, IntegerType} import org.apache.spark.sql.vectorized.ColumnarBatch @@ -59,6 +59,10 @@ abstract class GpuHashPartitioningBase(expressions: Seq[Expression], numPartitio sliceInternalGpuOrCpuAndClose(numRows, partitionIndexes, partitionColumns) } } + + def partitionIdExpression: GpuExpression = GpuPmod( + GpuMurmur3Hash(expressions, GpuHashPartitioningBase.DEFAULT_HASH_SEED), + GpuLiteral(numPartitions)) } object GpuHashPartitioningBase { diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala index 295480d24cc..9e26cf751f4 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala @@ -322,10 +322,11 @@ final class InsertIntoHadoopFsRelationCommandMeta( private var fileFormat: Option[ColumnarFileFormat] = None override def tagSelfForGpuInternal(): Unit = { - if (cmd.bucketSpec.isDefined) { - willNotWorkOnGpu("bucketing is not supported") + if (GpuBucketingUtils.isHiveHashBucketing(cmd.options)) { + GpuBucketingUtils.tagForHiveBucketingWrite(this, cmd.bucketSpec, cmd.outputColumns, false) + } else { + BucketIdMetaUtils.tagForBucketingWrite(this, cmd.bucketSpec, cmd.outputColumns) } - val spark = SparkSession.active val formatCls = cmd.fileFormat.getClass fileFormat = if (formatCls == classOf[CSVFileFormat]) { diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/GpuHiveFileFormat.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/GpuHiveFileFormat.scala index 21437a64481..69189b2600c 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/GpuHiveFileFormat.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/hive/rapids/GpuHiveFileFormat.scala @@ -24,6 +24,7 @@ import com.google.common.base.Charsets import com.nvidia.spark.rapids._ import com.nvidia.spark.rapids.Arm.withResource import com.nvidia.spark.rapids.jni.CastStrings +import com.nvidia.spark.rapids.shims.GpuBucketingUtils import org.apache.hadoop.mapreduce.{Job, TaskAttemptContext} import org.apache.spark.internal.Logging @@ -43,9 +44,8 @@ object GpuHiveFileFormat extends Logging { def tagGpuSupport(meta: GpuInsertIntoHiveTableMeta): Option[ColumnarFileFormat] = { val insertCmd = meta.wrapped // Bucketing write - if (insertCmd.table.bucketSpec.isDefined) { - meta.willNotWorkOnGpu("bucketed tables are not supported yet") - } + GpuBucketingUtils.tagForHiveBucketingWrite(meta, insertCmd.table.bucketSpec, + insertCmd.outputColumns, false) // Infer the file format from the serde string, similar as what Spark does in // RelationConversions for Hive. diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileFormatDataWriter.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileFormatDataWriter.scala index 4ceac365314..939a421e0b9 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileFormatDataWriter.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileFormatDataWriter.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019-2023, NVIDIA CORPORATION. + * Copyright (c) 2019-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,30 +17,30 @@ package org.apache.spark.sql.rapids import scala.collection.mutable -import scala.collection.mutable.{ArrayBuffer, ListBuffer} +import scala.collection.mutable.ListBuffer +import scala.util.hashing.{MurmurHash3 => ScalaMurmur3Hash} -import ai.rapids.cudf.{ColumnVector, OrderByArg, Table} +import ai.rapids.cudf.{OrderByArg, Table} import com.nvidia.spark.TimingUtils import com.nvidia.spark.rapids._ import com.nvidia.spark.rapids.Arm.{closeOnExcept, withResource} import com.nvidia.spark.rapids.RapidsPluginImplicits._ import com.nvidia.spark.rapids.RmmRapidsRetryIterator.withRetryNoSplit -import com.nvidia.spark.rapids.ScalableTaskCompletion.onTaskCompletion import com.nvidia.spark.rapids.shims.GpuFileFormatDataWriterShim import org.apache.hadoop.fs.Path import org.apache.hadoop.mapreduce.TaskAttemptContext -import org.apache.spark.TaskContext +import org.apache.spark.internal.Logging import org.apache.spark.internal.io.FileCommitProtocol import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.catalog.{BucketSpec, ExternalCatalogUtils} import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec -import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils -import org.apache.spark.sql.catalyst.expressions.{Ascending, Attribute, AttributeSet, Cast, Concat, Expression, Literal, NullsFirst, ScalaUDF, SortOrder, UnsafeProjection} +import org.apache.spark.sql.catalyst.expressions.{Ascending, Attribute, AttributeSet, Cast, Concat, Expression, Literal, Murmur3Hash, NullsFirst, ScalaUDF, UnsafeProjection} import org.apache.spark.sql.connector.write.DataWriter import org.apache.spark.sql.execution.datasources.{BucketingUtils, PartitioningUtils, WriteTaskResult} -import org.apache.spark.sql.rapids.GpuFileFormatDataWriter.{shouldSplitToFitMaxRecordsPerFile, splitToFitMaxRecordsAndClose} +import org.apache.spark.sql.rapids.GpuFileFormatDataWriter._ import org.apache.spark.sql.rapids.GpuFileFormatWriter.GpuConcurrentOutputWriterSpec -import org.apache.spark.sql.types.{DataType, StringType} +import org.apache.spark.sql.types._ import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.SerializableConfiguration @@ -50,7 +50,7 @@ object GpuFileFormatDataWriter { } def shouldSplitToFitMaxRecordsPerFile( - maxRecordsPerFile: Long, recordsInFile: Long, numRowsInBatch: Long) = { + maxRecordsPerFile: Long, recordsInFile: Long, numRowsInBatch: Long): Boolean = { maxRecordsPerFile > 0 && (recordsInFile + numRowsInBatch) > maxRecordsPerFile } @@ -88,13 +88,8 @@ object GpuFileFormatDataWriter { maxRecordsPerFile: Long, recordsInFile: Long): Array[SpillableColumnarBatch] = { val (types, splitIndexes) = closeOnExcept(batch) { _ => - val types = GpuColumnVector.extractTypes(batch) - val splitIndexes = - getSplitIndexes( - maxRecordsPerFile, - recordsInFile, - batch.numRows()) - (types, splitIndexes) + val splitIndexes = getSplitIndexes(maxRecordsPerFile, recordsInFile, batch.numRows()) + (GpuColumnVector.extractTypes(batch), splitIndexes) } if (splitIndexes.isEmpty) { // this should never happen, as `splitToFitMaxRecordsAndClose` is called when @@ -124,6 +119,31 @@ abstract class GpuFileFormatDataWriter( description: GpuWriteJobDescription, taskAttemptContext: TaskAttemptContext, committer: FileCommitProtocol) extends DataWriter[ColumnarBatch] { + + protected class WriterAndStatus { + var writer: ColumnarOutputWriter = _ + + /** Number of records in current file. */ + var recordsInFile: Long = 0 + + /** + * File counter for writing current partition or bucket. For same partition or bucket, + * we may have more than one file, due to number of records limit per file. + */ + var fileCounter: Int = 0 + + final def release(): Unit = { + if (writer != null) { + try { + writer.close() + statsTrackers.foreach(_.closeFile(writer.path())) + } finally { + writer = null + } + } + } + } + /** * Max number of files a single task writes out due to file size. In most cases the number of * files written should be very small. This is just a safe guard to protect some really bad @@ -131,28 +151,26 @@ abstract class GpuFileFormatDataWriter( */ protected val MAX_FILE_COUNTER: Int = 1000 * 1000 protected val updatedPartitions: mutable.Set[String] = mutable.Set[String]() - protected var currentWriter: ColumnarOutputWriter = _ + protected var currentWriterStatus: WriterAndStatus = new WriterAndStatus() /** Trackers for computing various statistics on the data as it's being written out. */ protected val statsTrackers: Seq[ColumnarWriteTaskStatsTracker] = description.statsTrackers.map(_.newTaskInstance()) - /** Release resources of `currentWriter`. */ - protected def releaseCurrentWriter(): Unit = { - if (currentWriter != null) { - try { - currentWriter.close() - statsTrackers.foreach(_.closeFile(currentWriter.path())) - } finally { - currentWriter = null - } - } + /** Release resources of a WriterStatus. */ + protected final def releaseOutWriter(status: WriterAndStatus): Unit = { + status.release() + } + + protected final def writeUpdateMetricsAndClose(scb: SpillableColumnarBatch, + writerStatus: WriterAndStatus): Unit = { + writerStatus.recordsInFile += writerStatus.writer.writeSpillableAndClose(scb, statsTrackers) } /** Release all resources. Public for testing */ def releaseResources(): Unit = { - // Call `releaseCurrentWriter()` by default, as this is the only resource to be released. - releaseCurrentWriter() + // Release current writer by default, as this is the only resource to be released. + releaseOutWriter(currentWriterStatus) } /** Write an iterator of column batch. */ @@ -211,8 +229,6 @@ class GpuSingleDirectoryDataWriter( taskAttemptContext: TaskAttemptContext, committer: FileCommitProtocol) extends GpuFileFormatDataWriter(description, taskAttemptContext, committer) { - private var fileCounter: Int = _ - private var recordsInFile: Long = _ // Initialize currentWriter and statsTrackers newOutputWriter() @@ -220,7 +236,8 @@ class GpuSingleDirectoryDataWriter( "msg=method newTaskTempFile in class FileCommitProtocol is deprecated" ) private def newOutputWriter(): Unit = { - recordsInFile = 0 + currentWriterStatus.recordsInFile = 0 + val fileCounter = currentWriterStatus.fileCounter releaseResources() val ext = description.outputWriterFactory.getFileExtension(taskAttemptContext) @@ -229,7 +246,7 @@ class GpuSingleDirectoryDataWriter( None, f"-c$fileCounter%03d" + ext) - currentWriter = description.outputWriterFactory.newInstance( + currentWriterStatus.writer = description.outputWriterFactory.newInstance( path = currentPath, dataSchema = description.dataColumns.toStructType, context = taskAttemptContext) @@ -237,32 +254,30 @@ class GpuSingleDirectoryDataWriter( statsTrackers.foreach(_.newFile(currentPath)) } - private def writeUpdateMetricsAndClose(scb: SpillableColumnarBatch): Unit = { - recordsInFile += currentWriter.writeSpillableAndClose(scb, statsTrackers) - } - override def write(batch: ColumnarBatch): Unit = { val maxRecordsPerFile = description.maxRecordsPerFile + val recordsInFile = currentWriterStatus.recordsInFile if (!shouldSplitToFitMaxRecordsPerFile( maxRecordsPerFile, recordsInFile, batch.numRows())) { writeUpdateMetricsAndClose( - SpillableColumnarBatch(batch, SpillPriorities.ACTIVE_ON_DECK_PRIORITY)) + SpillableColumnarBatch(batch, SpillPriorities.ACTIVE_ON_DECK_PRIORITY), + currentWriterStatus) } else { val partBatches = splitToFitMaxRecordsAndClose( batch, maxRecordsPerFile, recordsInFile) - var needNewWriter = recordsInFile >= maxRecordsPerFile + val needNewWriterForFirstPart = recordsInFile >= maxRecordsPerFile closeOnExcept(partBatches) { _ => partBatches.zipWithIndex.foreach { case (partBatch, partIx) => - if (needNewWriter) { - fileCounter += 1 + if (partIx > 0 || needNewWriterForFirstPart) { + currentWriterStatus.fileCounter += 1 + val fileCounter = currentWriterStatus.fileCounter assert(fileCounter <= MAX_FILE_COUNTER, s"File counter $fileCounter is beyond max value $MAX_FILE_COUNTER") newOutputWriter() } // null out the entry so that we don't double close partBatches(partIx) = null - writeUpdateMetricsAndClose(partBatch) - needNewWriter = true + writeUpdateMetricsAndClose(partBatch, currentWriterStatus) } } } @@ -280,35 +295,44 @@ class GpuDynamicPartitionDataSingleWriter( taskAttemptContext: TaskAttemptContext, committer: FileCommitProtocol) extends GpuFileFormatDataWriter(description, taskAttemptContext, committer) { + /** Wrapper class to index a unique concurrent output writer. */ + protected class WriterIndex( + var partitionPath: Option[String], + var bucketId: Option[Int]) extends Product2[Option[String], Option[Int]] { - /** Wrapper class for status of a unique single output writer. */ - protected class WriterStatus( - // output writer - var outputWriter: ColumnarOutputWriter, + override def hashCode(): Int = ScalaMurmur3Hash.productHash(this) - /** Number of records in current file. */ - var recordsInFile: Long = 0, + override def equals(obj: Any): Boolean = { + if (obj.isInstanceOf[WriterIndex]) { + val otherWI = obj.asInstanceOf[WriterIndex] + partitionPath == otherWI.partitionPath && bucketId == otherWI.bucketId + } else { + false + } + } - /** - * File counter for writing current partition or bucket. For same partition or bucket, - * we may have more than one file, due to number of records limit per file. - */ - var fileCounter: Int = 0 - ) + override def _1: Option[String] = partitionPath + override def _2: Option[Int] = bucketId + override def canEqual(that: Any): Boolean = that.isInstanceOf[WriterIndex] + } - /** Wrapper class for status and caches of a unique concurrent output writer. - * Used by `GpuDynamicPartitionDataConcurrentWriter` + /** + * A case class to hold the batch, the optional partition path and the optional bucket + * ID for a split group. All the rows in the batch belong to the group defined by the + * partition path and the bucket ID. */ - class WriterStatusWithCaches( - // writer status - var writerStatus: WriterStatus, - - // caches for this partition or writer - val tableCaches: ListBuffer[SpillableColumnarBatch] = ListBuffer(), - - // current device bytes for the above caches - var deviceBytes: Long = 0 - ) + private case class SplitPack(split: SpillableColumnarBatch, path: Option[String], + bucketId: Option[Int]) extends AutoCloseable { + override def close(): Unit = { + split.safeClose() + } + } + /** + * The index for current writer. Intentionally make the index mutable and reusable. + * Avoid JVM GC issue when many short-living `WriterIndex` objects are created + * if switching between concurrent writers frequently. + */ + private val currentWriterId: WriterIndex = new WriterIndex(None, None) /** Flag saying whether or not the data to be written out is partitioned. */ protected val isPartitioned: Boolean = description.partitionColumns.nonEmpty @@ -316,25 +340,17 @@ class GpuDynamicPartitionDataSingleWriter( /** Flag saying whether or not the data to be written out is bucketed. */ protected val isBucketed: Boolean = description.bucketSpec.isDefined - private var currentPartPath: String = "" - - private var currentWriterStatus: WriterStatus = _ - - // All data is sorted ascending with default null ordering - private val nullsSmallest = Ascending.defaultNullOrdering == NullsFirst - - if (isBucketed) { - throw new UnsupportedOperationException("Bucketing is not supported on the GPU yet.") - } - assert(isPartitioned || isBucketed, s"""GpuDynamicPartitionWriteTask should be used for writing out data that's either |partitioned or bucketed. In this case neither is true. |GpuWriteJobDescription: $description """.stripMargin) + // All data is sorted ascending with default null ordering + private val nullsSmallest = Ascending.defaultNullOrdering == NullsFirst + /** Extracts the partition values out of an input batch. */ - protected lazy val getPartitionColumnsAsBatch: ColumnarBatch => ColumnarBatch = { + private lazy val getPartitionColumnsAsBatch: ColumnarBatch => ColumnarBatch = { val expressions = GpuBindReferences.bindGpuReferences( description.partitionColumns, description.allColumns) @@ -343,20 +359,9 @@ class GpuDynamicPartitionDataSingleWriter( } } - /** Extracts the output values of an input batch. */ - private lazy val getOutputColumnsAsBatch: ColumnarBatch => ColumnarBatch= { + private lazy val getBucketIdColumnAsBatch: ColumnarBatch => ColumnarBatch = { val expressions = GpuBindReferences.bindGpuReferences( - description.dataColumns, - description.allColumns) - cb => { - GpuProjectExec.project(cb, expressions) - } - } - - /** Extracts the output values of an input batch. */ - protected lazy val getOutputCb: ColumnarBatch => ColumnarBatch = { - val expressions = GpuBindReferences.bindGpuReferences( - description.dataColumns, + Seq(description.bucketSpec.get.bucketIdExpression), description.allColumns) cb => { GpuProjectExec.project(cb, expressions) @@ -379,62 +384,58 @@ class GpuDynamicPartitionDataSingleWriter( /** Evaluates the `partitionPathExpression` above on a row of `partitionValues` and returns * the partition string. */ - protected lazy val getPartitionPath: InternalRow => String = { + private lazy val getPartitionPath: InternalRow => String = { val proj = UnsafeProjection.create(Seq(partitionPathExpression), description.partitionColumns) row => proj(row).getString(0) } - /** Release resources of writer. */ - private def releaseWriter(writer: ColumnarOutputWriter): Unit = { - if (writer != null) { - val path = writer.path() - writer.close() - statsTrackers.foreach(_.closeFile(path)) + /** Extracts the output values of an input batch. */ + protected lazy val getDataColumnsAsBatch: ColumnarBatch => ColumnarBatch = { + val expressions = GpuBindReferences.bindGpuReferences( + description.dataColumns, + description.allColumns) + cb => { + GpuProjectExec.project(cb, expressions) } } - /** - * Opens a new OutputWriter given a partition key and/or a bucket id. - * If bucket id is specified, we will append it to the end of the file name, but before the - * file extension, e.g. part-r-00009-ea518ad4-455a-4431-b471-d24e03814677-00002.gz.parquet - * - * @param partDir the partition directory - * @param bucketId the bucket which all tuples being written by this OutputWriter belong to, - * currently does not support `bucketId`, it's always None - * @param fileCounter integer indicating the number of files to be written to `partDir` - */ - @scala.annotation.nowarn( - "msg=method newTaskTempFile.* in class FileCommitProtocol is deprecated" - ) - def newWriter( - partDir: String, - bucketId: Option[Int], // Currently it's always None - fileCounter: Int - ): ColumnarOutputWriter = { - updatedPartitions.add(partDir) - // Currently will be empty - val bucketIdStr = bucketId.map(BucketingUtils.bucketIdToString).getOrElse("") - - // This must be in a form that matches our bucketing format. See BucketingUtils. - val ext = f"$bucketIdStr.c$fileCounter%03d" + - description.outputWriterFactory.getFileExtension(taskAttemptContext) - - val customPath = description.customPartitionLocations - .get(PartitioningUtils.parsePathFragment(partDir)) + protected def getKeysBatch(cb: ColumnarBatch): ColumnarBatch = { + val keysBatch = withResource(getPartitionColumnsAsBatch(cb)) { partCb => + if (isBucketed) { + withResource(getBucketIdColumnAsBatch(cb)) { bucketIdCb => + GpuColumnVector.combineColumns(partCb, bucketIdCb) + } + } else { + GpuColumnVector.incRefCounts(partCb) + } + } + require(keysBatch.numCols() > 0, "No sort key is specified") + keysBatch + } - val currentPath = if (customPath.isDefined) { - committer.newTaskTempFileAbsPath(taskAttemptContext, customPath.get, ext) + protected def genGetBucketIdFunc(keyHostCb: ColumnarBatch): Int => Option[Int] = { + if (isBucketed) { + // The last column is the bucket id column + val bucketIdCol = keyHostCb.column(keyHostCb.numCols() - 1) + i => Some(bucketIdCol.getInt(i)) } else { - committer.newTaskTempFile(taskAttemptContext, Option(partDir), ext) + _ => None } + } - val newWriter = description.outputWriterFactory.newInstance( - path = currentPath, - dataSchema = description.dataColumns.toStructType, - context = taskAttemptContext) - - statsTrackers.foreach(_.newFile(currentPath)) - newWriter + protected def genGetPartitionPathFunc(keyHostCb: ColumnarBatch): Int => Option[String] = { + if (isPartitioned) { + // Use the existing code to convert each row into a path. It would be nice to do this + // on the GPU, but the data should be small and there are things we cannot easily + // support on the GPU right now + import scala.collection.JavaConverters._ + val partCols = description.partitionColumns.indices.map(keyHostCb.column) + val iter = new ColumnarBatch(partCols.toArray, keyHostCb.numRows()).rowIterator() + .asScala.map(getPartitionPath) + _ => Some(iter.next) + } else { + _ => None + } } // distinct value sorted the same way the input data is sorted. @@ -461,282 +462,195 @@ class GpuDynamicPartitionDataSingleWriter( } } - override def write(batch: ColumnarBatch): Unit = { - // this single writer always passes `cachesMap` as None - write(batch, cachesMap = None) - } - - private case class SplitAndPath(var split: SpillableColumnarBatch, path: String) - extends AutoCloseable { - override def close(): Unit = { - split.safeClose() - split = null - } - } - /** - * Split a batch according to the sorted keys (partitions). Returns a tuple with an - * array of the splits as `ContiguousTable`'s, and an array of paths to use to - * write each partition. + * Split a batch according to the sorted keys (partitions + bucket ids). + * Returns a tuple with an array of the splits as `ContiguousTable`'s, an array of + * paths and bucket ids to use to write each partition and(or) bucket file. */ - private def splitBatchByKeyAndClose( - batch: ColumnarBatch, - partDataTypes: Array[DataType]): Array[SplitAndPath] = { - val (outputColumnsBatch, partitionColumnsBatch) = withResource(batch) { _ => - closeOnExcept(getOutputColumnsAsBatch(batch)) { outputColumnsBatch => - closeOnExcept(getPartitionColumnsAsBatch(batch)) { partitionColumnsBatch => - (outputColumnsBatch, partitionColumnsBatch) - } + private def splitBatchByKeyAndClose(batch: ColumnarBatch): Array[SplitPack] = { + val (keysCb, dataCb) = withResource(batch) { _ => + closeOnExcept(getDataColumnsAsBatch(batch)) { data => + (getKeysBatch(batch), data) } } - val (cbKeys, partitionIndexes) = closeOnExcept(outputColumnsBatch) { _ => - val partitionColumnsTbl = withResource(partitionColumnsBatch) { _ => - GpuColumnVector.from(partitionColumnsBatch) - } - withResource(partitionColumnsTbl) { _ => - withResource(distinctAndSort(partitionColumnsTbl)) { distinctKeysTbl => - val partitionIndexes = splitIndexes(partitionColumnsTbl, distinctKeysTbl) - val cbKeys = copyToHostAsBatch(distinctKeysTbl, partDataTypes) - (cbKeys, partitionIndexes) + val (keyHostCb, splitIds) = closeOnExcept(dataCb) { _ => + val (splitIds, distinctKeysTbl, keysCbTypes) = withResource(keysCb) { _ => + val keysCbTypes = GpuColumnVector.extractTypes(keysCb) + withResource(GpuColumnVector.from(keysCb)) { keysTable => + closeOnExcept(distinctAndSort(keysTable)) { distinctKeysTbl => + (splitIndexes(keysTable, distinctKeysTbl), distinctKeysTbl, keysCbTypes) + } } } + withResource(distinctKeysTbl) { _ => + (copyToHostAsBatch(distinctKeysTbl, keysCbTypes), splitIds) + } } - - val splits = closeOnExcept(cbKeys) { _ => - val spillableOutputColumnsBatch = - SpillableColumnarBatch(outputColumnsBatch, SpillPriorities.ACTIVE_ON_DECK_PRIORITY) - withRetryNoSplit(spillableOutputColumnsBatch) { spillable => - withResource(spillable.getColumnarBatch()) { outCb => + val splits = closeOnExcept(keyHostCb) { _ => + val scbOutput = closeOnExcept(dataCb)( _ => + SpillableColumnarBatch(dataCb, SpillPriorities.ACTIVE_ON_DECK_PRIORITY)) + withRetryNoSplit(scbOutput) { scb => + withResource(scb.getColumnarBatch()) { outCb => withResource(GpuColumnVector.from(outCb)) { outputColumnsTbl => withResource(outputColumnsTbl) { _ => - outputColumnsTbl.contiguousSplit(partitionIndexes: _*) + outputColumnsTbl.contiguousSplit(splitIds: _*) } } } } } - - val paths = closeOnExcept(splits) { _ => - withResource(cbKeys) { _ => - // Use the existing code to convert each row into a path. It would be nice to do this - // on the GPU, but the data should be small and there are things we cannot easily - // support on the GPU right now - import scala.collection.JavaConverters._ - // paths - cbKeys.rowIterator().asScala.map(getPartitionPath).toArray - } - } + // Build the split result withResource(splits) { _ => - // NOTE: the `zip` here has the effect that will remove an extra `ContiguousTable` - // added at the end of `splits` because we use `upperBound` to find the split points, - // and the last split point is the number of rows. - val outDataTypes = description.dataColumns.map(_.dataType).toArray - splits.zip(paths).zipWithIndex.map { case ((split, path), ix) => - splits(ix) = null - withResource(split) { _ => - SplitAndPath( - SpillableColumnarBatch( - split, outDataTypes, SpillPriorities.ACTIVE_BATCHING_PRIORITY), - path) - } + withResource(keyHostCb) { _ => + val getBucketId = genGetBucketIdFunc(keyHostCb) + val getNextPartPath = genGetPartitionPathFunc(keyHostCb) + val outDataTypes = description.dataColumns.map(_.dataType).toArray + (0 until keyHostCb.numRows()).safeMap { idx => + val split = splits(idx) + splits(idx) = null + closeOnExcept(split) { _ => + SplitPack( + SpillableColumnarBatch(split, outDataTypes, + SpillPriorities.ACTIVE_BATCHING_PRIORITY), + getNextPartPath(idx), getBucketId(idx)) + } + }.toArray } } } - private def getBatchToWrite( - partBatch: SpillableColumnarBatch, - savedStatus: Option[WriterStatusWithCaches]): SpillableColumnarBatch = { - val outDataTypes = description.dataColumns.map(_.dataType).toArray - if (savedStatus.isDefined && savedStatus.get.tableCaches.nonEmpty) { - // In the case where the concurrent partition writers fall back, we need to - // incorporate into the current part any pieces that are already cached - // in the `savedStatus`. Adding `partBatch` to what was saved could make a - // concatenated batch with number of rows larger than `maxRecordsPerFile`, - // so this concatenated result could be split later, which is not efficient. However, - // the concurrent writers are default off in Spark, so it is not clear if this - // code path is worth optimizing. - val concat: Table = - withResource(savedStatus.get.tableCaches) { subSpillableBatches => - val toConcat = subSpillableBatches :+ partBatch - - // clear the caches - savedStatus.get.tableCaches.clear() - - withRetryNoSplit(toConcat.toSeq) { spillables => - withResource(spillables.safeMap(_.getColumnarBatch())) { batches => - withResource(batches.map(GpuColumnVector.from)) { subTables => - Table.concatenate(subTables: _*) - } - } - } - } - withResource(concat) { _ => - SpillableColumnarBatch( - GpuColumnVector.from(concat, outDataTypes), - SpillPriorities.ACTIVE_ON_DECK_PRIORITY) - } - } else { - partBatch + /** + * Create a new writer according to the given writer id, and update the given + * writer status. It also closes the old writer in the writer status by default. + */ + protected final def renewOutWriter(newWriterId: WriterIndex, curWriterStatus: WriterAndStatus, + closeOldWriter: Boolean = true): Unit = { + if (closeOldWriter) { + releaseOutWriter(curWriterStatus) } + curWriterStatus.recordsInFile = 0 + curWriterStatus.writer = newWriter(newWriterId.partitionPath, newWriterId.bucketId, + curWriterStatus.fileCounter) + } + + /** + * Set up a writer to the given writer status for the given writer id. + * It will create a new one if needed. This is used when seeing a new partition + * and(or) a new bucket id. + */ + protected def setupCurrentWriter(newWriterId: WriterIndex, curWriterStatus: WriterAndStatus, + closeOldWriter: Boolean = true): Unit = { + renewOutWriter(newWriterId, curWriterStatus, closeOldWriter) } /** - * Write columnar batch. - * If the `cachesMap` is not empty, this single writer should restore the writers and caches in - * the `cachesMap`, this single writer should first combine the caches and current split data - * for a specific partition before write. + * Opens a new OutputWriter given a partition key and/or a bucket id. + * If bucket id is specified, we will append it to the end of the file name, but before the + * file extension, e.g. part-r-00009-ea518ad4-455a-4431-b471-d24e03814677-00002.gz.parquet * - * @param cb the column batch - * @param cachesMap used by `GpuDynamicPartitionDataConcurrentWriter` when fall back to single - * writer, single writer should handle the stored writers and the pending caches + * @param partDir the partition directory + * @param bucketId the bucket which all tuples being written by this OutputWriter belong to, + * currently does not support `bucketId`, it's always None + * @param fileCounter integer indicating the number of files to be written to `partDir` */ - protected def write( - batch: ColumnarBatch, - cachesMap: Option[mutable.HashMap[String, WriterStatusWithCaches]]): Unit = { - assert(isPartitioned) - assert(!isBucketed) + @scala.annotation.nowarn( + "msg=method newTaskTempFile.* in class FileCommitProtocol is deprecated" + ) + def newWriter(partDir: Option[String], bucketId: Option[Int], + fileCounter: Int): ColumnarOutputWriter = { + partDir.foreach(updatedPartitions.add) + // Currently will be empty + val bucketIdStr = bucketId.map(BucketingUtils.bucketIdToString).getOrElse("") - val maxRecordsPerFile = description.maxRecordsPerFile - val partDataTypes = description.partitionColumns.map(_.dataType).toArray - - // We have an entire batch that is sorted, so we need to split it up by key - // to get a batch per path - withResource(splitBatchByKeyAndClose(batch, partDataTypes)) { splitsAndPaths => - splitsAndPaths.zipWithIndex.foreach { case (SplitAndPath(partBatch, partPath), ix) => - // If we fall back from `GpuDynamicPartitionDataConcurrentWriter`, we should get the - // saved status - val savedStatus = updateCurrentWriterIfNeeded(partPath, cachesMap) - - // combine `partBatch` with any remnants for this partition for the concurrent - // writer fallback case in `savedStatus` - splitsAndPaths(ix) = null - val batchToWrite = getBatchToWrite(partBatch, savedStatus) - - // if the batch fits, write it as is, else split and write it. - if (!shouldSplitToFitMaxRecordsPerFile(maxRecordsPerFile, - currentWriterStatus.recordsInFile, batchToWrite.numRows())) { - writeUpdateMetricsAndClose(currentWriterStatus, batchToWrite) - } else { - // materialize an actual batch since we are going to split it - // on the GPU - val batchToSplit = withRetryNoSplit(batchToWrite) { _ => - batchToWrite.getColumnarBatch() - } - val maxRecordsPerFileSplits = splitToFitMaxRecordsAndClose( - batchToSplit, - maxRecordsPerFile, - currentWriterStatus.recordsInFile) - writeSplitBatchesAndClose(maxRecordsPerFileSplits, maxRecordsPerFile, partPath) - } - } + // This must be in a form that matches our bucketing format. See BucketingUtils. + val ext = f"$bucketIdStr.c$fileCounter%03d" + + description.outputWriterFactory.getFileExtension(taskAttemptContext) + + val customPath = partDir.flatMap { dir => + description.customPartitionLocations.get(PartitioningUtils.parsePathFragment(dir)) } + + val currentPath = if (customPath.isDefined) { + committer.newTaskTempFileAbsPath(taskAttemptContext, customPath.get, ext) + } else { + committer.newTaskTempFile(taskAttemptContext, partDir, ext) + } + + val outWriter = description.outputWriterFactory.newInstance( + path = currentPath, + dataSchema = description.dataColumns.toStructType, + context = taskAttemptContext) + + statsTrackers.foreach(_.newFile(currentPath)) + outWriter } - private def updateCurrentWriterIfNeeded( - partPath: String, - cachesMap: Option[mutable.HashMap[String, WriterStatusWithCaches]]): - Option[WriterStatusWithCaches] = { - var savedStatus: Option[WriterStatusWithCaches] = None - if (currentPartPath != partPath) { - val previousPartPath = currentPartPath - currentPartPath = partPath - - // see a new partition, close the old writer - val previousWriterStatus = currentWriterStatus - if (previousWriterStatus != null) { - releaseWriter(previousWriterStatus.outputWriter) - } + protected final def writeBatchPerMaxRecordsAndClose(scb: SpillableColumnarBatch, + writerId: WriterIndex, writerStatus: WriterAndStatus): Unit = { + val maxRecordsPerFile = description.maxRecordsPerFile + val recordsInFile = writerStatus.recordsInFile - if (cachesMap.isDefined) { - savedStatus = cachesMap.get.get(currentPartPath) - if (savedStatus.isDefined) { - // first try to restore the saved writer status, - // `GpuDynamicPartitionDataConcurrentWriter` may already opened the writer, and may - // have pending caches - currentWriterStatus = savedStatus.get.writerStatus - // entire batch that is sorted, see a new partition, the old write status is useless - cachesMap.get.remove(previousPartPath) - } else { - // create a new one - val writer = newWriter(partPath, None, 0) - currentWriterStatus = new WriterStatus(writer) - statsTrackers.foreach(_.newPartition()) + if (!shouldSplitToFitMaxRecordsPerFile(maxRecordsPerFile, recordsInFile, scb.numRows())) { + writeUpdateMetricsAndClose(scb, writerStatus) + } else { + val batch = withRetryNoSplit(scb) { scb => + scb.getColumnarBatch() + } + val splits = splitToFitMaxRecordsAndClose(batch, maxRecordsPerFile, recordsInFile) + withResource(splits) { _ => + val needNewWriterForFirstPart = recordsInFile >= maxRecordsPerFile + splits.zipWithIndex.foreach { case (part, partIx) => + if (partIx > 0 || needNewWriterForFirstPart) { + writerStatus.fileCounter += 1 + assert(writerStatus.fileCounter <= MAX_FILE_COUNTER, + s"File counter ${writerStatus.fileCounter} is beyond max value $MAX_FILE_COUNTER") + // will create a new file, so close the old writer + renewOutWriter(writerId, writerStatus) + } + splits(partIx) = null + writeUpdateMetricsAndClose(part, writerStatus) } - } else { - // create a new one - val writer = newWriter(partPath, None, 0) - currentWriterStatus = new WriterStatus(writer) - statsTrackers.foreach(_.newPartition()) } } - savedStatus } /** - * Write an array of spillable batches. + * Called just before updating the current writer status when seeing a new partition + * or a bucket. * - * Note: `spillableBatches` will be closed in this function. - * - * @param batches the SpillableColumnarBatch splits to be written - * @param maxRecordsPerFile the max number of rows per file - * @param partPath the partition directory + * @param curWriterId the current writer index */ - private def writeSplitBatchesAndClose( - spillableBatches: Array[SpillableColumnarBatch], - maxRecordsPerFile: Long, - partPath: String): Unit = { - var needNewWriter = currentWriterStatus.recordsInFile >= maxRecordsPerFile - withResource(spillableBatches) { _ => - spillableBatches.zipWithIndex.foreach { case (part, partIx) => - if (needNewWriter) { - currentWriterStatus.fileCounter += 1 - assert(currentWriterStatus.fileCounter <= MAX_FILE_COUNTER, - s"File counter ${currentWriterStatus.fileCounter} " + - s"is beyond max value $MAX_FILE_COUNTER") - - // will create a new file, close the old writer - if (currentWriterStatus != null) { - releaseWriter(currentWriterStatus.outputWriter) - } + protected def preUpdateCurrentWriterStatus(curWriterId: WriterIndex): Unit ={} - // create a new writer and update the writer in the status - currentWriterStatus.outputWriter = - newWriter(partPath, None, currentWriterStatus.fileCounter) - currentWriterStatus.recordsInFile = 0 + override def write(batch: ColumnarBatch): Unit = { + // The input batch that is entirely sorted, so split it up by partitions and (or) + // bucket ids, and write the split batches one by one. + withResource(splitBatchByKeyAndClose(batch)) { splitPacks => + splitPacks.zipWithIndex.foreach { case (SplitPack(sp, partPath, bucketId), i) => + val hasDiffPart = partPath != currentWriterId.partitionPath + val hasDiffBucket = bucketId != currentWriterId.bucketId + if (hasDiffPart || hasDiffBucket) { + preUpdateCurrentWriterStatus(currentWriterId) + if (hasDiffPart) { + currentWriterId.partitionPath = partPath + statsTrackers.foreach(_.newPartition()) + } + if (hasDiffBucket) { + currentWriterId.bucketId = bucketId + } + currentWriterStatus.fileCounter = 0 + setupCurrentWriter(currentWriterId, currentWriterStatus) } - spillableBatches(partIx) = null - writeUpdateMetricsAndClose(currentWriterStatus, part) - needNewWriter = true - } - } - } - - protected def writeUpdateMetricsAndClose( - writerStatus: WriterStatus, - spillableBatch: SpillableColumnarBatch): Unit = { - writerStatus.recordsInFile += - writerStatus.outputWriter.writeSpillableAndClose(spillableBatch, statsTrackers) - } - - /** Release all resources. */ - override def releaseResources(): Unit = { - // does not use `currentWriter`, single writer use `currentWriterStatus` - assert(currentWriter == null) - - if (currentWriterStatus != null) { - try { - currentWriterStatus.outputWriter.close() - statsTrackers.foreach(_.closeFile(currentWriterStatus.outputWriter.path())) - } finally { - currentWriterStatus = null + splitPacks(i) = null + writeBatchPerMaxRecordsAndClose(sp, currentWriterId, currentWriterStatus) } } } } /** - * Dynamic partition writer with concurrent writers, meaning multiple concurrent writers are opened - * for writing. + * Dynamic partition writer with concurrent writers, meaning multiple concurrent + * writers are opened for writing. * * The process has the following steps: * - Step 1: Maintain a map of output writers per each partition columns. Keep all @@ -754,18 +668,29 @@ class GpuDynamicPartitionDataConcurrentWriter( description: GpuWriteJobDescription, taskAttemptContext: TaskAttemptContext, committer: FileCommitProtocol, - spec: GpuConcurrentOutputWriterSpec, - taskContext: TaskContext) - extends GpuDynamicPartitionDataSingleWriter(description, taskAttemptContext, committer) { + spec: GpuConcurrentOutputWriterSpec) + extends GpuDynamicPartitionDataSingleWriter(description, taskAttemptContext, committer) + with Logging { - // Keep all the unclosed writers, key is partition directory string. - // Note: if fall back to sort-based mode, also use the opened writers in the map. - private val concurrentWriters = mutable.HashMap[String, WriterStatusWithCaches]() + /** Wrapper class for status and caches of a unique concurrent output writer. */ + private class WriterStatusWithBatches extends WriterAndStatus with AutoCloseable { + // caches for this partition or writer + val tableCaches: ListBuffer[SpillableColumnarBatch] = ListBuffer() - // guarantee to close the caches and writers when task is finished - onTaskCompletion(taskContext)(closeCachesAndWriters()) + // current device bytes for the above caches + var deviceBytes: Long = 0 - private val outDataTypes = description.dataColumns.map(_.dataType).toArray + override def close(): Unit = try { + releaseOutWriter(this) + } finally { + tableCaches.safeClose() + tableCaches.clear() + } + } + + // Keep all the unclosed writers, key is a partition path and(or) bucket id. + // Note: if fall back to sort-based mode, also use the opened writers in the map. + private val concurrentWriters = mutable.HashMap[WriterIndex, WriterStatusWithBatches]() private val partitionFlushSize = if (description.concurrentWriterPartitionFlushSize <= 0) { @@ -777,324 +702,196 @@ class GpuDynamicPartitionDataConcurrentWriter( description.concurrentWriterPartitionFlushSize } - // refer to current batch if should fall back to `single writer` - private var currentFallbackColumnarBatch: ColumnarBatch = _ + // Pending split batches that are not cached for the concurrent write because + // there are too many open writers, and it is going to fall back to the sorted + // sequential write. + private val pendingBatches: mutable.Queue[SpillableColumnarBatch] = mutable.Queue.empty - override def abort(): Unit = { - try { - closeCachesAndWriters() - } finally { - committer.abortTask(taskAttemptContext) + override def writeWithIterator(iterator: Iterator[ColumnarBatch]): Unit = { + // 1: try concurrent writer + while (iterator.hasNext && pendingBatches.isEmpty) { + // concurrent write and update the `concurrentWriters` map. + this.write(iterator.next()) } - } - /** - * State to indicate if we are falling back to sort-based writer. - * Because we first try to use concurrent writers, its initial value is false. - */ - private var fallBackToSortBased: Boolean = false + // 2: fall back to single write if the input is not all consumed. + if (pendingBatches.nonEmpty || iterator.hasNext) { + // sort the all the pending batches and ones in `iterator` + val pendingCbsIter = new Iterator[ColumnarBatch] { + override def hasNext: Boolean = pendingBatches.nonEmpty - private def writeWithSingleWriter(cb: ColumnarBatch): Unit = { - // invoke `GpuDynamicPartitionDataSingleWriter`.write, - // single writer will take care of the unclosed writers and the pending caches - // in `concurrentWriters` - super.write(cb, Some(concurrentWriters)) + override def next(): ColumnarBatch = { + if (!hasNext) { + throw new NoSuchElementException() + } + withResource(pendingBatches.dequeue())(_.getColumnarBatch()) + } + } + val sortIter = GpuOutOfCoreSortIterator(pendingCbsIter ++ iterator, + new GpuSorter(spec.sortOrder, spec.output), GpuSortExec.targetSize(spec.batchSize), + NoopMetric, NoopMetric, NoopMetric, NoopMetric) + while (sortIter.hasNext) { + // write with sort-based sequential writer + super.write(sortIter.next()) + } + } } - private def writeWithConcurrentWriter(cb: ColumnarBatch): Unit = { - this.write(cb) + /** This is for the fallback case, used to clean the writers map. */ + override def preUpdateCurrentWriterStatus(curWriterId: WriterIndex): Unit = { + concurrentWriters.remove(curWriterId) } - /** - * Write an iterator of column batch. - * - * @param iterator the iterator of column batch - */ - override def writeWithIterator(iterator: Iterator[ColumnarBatch]): Unit = { - // 1: try concurrent writer - while (iterator.hasNext && !fallBackToSortBased) { - // concurrently write and update the `concurrentWriters` map - // the `` will be updated - writeWithConcurrentWriter(iterator.next()) + /** This is for the fallback case, try to find the writer from cache first. */ + override def setupCurrentWriter(newWriterId: WriterIndex, writerStatus: WriterAndStatus, + closeOldWriter: Boolean): Unit = { + if (closeOldWriter) { + releaseOutWriter(writerStatus) } - - // 2: fall back to single writer - // Note single writer should restore writer status and handle the pending caches - if (fallBackToSortBased) { - // concat the put back batch and un-coming batches - val newIterator = Iterator.single(currentFallbackColumnarBatch) ++ iterator - // sort the all the batches in `iterator` - - val sortIterator: GpuOutOfCoreSortIterator = getSorted(newIterator) - while (sortIterator.hasNext) { - // write with sort-based single writer - writeWithSingleWriter(sortIterator.next()) - } + val oOpenStatus = concurrentWriters.get(newWriterId) + if (oOpenStatus.isDefined) { + val openStatus = oOpenStatus.get + writerStatus.writer = openStatus.writer + writerStatus.recordsInFile = openStatus.recordsInFile + writerStatus.fileCounter = openStatus.fileCounter + } else { + super.setupCurrentWriter(newWriterId, writerStatus, closeOldWriter = false) } } /** - * Sort the input iterator by out of core sort - * - * @param iterator the input iterator - * @return sorted iterator - */ - private def getSorted(iterator: Iterator[ColumnarBatch]): GpuOutOfCoreSortIterator = { - val gpuSortOrder: Seq[SortOrder] = spec.sortOrder - val output: Seq[Attribute] = spec.output - val sorter = new GpuSorter(gpuSortOrder, output) - - // use noop metrics below - val sortTime = NoopMetric - val opTime = NoopMetric - val outputBatch = NoopMetric - val outputRows = NoopMetric - - val targetSize = GpuSortExec.targetSize(spec.batchSize) - // out of core sort the entire iterator - GpuOutOfCoreSortIterator(iterator, sorter, targetSize, - opTime, sortTime, outputBatch, outputRows) - } - - /** - * concurrent write the columnar batch - * Note: if new partitions number in `cb` plus existing partitions number is greater than - * `maxWriters` limit, will put back the whole `cb` to 'single writer` + * The write path of concurrent writers * - * @param cb the columnar batch + * @param cb the columnar batch to be written */ override def write(cb: ColumnarBatch): Unit = { - assert(isPartitioned) - assert(!isBucketed) - if (cb.numRows() == 0) { // TODO https://github.com/NVIDIA/spark-rapids/issues/6453 // To solve above issue, I assume that an empty batch will be wrote for saving metadata. // If the assumption it's true, this concurrent writer should write the metadata here, // and should not run into below splitting and caching logic + cb.close() return } - // 1. combine partition columns and `cb` columns into a column array - val columnsWithPartition = ArrayBuffer[ColumnVector]() - - // this withResource is here to decrement the refcount of the partition columns - // that are projected out of `cb` - withResource(getPartitionColumnsAsBatch(cb)) { partitionColumnsBatch => - columnsWithPartition.appendAll(GpuColumnVector.extractBases(partitionColumnsBatch)) - } - - val cols = GpuColumnVector.extractBases(cb) - columnsWithPartition ++= cols - - // 2. group by the partition columns - // get sub-groups for each partition and get unique keys for each partition - val groupsAndKeys = withResource( - new Table(columnsWithPartition.toSeq: _*)) { colsWithPartitionTbl => - // [0, partition columns number - 1] - val partitionIndices = description.partitionColumns.indices - - // group by partition columns - val op = colsWithPartitionTbl.groupBy(partitionIndices: _*) - // return groups and uniq keys table - // Each row in uniq keys table is corresponding to a group - op.contiguousSplitGroupsAndGenUniqKeys() - } - - withResource(groupsAndKeys) { _ => - // groups number should equal to uniq keys number - assert(groupsAndKeys.getGroups.length == groupsAndKeys.getUniqKeyTable.getRowCount) - - val (groups, keys) = (groupsAndKeys.getGroups, groupsAndKeys.getUniqKeyTable) - - // 3. generate partition strings for all sub-groups in advance - val partDataTypes = description.partitionColumns.map(_.dataType).toArray - val dataTypes = GpuColumnVector.extractTypes(cb) - // generate partition string list for all groups - val partitionStrList = getPartitionStrList(keys, partDataTypes) - // key table is useless now - groupsAndKeys.closeUniqKeyTable() - - // 4. cache each group according to each partitionStr - withResource(groups) { _ => - - // first update fallBackToSortBased - withResource(cb) { _ => - var newPartitionNum = 0 - var groupIndex = 0 - while (!fallBackToSortBased && groupIndex < groups.length) { - // get the partition string - val partitionStr = partitionStrList(groupIndex) - groupIndex += 1 - if (!concurrentWriters.contains(partitionStr)) { - newPartitionNum += 1 - if (newPartitionNum + concurrentWriters.size >= spec.maxWriters) { - fallBackToSortBased = true - currentFallbackColumnarBatch = cb - // `cb` should be put back to single writer - GpuColumnVector.incRefCounts(cb) - } - } - } - } - - if (!fallBackToSortBased) { - // not fall, collect all caches - var groupIndex = 0 - while (groupIndex < groups.length) { - // get the partition string and group pair - val (partitionStr, group) = (partitionStrList(groupIndex), groups(groupIndex)) - val groupTable = group.getTable - groupIndex += 1 - - // create writer if encounter a new partition and put into `concurrentWriters` map - if (!concurrentWriters.contains(partitionStr)) { - val w = newWriter(partitionStr, None, 0) - val ws = new WriterStatus(w) - concurrentWriters.put(partitionStr, new WriterStatusWithCaches(ws)) - statsTrackers.foreach(_.newPartition()) - } - - // get data columns, tail part is data columns - val dataColumns = ArrayBuffer[ColumnVector]() - for (i <- description.partitionColumns.length until groupTable.getNumberOfColumns) { - dataColumns += groupTable.getColumn(i) - } - withResource(new Table(dataColumns.toSeq: _*)) { dataTable => - withResource(GpuColumnVector.from(dataTable, dataTypes)) { cb => - val outputCb = getOutputCb(cb) - // convert to spillable cache and add to the pending cache - val currWriterStatus = concurrentWriters(partitionStr) - // create SpillableColumnarBatch to take the owner of `outputCb` - currWriterStatus.tableCaches += SpillableColumnarBatch( - outputCb, SpillPriorities.ACTIVE_ON_DECK_PRIORITY) - currWriterStatus.deviceBytes += GpuColumnVector.getTotalDeviceMemoryUsed(outputCb) - } - } + // Split the batch and cache the result, along with opening the writers. + splitBatchToCacheAndClose(cb) + // Write the cached batches + val writeFunc: (WriterIndex, WriterStatusWithBatches) => Unit = + if (pendingBatches.nonEmpty) { + // Flush all the caches before going into sorted sequential write + writeOneCacheAndClose + } else { + // Still the concurrent write, so write out only partitions that size > threshold. + (wi, ws) => + if (ws.deviceBytes > partitionFlushSize) { + writeOneCacheAndClose(wi, ws) } - } } - } - - // 5. find all big enough partitions and write - if(!fallBackToSortBased) { - for ((partitionDir, ws) <- findBigPartitions(partitionFlushSize)) { - writeAndCloseCache(partitionDir, ws) - } - } - } - - private def getPartitionStrList( - uniqKeysTable: Table, partDataTypes: Array[DataType]): Array[String] = { - withResource(copyToHostAsBatch(uniqKeysTable, partDataTypes)) { oneRowCb => - import scala.collection.JavaConverters._ - oneRowCb.rowIterator().asScala.map(getPartitionPath).toArray + concurrentWriters.foreach { case (writerIdx, writerStatus) => + writeFunc(writerIdx, writerStatus) } } - private def writeAndCloseCache(partitionDir: String, status: WriterStatusWithCaches): Unit = { + private def writeOneCacheAndClose(writerId: WriterIndex, + status: WriterStatusWithBatches): Unit = { assert(status.tableCaches.nonEmpty) + // Concat tables if needed + val scbToWrite = GpuBatchUtils.concatSpillBatchesAndClose(status.tableCaches.toSeq).get + status.tableCaches.clear() + status.deviceBytes = 0 + writeBatchPerMaxRecordsAndClose(scbToWrite, writerId, status) + } - // get concat table or the single table - val spillableToWrite = if (status.tableCaches.length >= 2) { - // concat the sub batches to write in once. - val concatted = withRetryNoSplit(status.tableCaches.toSeq) { spillableSubBatches => - withResource(spillableSubBatches.safeMap(_.getColumnarBatch())) { subBatches => - withResource(subBatches.map(GpuColumnVector.from)) { subTables => - Table.concatenate(subTables: _*) - } - } + private def splitBatchToCacheAndClose(batch: ColumnarBatch): Unit = { + // Split batch to groups by sort columns, [partition and(or) bucket id column]. + val (keysAndGroups, keyTypes) = withResource(batch) { _ => + val (opBatch, keyTypes) = withResource(getKeysBatch(batch)) { keysBatch => + val combinedCb = GpuColumnVector.combineColumns(keysBatch, batch) + (combinedCb, GpuColumnVector.extractTypes(keysBatch)) } - withResource(concatted) { _ => - SpillableColumnarBatch( - GpuColumnVector.from(concatted, outDataTypes), - SpillPriorities.ACTIVE_ON_DECK_PRIORITY) + withResource(opBatch) { _ => + withResource(GpuColumnVector.from(opBatch)) { opTable => + (opTable.groupBy(keyTypes.indices: _*).contiguousSplitGroupsAndGenUniqKeys(), + keyTypes) + } } - } else { - // only one single table - status.tableCaches.head } - - status.tableCaches.clear() - - val maxRecordsPerFile = description.maxRecordsPerFile - if (!shouldSplitToFitMaxRecordsPerFile( - maxRecordsPerFile, status.writerStatus.recordsInFile, spillableToWrite.numRows())) { - writeUpdateMetricsAndClose(status.writerStatus, spillableToWrite) - } else { - val batchToSplit = withRetryNoSplit(spillableToWrite) { _ => - spillableToWrite.getColumnarBatch() - } - val splits = splitToFitMaxRecordsAndClose( - batchToSplit, - maxRecordsPerFile, - status.writerStatus.recordsInFile) - var needNewWriter = status.writerStatus.recordsInFile >= maxRecordsPerFile - withResource(splits) { _ => - splits.zipWithIndex.foreach { case (split, partIndex) => - if (needNewWriter) { - status.writerStatus.fileCounter += 1 - assert(status.writerStatus.fileCounter <= MAX_FILE_COUNTER, - s"File counter ${status.writerStatus.fileCounter} " + - s"is beyond max value $MAX_FILE_COUNTER") - status.writerStatus.outputWriter.close() - // start a new writer - val w = newWriter(partitionDir, None, status.writerStatus.fileCounter) - status.writerStatus.outputWriter = w - status.writerStatus.recordsInFile = 0L + // Copy keys table to host and make group batches spillable + val (keyHostCb, groups) = withResource(keysAndGroups) { _ => + // groups number should equal to uniq keys number + assert(keysAndGroups.getGroups.length == keysAndGroups.getUniqKeyTable.getRowCount) + closeOnExcept(copyToHostAsBatch(keysAndGroups.getUniqKeyTable, keyTypes)) { keyHostCb => + keysAndGroups.closeUniqKeyTable() + val allTypes = description.allColumns.map(_.dataType).toArray + val allColsIds = allTypes.indices.map(_ + keyTypes.length) + val gps = keysAndGroups.getGroups.safeMap { gp => + withResource(gp.getTable) { gpTable => + withResource(new Table(allColsIds.map(gpTable.getColumn): _*)) { allTable => + SpillableColumnarBatch(GpuColumnVector.from(allTable, allTypes), + SpillPriorities.ACTIVE_BATCHING_PRIORITY) + } } - splits(partIndex) = null - writeUpdateMetricsAndClose(status.writerStatus, split) - needNewWriter = true } + (keyHostCb, gps) } } - status.tableCaches.clear() - status.deviceBytes = 0 - } - - def closeCachesAndWriters(): Unit = { - // collect all caches and writers - val allResources = ArrayBuffer[AutoCloseable]() - allResources ++= concurrentWriters.values.flatMap(ws => ws.tableCaches) - allResources ++= concurrentWriters.values.map { ws => - new AutoCloseable() { - override def close(): Unit = { - ws.writerStatus.outputWriter.close() - statsTrackers.foreach(_.closeFile(ws.writerStatus.outputWriter.path())) + // Cache the result to either the map or the pending queue. + withResource(groups) { _ => + withResource(keyHostCb) { _ => + val getBucketId = genGetBucketIdFunc(keyHostCb) + val getNextPartPath = genGetPartitionPathFunc(keyHostCb) + var idx = 0 + while (idx < groups.length && concurrentWriters.size < spec.maxWriters) { + val writerId = new WriterIndex(getNextPartPath(idx), getBucketId(idx)) + val writerStatus = + concurrentWriters.getOrElseUpdate(writerId, new WriterStatusWithBatches) + if (writerStatus.writer == null) { + // a new partition or bucket, so create a writer + renewOutWriter(writerId, writerStatus, closeOldWriter = false) + } + withResource(groups(idx)) { gp => + groups(idx) = null + withResource(gp.getColumnarBatch()) { cb => + val dataScb = SpillableColumnarBatch(getDataColumnsAsBatch(cb), + SpillPriorities.ACTIVE_BATCHING_PRIORITY) + writerStatus.tableCaches.append(dataScb) + writerStatus.deviceBytes += dataScb.sizeInBytes + } + } + idx += 1 + } + if (idx < groups.length) { + // The open writers number reaches the limit, and still some partitions are + // not cached. Append to the queue for the coming fallback to the sorted + // sequential write. + groups.drop(idx).foreach(g => pendingBatches.enqueue(g)) + // Set to null to avoid double close + (idx until groups.length).foreach(groups(_) = null) + logInfo(s"Number of concurrent writers ${concurrentWriters.size} reaches " + + "the threshold. Fall back from concurrent writers to sort-based sequential" + + " writer.") } } } - - // safe close all the caches and writers - allResources.safeClose() - - // clear `concurrentWriters` map - concurrentWriters.values.foreach(ws => ws.tableCaches.clear()) - concurrentWriters.clear() } /** Release all resources. */ override def releaseResources(): Unit = { - // does not use `currentWriter`, only use the writers in the concurrent writer map - assert(currentWriter == null) - - if (fallBackToSortBased) { - // Note: we should close the last partition writer in the single writer. - super.releaseResources() - } + pendingBatches.safeClose() + pendingBatches.clear() // write all caches - concurrentWriters.filter(pair => pair._2.tableCaches.nonEmpty) - .foreach(pair => writeAndCloseCache(pair._1, pair._2)) + concurrentWriters.foreach { case (wi, ws) => + if (ws.tableCaches.nonEmpty) { + writeOneCacheAndClose(wi, ws) + } + } // close all resources - closeCachesAndWriters() - } - - private def findBigPartitions( - sizeThreshold: Long): mutable.Map[String, WriterStatusWithCaches] = { - concurrentWriters.filter(pair => pair._2.deviceBytes >= sizeThreshold) + concurrentWriters.values.toSeq.safeClose() + concurrentWriters.clear() + super.releaseResources() } } @@ -1105,7 +902,7 @@ class GpuDynamicPartitionDataConcurrentWriter( * @param bucketFileNamePrefix Prefix of output file name based on bucket id. */ case class GpuWriterBucketSpec( - bucketIdExpression: Expression, + bucketIdExpression: GpuExpression, bucketFileNamePrefix: Int => String) /** @@ -1134,4 +931,23 @@ class GpuWriteJobDescription( |Partition columns: ${partitionColumns.mkString(", ")} |Data columns: ${dataColumns.mkString(", ")} """.stripMargin) -} \ No newline at end of file +} + +object BucketIdMetaUtils { + // Tag for the bucketing write using Spark Murmur3Hash + def tagForBucketingWrite(meta: RapidsMeta[_, _, _], bucketSpec: Option[BucketSpec], + outputColumns: Seq[Attribute]): Unit = { + bucketSpec.foreach { bSpec => + // Create a Murmur3Hash expression to leverage the overriding types check. + val expr = Murmur3Hash( + bSpec.bucketColumnNames.map(n => outputColumns.find(_.name == n).get), + GpuHashPartitioningBase.DEFAULT_HASH_SEED) + val hashMeta = GpuOverrides.wrapExpr(expr, meta.conf, None) + hashMeta.tagForGpu() + if(!hashMeta.canThisBeReplaced) { + meta.willNotWorkOnGpu(s"Hashing for generating bucket IDs can not run" + + s" on GPU. Details: ${hashMeta.explain(all=false)}") + } + } + } +} diff --git a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/CreateDataSourceTableAsSelectCommandMetaShims.scala b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/CreateDataSourceTableAsSelectCommandMetaShims.scala index de066a5486d..d1a26dc80fc 100644 --- a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/CreateDataSourceTableAsSelectCommandMetaShims.scala +++ b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/CreateDataSourceTableAsSelectCommandMetaShims.scala @@ -41,7 +41,7 @@ import org.apache.spark.sql.SparkSession import org.apache.spark.sql.execution.command.CreateDataSourceTableAsSelectCommand import org.apache.spark.sql.execution.datasources.FileFormat import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat -import org.apache.spark.sql.rapids.{GpuDataSourceBase, GpuOrcFileFormat} +import org.apache.spark.sql.rapids.{BucketIdMetaUtils, GpuDataSourceBase, GpuOrcFileFormat} import org.apache.spark.sql.rapids.shims.GpuCreateDataSourceTableAsSelectCommand @@ -56,9 +56,7 @@ final class CreateDataSourceTableAsSelectCommandMeta( private var gpuProvider: Option[ColumnarFileFormat] = None override def tagSelfForGpuInternal(): Unit = { - if (cmd.table.bucketSpec.isDefined) { - willNotWorkOnGpu("bucketing is not supported") - } + BucketIdMetaUtils.tagForBucketingWrite(this, cmd.table.bucketSpec, cmd.outputColumns) if (cmd.table.provider.isEmpty) { willNotWorkOnGpu("provider must be defined") } @@ -94,4 +92,4 @@ final class CreateDataSourceTableAsSelectCommandMeta( conf.stableSort, conf.concurrentWriterPartitionFlushSize) } -} \ No newline at end of file +} diff --git a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/GpuOptimizedCreateHiveTableAsSelectCommandShims.scala b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/GpuOptimizedCreateHiveTableAsSelectCommandShims.scala index 5e2601a0467..55d9bc53704 100644 --- a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/GpuOptimizedCreateHiveTableAsSelectCommandShims.scala +++ b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/GpuOptimizedCreateHiveTableAsSelectCommandShims.scala @@ -184,9 +184,8 @@ final class OptimizedCreateHiveTableAsSelectCommandMeta( willNotWorkOnGpu("partitioned writes are not supported") } - if (tableDesc.bucketSpec.isDefined) { - willNotWorkOnGpu("bucketing is not supported") - } + GpuBucketingUtils.tagForHiveBucketingWrite(this, tableDesc.bucketSpec, + cmd.outputColumns, false) val serde = tableDesc.storage.serde.getOrElse("").toLowerCase(Locale.ROOT) if (serde.contains("parquet")) { diff --git a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/spark311/GpuBucketingUtils.scala b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/spark311/GpuBucketingUtils.scala new file mode 100644 index 00000000000..a604267d1d9 --- /dev/null +++ b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/spark311/GpuBucketingUtils.scala @@ -0,0 +1,77 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed 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. + */ +/*** spark-rapids-shim-json-lines +{"spark": "311"} +{"spark": "312"} +{"spark": "313"} +{"spark": "320"} +{"spark": "321"} +{"spark": "321cdh"} +{"spark": "322"} +{"spark": "323"} +{"spark": "324"} +spark-rapids-shim-json-lines ***/ +package com.nvidia.spark.rapids.shims + +import com.nvidia.spark.rapids.RapidsMeta + +import org.apache.spark.sql.catalyst.catalog.BucketSpec +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.rapids.{BucketIdMetaUtils, GpuWriterBucketSpec} + +object GpuBucketingUtils { + + def getWriterBucketSpec( + bucketSpec: Option[BucketSpec], + dataColumns: Seq[Attribute], + options: Map[String, String], + forceHiveHash: Boolean): Option[GpuWriterBucketSpec] = { + bucketSpec.map { spec => + val bucketColumns = spec.bucketColumnNames.map(c => dataColumns.find(_.name == c).get) + if (forceHiveHash) { + // Forcely use HiveHash for Hive write commands for some customized Spark binaries. + // TODO: Cannot support this until we support Hive hash partitioning on the GPU + throw new UnsupportedOperationException("Hive hash partitioning is not supported" + + " on GPU") + } else { + // Spark bucketed table: use `HashPartitioning.partitionIdExpression` as bucket id + // expression, so that we can guarantee the data distribution is same between shuffle and + // bucketed data source, which enables us to only shuffle one side when join a bucketed + // table and a normal one. + val bucketIdExpression = GpuHashPartitioning(bucketColumns, spec.numBuckets) + .partitionIdExpression + GpuWriterBucketSpec(bucketIdExpression, (_: Int) => "") + } + } + } + + def isHiveHashBucketing(options: Map[String, String]): Boolean = false + + def getOptionsWithHiveBucketWrite(bucketSpec: Option[BucketSpec]): Map[String, String] = { + Map.empty + } + + def tagForHiveBucketingWrite(meta: RapidsMeta[_, _, _], bucketSpec: Option[BucketSpec], + outColumns: Seq[Attribute], forceHiveHash: Boolean): Unit = { + if (forceHiveHash) { + bucketSpec.foreach(_ => + meta.willNotWorkOnGpu("Hive Hashing for generating bucket IDs is not supported yet") + ) + } else { + BucketIdMetaUtils.tagForBucketingWrite(meta, bucketSpec, outColumns) + } + } +} diff --git a/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/hive/rapids/shims/GpuCreateHiveTableAsSelectCommand.scala b/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/hive/rapids/shims/GpuCreateHiveTableAsSelectCommand.scala index 034567d60e5..acdd53b74ab 100644 --- a/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/hive/rapids/shims/GpuCreateHiveTableAsSelectCommand.scala +++ b/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/hive/rapids/shims/GpuCreateHiveTableAsSelectCommand.scala @@ -36,7 +36,7 @@ spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.hive.rapids.shims import com.nvidia.spark.rapids.{DataFromReplacementRule, DataWritingCommandMeta, GpuDataWritingCommand, GpuOverrides, RapidsConf, RapidsMeta} -import com.nvidia.spark.rapids.shims.GpuCreateHiveTableAsSelectBase +import com.nvidia.spark.rapids.shims.{GpuBucketingUtils, GpuCreateHiveTableAsSelectBase} import org.apache.spark.sql.{SaveMode, SparkSession} import org.apache.spark.sql.catalyst.catalog.{CatalogTable, SessionCatalog} @@ -61,9 +61,8 @@ final class GpuCreateHiveTableAsSelectCommandMeta(cmd: CreateHiveTableAsSelectCo willNotWorkOnGpu("partitioned writes are not supported") } - if (tableDesc.bucketSpec.isDefined) { - willNotWorkOnGpu("bucketing is not supported") - } + GpuBucketingUtils.tagForHiveBucketingWrite(this, tableDesc.bucketSpec, + cmd.outputColumns, false) val catalog = spark.sessionState.catalog val tableExists = catalog.tableExists(tableDesc.identifier) @@ -137,4 +136,4 @@ case class GpuCreateHiveTableAsSelectCommand( // Do not support partitioned or bucketed writes override def requireSingleBatch: Boolean = false -} \ No newline at end of file +} diff --git a/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/hive/rapids/shims/GpuInsertIntoHiveTable.scala b/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/hive/rapids/shims/GpuInsertIntoHiveTable.scala index 2ea0301fa2c..3f59d6565a5 100644 --- a/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/hive/rapids/shims/GpuInsertIntoHiveTable.scala +++ b/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/hive/rapids/shims/GpuInsertIntoHiveTable.scala @@ -38,6 +38,7 @@ package org.apache.spark.sql.hive.rapids.shims import java.util.Locale import com.nvidia.spark.rapids.{ColumnarFileFormat, DataFromReplacementRule, DataWritingCommandMeta, GpuDataWritingCommand, RapidsConf, RapidsMeta} +import com.nvidia.spark.rapids.shims.GpuBucketingUtils import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.conf.HiveConf @@ -216,7 +217,9 @@ case class GpuInsertIntoHiveTable( hadoopConf = hadoopConf, fileFormat = fileFormat, outputLocation = tmpLocation.toString, - partitionAttributes = partitionAttributes) + partitionAttributes = partitionAttributes, + bucketSpec = table.bucketSpec, + options = GpuBucketingUtils.getOptionsWithHiveBucketWrite(table.bucketSpec)) if (partition.nonEmpty) { if (numDynamicPartitions > 0) { diff --git a/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/GpuFileFormatWriter.scala b/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/GpuFileFormatWriter.scala index f788971a85f..4adbd7b2ef5 100644 --- a/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/GpuFileFormatWriter.scala +++ b/sql-plugin/src/main/spark311/scala/org/apache/spark/sql/rapids/GpuFileFormatWriter.scala @@ -39,7 +39,7 @@ import java.util.{Date, UUID} import com.nvidia.spark.TimingUtils import com.nvidia.spark.rapids._ -import com.nvidia.spark.rapids.shims.RapidsFileSourceMetaUtils +import com.nvidia.spark.rapids.shims.{GpuBucketingUtils, RapidsFileSourceMetaUtils} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.mapreduce._ @@ -136,13 +136,8 @@ object GpuFileFormatWriter extends Logging { if (projectList.nonEmpty) GpuProjectExec(projectList, plan)() else plan } - val writerBucketSpec: Option[GpuWriterBucketSpec] = bucketSpec.map { spec => - // TODO: Cannot support this until we: - // support Hive hash partitioning on the GPU - throw new UnsupportedOperationException("GPU hash partitioning for bucketed data is not " - + "compatible with the CPU version") - } - + val writerBucketSpec = GpuBucketingUtils.getWriterBucketSpec(bucketSpec, dataColumns, + options, false) val sortColumns = bucketSpec.toSeq.flatMap { spec => spec.sortColumnNames.map(c => dataColumns.find(_.name == c).get) } @@ -328,8 +323,8 @@ object GpuFileFormatWriter extends Logging { } else { concurrentOutputWriterSpec match { case Some(spec) => - new GpuDynamicPartitionDataConcurrentWriter( - description, taskAttemptContext, committer, spec, TaskContext.get()) + new GpuDynamicPartitionDataConcurrentWriter(description, taskAttemptContext, + committer, spec) case _ => new GpuDynamicPartitionDataSingleWriter(description, taskAttemptContext, committer) } diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/spark330/GpuBucketingUtils.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/spark330/GpuBucketingUtils.scala new file mode 100644 index 00000000000..feb562fa9b8 --- /dev/null +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/spark330/GpuBucketingUtils.scala @@ -0,0 +1,88 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed 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. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "330"} +{"spark": "330cdh"} +{"spark": "330db"} +{"spark": "331"} +{"spark": "332"} +{"spark": "332cdh"} +{"spark": "332db"} +{"spark": "333"} +{"spark": "334"} +{"spark": "340"} +{"spark": "341"} +{"spark": "341db"} +{"spark": "342"} +{"spark": "343"} +{"spark": "350"} +{"spark": "351"} +spark-rapids-shim-json-lines ***/ +package com.nvidia.spark.rapids.shims + +import com.nvidia.spark.rapids.RapidsMeta + +import org.apache.spark.sql.catalyst.catalog.BucketSpec +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.execution.datasources.BucketingUtils +import org.apache.spark.sql.rapids.GpuWriterBucketSpec + +object GpuBucketingUtils { + + def getWriterBucketSpec( + bucketSpec: Option[BucketSpec], + dataColumns: Seq[Attribute], + options: Map[String, String], + forceHiveHash: Boolean): Option[GpuWriterBucketSpec] = { + bucketSpec.map { spec => + val bucketColumns = spec.bucketColumnNames.map(c => dataColumns.find(_.name == c).get) + val shouldHiveCompatibleWrite = options.getOrElse( + BucketingUtils.optionForHiveCompatibleBucketWrite, "false").toBoolean + if (shouldHiveCompatibleWrite) { + // TODO: Cannot support this until we support Hive hash partitioning on the GPU + throw new UnsupportedOperationException("Hive hash partitioning is not supported" + + " on GPU") + } else { + // Spark bucketed table: use `HashPartitioning.partitionIdExpression` as bucket id + // expression, so that we can guarantee the data distribution is same between shuffle and + // bucketed data source, which enables us to only shuffle one side when join a bucketed + // table and a normal one. + val bucketIdExpression = GpuHashPartitioning(bucketColumns, spec.numBuckets) + .partitionIdExpression + GpuWriterBucketSpec(bucketIdExpression, (_: Int) => "") + } + } + } + + def isHiveHashBucketing(options: Map[String, String]): Boolean = { + options.getOrElse(BucketingUtils.optionForHiveCompatibleBucketWrite, "false").toBoolean + } + + def getOptionsWithHiveBucketWrite(bucketSpec: Option[BucketSpec]): Map[String, String] = { + bucketSpec + .map(_ => Map(BucketingUtils.optionForHiveCompatibleBucketWrite -> "true")) + .getOrElse(Map.empty) + } + + def tagForHiveBucketingWrite(meta: RapidsMeta[_, _, _], bucketSpec: Option[BucketSpec], + outColumns: Seq[Attribute], forceHiveHash: Boolean): Unit = { + bucketSpec.foreach(_ => + // From Spark330, Hive write always uses HiveHash to generate bucket IDs. + meta.willNotWorkOnGpu("Hive Hashing for generating bucket IDs is not supported yet") + ) + } +} diff --git a/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/CreateDataSourceTableAsSelectCommandMetaShims.scala b/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/CreateDataSourceTableAsSelectCommandMetaShims.scala index faa550c0cb6..f51bd984bdc 100644 --- a/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/CreateDataSourceTableAsSelectCommandMetaShims.scala +++ b/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/CreateDataSourceTableAsSelectCommandMetaShims.scala @@ -30,10 +30,10 @@ package com.nvidia.spark.rapids.shims import com.nvidia.spark.rapids._ import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.execution.command.CreateDataSourceTableAsSelectCommand +import org.apache.spark.sql.execution.command.{CreateDataSourceTableAsSelectCommand, DataWritingCommand} import org.apache.spark.sql.execution.datasources.FileFormat import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat -import org.apache.spark.sql.rapids.{GpuDataSourceBase, GpuOrcFileFormat} +import org.apache.spark.sql.rapids.{BucketIdMetaUtils, GpuDataSourceBase, GpuOrcFileFormat} import org.apache.spark.sql.rapids.shims.GpuCreateDataSourceTableAsSelectCommand final class CreateDataSourceTableAsSelectCommandMeta( @@ -46,9 +46,9 @@ final class CreateDataSourceTableAsSelectCommandMeta( private var origProvider: Class[_] = _ override def tagSelfForGpu(): Unit = { - if (cmd.table.bucketSpec.isDefined) { - willNotWorkOnGpu("bucketing is not supported") - } + val outputColumns = + DataWritingCommand.logicalPlanOutputWithNames(cmd.query, cmd.outputColumnNames) + BucketIdMetaUtils.tagForBucketingWrite(this, cmd.table.bucketSpec, outputColumns) if (cmd.table.provider.isEmpty) { willNotWorkOnGpu("provider must be defined") } @@ -76,4 +76,4 @@ final class CreateDataSourceTableAsSelectCommandMeta( cmd.outputColumnNames, origProvider) } -} \ No newline at end of file +} diff --git a/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuInsertIntoHiveTable.scala b/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuInsertIntoHiveTable.scala index 42fd5941025..b3103c3c76e 100644 --- a/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuInsertIntoHiveTable.scala +++ b/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuInsertIntoHiveTable.scala @@ -30,6 +30,7 @@ package org.apache.spark.sql.hive.rapids.shims import java.util.Locale import com.nvidia.spark.rapids.{ColumnarFileFormat, DataFromReplacementRule, DataWritingCommandMeta, GpuDataWritingCommand, RapidsConf, RapidsMeta} +import com.nvidia.spark.rapids.shims.GpuBucketingUtils import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.conf.HiveConf @@ -205,7 +206,9 @@ case class GpuInsertIntoHiveTable( hadoopConf = hadoopConf, fileFormat = fileFormat, outputLocation = tmpLocation.toString, - partitionAttributes = partitionAttributes) + partitionAttributes = partitionAttributes, + bucketSpec = table.bucketSpec, + options = GpuBucketingUtils.getOptionsWithHiveBucketWrite(table.bucketSpec)) if (partition.nonEmpty) { if (numDynamicPartitions > 0) { @@ -349,4 +352,4 @@ case class GpuInsertIntoHiveTable( } override def requireSingleBatch: Boolean = false // TODO: Re-evaluate. If partitioned or bucketed? -} \ No newline at end of file +} diff --git a/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuOptimizedCreateHiveTableAsSelectCommandShims.scala b/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuOptimizedCreateHiveTableAsSelectCommandShims.scala index 53c17d2f946..e74bf979af9 100644 --- a/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuOptimizedCreateHiveTableAsSelectCommandShims.scala +++ b/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuOptimizedCreateHiveTableAsSelectCommandShims.scala @@ -197,9 +197,9 @@ final class OptimizedCreateHiveTableAsSelectCommandMeta( willNotWorkOnGpu("partitioned writes are not supported") } - if (tableDesc.bucketSpec.isDefined) { - willNotWorkOnGpu("bucketing is not supported") - } + val outputColumns = + DataWritingCommand.logicalPlanOutputWithNames(cmd.query, cmd.outputColumnNames) + GpuBucketingUtils.tagForHiveBucketingWrite(this, tableDesc.bucketSpec, outputColumns, false) val serde = tableDesc.storage.serde.getOrElse("").toLowerCase(Locale.ROOT) if (serde.contains("parquet")) { diff --git a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/GpuFileFormatWriter.scala b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/GpuFileFormatWriter.scala index e7b3561f5fd..874d89353aa 100644 --- a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/GpuFileFormatWriter.scala +++ b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/rapids/GpuFileFormatWriter.scala @@ -31,7 +31,7 @@ import java.util.{Date, UUID} import com.nvidia.spark.TimingUtils import com.nvidia.spark.rapids._ -import com.nvidia.spark.rapids.shims.RapidsFileSourceMetaUtils +import com.nvidia.spark.rapids.shims.{GpuBucketingUtils, RapidsFileSourceMetaUtils} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.mapreduce._ @@ -119,13 +119,8 @@ object GpuFileFormatWriter extends Logging { .map(RapidsFileSourceMetaUtils.cleanupFileSourceMetadataInformation)) val dataColumns = finalOutputSpec.outputColumns.filterNot(partitionSet.contains) - val writerBucketSpec: Option[GpuWriterBucketSpec] = bucketSpec.map { spec => - // TODO: Cannot support this until we: - // support Hive hash partitioning on the GPU - throw new UnsupportedOperationException("GPU hash partitioning for bucketed data is not " - + "compatible with the CPU version") - } - + val writerBucketSpec = GpuBucketingUtils.getWriterBucketSpec(bucketSpec, dataColumns, + options, false) val sortColumns = bucketSpec.toSeq.flatMap { spec => spec.sortColumnNames.map(c => dataColumns.find(_.name == c).get) } @@ -419,8 +414,8 @@ object GpuFileFormatWriter extends Logging { } else { concurrentOutputWriterSpec match { case Some(spec) => - new GpuDynamicPartitionDataConcurrentWriter( - description, taskAttemptContext, committer, spec, TaskContext.get()) + new GpuDynamicPartitionDataConcurrentWriter(description, taskAttemptContext, + committer, spec) case _ => new GpuDynamicPartitionDataSingleWriter(description, taskAttemptContext, committer) } diff --git a/tests/src/test/scala/org/apache/spark/sql/rapids/GpuFileFormatDataWriterSuite.scala b/tests/src/test/scala/org/apache/spark/sql/rapids/GpuFileFormatDataWriterSuite.scala index 5aaeae2c7b9..d52c8b47ae7 100644 --- a/tests/src/test/scala/org/apache/spark/sql/rapids/GpuFileFormatDataWriterSuite.scala +++ b/tests/src/test/scala/org/apache/spark/sql/rapids/GpuFileFormatDataWriterSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023, NVIDIA CORPORATION. + * Copyright (c) 2023-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -16,7 +16,7 @@ package org.apache.spark.sql.rapids import ai.rapids.cudf.TableWriter -import com.nvidia.spark.rapids.{ColumnarOutputWriter, ColumnarOutputWriterFactory, GpuBoundReference, GpuColumnVector, RapidsBufferCatalog, RapidsDeviceMemoryStore, ScalableTaskCompletion} +import com.nvidia.spark.rapids.{ColumnarOutputWriter, ColumnarOutputWriterFactory, GpuColumnVector, GpuLiteral, RapidsBufferCatalog, RapidsDeviceMemoryStore, ScalableTaskCompletion} import com.nvidia.spark.rapids.Arm.{closeOnExcept, withResource} import com.nvidia.spark.rapids.jni.{GpuRetryOOM, GpuSplitAndRetryOOM} import org.apache.hadoop.conf.Configuration @@ -28,7 +28,6 @@ import org.scalatest.BeforeAndAfterEach import org.scalatest.funsuite.AnyFunSuite import org.scalatestplus.mockito.MockitoSugar.mock -import org.apache.spark.TaskContext import org.apache.spark.internal.io.FileCommitProtocol import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.{Ascending, AttributeReference, ExprId, SortOrder} @@ -39,7 +38,6 @@ import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} class GpuFileFormatDataWriterSuite extends AnyFunSuite with BeforeAndAfterEach { private var mockJobDescription: GpuWriteJobDescription = _ - private var mockTaskContext: TaskContext = _ private var mockTaskAttemptContext: TaskAttemptContext = _ private var mockCommitter: FileCommitProtocol = _ private var mockOutputWriterFactory: ColumnarOutputWriterFactory = _ @@ -48,6 +46,7 @@ class GpuFileFormatDataWriterSuite extends AnyFunSuite with BeforeAndAfterEach { private var allCols: Seq[AttributeReference] = _ private var partSpec: Seq[AttributeReference] = _ private var dataSpec: Seq[AttributeReference] = _ + private var bucketSpec: Option[GpuWriterBucketSpec] = None private var includeRetry: Boolean = false class NoTransformColumnarOutputWriter( @@ -102,9 +101,9 @@ class GpuFileFormatDataWriterSuite extends AnyFunSuite with BeforeAndAfterEach { allCols = null partSpec = null dataSpec = null + bucketSpec = None mockJobDescription = mock[GpuWriteJobDescription] when(mockJobDescription.statsTrackers).thenReturn(Seq.empty) - mockTaskContext = mock[TaskContext] mockTaskAttemptContext = mock[TaskAttemptContext] mockCommitter = mock[FileCommitProtocol] mockOutputWriterFactory = mock[ColumnarOutputWriterFactory] @@ -130,8 +129,12 @@ class GpuFileFormatDataWriterSuite extends AnyFunSuite with BeforeAndAfterEach { * It is used to setup certain mocks before `body` is executed. After execution, the * columns in the batches are checked for `refCount==0` (e.g. that they were closed). * @note it is assumed that the schema of each batch is identical. + * numBuckets > 0: Bucketing only + * numBuckets == 0: Partition only + * numBuckets < 0: Both partition and bucketing */ - def withColumnarBatchesVerifyClosed[V](cbs: Seq[ColumnarBatch])(body: => V): Unit = { + def withColumnarBatchesVerifyClosed[V]( + cbs: Seq[ColumnarBatch], numBuckets: Int = 0)(body: => V): Unit = { val allTypes = cbs.map(GpuColumnVector.extractTypes) allCols = Seq.empty dataSpec = Seq.empty @@ -140,8 +143,17 @@ class GpuFileFormatDataWriterSuite extends AnyFunSuite with BeforeAndAfterEach { allCols = allTypes.head.zipWithIndex.map { case (dataType, colIx) => AttributeReference(s"col_$colIx", dataType, nullable = false)(ExprId(colIx)) } - partSpec = Seq(allCols.head) - dataSpec = allCols.tail + if (numBuckets <= 0) { + partSpec = Seq(allCols.head) + dataSpec = allCols.tail + } else { + dataSpec = allCols + } + if (numBuckets != 0) { + bucketSpec = Some(GpuWriterBucketSpec( + GpuPmod(GpuMurmur3Hash(Seq(allCols.last), 42), GpuLiteral(Math.abs(numBuckets))), + _ => "")) + } } val fields = new Array[StructField](allCols.size) allCols.zipWithIndex.foreach { case (col, ix) => @@ -153,6 +165,7 @@ class GpuFileFormatDataWriterSuite extends AnyFunSuite with BeforeAndAfterEach { } when(mockJobDescription.dataColumns).thenReturn(dataSpec) when(mockJobDescription.partitionColumns).thenReturn(partSpec) + when(mockJobDescription.bucketSpec).thenReturn(bucketSpec) when(mockJobDescription.allColumns).thenReturn(allCols) try { body @@ -187,6 +200,20 @@ class GpuFileFormatDataWriterSuite extends AnyFunSuite with BeforeAndAfterEach { new ColumnarBatch(cols, rowCount) } + def buildBatchWithPartitionedAndBucketCols( + partInts: Seq[Int], bucketInts: Seq[Int]): ColumnarBatch = { + assert(partInts.length == bucketInts.length) + val rowCount = partInts.size + val cols: Array[ColumnVector] = new Array[ColumnVector](3) + val partCol = ai.rapids.cudf.ColumnVector.fromInts(partInts: _*) + val dataCol = ai.rapids.cudf.ColumnVector.fromStrings(partInts.map(_.toString): _*) + val bucketCol = ai.rapids.cudf.ColumnVector.fromInts(bucketInts: _*) + cols(0) = GpuColumnVector.from(partCol, IntegerType) + cols(1) = GpuColumnVector.from(dataCol, StringType) + cols(2) = GpuColumnVector.from(bucketCol, IntegerType) + new ColumnarBatch(cols, rowCount) + } + def verifyClosed(cbs: Seq[ColumnarBatch]): Unit = { cbs.foreach { cb => val cols = GpuColumnVector.extractBases(cb) @@ -198,7 +225,6 @@ class GpuFileFormatDataWriterSuite extends AnyFunSuite with BeforeAndAfterEach { def prepareDynamicPartitionSingleWriter(): GpuDynamicPartitionDataSingleWriter = { - when(mockJobDescription.bucketSpec).thenReturn(None) when(mockJobDescription.customPartitionLocations) .thenReturn(Map.empty[TablePartitionSpec, String]) @@ -212,13 +238,10 @@ class GpuFileFormatDataWriterSuite extends AnyFunSuite with BeforeAndAfterEach { GpuDynamicPartitionDataConcurrentWriter = { val mockConfig = new Configuration() when(mockTaskAttemptContext.getConfiguration).thenReturn(mockConfig) - when(mockJobDescription.bucketSpec).thenReturn(None) when(mockJobDescription.customPartitionLocations) .thenReturn(Map.empty[TablePartitionSpec, String]) - // assume the first column is the partition-by column - val sortExpr = - GpuBoundReference(0, partSpec.head.dataType, nullable = false)(ExprId(0), "") - val sortSpec = Seq(SortOrder(sortExpr, Ascending)) + val sortSpec = (partSpec ++ bucketSpec.map(_.bucketIdExpression)) + .map(SortOrder(_, Ascending)) val concurrentSpec = GpuConcurrentOutputWriterSpec( maxWriters, allCols, batchSize, sortSpec) @@ -226,8 +249,7 @@ class GpuFileFormatDataWriterSuite extends AnyFunSuite with BeforeAndAfterEach { mockJobDescription, mockTaskAttemptContext, mockCommitter, - concurrentSpec, - mockTaskContext)) + concurrentSpec)) } test("empty directory data writer") { @@ -317,18 +339,6 @@ class GpuFileFormatDataWriterSuite extends AnyFunSuite with BeforeAndAfterEach { } } - test("dynamic partition data writer doesn't support bucketing") { - resetMocksWithAndWithoutRetry { - withColumnarBatchesVerifyClosed(Seq.empty) { - when(mockJobDescription.bucketSpec).thenReturn(Some(GpuWriterBucketSpec(null, null))) - assertThrows[UnsupportedOperationException] { - new GpuDynamicPartitionDataSingleWriter( - mockJobDescription, mockTaskAttemptContext, mockCommitter) - } - } - } - } - test("dynamic partition data writer without splits") { resetMocksWithAndWithoutRetry { // 4 partitions @@ -353,6 +363,35 @@ class GpuFileFormatDataWriterSuite extends AnyFunSuite with BeforeAndAfterEach { } } + test("dynamic partition data writer bucketing write without splits") { + Seq(5, -5).foreach { numBuckets => + val (numWrites, numNewWriters) = if (numBuckets > 0) { // Bucket only + (6, 6) // 3 buckets + 3 buckets + } else { // partition and bucket + (10, 10) // 5 pairs + 5 pairs + } + resetMocksWithAndWithoutRetry { + val cb = buildBatchWithPartitionedAndBucketCols( + IndexedSeq(1, 1, 2, 2, 3, 3, 4, 4), + IndexedSeq(1, 1, 1, 1, 2, 2, 2, 3)) + val cb2 = buildBatchWithPartitionedAndBucketCols( + IndexedSeq(1, 2, 3, 4, 5), + IndexedSeq(1, 1, 2, 2, 3)) + val cbs = Seq(spy(cb), spy(cb2)) + withColumnarBatchesVerifyClosed(cbs, numBuckets) { + // setting to 9 then the writer won't split as no group has more than 9 rows + when(mockJobDescription.maxRecordsPerFile).thenReturn(9) + val dynamicSingleWriter = prepareDynamicPartitionSingleWriter() + dynamicSingleWriter.writeWithIterator(cbs.iterator) + dynamicSingleWriter.commit() + verify(mockOutputWriter, times(numWrites)).writeSpillableAndClose(any(), any()) + verify(dynamicSingleWriter, times(numNewWriters)).newWriter(any(), any(), any()) + verify(mockOutputWriter, times(numNewWriters)).close() + } + } + } + } + test("dynamic partition data writer with splits") { resetMocksWithAndWithoutRetry { val cb = buildBatchWithPartitionedCol(1, 1, 2, 2, 3, 3, 4, 4) @@ -399,6 +438,38 @@ class GpuFileFormatDataWriterSuite extends AnyFunSuite with BeforeAndAfterEach { } } + test("dynamic partition concurrent data writer bucketing write without splits") { + Seq(5, -5).foreach { numBuckets => + val (numWrites, numNewWriters) = if (numBuckets > 0) { // Bucket only + (3, 3) // 3 distinct buckets in total + } else { // partition and bucket + (6, 6) // 6 distinct pairs in total + } + resetMocksWithAndWithoutRetry { + val cb = buildBatchWithPartitionedAndBucketCols( + IndexedSeq(1, 1, 2, 2, 3, 3, 4, 4), + IndexedSeq(1, 1, 1, 1, 2, 2, 2, 3)) + val cb2 = buildBatchWithPartitionedAndBucketCols( + IndexedSeq(1, 2, 3, 4, 5), + IndexedSeq(1, 1, 2, 2, 3)) + val cbs = Seq(spy(cb), spy(cb2)) + withColumnarBatchesVerifyClosed(cbs, numBuckets) { + // setting to 9 then the writer won't split as no group has more than 9 rows + when(mockJobDescription.maxRecordsPerFile).thenReturn(9) + // I would like to not flush on the first iteration of the `write` method + when(mockJobDescription.concurrentWriterPartitionFlushSize).thenReturn(1000) + val dynamicConcurrentWriter = + prepareDynamicPartitionConcurrentWriter(maxWriters = 20, batchSize = 100) + dynamicConcurrentWriter.writeWithIterator(cbs.iterator) + dynamicConcurrentWriter.commit() + verify(mockOutputWriter, times(numWrites)).writeSpillableAndClose(any(), any()) + verify(dynamicConcurrentWriter, times(numNewWriters)).newWriter(any(), any(), any()) + verify(mockOutputWriter, times(numNewWriters)).close() + } + } + } + } + test("dynamic partition concurrent data writer with splits and flush") { resetMocksWithAndWithoutRetry { val cb = buildBatchWithPartitionedCol(1, 1, 2, 2, 3, 3, 4, 4) @@ -438,8 +509,9 @@ class GpuFileFormatDataWriterSuite extends AnyFunSuite with BeforeAndAfterEach { prepareDynamicPartitionConcurrentWriter(maxWriters = 1, batchSize = 1) dynamicConcurrentWriter.writeWithIterator(cbs.iterator) dynamicConcurrentWriter.commit() - // 5 batches written, one per partition (no splitting) - verify(mockOutputWriter, times(5)) + // 6 batches written, one per partition (no splitting) plus one written by + // the concurrent writer. + verify(mockOutputWriter, times(6)) .writeSpillableAndClose(any(), any()) verify(dynamicConcurrentWriter, times(5)).newWriter(any(), any(), any()) // 5 files written because this is the single writer mode From 18ec4b2530f68ad2703e661cfb5a06aaaa2b2dea Mon Sep 17 00:00:00 2001 From: YanxuanLiu <104543031+YanxuanLiu@users.noreply.github.com> Date: Tue, 25 Jun 2024 09:04:37 +0800 Subject: [PATCH 53/68] upgrade actions version (#11086) Signed-off-by: YanxuanLiu --- .github/workflows/blossom-ci.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/blossom-ci.yml b/.github/workflows/blossom-ci.yml index 4b8071303c1..447f3d5049b 100644 --- a/.github/workflows/blossom-ci.yml +++ b/.github/workflows/blossom-ci.yml @@ -90,7 +90,7 @@ jobs: runs-on: ubuntu-latest steps: - name: Checkout code - uses: actions/checkout@v3 + uses: actions/checkout@v4 with: repository: ${{ fromJson(needs.Authorization.outputs.args).repo }} ref: ${{ fromJson(needs.Authorization.outputs.args).ref }} @@ -98,7 +98,7 @@ jobs: # repo specific steps - name: Setup java - uses: actions/setup-java@v3 + uses: actions/setup-java@v4 with: distribution: adopt java-version: 8 From 86a905aac1544fef0554bad188c150b8e9720f91 Mon Sep 17 00:00:00 2001 From: Raza Jafri Date: Mon, 24 Jun 2024 22:54:55 -0700 Subject: [PATCH 54/68] Fixed Failing tests in arithmetic_ops_tests for Spark 4.0.0 [databricks] (#11044) * Fixed arithmetic_ops_tests * Signing off Signed-off-by: Raza Jafri * Added a mechanism to add ansi mode per test * Reverted unnecessary change to spark_init_internal.py * Corrected the year in the licence * Only set ansi conf to false when ansi_mode_disabled is set * Addressed review comments * Fixed the method name * Update integration_tests/src/main/python/conftest.py This handles cases like `cache_test.py` which should run with the default conf for `spark.sql.ansi.enabled`. --------- Signed-off-by: Raza Jafri Co-authored-by: MithunR --- .../src/main/python/arithmetic_ops_test.py | 77 ++++++++++++++----- integration_tests/src/main/python/conftest.py | 10 +++ integration_tests/src/main/python/marks.py | 3 +- .../src/main/python/spark_session.py | 9 ++- 4 files changed, 76 insertions(+), 23 deletions(-) diff --git a/integration_tests/src/main/python/arithmetic_ops_test.py b/integration_tests/src/main/python/arithmetic_ops_test.py index b75872ed8b2..d7fd941b97b 100644 --- a/integration_tests/src/main/python/arithmetic_ops_test.py +++ b/integration_tests/src/main/python/arithmetic_ops_test.py @@ -1,4 +1,4 @@ -# Copyright (c) 2020-2023, NVIDIA CORPORATION. +# Copyright (c) 2020-2024, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -17,7 +17,7 @@ from asserts import assert_gpu_and_cpu_are_equal_collect, assert_gpu_and_cpu_error, assert_gpu_fallback_collect, assert_gpu_and_cpu_are_equal_sql from data_gen import * -from marks import ignore_order, incompat, approximate_float, allow_non_gpu, datagen_overrides +from marks import ignore_order, incompat, approximate_float, allow_non_gpu, datagen_overrides, disable_ansi_mode from pyspark.sql.types import * from pyspark.sql.types import IntegralType from spark_session import * @@ -25,6 +25,10 @@ import pyspark.sql.utils from datetime import timedelta +_arithmetic_exception_string = 'java.lang.ArithmeticException' if is_before_spark_330() else \ + 'org.apache.spark.SparkArithmeticException' if is_before_spark_400() else \ + 'pyspark.errors.exceptions.captured.ArithmeticException' + # No overflow gens here because we just focus on verifying the fallback to CPU when # enabling ANSI mode. But overflows will fail the tests because CPU runs raise # exceptions. @@ -95,6 +99,7 @@ def _get_overflow_df(spark, data, data_type, expr): ).selectExpr(expr) @pytest.mark.parametrize('data_gen', _arith_data_gens, ids=idfn) +@disable_ansi_mode def test_addition(data_gen): data_type = data_gen.data_type assert_gpu_and_cpu_are_equal_collect( @@ -119,6 +124,7 @@ def test_addition_ansi_no_overflow(data_gen): conf=ansi_enabled_conf) @pytest.mark.parametrize('data_gen', _arith_data_gens, ids=idfn) +@disable_ansi_mode def test_subtraction(data_gen): data_type = data_gen.data_type assert_gpu_and_cpu_are_equal_collect( @@ -136,6 +142,7 @@ def test_subtraction(data_gen): DecimalGen(10, -2), DecimalGen(15, 3), DecimalGen(30, 12), DecimalGen(3, -3), DecimalGen(27, 7), DecimalGen(20, -3)], ids=idfn) @pytest.mark.parametrize('addOrSub', ['+', '-']) +@disable_ansi_mode def test_addition_subtraction_mixed(lhs, rhs, addOrSub): assert_gpu_and_cpu_are_equal_collect( lambda spark : two_col_df(spark, lhs, rhs).selectExpr(f"a {addOrSub} b") @@ -160,6 +167,7 @@ def test_subtraction_ansi_no_overflow(data_gen): _decimal_gen_38_10, _decimal_gen_38_neg10 ], ids=idfn) +@disable_ansi_mode def test_multiplication(data_gen): data_type = data_gen.data_type assert_gpu_and_cpu_are_equal_collect( @@ -203,6 +211,7 @@ def test_multiplication_ansi_overflow(): @pytest.mark.parametrize('rhs', [byte_gen, short_gen, int_gen, long_gen, DecimalGen(6, 3), DecimalGen(10, -2), DecimalGen(15, 3), DecimalGen(30, 12), DecimalGen(3, -3), DecimalGen(27, 7), DecimalGen(20, -3)], ids=idfn) +@disable_ansi_mode def test_multiplication_mixed(lhs, rhs): assert_gpu_and_cpu_are_equal_collect( lambda spark : two_col_df(spark, lhs, rhs).select( @@ -220,6 +229,7 @@ def test_float_multiplication_mixed(lhs, rhs): @pytest.mark.parametrize('data_gen', [double_gen, decimal_gen_32bit_neg_scale, DecimalGen(6, 3), DecimalGen(5, 5), DecimalGen(6, 0), DecimalGen(7, 4), DecimalGen(15, 0), DecimalGen(18, 0), DecimalGen(17, 2), DecimalGen(16, 4), DecimalGen(38, 21), DecimalGen(21, 17), DecimalGen(3, -2)], ids=idfn) +@disable_ansi_mode def test_division(data_gen): data_type = data_gen.data_type assert_gpu_and_cpu_are_equal_collect( @@ -232,6 +242,7 @@ def test_division(data_gen): @pytest.mark.parametrize('rhs', [byte_gen, short_gen, int_gen, long_gen, DecimalGen(4, 1), DecimalGen(5, 0), DecimalGen(5, 1), DecimalGen(10, 5)], ids=idfn) @pytest.mark.parametrize('lhs', [byte_gen, short_gen, int_gen, long_gen, DecimalGen(5, 3), DecimalGen(4, 2), DecimalGen(1, -2), DecimalGen(16, 1)], ids=idfn) +@disable_ansi_mode def test_division_mixed(lhs, rhs): assert_gpu_and_cpu_are_equal_collect( lambda spark : two_col_df(spark, lhs, rhs).select( @@ -242,12 +253,14 @@ def test_division_mixed(lhs, rhs): # instead of increasing the precision. So we have a second test that deals with a few of these use cases @pytest.mark.parametrize('rhs', [DecimalGen(30, 10), DecimalGen(28, 18)], ids=idfn) @pytest.mark.parametrize('lhs', [DecimalGen(27, 7), DecimalGen(20, -3)], ids=idfn) +@disable_ansi_mode def test_division_mixed_larger_dec(lhs, rhs): assert_gpu_and_cpu_are_equal_collect( lambda spark : two_col_df(spark, lhs, rhs).select( f.col('a'), f.col('b'), f.col('a') / f.col('b'))) +@disable_ansi_mode def test_special_decimal_division(): for precision in range(1, 39): for scale in range(-3, precision + 1): @@ -260,6 +273,7 @@ def test_special_decimal_division(): @approximate_float # we should get the perfectly correct answer for floats except when casting a decimal to a float in some corner cases. @pytest.mark.parametrize('rhs', [float_gen, double_gen], ids=idfn) @pytest.mark.parametrize('lhs', [DecimalGen(5, 3), DecimalGen(4, 2), DecimalGen(1, -2), DecimalGen(16, 1)], ids=idfn) +@disable_ansi_mode def test_float_division_mixed(lhs, rhs): assert_gpu_and_cpu_are_equal_collect( lambda spark : two_col_df(spark, lhs, rhs).select( @@ -269,6 +283,7 @@ def test_float_division_mixed(lhs, rhs): @pytest.mark.parametrize('data_gen', integral_gens + [ decimal_gen_32bit, decimal_gen_64bit, _decimal_gen_7_7, _decimal_gen_18_3, _decimal_gen_30_2, _decimal_gen_36_5, _decimal_gen_38_0], ids=idfn) +@disable_ansi_mode def test_int_division(data_gen): string_type = to_cast_string(data_gen.data_type) assert_gpu_and_cpu_are_equal_collect( @@ -282,12 +297,14 @@ def test_int_division(data_gen): @pytest.mark.parametrize('lhs', [DecimalGen(6, 5), DecimalGen(5, 4), DecimalGen(3, -2), _decimal_gen_30_2], ids=idfn) @pytest.mark.parametrize('rhs', [DecimalGen(13, 2), DecimalGen(6, 3), _decimal_gen_38_0, pytest.param(_decimal_gen_36_neg5, marks=pytest.mark.skipif(not is_before_spark_340() or is_databricks113_or_later(), reason='SPARK-41207'))], ids=idfn) +@disable_ansi_mode def test_int_division_mixed(lhs, rhs): assert_gpu_and_cpu_are_equal_collect( lambda spark : two_col_df(spark, lhs, rhs).selectExpr( 'a DIV b')) @pytest.mark.parametrize('data_gen', _arith_data_gens, ids=idfn) +@disable_ansi_mode def test_mod(data_gen): data_type = data_gen.data_type assert_gpu_and_cpu_are_equal_collect( @@ -308,6 +325,7 @@ def test_mod(data_gen): _decimal_gen_7_7] @pytest.mark.parametrize('data_gen', _pmod_gens, ids=idfn) +@disable_ansi_mode def test_pmod(data_gen): string_type = to_cast_string(data_gen.data_type) assert_gpu_and_cpu_are_equal_collect( @@ -321,6 +339,7 @@ def test_pmod(data_gen): @allow_non_gpu("ProjectExec", "Pmod") @pytest.mark.parametrize('data_gen', test_pmod_fallback_decimal_gens + [_decimal_gen_38_0, _decimal_gen_38_10], ids=idfn) +@disable_ansi_mode def test_pmod_fallback(data_gen): string_type = to_cast_string(data_gen.data_type) assert_gpu_fallback_collect( @@ -372,8 +391,10 @@ def test_cast_neg_to_decimal_err(): data_gen = _decimal_gen_7_7 if is_before_spark_322(): exception_content = "Decimal(compact,-120000000,20,0}) cannot be represented as Decimal(7, 7)" - elif is_databricks113_or_later() or not is_before_spark_340(): + elif is_databricks113_or_later() or not is_before_spark_340() and is_before_spark_400(): exception_content = "[NUMERIC_VALUE_OUT_OF_RANGE] -12 cannot be represented as Decimal(7, 7)" + elif not is_before_spark_400(): + exception_content = "[NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION] -12 cannot be represented as Decimal(7, 7)" else: exception_content = "Decimal(compact, -120000000, 20, 0) cannot be represented as Decimal(7, 7)" @@ -410,6 +431,7 @@ def test_mod_pmod_by_zero_not_ansi(data_gen): @pytest.mark.parametrize('rhs', [byte_gen, short_gen, int_gen, long_gen, DecimalGen(6, 3), DecimalGen(10, -2), DecimalGen(15, 3), DecimalGen(30, 12), DecimalGen(3, -3), DecimalGen(27, 7), DecimalGen(20, -3)], ids=idfn) +@disable_ansi_mode def test_mod_mixed(lhs, rhs): assert_gpu_and_cpu_are_equal_collect( lambda spark : two_col_df(spark, lhs, rhs).selectExpr(f"a % b")) @@ -417,6 +439,7 @@ def test_mod_mixed(lhs, rhs): # @pytest.mark.skipif(not is_databricks113_or_later() and not is_spark_340_or_later(), reason="https://github.com/NVIDIA/spark-rapids/issues/8330") @pytest.mark.parametrize('lhs', [DecimalGen(38,0), DecimalGen(37,2), DecimalGen(38,5), DecimalGen(38,-10), DecimalGen(38,7)], ids=idfn) @pytest.mark.parametrize('rhs', [DecimalGen(27,7), DecimalGen(30,10), DecimalGen(38,1), DecimalGen(36,0), DecimalGen(28,-7)], ids=idfn) +@disable_ansi_mode def test_mod_mixed_decimal128(lhs, rhs): assert_gpu_and_cpu_are_equal_collect( lambda spark : two_col_df(spark, lhs, rhs).selectExpr("a", "b", f"a % b")) @@ -424,6 +447,7 @@ def test_mod_mixed_decimal128(lhs, rhs): # Split into 4 tests to permute https://github.com/NVIDIA/spark-rapids/issues/7553 failures @pytest.mark.parametrize('lhs', [byte_gen, short_gen, int_gen, long_gen], ids=idfn) @pytest.mark.parametrize('rhs', [byte_gen, short_gen, int_gen, long_gen], ids=idfn) +@disable_ansi_mode def test_pmod_mixed_numeric(lhs, rhs): assert_gpu_and_cpu_are_equal_collect( lambda spark : two_col_df(spark, lhs, rhs).selectExpr(f"pmod(a, b)")) @@ -433,6 +457,7 @@ def test_pmod_mixed_numeric(lhs, rhs): DecimalGen(4, 2), DecimalGen(3, -2), DecimalGen(16, 7), DecimalGen(19, 0), DecimalGen(30, 10) ], ids=idfn) @pytest.mark.parametrize('rhs', [byte_gen, short_gen, int_gen, long_gen], ids=idfn) +@disable_ansi_mode def test_pmod_mixed_decimal_lhs(lhs, rhs): assert_gpu_fallback_collect( lambda spark : two_col_df(spark, lhs, rhs).selectExpr(f"pmod(a, b)"), @@ -443,6 +468,7 @@ def test_pmod_mixed_decimal_lhs(lhs, rhs): @pytest.mark.parametrize('rhs', [DecimalGen(6, 3), DecimalGen(10, -2), DecimalGen(15, 3), DecimalGen(30, 12), DecimalGen(3, -3), DecimalGen(27, 7), DecimalGen(20, -3) ], ids=idfn) +@disable_ansi_mode def test_pmod_mixed_decimal_rhs(lhs, rhs): assert_gpu_fallback_collect( lambda spark : two_col_df(spark, lhs, rhs).selectExpr(f"pmod(a, b)"), @@ -455,6 +481,7 @@ def test_pmod_mixed_decimal_rhs(lhs, rhs): @pytest.mark.parametrize('rhs', [DecimalGen(6, 3), DecimalGen(10, -2), DecimalGen(15, 3), DecimalGen(30, 12), DecimalGen(3, -3), DecimalGen(27, 7), DecimalGen(20, -3) ], ids=idfn) +@disable_ansi_mode def test_pmod_mixed_decimal(lhs, rhs): assert_gpu_fallback_collect( lambda spark : two_col_df(spark, lhs, rhs).selectExpr(f"pmod(a, b)"), @@ -466,6 +493,7 @@ def test_signum(data_gen): lambda spark : unary_op_df(spark, data_gen).selectExpr('signum(a)')) @pytest.mark.parametrize('data_gen', numeric_gens + _arith_decimal_gens_low_precision, ids=idfn) +@disable_ansi_mode def test_unary_minus(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark : unary_op_df(spark, data_gen).selectExpr('-a')) @@ -504,8 +532,7 @@ def test_unary_minus_ansi_overflow(data_type, value): assert_gpu_and_cpu_error( df_fun=lambda spark: _get_overflow_df(spark, [value], data_type, '-a').collect(), conf=ansi_enabled_conf, - error_message='java.lang.ArithmeticException' if is_before_spark_330() else \ - 'org.apache.spark.SparkArithmeticException') + error_message=_arithmetic_exception_string) # This just ends up being a pass through. There is no good way to force # a unary positive into a plan, because it gets optimized out, but this @@ -516,6 +543,7 @@ def test_unary_positive(data_gen): lambda spark : unary_op_df(spark, data_gen).selectExpr('+a')) @pytest.mark.parametrize('data_gen', numeric_gens + _arith_decimal_gens_low_precision, ids=idfn) +@disable_ansi_mode def test_abs(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark : unary_op_df(spark, data_gen).selectExpr('abs(a)')) @@ -556,10 +584,9 @@ def test_abs_ansi_overflow(data_type, value): GPU: One or more rows overflow for abs operation. """ assert_gpu_and_cpu_error( - df_fun=lambda spark: _get_overflow_df(spark, [value], data_type, 'abs(a)').collect(), - conf=ansi_enabled_conf, - error_message='java.lang.ArithmeticException' if is_before_spark_330() else \ - 'org.apache.spark.SparkArithmeticException') + df_fun=lambda spark: _get_overflow_df(spark, [value], data_type, 'abs(a)').collect(), + conf=ansi_enabled_conf, + error_message=_arithmetic_exception_string) @approximate_float @pytest.mark.parametrize('data_gen', double_gens, ids=idfn) @@ -613,7 +640,8 @@ def test_ceil_scale_zero(data_gen): @pytest.mark.parametrize('data_gen', [_decimal_gen_36_neg5, _decimal_gen_38_neg10], ids=idfn) def test_floor_ceil_overflow(data_gen): exception_type = "java.lang.ArithmeticException" if is_before_spark_330() and not is_databricks104_or_later() \ - else "SparkArithmeticException" + else "SparkArithmeticException" if is_before_spark_400() else \ + "pyspark.errors.exceptions.captured.ArithmeticException: [NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION]" assert_gpu_and_cpu_error( lambda spark: unary_op_df(spark, data_gen).selectExpr('floor(a)').collect(), conf={}, @@ -678,6 +706,7 @@ def test_shift_right_unsigned(data_gen): @approximate_float @datagen_overrides(seed=0, reason="https://github.com/NVIDIA/spark-rapids/issues/9350") @pytest.mark.parametrize('data_gen', _arith_data_gens_for_round, ids=idfn) +@disable_ansi_mode def test_decimal_bround(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark: unary_op_df(spark, data_gen).selectExpr( @@ -692,6 +721,7 @@ def test_decimal_bround(data_gen): @approximate_float @datagen_overrides(seed=0, reason="https://github.com/NVIDIA/spark-rapids/issues/9847") @pytest.mark.parametrize('data_gen', _arith_data_gens_for_round, ids=idfn) +@disable_ansi_mode def test_decimal_round(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark: unary_op_df(spark, data_gen).selectExpr( @@ -726,6 +756,7 @@ def doit(spark): @incompat @approximate_float +@disable_ansi_mode def test_non_decimal_round_overflow(): gen = StructGen([('byte_c', byte_gen), ('short_c', short_gen), ('int_c', int_gen), ('long_c', long_gen), @@ -1057,7 +1088,8 @@ def _div_overflow_exception_when(expr, ansi_enabled, is_lit=False): ansi_conf = {'spark.sql.ansi.enabled': ansi_enabled} err_exp = 'java.lang.ArithmeticException' if is_before_spark_330() else \ 'org.apache.spark.SparkArithmeticException' \ - if not is_lit or not is_spark_340_or_later() else "pyspark.errors.exceptions.captured.ArithmeticException" + if (not is_lit or not is_spark_340_or_later()) and is_before_spark_400() else \ + "pyspark.errors.exceptions.captured.ArithmeticException" err_mess = ': Overflow in integral divide' \ if is_before_spark_340() and not is_databricks113_or_later() else \ ': [ARITHMETIC_OVERFLOW] Overflow in integral divide' @@ -1123,7 +1155,7 @@ def test_add_overflow_with_ansi_enabled(data, tp, expr): assert_gpu_and_cpu_error( lambda spark: _get_overflow_df(spark, data, tp, expr).collect(), conf=ansi_enabled_conf, - error_message='java.lang.ArithmeticException' if is_before_spark_330() else 'SparkArithmeticException') + error_message=_arithmetic_exception_string) elif isinstance(tp, DecimalType): assert_gpu_and_cpu_error( lambda spark: _get_overflow_df(spark, data, tp, expr).collect(), @@ -1152,7 +1184,8 @@ def test_subtraction_overflow_with_ansi_enabled(data, tp, expr): assert_gpu_and_cpu_error( lambda spark: _get_overflow_df(spark, data, tp, expr).collect(), conf=ansi_enabled_conf, - error_message='java.lang.ArithmeticException' if is_before_spark_330() else 'SparkArithmeticException') + error_message='java.lang.ArithmeticException' if is_before_spark_330() else 'SparkArithmeticException' \ + if is_before_spark_400() else "pyspark.errors.exceptions.captured.ArithmeticException:") elif isinstance(tp, DecimalType): assert_gpu_and_cpu_error( lambda spark: _get_overflow_df(spark, data, tp, expr).collect(), @@ -1183,7 +1216,7 @@ def test_unary_minus_ansi_overflow_day_time_interval(ansi_enabled): assert_gpu_and_cpu_error( df_fun=lambda spark: _get_overflow_df(spark, [timedelta(microseconds=LONG_MIN)], DayTimeIntervalType(), '-a').collect(), conf={'spark.sql.ansi.enabled': ansi_enabled}, - error_message='SparkArithmeticException') + error_message='SparkArithmeticException' if is_before_spark_400() else "ArithmeticException") @pytest.mark.skipif(is_before_spark_330(), reason='DayTimeInterval is not supported before Pyspark 3.3.0') @pytest.mark.parametrize('ansi_enabled', ['false', 'true']) @@ -1224,7 +1257,7 @@ def test_add_overflow_with_ansi_enabled_day_time_interval(ansi_enabled): StructType([StructField('a', DayTimeIntervalType()), StructField('b', DayTimeIntervalType())]) ).selectExpr('a + b').collect(), conf={'spark.sql.ansi.enabled': ansi_enabled}, - error_message='SparkArithmeticException') + error_message=_arithmetic_exception_string) @pytest.mark.skipif(is_before_spark_330(), reason='DayTimeInterval is not supported before Pyspark 3.3.0') @pytest.mark.parametrize('ansi_enabled', ['false', 'true']) @@ -1244,7 +1277,7 @@ def test_subtraction_overflow_with_ansi_enabled_day_time_interval(ansi_enabled): StructType([StructField('a', DayTimeIntervalType()), StructField('b', DayTimeIntervalType())]) ).selectExpr('a - b').collect(), conf={'spark.sql.ansi.enabled': ansi_enabled}, - error_message='SparkArithmeticException') + error_message='SparkArithmeticException' if is_before_spark_400() else "ArithmeticException") @pytest.mark.skipif(is_before_spark_330(), reason='DayTimeInterval is not supported before Pyspark 3.3.0') def test_unary_positive_day_time_interval(): @@ -1303,7 +1336,8 @@ def _get_overflow_df_2cols(spark, data_types, values, expr): def test_day_time_interval_division_overflow(data_type, value_pair): exception_message = "SparkArithmeticException: Overflow in integral divide." \ if is_before_spark_340() and not is_databricks113_or_later() else \ - "SparkArithmeticException: [ARITHMETIC_OVERFLOW] Overflow in integral divide." + "SparkArithmeticException: [ARITHMETIC_OVERFLOW] Overflow in integral divide." if is_before_spark_400() else \ + "ArithmeticException: [ARITHMETIC_OVERFLOW] Overflow in integral divide." assert_gpu_and_cpu_error( df_fun=lambda spark: _get_overflow_df_2cols(spark, [DayTimeIntervalType(), data_type], value_pair, 'a / b').collect(), conf={}, @@ -1338,7 +1372,8 @@ def test_day_time_interval_division_round_overflow(data_type, value_pair): def test_day_time_interval_divided_by_zero(data_type, value_pair): exception_message = "SparkArithmeticException: Division by zero." \ if is_before_spark_340() and not is_databricks113_or_later() else \ - "SparkArithmeticException: [INTERVAL_DIVIDED_BY_ZERO] Division by zero" + "SparkArithmeticException: [INTERVAL_DIVIDED_BY_ZERO] Division by zero" if is_before_spark_400() else \ + "ArithmeticException: [INTERVAL_DIVIDED_BY_ZERO] Division by zero" assert_gpu_and_cpu_error( df_fun=lambda spark: _get_overflow_df_2cols(spark, [DayTimeIntervalType(), data_type], value_pair, 'a / b').collect(), conf={}, @@ -1349,7 +1384,8 @@ def test_day_time_interval_divided_by_zero(data_type, value_pair): def test_day_time_interval_divided_by_zero_scalar(zero_literal): exception_message = "SparkArithmeticException: Division by zero." \ if is_before_spark_340() and not is_databricks113_or_later() else \ - "SparkArithmeticException: [INTERVAL_DIVIDED_BY_ZERO] Division by zero." + "SparkArithmeticException: [INTERVAL_DIVIDED_BY_ZERO] Division by zero." if is_before_spark_400() else \ + "ArithmeticException: [INTERVAL_DIVIDED_BY_ZERO] Division by zero" assert_gpu_and_cpu_error( df_fun=lambda spark: _get_overflow_df_1col(spark, DayTimeIntervalType(), [timedelta(seconds=1)], 'a / ' + zero_literal).collect(), conf={}, @@ -1369,7 +1405,8 @@ def test_day_time_interval_divided_by_zero_scalar(zero_literal): def test_day_time_interval_scalar_divided_by_zero(data_type, value): exception_message = "SparkArithmeticException: Division by zero." \ if is_before_spark_340() and not is_databricks113_or_later() else \ - "SparkArithmeticException: [INTERVAL_DIVIDED_BY_ZERO] Division by zero." + "SparkArithmeticException: [INTERVAL_DIVIDED_BY_ZERO] Division by zero." if is_before_spark_400() else \ + "ArithmeticException: [INTERVAL_DIVIDED_BY_ZERO] Division by zero" assert_gpu_and_cpu_error( df_fun=lambda spark: _get_overflow_df_1col(spark, data_type, [value], 'INTERVAL 1 SECOND / a').collect(), conf={}, diff --git a/integration_tests/src/main/python/conftest.py b/integration_tests/src/main/python/conftest.py index 1adeb6964fd..6af40b99768 100644 --- a/integration_tests/src/main/python/conftest.py +++ b/integration_tests/src/main/python/conftest.py @@ -54,6 +54,7 @@ def array_columns_to_sort_locally(): _allow_any_non_gpu = False _non_gpu_allowed = [] +_per_test_ansi_mode_enabled = None def is_allowing_any_non_gpu(): return _allow_any_non_gpu @@ -61,6 +62,11 @@ def is_allowing_any_non_gpu(): def get_non_gpu_allowed(): return _non_gpu_allowed + +def is_per_test_ansi_mode_enabled(): + return _per_test_ansi_mode_enabled + + def get_validate_execs_in_gpu_plan(): return _validate_execs_in_gpu_plan @@ -210,10 +216,14 @@ def pytest_runtest_setup(item): global _allow_any_non_gpu global _non_gpu_allowed + global _per_test_ansi_mode_enabled _non_gpu_allowed_databricks = [] _allow_any_non_gpu_databricks = False non_gpu_databricks = item.get_closest_marker('allow_non_gpu_databricks') non_gpu = item.get_closest_marker('allow_non_gpu') + _per_test_ansi_mode_enabled = None if item.get_closest_marker('disable_ansi_mode') is None \ + else not item.get_closest_marker('disable_ansi_mode') + if non_gpu_databricks: if is_databricks_runtime(): if non_gpu_databricks.kwargs and non_gpu_databricks.kwargs['any']: diff --git a/integration_tests/src/main/python/marks.py b/integration_tests/src/main/python/marks.py index 1f326a75505..9a0bde11113 100644 --- a/integration_tests/src/main/python/marks.py +++ b/integration_tests/src/main/python/marks.py @@ -1,4 +1,4 @@ -# Copyright (c) 2020-2023, NVIDIA CORPORATION. +# Copyright (c) 2020-2024, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -16,6 +16,7 @@ allow_non_gpu_databricks = pytest.mark.allow_non_gpu_databricks allow_non_gpu = pytest.mark.allow_non_gpu +disable_ansi_mode = pytest.mark.disable_ansi_mode validate_execs_in_gpu_plan = pytest.mark.validate_execs_in_gpu_plan approximate_float = pytest.mark.approximate_float ignore_order = pytest.mark.ignore_order diff --git a/integration_tests/src/main/python/spark_session.py b/integration_tests/src/main/python/spark_session.py index c55f1976497..26388617fff 100644 --- a/integration_tests/src/main/python/spark_session.py +++ b/integration_tests/src/main/python/spark_session.py @@ -16,7 +16,7 @@ import calendar, time from datetime import date, datetime from contextlib import contextmanager, ExitStack -from conftest import is_allowing_any_non_gpu, get_non_gpu_allowed, get_validate_execs_in_gpu_plan, is_databricks_runtime, is_at_least_precommit_run, get_inject_oom_conf +from conftest import is_allowing_any_non_gpu, get_non_gpu_allowed, get_validate_execs_in_gpu_plan, is_databricks_runtime, is_at_least_precommit_run, get_inject_oom_conf, is_per_test_ansi_mode_enabled from pyspark.sql import DataFrame from pyspark.sql.types import TimestampType, DateType, _acceptable_types from spark_init_internal import get_spark_i_know_what_i_am_doing, spark_version @@ -41,7 +41,6 @@ def _from_scala_map(scala_map): # Many of these are redundant with default settings for the configs but are set here explicitly # to ensure any cluster settings do not interfere with tests that assume the defaults. _default_conf = { - 'spark.ansi.enabled': 'false', 'spark.rapids.sql.castDecimalToFloat.enabled': 'false', 'spark.rapids.sql.castFloatToDecimal.enabled': 'false', 'spark.rapids.sql.castFloatToIntegralTypes.enabled': 'false', @@ -127,6 +126,9 @@ def with_spark_session(func, conf={}): """Run func that takes a spark session as input with the given configs set.""" reset_spark_session_conf() _add_job_description(conf) + # Only set the ansi conf if not set by the test explicitly by setting the value in the dict + if "spark.sql.ansi.enabled" not in conf and is_per_test_ansi_mode_enabled() is not None: + conf["spark.sql.ansi.enabled"] = is_per_test_ansi_mode_enabled() _set_all_confs(conf) ret = func(_spark) _check_for_proper_return_values(ret) @@ -205,6 +207,9 @@ def is_before_spark_350(): def is_before_spark_351(): return spark_version() < "3.5.1" +def is_before_spark_400(): + return spark_version() < "4.0.0" + def is_spark_320_or_later(): return spark_version() >= "3.2.0" From 7a8690f5e2e4e9009c121e08c6429e2496b4f01c Mon Sep 17 00:00:00 2001 From: "Hongbin Ma (Mahone)" Date: Tue, 25 Jun 2024 15:39:47 +0800 Subject: [PATCH 55/68] fix duplicate counted metrics like op time for GpuCoalesceBatches (#11062) * with call site print, not good because some test cases by design will dup Signed-off-by: Hongbin Ma (Mahone) * done Signed-off-by: Hongbin Ma (Mahone) * add file Signed-off-by: Hongbin Ma (Mahone) * fix comiple Signed-off-by: Hongbin Ma (Mahone) * address review comments Signed-off-by: Hongbin Ma (Mahone) --------- Signed-off-by: Hongbin Ma (Mahone) --- .../spark/rapids/GpuCoalesceBatches.scala | 6 +- .../com/nvidia/spark/rapids/GpuExec.scala | 30 ++++++-- .../nvidia/spark/rapids/NvtxWithMetrics.scala | 18 +++-- .../nvidia/spark/rapids/MetricsSuite.scala | 68 +++++++++++++++++++ 4 files changed, 109 insertions(+), 13 deletions(-) create mode 100644 tests/src/test/scala/com/nvidia/spark/rapids/MetricsSuite.scala diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCoalesceBatches.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCoalesceBatches.scala index e6dc216d7e6..1afc03b177b 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCoalesceBatches.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCoalesceBatches.scala @@ -462,7 +462,7 @@ abstract class AbstractGpuCoalesceIterator( // If we have reached the cuDF limit once, proactively filter batches // after that first limit is reached. GpuFilter.filterAndClose(cbFromIter, inputFilterTier.get, - NoopMetric, NoopMetric, opTime) + NoopMetric, NoopMetric, NoopMetric) } else { Iterator(cbFromIter) } @@ -499,7 +499,7 @@ abstract class AbstractGpuCoalesceIterator( var filteredBytes = 0L if (hasAnyToConcat) { val filteredDowIter = GpuFilter.filterAndClose(concatAllAndPutOnGPU(), - filterTier, NoopMetric, NoopMetric, opTime) + filterTier, NoopMetric, NoopMetric, NoopMetric) while (filteredDowIter.hasNext) { closeOnExcept(filteredDowIter.next()) { filteredDownCb => filteredNumRows += filteredDownCb.numRows() @@ -512,7 +512,7 @@ abstract class AbstractGpuCoalesceIterator( // filterAndClose takes ownership of CB so we should not close it on a failure // anymore... val filteredCbIter = GpuFilter.filterAndClose(cb.release, filterTier, - NoopMetric, NoopMetric, opTime) + NoopMetric, NoopMetric, NoopMetric) while (filteredCbIter.hasNext) { closeOnExcept(filteredCbIter.next()) { filteredCb => val filteredWouldBeRows = filteredNumRows + filteredCb.numRows() diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExec.scala index ec87dd62d6c..d83f20113b2 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExec.scala @@ -152,12 +152,34 @@ sealed abstract class GpuMetric extends Serializable { def +=(v: Long): Unit def add(v: Long): Unit + private var isTimerActive = false + + final def tryActivateTimer(): Boolean = { + if (!isTimerActive) { + isTimerActive = true + true + } else { + false + } + } + + final def deactivateTimer(duration: Long): Unit = { + if (isTimerActive) { + isTimerActive = false + add(duration) + } + } + final def ns[T](f: => T): T = { - val start = System.nanoTime() - try { + if (tryActivateTimer()) { + val start = System.nanoTime() + try { + f + } finally { + deactivateTimer(System.nanoTime() - start) + } + } else { f - } finally { - add(System.nanoTime() - start) } } } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/NvtxWithMetrics.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/NvtxWithMetrics.scala index 92a11f56123..538f117e50f 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/NvtxWithMetrics.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/NvtxWithMetrics.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019-2021, NVIDIA CORPORATION. + * Copyright (c) 2019-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -32,26 +32,32 @@ object NvtxWithMetrics { * by the amount of time spent in the range */ class NvtxWithMetrics(name: String, color: NvtxColor, val metrics: GpuMetric*) - extends NvtxRange(name, color) { + extends NvtxRange(name, color) { + val needTracks = metrics.map(_.tryActivateTimer()) private val start = System.nanoTime() override def close(): Unit = { val time = System.nanoTime() - start - metrics.foreach { metric => - metric += time + metrics.toSeq.zip(needTracks).foreach { pair => + if (pair._2) { + pair._1.deactivateTimer(time) + } } super.close() } } class MetricRange(val metrics: GpuMetric*) extends AutoCloseable { + val needTracks = metrics.map(_.tryActivateTimer()) private val start = System.nanoTime() override def close(): Unit = { val time = System.nanoTime() - start - metrics.foreach { metric => - metric += time + metrics.toSeq.zip(needTracks).foreach { pair => + if (pair._2) { + pair._1.deactivateTimer(time) + } } } } diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/MetricsSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/MetricsSuite.scala new file mode 100644 index 00000000000..580c5a2ed55 --- /dev/null +++ b/tests/src/test/scala/com/nvidia/spark/rapids/MetricsSuite.scala @@ -0,0 +1,68 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed 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 com.nvidia.spark.rapids + +import ai.rapids.cudf.NvtxColor +import com.nvidia.spark.rapids.Arm.withResource +import org.scalatest.funsuite.AnyFunSuite + +class MetricsSuite extends AnyFunSuite { + + test("GpuMetric.ns: duplicate timing on the same metrics") { + val m1 = new LocalGpuMetric() + m1.ns( + m1.ns( + Thread.sleep(100) + ) + ) + // if the timing is duplicated, the value should be around 200,000,000 + assert(m1.value < 100000000 * 1.5) + assert(m1.value > 100000000 * 0.5) + } + + test("MetricRange: duplicate timing on the same metrics") { + val m1 = new LocalGpuMetric() + val m2 = new LocalGpuMetric() + withResource(new MetricRange(m1, m2)) { _ => + withResource(new MetricRange(m2, m1)) { _ => + Thread.sleep(100) + } + } + + // if the timing is duplicated, the value should be around 200,000,000 + assert(m1.value < 100000000 * 1.5) + assert(m1.value > 100000000 * 0.5) + assert(m2.value < 100000000 * 1.5) + assert(m2.value > 100000000 * 0.5) + } + + test("NvtxWithMetrics: duplicate timing on the same metrics") { + val m1 = new LocalGpuMetric() + val m2 = new LocalGpuMetric() + withResource(new NvtxWithMetrics("a", NvtxColor.BLUE, m1, m2)) { _ => + withResource(new NvtxWithMetrics("b", NvtxColor.BLUE, m2, m1)) { _ => + Thread.sleep(100) + } + } + + // if the timing is duplicated, the value should be around 200,000,000 + assert(m1.value < 100000000 * 1.5) + assert(m1.value > 100000000 * 0.5) + assert(m2.value < 100000000 * 1.5) + assert(m2.value > 100000000 * 0.5) + } +} From b3b5b5e259a84aabca84bd960ba21ab70f672a9e Mon Sep 17 00:00:00 2001 From: Raza Jafri Date: Tue, 25 Jun 2024 06:29:34 -0700 Subject: [PATCH 56/68] Add GpuBucketingUtils shim to Spark 4.0.0 (#11092) * Add GpuBucketingUtils shim to Spark 4.0.0 * Signing off Signed-off-by: Raza Jafri --------- Signed-off-by: Raza Jafri --- .../nvidia/spark/rapids/shims/spark330/GpuBucketingUtils.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/spark330/GpuBucketingUtils.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/spark330/GpuBucketingUtils.scala index feb562fa9b8..0f7c9b4fd62 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/spark330/GpuBucketingUtils.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/spark330/GpuBucketingUtils.scala @@ -31,6 +31,7 @@ {"spark": "343"} {"spark": "350"} {"spark": "351"} +{"spark": "400"} spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims From 6455396a2c06f3cd8dfa9bf7692eebc57902440e Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 25 Jun 2024 11:19:53 -0700 Subject: [PATCH 57/68] Improve the diagnostics for 'conv' fallback explain (#11076) * Improve the diagnostics for 'conv' fallback explain Signed-off-by: Jihoon Son * don't use nil Signed-off-by: Jihoon Son * the bases should not be an empty string in the error message when the user input is not Signed-off-by: Jihoon Son * more user-friendly message * Update sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringFunctions.scala Co-authored-by: Gera Shegalov --------- Signed-off-by: Jihoon Son Co-authored-by: Gera Shegalov --- .../src/main/python/string_test.py | 16 +++++++++++++++ .../spark/sql/rapids/stringFunctions.scala | 20 ++++++++++++++++--- 2 files changed, 33 insertions(+), 3 deletions(-) diff --git a/integration_tests/src/main/python/string_test.py b/integration_tests/src/main/python/string_test.py index 5631f13f13d..6ca0e1a1967 100644 --- a/integration_tests/src/main/python/string_test.py +++ b/integration_tests/src/main/python/string_test.py @@ -820,6 +820,22 @@ def test_conv_dec_to_from_hex(from_base, to_base, pattern): conf={'spark.rapids.sql.expression.Conv': True} ) +@pytest.mark.parametrize('from_base,to_base,expected_err_msg_prefix', + [ + pytest.param(10, 15, '15 is not a supported target radix', id='to_base_unsupported'), + pytest.param(11, 16, '11 is not a supported source radix', id='from_base_unsupported'), + pytest.param(9, 17, 'both 9 and 17 are not a supported radix', id='both_base_unsupported') + ]) +def test_conv_unsupported_base(from_base, to_base, expected_err_msg_prefix): + def do_conv(spark): + gen = StringGen() + df = unary_op_df(spark, gen).select('a', f.conv(f.col('a'), from_base, to_base)) + explain_str = spark.sparkContext._jvm.com.nvidia.spark.rapids.ExplainPlan.explainPotentialGpuPlan(df._jdf, "ALL") + unsupported_base_str = f'{expected_err_msg_prefix}, only literal 10 or 16 are supported for source and target radixes' + assert unsupported_base_str in explain_str + + with_cpu_session(do_conv) + format_number_gens = integral_gens + [DecimalGen(precision=7, scale=7), DecimalGen(precision=18, scale=0), DecimalGen(precision=18, scale=3), DecimalGen(precision=36, scale=5), DecimalGen(precision=36, scale=-5), DecimalGen(precision=38, scale=10), diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringFunctions.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringFunctions.scala index dc2845e4461..a435988686d 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringFunctions.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringFunctions.scala @@ -2084,11 +2084,25 @@ class GpuConvMeta( override def tagExprForGpu(): Unit = { val fromBaseLit = GpuOverrides.extractLit(expr.fromBaseExpr) val toBaseLit = GpuOverrides.extractLit(expr.toBaseExpr) + val errorPostfix = "only literal 10 or 16 are supported for source and target radixes" (fromBaseLit, toBaseLit) match { - case (Some(Literal(fromBaseVal, IntegerType)), Some(Literal(toBaseVal, IntegerType))) - if Set(fromBaseVal, toBaseVal).subsetOf(Set(10, 16)) => () + case (Some(Literal(fromBaseVal, IntegerType)), Some(Literal(toBaseVal, IntegerType))) => + def isBaseSupported(base: Any): Boolean = base == 10 || base == 16 + if (!isBaseSupported(fromBaseVal) && !isBaseSupported(toBaseVal)) { + willNotWorkOnGpu(because = s"both ${fromBaseVal} and ${toBaseVal} are not " + + s"a supported radix, ${errorPostfix}") + } else if (!isBaseSupported(fromBaseVal)) { + willNotWorkOnGpu(because = s"${fromBaseVal} is not a supported source radix, " + + s"${errorPostfix}") + } else if (!isBaseSupported(toBaseVal)) { + willNotWorkOnGpu(because = s"${toBaseVal} is not a supported target radix, " + + s"${errorPostfix}") + } case _ => - willNotWorkOnGpu(because = "only literal 10 or 16 for from_base and to_base are supported") + // This will never happen in production as the function signature enforces + // integer types for the bases, but nice to have an edge case handling. + willNotWorkOnGpu(because = "either source radix or target radix is not an integer " + + "literal, " + errorPostfix) } } From 34e6bc88b583fcb6e100fa507faff088994fdda3 Mon Sep 17 00:00:00 2001 From: MithunR Date: Tue, 25 Jun 2024 17:09:27 -0700 Subject: [PATCH 58/68] Disable ANSI mode for window function tests [databricks] (#11073) * Disable ANSI mode for window function tests. Fixes #11019. Window function tests fail on Spark 4.0 because of #5114 (and #5120 broadly), because spark-rapids does not support SUM, COUNT, and certain other aggregations in ANSI mode. This commit disables ANSI mode tests for the failing window function tests. These may be revisited, once error/overflow checking is available for ANSI mode in spark-rapids. Signed-off-by: MithunR * Switch from @ansi_mode_disabled to @disable_ansi_mode. --------- Signed-off-by: MithunR --- .../src/main/python/window_function_test.py | 41 +++++++++++++++++++ 1 file changed, 41 insertions(+) diff --git a/integration_tests/src/main/python/window_function_test.py b/integration_tests/src/main/python/window_function_test.py index af8bbbb55b3..44bc2a07d57 100644 --- a/integration_tests/src/main/python/window_function_test.py +++ b/integration_tests/src/main/python/window_function_test.py @@ -165,6 +165,8 @@ def test_float_window_min_max_all_nans(data_gen): .withColumn("max_b", f.max('a').over(w)) ) + +@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114 @ignore_order @pytest.mark.parametrize('data_gen', [decimal_gen_128bit], ids=idfn) def test_decimal128_count_window(data_gen): @@ -177,6 +179,8 @@ def test_decimal128_count_window(data_gen): ' rows between 2 preceding and 10 following) as count_c_asc ' 'from window_agg_table') + +@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114 @ignore_order @pytest.mark.parametrize('data_gen', [decimal_gen_128bit], ids=idfn) def test_decimal128_count_window_no_part(data_gen): @@ -189,6 +193,8 @@ def test_decimal128_count_window_no_part(data_gen): ' rows between 2 preceding and 10 following) as count_b_asc ' 'from window_agg_table') + +@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114 @ignore_order @pytest.mark.parametrize('data_gen', decimal_gens, ids=idfn) def test_decimal_sum_window(data_gen): @@ -201,6 +207,8 @@ def test_decimal_sum_window(data_gen): ' rows between 2 preceding and 10 following) as sum_c_asc ' 'from window_agg_table') + +@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114 @ignore_order @pytest.mark.parametrize('data_gen', decimal_gens, ids=idfn) def test_decimal_sum_window_no_part(data_gen): @@ -214,6 +222,7 @@ def test_decimal_sum_window_no_part(data_gen): 'from window_agg_table') +@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114 @ignore_order @pytest.mark.parametrize('data_gen', decimal_gens, ids=idfn) def test_decimal_running_sum_window(data_gen): @@ -227,6 +236,8 @@ def test_decimal_running_sum_window(data_gen): 'from window_agg_table', conf = {'spark.rapids.sql.batchSizeBytes': '100'}) + +@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114 @ignore_order @pytest.mark.parametrize('data_gen', decimal_gens, ids=idfn) def test_decimal_running_sum_window_no_part(data_gen): @@ -302,6 +313,7 @@ def test_window_aggs_for_ranges_numeric_long_overflow(data_gen): 'from window_agg_table') +@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114 # In a distributed setup the order of the partitions returned might be different, so we must ignore the order # but small batch sizes can make sort very slow, so do the final order by locally @ignore_order(local=True) @@ -352,6 +364,7 @@ def test_window_aggs_for_range_numeric_date(data_gen, batch_size): conf = conf) +@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114 # In a distributed setup the order of the partitions returned might be different, so we must ignore the order # but small batch sizes can make sort very slow, so do the final order by locally @ignore_order(local=True) @@ -396,6 +409,7 @@ def test_window_aggs_for_rows(data_gen, batch_size): conf = conf) +@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114 @ignore_order(local=True) @pytest.mark.parametrize('batch_size', ['1000', '1g'], ids=idfn) @pytest.mark.parametrize('data_gen', [ @@ -482,6 +496,8 @@ def test_window_batched_unbounded(b_gen, batch_size): validate_execs_in_gpu_plan = ['GpuCachedDoublePassWindowExec'], conf = conf) + +@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114 # This is for aggregations that work with a running window optimization. They don't need to be batched # specially, but it only works if all of the aggregations can support this. # the order returned should be consistent because the data ends up in a single task (no partitioning) @@ -520,6 +536,7 @@ def test_rows_based_running_window_unpartitioned(b_gen, batch_size): conf = conf) +@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114 @pytest.mark.parametrize('batch_size', ['1000', '1g'], ids=idfn) # Testing multiple batch sizes. @pytest.mark.parametrize('a_gen', integral_gens + [string_gen, date_gen, timestamp_gen], ids=meta_idfn('data:')) @allow_non_gpu(*non_utc_allow) @@ -694,6 +711,7 @@ def test_window_running_rank(data_gen): conf = conf) +@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114 # This is for aggregations that work with a running window optimization. They don't need to be batched # specially, but it only works if all of the aggregations can support this. # In a distributed setup the order of the partitions returned might be different, so we must ignore the order @@ -738,6 +756,8 @@ def test_rows_based_running_window_partitioned(b_gen, c_gen, batch_size): conf = conf) + +@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114 @ignore_order(local=True) @pytest.mark.parametrize('batch_size', ['1000', '1g'], ids=idfn) # Test different batch sizes. @pytest.mark.parametrize('part_gen', [int_gen, long_gen], ids=idfn) # Partitioning is not really the focus of the test. @@ -805,6 +825,7 @@ def must_test_sum_aggregation(gen): conf=conf) +@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114 # Test that we can do a running window sum on floats and doubles and decimal. This becomes problematic because we do the agg in parallel # which means that the result can switch back and forth from Inf to not Inf depending on the order of aggregations. # We test this by limiting the range of the values in the sum to never hit Inf, and by using abs so we don't have @@ -836,6 +857,7 @@ def test_window_running_float_decimal_sum(batch_size): conf = conf) +@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114 @approximate_float @ignore_order(local=True) @pytest.mark.parametrize('batch_size', ['1000', '1g'], ids=idfn) # Test different batch sizes. @@ -879,6 +901,7 @@ def window(oby_column): conf=conf) +@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114 # In a distributed setup the order of the partitions returned might be different, so we must ignore the order # but small batch sizes can make sort very slow, so do the final order by locally @ignore_order(local=True) @@ -1000,6 +1023,7 @@ def test_window_aggs_for_rows_lead_lag_on_arrays(a_gen, b_gen, c_gen, d_gen): ''') +@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114 # lead and lag don't currently work for string columns, so redo the tests, but just for strings # without lead and lag # In a distributed setup the order of the partitions returned might be different, so we must ignore the order @@ -1107,6 +1131,8 @@ def test_window_aggs_lag_ignore_nulls_fallback(a_gen, b_gen, c_gen, d_gen): FROM window_agg_table ''') + +@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114 # Test for RANGE queries, with timestamp order-by expressions. # In a distributed setup the order of the partitions returned might be different, so we must ignore the order # but small batch sizes can make sort very slow, so do the final order by locally @@ -1155,6 +1181,7 @@ def test_window_aggs_for_ranges_timestamps(data_gen): conf = {'spark.rapids.sql.castFloatToDecimal.enabled': True}) +@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114 # In a distributed setup the order of the partitions returned might be different, so we must ignore the order # but small batch sizes can make sort very slow, so do the final order by locally @ignore_order(local=True) @@ -1201,6 +1228,7 @@ def test_window_aggregations_for_decimal_and_float_ranges(data_gen): conf={}) +@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114 # In a distributed setup the order of the partitions returned might be different, so we must ignore the order # but small batch sizes can make sort very slow, so do the final order by locally @ignore_order(local=True) @@ -1306,6 +1334,7 @@ def test_window_aggs_for_rows_collect_list(): conf={'spark.rapids.sql.window.collectList.enabled': True}) +@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114 # SortExec does not support array type, so sort the result locally. @ignore_order(local=True) # This test is more directed at Databricks and their running window optimization instead of ours @@ -1347,6 +1376,8 @@ def test_running_window_function_exec_for_all_aggs(): ''', conf={'spark.rapids.sql.window.collectList.enabled': True}) + +@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114 # Test the Databricks WindowExec which combines a WindowExec with a ProjectExec and provides the output # fields that we need to handle with an extra GpuProjectExec and we need the input expressions to compute # a window function of another window function case @@ -1668,6 +1699,8 @@ def do_it(spark): assert_gpu_fallback_collect(do_it, 'WindowExec') + +@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114 @ignore_order(local=True) # single-level structs (no nested structs) are now supported by the plugin @pytest.mark.parametrize('part_gen', [StructGen([["a", long_gen]])], ids=meta_idfn('partBy:')) @@ -1731,6 +1764,8 @@ def do_it(spark): assert_gpu_and_cpu_are_equal_collect(do_it) + +@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114 @ignore_order def test_unbounded_to_unbounded_window(): # This is specifically to test a bug that caused overflow issues when calculating @@ -1784,6 +1819,7 @@ def test_window_first_last_nth_ignore_nulls(data_gen): 'FROM window_agg_table') +@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114 @tz_sensitive_test @allow_non_gpu(*non_supported_tz_allow) @ignore_order(local=True) @@ -1825,6 +1861,7 @@ def test_to_date_with_window_functions(): ) +@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114 @ignore_order(local=True) @approximate_float @pytest.mark.parametrize('batch_size', ['1000', '1g'], ids=idfn) @@ -1881,6 +1918,7 @@ def spark_bugs_in_decimal_sorting(): return v < "3.1.4" or v < "3.3.1" or v < "3.2.3" or v < "3.4.0" +@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114 @ignore_order(local=True) @approximate_float @pytest.mark.parametrize('batch_size', ['1g'], ids=idfn) @@ -1925,6 +1963,7 @@ def test_window_aggs_for_negative_rows_unpartitioned(data_gen, batch_size): conf=conf) +@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114 @ignore_order(local=True) @pytest.mark.parametrize('batch_size', ['1000', '1g'], ids=idfn) @pytest.mark.parametrize('data_gen', [ @@ -1964,6 +2003,7 @@ def test_window_aggs_for_batched_finite_row_windows_partitioned(data_gen, batch_ conf=conf) +@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114 @ignore_order(local=True) @pytest.mark.parametrize('batch_size', ['1000', '1g'], ids=idfn) @pytest.mark.parametrize('data_gen', [ @@ -2003,6 +2043,7 @@ def test_window_aggs_for_batched_finite_row_windows_unpartitioned(data_gen, batc conf=conf) +@disable_ansi_mode # https://github.com/NVIDIA/spark-rapids/issues/5114 @ignore_order(local=True) @pytest.mark.parametrize('data_gen', [_grpkey_int_with_nulls,], ids=idfn) def test_window_aggs_for_batched_finite_row_windows_fallback(data_gen): From 3cb54c4ae2d43c7ba391dd619cf9f4d32c960e89 Mon Sep 17 00:00:00 2001 From: Haoyang Li Date: Thu, 27 Jun 2024 10:49:32 +0800 Subject: [PATCH 59/68] Fix some test issues in Spark UT and keep RapidsTestSettings update-to-date (#10997) * wip Signed-off-by: Haoyang Li * fix json suite Signed-off-by: Haoyang Li * wip Signed-off-by: Haoyang Li * update Signed-off-by: Haoyang Li * Remove all utc config and clean up Signed-off-by: Haoyang Li * hardcode timezone to LA in ci Signed-off-by: Haoyang Li * remove concat Signed-off-by: Haoyang Li * remove spark timezone settings and only keep java timezone settings Signed-off-by: Haoyang Li * remove unintensional comment Signed-off-by: Haoyang Li * delete a comment Signed-off-by: Haoyang Li * set timezone to utc for two suites to avoid fallback Signed-off-by: Haoyang Li * style Signed-off-by: Haoyang Li * after all Signed-off-by: Haoyang Li * add cast string to timestamp back to exclude after upmerge Signed-off-by: Haoyang Li --------- Signed-off-by: Haoyang Li --- .../sql/rapids/suites/RapidsCastSuite.scala | 49 +++++++++++++++ .../suites/RapidsJsonExpressionsSuite.scala | 16 ++++- .../suites/RapidsJsonFunctionsSuite.scala | 18 +++++- .../utils/RapidsSQLTestsBaseTrait.scala | 6 +- .../sql/rapids/utils/RapidsTestSettings.scala | 63 +++++++++---------- .../sql/rapids/utils/RapidsTestsTrait.scala | 5 +- 6 files changed, 113 insertions(+), 44 deletions(-) diff --git a/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/suites/RapidsCastSuite.scala b/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/suites/RapidsCastSuite.scala index f3fec27f7f6..9b032c39e7c 100644 --- a/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/suites/RapidsCastSuite.scala +++ b/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/suites/RapidsCastSuite.scala @@ -19,7 +19,12 @@ spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.suites +import java.sql.Timestamp +import java.time.{LocalDateTime, ZoneId} + import org.apache.spark.sql.catalyst.expressions.{Cast, CastBase, CastSuite, Expression, Literal} +import org.apache.spark.sql.catalyst.util.DateTimeTestUtils._ +import org.apache.spark.sql.catalyst.util.DateTimeUtils.getZoneId import org.apache.spark.sql.rapids.utils.RapidsTestsTrait import org.apache.spark.sql.types._ @@ -36,4 +41,48 @@ class RapidsCastSuite extends CastSuite with RapidsTestsTrait { Cast(lit, targetType, timeZoneId) } } + + private val specialTs = Seq( + "0001-01-01T00:00:00", // the fist timestamp of Common Era + "1582-10-15T23:59:59", // the cutover date from Julian to Gregorian calendar + "1970-01-01T00:00:00", // the epoch timestamp + "9999-12-31T23:59:59" // the last supported timestamp according to SQL standard + ) + + val outstandingTimezonesIds: Seq[String] = Seq( + "UTC", + PST.getId, + CET.getId, + "Africa/Dakar", + LA.getId, + "Asia/Urumqi", + "Asia/Hong_Kong", + "Europe/Brussels") + val outstandingZoneIds: Seq[ZoneId] = outstandingTimezonesIds.map(getZoneId) + + testRapids("SPARK-35711: cast timestamp without time zone to timestamp with local time zone") { + outstandingZoneIds.foreach { zoneId => + println(s"zoneId: $zoneId") + withDefaultTimeZone(zoneId) { + specialTs.foreach { s => + val input = LocalDateTime.parse(s) + val expectedTs = Timestamp.valueOf(s.replace("T", " ")) + checkEvaluation(cast(input, TimestampType), expectedTs) + } + } + } + } + + testRapids("SPARK-35719: cast timestamp with local time zone to timestamp without timezone") { + outstandingZoneIds.foreach { zoneId => + println(s"zoneId: $zoneId") + withDefaultTimeZone(zoneId) { + specialTs.foreach { s => + val input = Timestamp.valueOf(s.replace("T", " ")) + val expectedTs = LocalDateTime.parse(s) + checkEvaluation(cast(input, TimestampNTZType), expectedTs) + } + } + } + } } diff --git a/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/suites/RapidsJsonExpressionsSuite.scala b/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/suites/RapidsJsonExpressionsSuite.scala index eb5fdc535e8..1c390e2479d 100644 --- a/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/suites/RapidsJsonExpressionsSuite.scala +++ b/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/suites/RapidsJsonExpressionsSuite.scala @@ -19,8 +19,22 @@ spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.suites +import java.util.TimeZone + import org.apache.spark.sql.catalyst.expressions.JsonExpressionsSuite import org.apache.spark.sql.rapids.utils.{RapidsJsonConfTrait, RapidsTestsTrait} class RapidsJsonExpressionsSuite - extends JsonExpressionsSuite with RapidsTestsTrait with RapidsJsonConfTrait {} + extends JsonExpressionsSuite with RapidsTestsTrait with RapidsJsonConfTrait { + + override def beforeAll(): Unit = { + super.beforeAll() + // Set timezone to UTC to avoid fallback, so that tests run on GPU to detect bugs + TimeZone.setDefault(TimeZone.getTimeZone("UTC")) + } + + override def afterAll(): Unit = { + TimeZone.setDefault(originalTimeZone) + super.afterAll() + } +} diff --git a/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/suites/RapidsJsonFunctionsSuite.scala b/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/suites/RapidsJsonFunctionsSuite.scala index ebddc498202..1c39477134d 100644 --- a/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/suites/RapidsJsonFunctionsSuite.scala +++ b/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/suites/RapidsJsonFunctionsSuite.scala @@ -19,8 +19,24 @@ spark-rapids-shim-json-lines ***/ package org.apache.spark.sql.rapids.suites +import java.util.TimeZone + import org.apache.spark.sql.JsonFunctionsSuite import org.apache.spark.sql.rapids.utils.{RapidsJsonConfTrait, RapidsSQLTestsTrait} class RapidsJsonFunctionsSuite - extends JsonFunctionsSuite with RapidsSQLTestsTrait with RapidsJsonConfTrait {} + extends JsonFunctionsSuite with RapidsSQLTestsTrait with RapidsJsonConfTrait { + + val originalTimeZone = TimeZone.getDefault + + override def beforeAll(): Unit = { + super.beforeAll() + // Set timezone to UTC to avoid fallback, so that tests run on GPU to detect bugs + TimeZone.setDefault(TimeZone.getTimeZone("UTC")) + } + + override def afterAll(): Unit = { + TimeZone.setDefault(originalTimeZone) + super.afterAll() + } +} diff --git a/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/utils/RapidsSQLTestsBaseTrait.scala b/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/utils/RapidsSQLTestsBaseTrait.scala index f8b9d21d169..533f05be68b 100644 --- a/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/utils/RapidsSQLTestsBaseTrait.scala +++ b/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/utils/RapidsSQLTestsBaseTrait.scala @@ -151,10 +151,10 @@ object RapidsSQLTestsBaseTrait extends Logging { } def nativeSparkConf(origin: SparkConf, warehouse: String): SparkConf = { - // Timezone is fixed to UTC to allow timestamps to work by default - TimeZone.setDefault(TimeZone.getTimeZone("UTC")) // Add Locale setting Locale.setDefault(Locale.US) + // Spark use "America/Los_Angeles" as default timezone in tests + TimeZone.setDefault(TimeZone.getTimeZone("America/Los_Angeles")) val conf = origin .set("spark.rapids.sql.enabled", "true") @@ -163,8 +163,6 @@ object RapidsSQLTestsBaseTrait extends Logging { "org.apache.spark.sql.rapids.ExecutionPlanCaptureCallback") .set("spark.sql.warehouse.dir", warehouse) .set("spark.sql.cache.serializer", "com.nvidia.spark.ParquetCachedBatchSerializer") - // TODO: remove hard coded UTC https://github.com/NVIDIA/spark-rapids/issues/10874 - .set("spark.sql.session.timeZone", "UTC") .set("spark.rapids.sql.explain", "ALL") // uncomment below config to run `strict mode`, where fallback to CPU is treated as fail // .set("spark.rapids.sql.test.enabled", "true") diff --git a/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/utils/RapidsTestSettings.scala b/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/utils/RapidsTestSettings.scala index 63649376829..4f2e114f515 100644 --- a/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/utils/RapidsTestSettings.scala +++ b/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/utils/RapidsTestSettings.scala @@ -27,14 +27,13 @@ import org.apache.spark.sql.rapids.suites.{RapidsCastSuite, RapidsDataFrameAggre class RapidsTestSettings extends BackendTestSettings { enableSuite[RapidsCastSuite] - .exclude("Process Infinity, -Infinity, NaN in case insensitive manner", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10771")) - .exclude("SPARK-35711: cast timestamp without time zone to timestamp with local time zone", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10771")) - .exclude("SPARK-35719: cast timestamp with local time zone to timestamp without timezone", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10771")) - .exclude("SPARK-35112: Cast string to day-time interval", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10771")) - .exclude("SPARK-35735: Take into account day-time interval fields in cast", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10771")) - .exclude("casting to fixed-precision decimals", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10771")) - .exclude("SPARK-32828: cast from a derived user-defined type to a base type", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10771")) - .exclude("cast string to timestamp", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10771")) + .exclude("SPARK-35711: cast timestamp without time zone to timestamp with local time zone", WONT_FIX_ISSUE("https://issues.apache.org/jira/browse/SPARK-40851")) + .exclude("SPARK-35719: cast timestamp with local time zone to timestamp without timezone", WONT_FIX_ISSUE("https://issues.apache.org/jira/browse/SPARK-40851")) + .exclude("SPARK-35112: Cast string to day-time interval", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10980")) + .exclude("SPARK-35735: Take into account day-time interval fields in cast", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10980")) + .exclude("casting to fixed-precision decimals", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10809")) + .exclude("SPARK-32828: cast from a derived user-defined type to a base type", WONT_FIX_ISSUE("User-defined types are not supported")) + .exclude("cast string to timestamp", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/blob/main/docs/compatibility.md#string-to-timestamp")) .exclude("cast string to date", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10771")) enableSuite[RapidsDataFrameAggregateSuite] .exclude("collect functions", ADJUST_UT("order of elements in the array is non-deterministic in collect")) @@ -44,38 +43,32 @@ class RapidsTestSettings extends BackendTestSettings { .exclude("SPARK-19471: AggregationIterator does not initialize the generated result projection before using it", WONT_FIX_ISSUE("Codegen related UT, not applicable for GPU")) .exclude("SPARK-24788: RelationalGroupedDataset.toString with unresolved exprs should not fail", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10801")) enableSuite[RapidsJsonExpressionsSuite] - .exclude("from_json - invalid data", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10849")) - .exclude("from_json - input=empty array, schema=struct, output=single row with null", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10849")) - .exclude("from_json - input=empty object, schema=struct, output=single row with null", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10849")) - .exclude("SPARK-20549: from_json bad UTF-8", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10849")) - .exclude("to_json - array", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10849")) - .exclude("to_json - array with single empty row", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10849")) - .exclude("to_json - empty array", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10849")) - .exclude("SPARK-21513: to_json support map[string, struct] to json", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10849")) - .exclude("SPARK-21513: to_json support map[struct, struct] to json", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10849")) - .exclude("SPARK-21513: to_json support map[string, integer] to json", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10849")) - .exclude("to_json - array with maps", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10849")) - .exclude("to_json - array with single map", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10849")) - .exclude("from_json missing fields", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10849")) + .exclude("from_json - invalid data", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10891")) + .exclude("from_json - input=empty array, schema=struct, output=single row with null", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10907")) + .exclude("from_json - input=empty object, schema=struct, output=single row with null", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10910")) + .exclude("SPARK-20549: from_json bad UTF-8", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10911")) + .exclude("to_json - array", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10915")) + .exclude("to_json - array with single empty row", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10923")) + .exclude("to_json - empty array", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10924")) + .exclude("SPARK-21513: to_json support map[string, struct] to json", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10916")) + .exclude("SPARK-21513: to_json support map[struct, struct] to json", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10918")) + .exclude("SPARK-21513: to_json support map[string, integer] to json", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10919")) + .exclude("to_json - array with maps", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10920")) + .exclude("to_json - array with single map", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10921")) + .exclude("from_json missing fields", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10922")) enableSuite[RapidsJsonFunctionsSuite] - .exclude("from_json invalid json", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10852")) - .exclude("to_json - array", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10852")) - .exclude("to_json - map", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10852")) - .exclude("to_json - array of primitive types", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10852")) - .exclude("SPARK-33134: return partial results only for root JSON objects", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10852")) + .exclude("from_json invalid json", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10891")) + .exclude("to_json - array", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10894")) + .exclude("to_json - map", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10895")) + .exclude("to_json - array of primitive types", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10896")) + .exclude("SPARK-33134: return partial results only for root JSON objects", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10901")) enableSuite[RapidsJsonSuite] - .exclude("Casting long as timestamp", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10773")) - .exclude("Write timestamps correctly with timestampFormat option and timeZone option", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10773")) - .exclude("exception mode for parsing date/timestamp string", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10773")) enableSuite[RapidsMathFunctionsSuite] enableSuite[RapidsRegexpExpressionsSuite] enableSuite[RapidsStringExpressionsSuite] - .exclude("string substring_index function", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10775")) - .exclude("SPARK-22498: Concat should not generate codes beyond 64KB", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10775")) - .exclude("SPARK-22549: ConcatWs should not generate codes beyond 64KB", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10775")) - .exclude("SPARK-22550: Elt should not generate codes beyond 64KB", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10775")) - .exclude("SPARK-22603: FormatString should not generate codes beyond 64KB", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10775")) - .exclude("ParseUrl", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10775")) + .exclude("string substring_index function", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/8750")) + .exclude("SPARK-22550: Elt should not generate codes beyond 64KB", WONT_FIX_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10775")) + .exclude("SPARK-22603: FormatString should not generate codes beyond 64KB", WONT_FIX_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10775")) enableSuite[RapidsStringFunctionsSuite] } // scalastyle:on line.size.limit diff --git a/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/utils/RapidsTestsTrait.scala b/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/utils/RapidsTestsTrait.scala index 69bd4532c71..d3aa8516679 100644 --- a/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/utils/RapidsTestsTrait.scala +++ b/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/utils/RapidsTestsTrait.scala @@ -56,7 +56,8 @@ trait RapidsTestsTrait extends RapidsTestsCommonTrait { super.beforeAll() initializeSession() _spark.sparkContext.setLogLevel("WARN") - TimeZone.setDefault(TimeZone.getTimeZone("UTC")) + // Spark use "America/Los_Angeles" as default timezone in tests + TimeZone.setDefault(TimeZone.getTimeZone("America/Los_Angeles")) } override def afterAll(): Unit = { @@ -102,8 +103,6 @@ trait RapidsTestsTrait extends RapidsTestsCommonTrait { .config("spark.sql.queryExecutionListeners", "org.apache.spark.sql.rapids.ExecutionPlanCaptureCallback") .config("spark.sql.warehouse.dir", warehouse) - // TODO: remove hard coded UTC https://github.com/NVIDIA/spark-rapids/issues/10874 - .config("spark.sql.session.timeZone","UTC") .config("spark.rapids.sql.explain", "ALL") .config("spark.rapids.sql.test.isFoldableNonLitAllowed", "true") // uncomment below config to run `strict mode`, where fallback to CPU is treated as fail From 9dafc54c7c7287906d6cdc6ac4ef102c9d9a7f7b Mon Sep 17 00:00:00 2001 From: Haoyang Li Date: Thu, 27 Jun 2024 13:33:12 +0800 Subject: [PATCH 60/68] exclude a case based on JDK version (#11083) Signed-off-by: Haoyang Li --- .../rapids/utils/BackendTestSettings.scala | 40 +++++++++++++++++-- .../sql/rapids/utils/RapidsTestSettings.scala | 2 +- 2 files changed, 38 insertions(+), 4 deletions(-) diff --git a/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/utils/BackendTestSettings.scala b/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/utils/BackendTestSettings.scala index e1aec1ffebc..a57b7802c9d 100644 --- a/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/utils/BackendTestSettings.scala +++ b/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/utils/BackendTestSettings.scala @@ -86,6 +86,19 @@ abstract class BackendTestSettings { case class ADJUST_UT(reason: String) extends ExcludeReason case class WONT_FIX_ISSUE(reason: String) extends ExcludeReason + protected def getJavaMajorVersion(): Int = { + val version = System.getProperty("java.version") + // Allow these formats: + // 1.8.0_72-ea + // 9-ea + // 9 + // 11.0.1 + val versionRegex = """(1\.)?(\d+)([._].+)?""".r + version match { + case versionRegex(_, major, _) => major.toInt + case _ => throw new IllegalStateException(s"Cannot parse java version: $version") + } + } final protected class SuiteSettings { private[utils] val inclusion: util.List[IncludeBase] = new util.ArrayList() @@ -96,42 +109,54 @@ abstract class BackendTestSettings { inclusion.add(Include(testNames: _*)) this } - def exclude(testNames: String, reason: ExcludeReason): SuiteSettings = { - exclusion.add(Exclude(testNames)) - excludeReasons.add(reason) + + def exclude(testNames: String, reason: ExcludeReason, condition: Boolean = true): + SuiteSettings = { + if (condition) { + exclusion.add(Exclude(testNames)) + excludeReasons.add(reason) + } this } + def includeRapidsTest(testName: String*): SuiteSettings = { inclusion.add(IncludeRapidsTest(testName: _*)) this } + def excludeRapidsTest(testName: String, reason: ExcludeReason): SuiteSettings = { exclusion.add(ExcludeRapidsTest(testName)) excludeReasons.add(reason) this } + def includeByPrefix(prefixes: String*): SuiteSettings = { inclusion.add(IncludeByPrefix(prefixes: _*)) this } + def excludeByPrefix(prefixes: String, reason: ExcludeReason): SuiteSettings = { exclusion.add(ExcludeByPrefix(prefixes)) excludeReasons.add(reason) this } + def includeRapidsTestsByPrefix(prefixes: String*): SuiteSettings = { inclusion.add(IncludeRapidsTestByPrefix(prefixes: _*)) this } + def excludeRapidsTestsByPrefix(prefixes: String, reason: ExcludeReason): SuiteSettings = { exclusion.add(ExcludeRadpisTestByPrefix(prefixes)) excludeReasons.add(reason) this } + def includeAllRapidsTests(): SuiteSettings = { inclusion.add(IncludeByPrefix(RAPIDS_TEST)) this } + def excludeAllRapidsTests(reason: ExcludeReason): SuiteSettings = { exclusion.add(ExcludeByPrefix(RAPIDS_TEST)) excludeReasons.add(reason) @@ -142,25 +167,31 @@ abstract class BackendTestSettings { protected trait IncludeBase { def isIncluded(testName: String): Boolean } + protected trait ExcludeBase { def isExcluded(testName: String): Boolean } + private case class Include(testNames: String*) extends IncludeBase { val nameSet: Set[String] = Set(testNames: _*) override def isIncluded(testName: String): Boolean = nameSet.contains(testName) } + private case class Exclude(testNames: String*) extends ExcludeBase { val nameSet: Set[String] = Set(testNames: _*) override def isExcluded(testName: String): Boolean = nameSet.contains(testName) } + private case class IncludeRapidsTest(testNames: String*) extends IncludeBase { val nameSet: Set[String] = testNames.map(name => RAPIDS_TEST + name).toSet override def isIncluded(testName: String): Boolean = nameSet.contains(testName) } + private case class ExcludeRapidsTest(testNames: String*) extends ExcludeBase { val nameSet: Set[String] = testNames.map(name => RAPIDS_TEST + name).toSet override def isExcluded(testName: String): Boolean = nameSet.contains(testName) } + private case class IncludeByPrefix(prefixes: String*) extends IncludeBase { override def isIncluded(testName: String): Boolean = { if (prefixes.exists(prefix => testName.startsWith(prefix))) { @@ -169,6 +200,7 @@ abstract class BackendTestSettings { false } } + private case class ExcludeByPrefix(prefixes: String*) extends ExcludeBase { override def isExcluded(testName: String): Boolean = { if (prefixes.exists(prefix => testName.startsWith(prefix))) { @@ -177,6 +209,7 @@ abstract class BackendTestSettings { false } } + private case class IncludeRapidsTestByPrefix(prefixes: String*) extends IncludeBase { override def isIncluded(testName: String): Boolean = { if (prefixes.exists(prefix => testName.startsWith(RAPIDS_TEST + prefix))) { @@ -185,6 +218,7 @@ abstract class BackendTestSettings { false } } + private case class ExcludeRadpisTestByPrefix(prefixes: String*) extends ExcludeBase { override def isExcluded(testName: String): Boolean = { if (prefixes.exists(prefix => testName.startsWith(RAPIDS_TEST + prefix))) { diff --git a/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/utils/RapidsTestSettings.scala b/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/utils/RapidsTestSettings.scala index 4f2e114f515..ba8dcbe6efe 100644 --- a/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/utils/RapidsTestSettings.scala +++ b/tests/src/test/spark330/scala/org/apache/spark/sql/rapids/utils/RapidsTestSettings.scala @@ -41,7 +41,7 @@ class RapidsTestSettings extends BackendTestSettings { .exclude("collect functions should be able to cast to array type with no null values", ADJUST_UT("order of elements in the array is non-deterministic in collect")) .exclude("SPARK-17641: collect functions should not collect null values", ADJUST_UT("order of elements in the array is non-deterministic in collect")) .exclude("SPARK-19471: AggregationIterator does not initialize the generated result projection before using it", WONT_FIX_ISSUE("Codegen related UT, not applicable for GPU")) - .exclude("SPARK-24788: RelationalGroupedDataset.toString with unresolved exprs should not fail", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10801")) + .exclude("SPARK-24788: RelationalGroupedDataset.toString with unresolved exprs should not fail", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10801"), (getJavaMajorVersion() >= 17)) enableSuite[RapidsJsonExpressionsSuite] .exclude("from_json - invalid data", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10891")) .exclude("from_json - input=empty array, schema=struct, output=single row with null", KNOWN_ISSUE("https://github.com/NVIDIA/spark-rapids/issues/10907")) From 3b6c5cd44cb906374e4e56437b36129a7a78e9a0 Mon Sep 17 00:00:00 2001 From: Raza Jafri Date: Thu, 27 Jun 2024 23:15:24 -0700 Subject: [PATCH 61/68] Replaced spark3xx-common references to spark-shared [databricks] (#11066) * Replaced spark3xx-common references to spark-shared * Signing off Signed-off-by: Raza Jafri * addressed review comments * addressed review comments * removed todo as per review comment * Moving dependency to the related module because it was causing an error while running code coverage * Addressed review comments * Regenerated 2.13 poms --------- Signed-off-by: Raza Jafri --- build/coverage-report | 6 ++-- delta-lake/delta-20x/pom.xml | 4 +++ delta-lake/delta-21x/pom.xml | 4 +++ delta-lake/delta-22x/pom.xml | 4 +++ delta-lake/delta-23x/pom.xml | 4 +++ delta-lake/delta-24x/pom.xml | 4 +++ delta-lake/delta-spark330db/pom.xml | 4 +++ delta-lake/delta-spark332db/pom.xml | 4 +++ delta-lake/delta-spark341db/pom.xml | 4 +++ dist/scripts/binary-dedupe.sh | 5 ---- docs/dev/shims.md | 30 +++++++++---------- jenkins/spark-premerge-build.sh | 6 ++-- scala2.13/delta-lake/delta-20x/pom.xml | 4 +++ scala2.13/delta-lake/delta-21x/pom.xml | 4 +++ scala2.13/delta-lake/delta-22x/pom.xml | 4 +++ scala2.13/delta-lake/delta-23x/pom.xml | 4 +++ scala2.13/delta-lake/delta-24x/pom.xml | 4 +++ scala2.13/delta-lake/delta-spark330db/pom.xml | 4 +++ scala2.13/delta-lake/delta-spark332db/pom.xml | 4 +++ scala2.13/delta-lake/delta-spark341db/pom.xml | 4 +++ scala2.13/sql-plugin/pom.xml | 4 --- sql-plugin/pom.xml | 4 --- 22 files changed, 85 insertions(+), 34 deletions(-) diff --git a/build/coverage-report b/build/coverage-report index 69c96c050dc..75c86e55258 100755 --- a/build/coverage-report +++ b/build/coverage-report @@ -1,7 +1,7 @@ #!/bin/bash # -# Copyright (c) 2020-2022, NVIDIA CORPORATION. All rights reserved. +# Copyright (c) 2020-2024, NVIDIA CORPORATION. All rights reserved. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -33,11 +33,11 @@ SOURCE_WITH_ARGS="--sourcefiles "$(echo $SOURCE_DIRS | sed -e 's/:/ --sourcefile rm -rf "$TMP_CLASS" mkdir -p "$TMP_CLASS" pushd "$TMP_CLASS" -jar xf "$DIST_JAR" com org rapids spark3xx-common "spark${SPK_VER}/" +jar xf "$DIST_JAR" com org rapids spark-shared "spark${SPK_VER}/" # extract the .class files in udf jar and replace the existing ones in spark3xx-ommon and spark$SPK_VER # because the class files in udf jar will be modified in aggregator's shade phase jar xf "$UDF_JAR" com/nvidia/spark/udf -rm -rf com/nvidia/shaded/ org/openucx/ spark3xx-common/com/nvidia/spark/udf/ spark${SPK_VER}/com/nvidia/spark/udf/ +rm -rf com/nvidia/shaded/ org/openucx/ spark-shared/com/nvidia/spark/udf/ spark${SPK_VER}/com/nvidia/spark/udf/ popd if [ ! -f "$JDEST" ]; then diff --git a/delta-lake/delta-20x/pom.xml b/delta-lake/delta-20x/pom.xml index dfc34fad987..be646b807f5 100644 --- a/delta-lake/delta-20x/pom.xml +++ b/delta-lake/delta-20x/pom.xml @@ -39,6 +39,10 @@ + + org.roaringbitmap + RoaringBitmap + com.nvidia rapids-4-spark-sql_${scala.binary.version} diff --git a/delta-lake/delta-21x/pom.xml b/delta-lake/delta-21x/pom.xml index 8770a7e3d9d..f2915278625 100644 --- a/delta-lake/delta-21x/pom.xml +++ b/delta-lake/delta-21x/pom.xml @@ -39,6 +39,10 @@ + + org.roaringbitmap + RoaringBitmap + com.nvidia rapids-4-spark-sql_${scala.binary.version} diff --git a/delta-lake/delta-22x/pom.xml b/delta-lake/delta-22x/pom.xml index be9e122e5df..0be1371f4a2 100644 --- a/delta-lake/delta-22x/pom.xml +++ b/delta-lake/delta-22x/pom.xml @@ -39,6 +39,10 @@ + + org.roaringbitmap + RoaringBitmap + com.nvidia rapids-4-spark-sql_${scala.binary.version} diff --git a/delta-lake/delta-23x/pom.xml b/delta-lake/delta-23x/pom.xml index 97207dca741..b4bd721dd55 100644 --- a/delta-lake/delta-23x/pom.xml +++ b/delta-lake/delta-23x/pom.xml @@ -39,6 +39,10 @@ + + org.roaringbitmap + RoaringBitmap + com.nvidia rapids-4-spark-sql_${scala.binary.version} diff --git a/delta-lake/delta-24x/pom.xml b/delta-lake/delta-24x/pom.xml index 19e0be3c90d..5573709b446 100644 --- a/delta-lake/delta-24x/pom.xml +++ b/delta-lake/delta-24x/pom.xml @@ -39,6 +39,10 @@ + + org.roaringbitmap + RoaringBitmap + com.nvidia rapids-4-spark-sql_${scala.binary.version} diff --git a/delta-lake/delta-spark330db/pom.xml b/delta-lake/delta-spark330db/pom.xml index abc57f793f8..3ee901eef2c 100644 --- a/delta-lake/delta-spark330db/pom.xml +++ b/delta-lake/delta-spark330db/pom.xml @@ -39,6 +39,10 @@ + + org.roaringbitmap + RoaringBitmap + com.nvidia rapids-4-spark-sql_${scala.binary.version} diff --git a/delta-lake/delta-spark332db/pom.xml b/delta-lake/delta-spark332db/pom.xml index 4511a76be71..92964a59e17 100644 --- a/delta-lake/delta-spark332db/pom.xml +++ b/delta-lake/delta-spark332db/pom.xml @@ -39,6 +39,10 @@ + + org.roaringbitmap + RoaringBitmap + com.nvidia rapids-4-spark-sql_${scala.binary.version} diff --git a/delta-lake/delta-spark341db/pom.xml b/delta-lake/delta-spark341db/pom.xml index 6532062af68..6eaa3fb8c32 100644 --- a/delta-lake/delta-spark341db/pom.xml +++ b/delta-lake/delta-spark341db/pom.xml @@ -38,6 +38,10 @@ + + org.roaringbitmap + RoaringBitmap + com.nvidia rapids-4-spark-sql_${scala.binary.version} diff --git a/dist/scripts/binary-dedupe.sh b/dist/scripts/binary-dedupe.sh index 356b0b4dbae..970840f59e3 100755 --- a/dist/scripts/binary-dedupe.sh +++ b/dist/scripts/binary-dedupe.sh @@ -127,11 +127,6 @@ done mv "$SPARK_SHARED_DIR" parallel-world/ -# TODO further dedupe by FEATURE version lines: -# spark30x-common -# spark31x-common -# spark32x-common - # Verify that all class files in the conventional jar location are bitwise # identical regardless of the Spark-version-specific jar. # diff --git a/docs/dev/shims.md b/docs/dev/shims.md index b6290287314..218adbc99d4 100644 --- a/docs/dev/shims.md +++ b/docs/dev/shims.md @@ -69,7 +69,7 @@ Spark 3.0.2's URLs: ```text jar:file:/home/spark/rapids-4-spark_2.12-24.08.0.jar!/ -jar:file:/home/spark/rapids-4-spark_2.12-24.08.0.jar!/spark3xx-common/ +jar:file:/home/spark/rapids-4-spark_2.12-24.08.0.jar!/spark-shared/ jar:file:/home/spark/rapids-4-spark_2.12-24.08.0.jar!/spark302/ ``` @@ -77,7 +77,7 @@ Spark 3.2.0's URLs : ```text jar:file:/home/spark/rapids-4-spark_2.12-24.08.0.jar!/ -jar:file:/home/spark/rapids-4-spark_2.12-24.08.0.jar!/spark3xx-common/ +jar:file:/home/spark/rapids-4-spark_2.12-24.08.0.jar!/spark-shared/ jar:file:/home/spark/rapids-4-spark_2.12-24.08.0.jar!/spark320/ ``` @@ -143,7 +143,7 @@ This has two pre-requisites: 1. The .class file with the bytecode is bitwise-identical among the currently supported Spark versions. To verify this you can inspect the dist jar and check -if the class file is under `spark3xx-common` jar entry. If this is not the case then +if the class file is under `spark-shared` jar entry. If this is not the case then code should be refactored until all discrepancies are shimmed away. 1. The transitive closure of the classes compile-time-referenced by `A` should have the property above. @@ -181,28 +181,28 @@ mv org com ai public/ and you will see the dependencies of `public` classes. By design `public` classes should have only edges only to other `public` classes in the dist jar. -Execute `jdeps` against `public`, `spark3xx-common` and an *exactly one* parallel +Execute `jdeps` against `public`, `spark-shared` and an *exactly one* parallel world such as `spark330` ```bash ${JAVA_HOME}/bin/jdeps -v \ -dotoutput /tmp/jdeps330 \ -regex '(com|org)\..*\.rapids\..*' \ - public spark3xx-common spark330 + public spark-shared spark330 ``` This will produce three DOT files for each "archive" with directed edges for a class in the archive to a class either in this or another archive. -Looking at an output file, e.g. `/tmp/jdeps330/spark3xx-common.dot`, +Looking at an output file, e.g. `/tmp/jdeps330/spark-shared.dot`, unfortunately you see that jdeps does not label the source class node but labels the target class node of an edge. Thus the graph is incorrect as it breaks paths if a node has both incoming and outgoing edges. ```bash -$ grep 'com.nvidia.spark.rapids.GpuFilterExec\$' spark3xx-common.dot +$ grep 'com.nvidia.spark.rapids.GpuFilterExec\$' spark-shared.dot "com.nvidia.spark.rapids.GpuFilterExec$" -> "com.nvidia.spark.rapids.GpuFilterExec (spark330)"; - "com.nvidia.spark.rapids.GpuOverrides$$anon$204" -> "com.nvidia.spark.rapids.GpuFilterExec$ (spark3xx-common)"; + "com.nvidia.spark.rapids.GpuOverrides$$anon$204" -> "com.nvidia.spark.rapids.GpuFilterExec$ (spark-shared)"; ``` So first create and `cd` to some other directory `/tmp/jdep330.processed` to massage @@ -214,8 +214,8 @@ that the source nodes are guaranteed to be from the ``. ```bash sed 's/"\([^(]*\)"\(\s*->.*;\)/"\1 (public)"\2/' \ /tmp/jdeps330/public.dot > public.dot -sed 's/"\([^(]*\)"\(\s*->.*;\)/"\1 (spark3xx-common)"\2/' \ - /tmp/jdeps330/spark3xx-common.dot > spark3xx-common.dot +sed 's/"\([^(]*\)"\(\s*->.*;\)/"\1 (spark-shared)"\2/' \ + /tmp/jdeps330/spark-shared.dot > spark-shared.dot sed 's/"\([^(]*\)"\(\s*->.*;\)/"\1 (spark330)"\2/' \ /tmp/jdeps330/spark330.dot > spark330.dot ``` @@ -224,7 +224,7 @@ Next you need to union edges of all three graphs into a single graph to be able to analyze cross-archive paths. ```bash -cat public.dot spark3xx-common.dot spark330.dot | \ +cat public.dot spark-shared.dot spark330.dot | \ tr '\n' '\r' | \ sed 's/}\rdigraph "[^"]*" {\r//g' | \ tr '\r' '\n' > merged.dot @@ -245,7 +245,7 @@ GpuTypeColumnVector needs refactoring prior externalization as of the time of this writing: ```bash -$ dijkstra -d -p "com.nvidia.spark.rapids.GpuColumnVector (spark3xx-common)" merged.dot | \ +$ dijkstra -d -p "com.nvidia.spark.rapids.GpuColumnVector (spark-shared)" merged.dot | \ grep '\[dist=' | grep '(spark330)' "org.apache.spark.sql.rapids.GpuFileSourceScanExec (spark330)" [dist=5.000, "com.nvidia.spark.rapids.GpuExec (spark330)" [dist=3.000, @@ -255,9 +255,9 @@ $ dijkstra -d -p "com.nvidia.spark.rapids.GpuColumnVector (spark3xx-common)" mer RegexReplace could be externalized safely: ```bash -$ dijkstra -d -p "org.apache.spark.sql.rapids.RegexReplace (spark3xx-common)" merged.dot | grep '\[dist=' - "org.apache.spark.sql.rapids.RegexReplace (spark3xx-common)" [dist=0.000]; - "org.apache.spark.sql.rapids.RegexReplace$ (spark3xx-common)" [dist=1.000, +$ dijkstra -d -p "org.apache.spark.sql.rapids.RegexReplace (spark-shared)" merged.dot | grep '\[dist=' + "org.apache.spark.sql.rapids.RegexReplace (spark-shared)" [dist=0.000]; + "org.apache.spark.sql.rapids.RegexReplace$ (spark-shared)" [dist=1.000, ``` because it is self-contained. diff --git a/jenkins/spark-premerge-build.sh b/jenkins/spark-premerge-build.sh index 697722c0138..ec4b74cc902 100755 --- a/jenkins/spark-premerge-build.sh +++ b/jenkins/spark-premerge-build.sh @@ -1,6 +1,6 @@ #!/bin/bash # -# Copyright (c) 2020-2023, NVIDIA CORPORATION. All rights reserved. +# Copyright (c) 2020-2024, NVIDIA CORPORATION. All rights reserved. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -88,11 +88,11 @@ mvn_verify() { FILE=$(ls dist/target/rapids-4-spark_2.12-*.jar | grep -v test | xargs readlink -f) UDF_JAR=$(ls ./udf-compiler/target/spark${SPK_VER}/rapids-4-spark-udf_2.12-*-spark${SPK_VER}.jar | grep -v test | xargs readlink -f) pushd target/jacoco_classes/ - jar xf $FILE com org rapids spark3xx-common "spark${JACOCO_SPARK_VER:-311}/" + jar xf $FILE com org rapids spark-shared "spark${JACOCO_SPARK_VER:-311}/" # extract the .class files in udf jar and replace the existing ones in spark3xx-ommon and spark$SPK_VER # because the class files in udf jar will be modified in aggregator's shade phase jar xf "$UDF_JAR" com/nvidia/spark/udf - rm -rf com/nvidia/shaded/ org/openucx/ spark3xx-common/com/nvidia/spark/udf/ spark${SPK_VER}/com/nvidia/spark/udf/ + rm -rf com/nvidia/shaded/ org/openucx/ spark-shared/com/nvidia/spark/udf/ spark${SPK_VER}/com/nvidia/spark/udf/ popd # Triggering here until we change the jenkins file diff --git a/scala2.13/delta-lake/delta-20x/pom.xml b/scala2.13/delta-lake/delta-20x/pom.xml index 6b098f9bdb1..40547c3f510 100644 --- a/scala2.13/delta-lake/delta-20x/pom.xml +++ b/scala2.13/delta-lake/delta-20x/pom.xml @@ -39,6 +39,10 @@ + + org.roaringbitmap + RoaringBitmap + com.nvidia rapids-4-spark-sql_${scala.binary.version} diff --git a/scala2.13/delta-lake/delta-21x/pom.xml b/scala2.13/delta-lake/delta-21x/pom.xml index 6f58d97b9c9..0f228709016 100644 --- a/scala2.13/delta-lake/delta-21x/pom.xml +++ b/scala2.13/delta-lake/delta-21x/pom.xml @@ -39,6 +39,10 @@ + + org.roaringbitmap + RoaringBitmap + com.nvidia rapids-4-spark-sql_${scala.binary.version} diff --git a/scala2.13/delta-lake/delta-22x/pom.xml b/scala2.13/delta-lake/delta-22x/pom.xml index 500405c9f1a..c315b05ad36 100644 --- a/scala2.13/delta-lake/delta-22x/pom.xml +++ b/scala2.13/delta-lake/delta-22x/pom.xml @@ -39,6 +39,10 @@ + + org.roaringbitmap + RoaringBitmap + com.nvidia rapids-4-spark-sql_${scala.binary.version} diff --git a/scala2.13/delta-lake/delta-23x/pom.xml b/scala2.13/delta-lake/delta-23x/pom.xml index c6e9cf634a6..7ac7c1156dd 100644 --- a/scala2.13/delta-lake/delta-23x/pom.xml +++ b/scala2.13/delta-lake/delta-23x/pom.xml @@ -39,6 +39,10 @@ + + org.roaringbitmap + RoaringBitmap + com.nvidia rapids-4-spark-sql_${scala.binary.version} diff --git a/scala2.13/delta-lake/delta-24x/pom.xml b/scala2.13/delta-lake/delta-24x/pom.xml index 987d24b0172..2d1a1b43e82 100644 --- a/scala2.13/delta-lake/delta-24x/pom.xml +++ b/scala2.13/delta-lake/delta-24x/pom.xml @@ -39,6 +39,10 @@ + + org.roaringbitmap + RoaringBitmap + com.nvidia rapids-4-spark-sql_${scala.binary.version} diff --git a/scala2.13/delta-lake/delta-spark330db/pom.xml b/scala2.13/delta-lake/delta-spark330db/pom.xml index b5f5a9e475b..46d6f8798d9 100644 --- a/scala2.13/delta-lake/delta-spark330db/pom.xml +++ b/scala2.13/delta-lake/delta-spark330db/pom.xml @@ -39,6 +39,10 @@ + + org.roaringbitmap + RoaringBitmap + com.nvidia rapids-4-spark-sql_${scala.binary.version} diff --git a/scala2.13/delta-lake/delta-spark332db/pom.xml b/scala2.13/delta-lake/delta-spark332db/pom.xml index 2ea75281f26..93a942e9d44 100644 --- a/scala2.13/delta-lake/delta-spark332db/pom.xml +++ b/scala2.13/delta-lake/delta-spark332db/pom.xml @@ -39,6 +39,10 @@ + + org.roaringbitmap + RoaringBitmap + com.nvidia rapids-4-spark-sql_${scala.binary.version} diff --git a/scala2.13/delta-lake/delta-spark341db/pom.xml b/scala2.13/delta-lake/delta-spark341db/pom.xml index 30c513c6f3d..6b5e545291b 100644 --- a/scala2.13/delta-lake/delta-spark341db/pom.xml +++ b/scala2.13/delta-lake/delta-spark341db/pom.xml @@ -38,6 +38,10 @@ + + org.roaringbitmap + RoaringBitmap + com.nvidia rapids-4-spark-sql_${scala.binary.version} diff --git a/scala2.13/sql-plugin/pom.xml b/scala2.13/sql-plugin/pom.xml index eb6f240a3f6..df3532a3592 100644 --- a/scala2.13/sql-plugin/pom.xml +++ b/scala2.13/sql-plugin/pom.xml @@ -97,10 +97,6 @@ org.alluxio alluxio-shaded-client - - org.roaringbitmap - RoaringBitmap - org.mockito mockito-core diff --git a/sql-plugin/pom.xml b/sql-plugin/pom.xml index 08657a9d40b..961e6f08372 100644 --- a/sql-plugin/pom.xml +++ b/sql-plugin/pom.xml @@ -97,10 +97,6 @@ org.alluxio alluxio-shaded-client - - org.roaringbitmap - RoaringBitmap - org.mockito mockito-core From 7dc52bcefea19ebfa5a3c7985205a4e5e7fc9122 Mon Sep 17 00:00:00 2001 From: Raza Jafri Date: Fri, 28 Jun 2024 09:07:31 -0700 Subject: [PATCH 62/68] Fixed some cast_tests (#11049) Signed-off-by: Raza Jafri --- integration_tests/src/main/python/cast_test.py | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/integration_tests/src/main/python/cast_test.py b/integration_tests/src/main/python/cast_test.py index 2621124b4b8..a56afe29ab9 100644 --- a/integration_tests/src/main/python/cast_test.py +++ b/integration_tests/src/main/python/cast_test.py @@ -1,4 +1,4 @@ -# Copyright (c) 2021-2023, NVIDIA CORPORATION. +# Copyright (c) 2021-2024, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -427,7 +427,10 @@ def fun(spark): data = [invalid_value] df = spark.createDataFrame(data, type) return df.select(f.col('value').cast(TimestampType())).collect() - assert_gpu_and_cpu_error(fun, {"spark.sql.ansi.enabled": True}, "SparkDateTimeException") + + assert_gpu_and_cpu_error(fun, {"spark.sql.ansi.enabled": True}, + error_message="SparkDateTimeException" + if is_before_spark_400() else "DateTimeException") # if float.floor > Long.max or float.ceil < Long.min, throw exception @pytest.mark.skipif(is_before_spark_330(), reason="ansi cast throws exception only in 3.3.0+") @@ -583,7 +586,7 @@ def fun(spark): data=[invalid_string] df = spark.createDataFrame(data, StringType()) return df.select(f.col('value').cast(dtType)).collect() - assert_gpu_and_cpu_error(fun, {}, "java.lang.IllegalArgumentException") + assert_gpu_and_cpu_error(fun, {}, "IllegalArgumentException") @pytest.mark.skipif(is_before_spark_330(), reason='casting between interval and integral is not supported before Pyspark 3.3.0') def test_cast_day_time_interval_to_integral_no_overflow(): From dd62000d91d7563b20a1af8b1e015a81bd056061 Mon Sep 17 00:00:00 2001 From: Raza Jafri Date: Fri, 28 Jun 2024 09:31:36 -0700 Subject: [PATCH 63/68] Fixed array_tests for Spark 4.0.0 [databricks] (#11048) * Fixed array_tests * Signing off Signed-off-by: Raza Jafri * Disable ANSI for failing tests --------- Signed-off-by: Raza Jafri --- .../src/main/python/array_test.py | 20 ++++++++++++++----- 1 file changed, 15 insertions(+), 5 deletions(-) diff --git a/integration_tests/src/main/python/array_test.py b/integration_tests/src/main/python/array_test.py index 0b98bd23439..f3d84aff372 100644 --- a/integration_tests/src/main/python/array_test.py +++ b/integration_tests/src/main/python/array_test.py @@ -17,8 +17,8 @@ from asserts import assert_gpu_and_cpu_are_equal_collect, assert_gpu_and_cpu_are_equal_sql, assert_gpu_and_cpu_error, assert_gpu_fallback_collect from data_gen import * from conftest import is_databricks_runtime -from marks import incompat, allow_non_gpu -from spark_session import is_before_spark_313, is_before_spark_330, is_databricks113_or_later, is_spark_330_or_later, is_databricks104_or_later, is_spark_33X, is_spark_340_or_later, is_spark_330, is_spark_330cdh +from marks import incompat, allow_non_gpu, disable_ansi_mode +from spark_session import * from pyspark.sql.types import * from pyspark.sql.types import IntegralType from pyspark.sql.functions import array_contains, col, element_at, lit, array @@ -103,11 +103,13 @@ @pytest.mark.parametrize('data_gen', array_item_test_gens, ids=idfn) @pytest.mark.parametrize('index_gen', array_index_gens, ids=idfn) +@disable_ansi_mode def test_array_item(data_gen, index_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark: two_col_df(spark, data_gen, index_gen).selectExpr('a[b]')) @pytest.mark.parametrize('data_gen', array_item_test_gens, ids=idfn) +@disable_ansi_mode def test_array_item_lit_ordinal(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark: unary_op_df(spark, data_gen).selectExpr( @@ -145,8 +147,10 @@ def test_array_item_with_strict_index(strict_index_enabled, index): # No need to test this for multiple data types for array. Only one is enough, but with two kinds of invalid index. @pytest.mark.parametrize('index', [-2, 100, array_neg_index_gen, array_out_index_gen], ids=idfn) +@disable_ansi_mode def test_array_item_ansi_fail_invalid_index(index): - message = "SparkArrayIndexOutOfBoundsException" if (is_databricks104_or_later() or is_spark_330_or_later()) else "java.lang.ArrayIndexOutOfBoundsException" + message = "SparkArrayIndexOutOfBoundsException" if (is_databricks104_or_later() or is_spark_330_or_later() and is_before_spark_400()) else \ + "ArrayIndexOutOfBoundsException" if isinstance(index, int): test_func = lambda spark: unary_op_df(spark, ArrayGen(int_gen)).select(col('a')[index]).collect() else: @@ -171,6 +175,7 @@ def test_array_item_ansi_not_fail_all_null_data(): decimal_gen_32bit, decimal_gen_64bit, decimal_gen_128bit, binary_gen, StructGen([['child0', StructGen([['child01', IntegerGen()]])], ['child1', string_gen], ['child2', float_gen]], nullable=False), StructGen([['child0', byte_gen], ['child1', string_gen], ['child2', float_gen]], nullable=False)], ids=idfn) +@disable_ansi_mode def test_make_array(data_gen): (s1, s2) = with_cpu_session( lambda spark: gen_scalars_for_sql(data_gen, 2, force_no_nulls=not isinstance(data_gen, NullGen))) @@ -212,6 +217,7 @@ def test_orderby_array_of_structs(data_gen): @pytest.mark.parametrize('data_gen', [byte_gen, short_gen, int_gen, long_gen, float_gen, double_gen, string_gen, boolean_gen, date_gen, timestamp_gen], ids=idfn) +@disable_ansi_mode def test_array_contains(data_gen): arr_gen = ArrayGen(data_gen) literal = with_cpu_session(lambda spark: gen_scalar(data_gen, force_no_nulls=True)) @@ -239,6 +245,7 @@ def test_array_contains_for_nans(data_gen): @pytest.mark.parametrize('data_gen', array_item_test_gens, ids=idfn) +@disable_ansi_mode def test_array_element_at(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark: two_col_df(spark, data_gen, array_no_zero_index_gen).selectExpr( @@ -252,8 +259,9 @@ def test_array_element_at(data_gen): # No need tests for multiple data types for list data. Only one is enough. @pytest.mark.parametrize('index', [100, array_out_index_gen], ids=idfn) +@disable_ansi_mode def test_array_element_at_ansi_fail_invalid_index(index): - message = "ArrayIndexOutOfBoundsException" if is_before_spark_330() else "SparkArrayIndexOutOfBoundsException" + message = "ArrayIndexOutOfBoundsException" if is_before_spark_330() or not is_before_spark_400() else "SparkArrayIndexOutOfBoundsException" if isinstance(index, int): test_func = lambda spark: unary_op_df(spark, ArrayGen(int_gen)).select( element_at(col('a'), index)).collect() @@ -282,9 +290,10 @@ def test_array_element_at_ansi_not_fail_all_null_data(): @pytest.mark.parametrize('index', [0, array_zero_index_gen], ids=idfn) @pytest.mark.parametrize('ansi_enabled', [False, True], ids=idfn) +@disable_ansi_mode def test_array_element_at_zero_index_fail(index, ansi_enabled): if is_spark_340_or_later(): - message = "org.apache.spark.SparkRuntimeException: [INVALID_INDEX_OF_ZERO] The index 0 is invalid" + message = "SparkRuntimeException: [INVALID_INDEX_OF_ZERO] The index 0 is invalid" elif is_databricks113_or_later(): message = "org.apache.spark.SparkRuntimeException: [ELEMENT_AT_BY_INDEX_ZERO] The index 0 is invalid" else: @@ -303,6 +312,7 @@ def test_array_element_at_zero_index_fail(index, ansi_enabled): @pytest.mark.parametrize('data_gen', array_gens_sample, ids=idfn) +@disable_ansi_mode def test_array_transform(data_gen): def do_it(spark): columns = ['a', 'b', From f95402640fb5b85ae96078d14b358aef4424ef04 Mon Sep 17 00:00:00 2001 From: "Hongbin Ma (Mahone)" Date: Sat, 29 Jun 2024 09:02:27 +0800 Subject: [PATCH 64/68] Add a heuristic to skip second or third agg pass (#10950) * add a heristic to skip agg pass Signed-off-by: Hongbin Ma (Mahone) * commit doc change Signed-off-by: Hongbin Ma (Mahone) * refine naming Signed-off-by: Hongbin Ma (Mahone) * fix only reduction case Signed-off-by: Hongbin Ma (Mahone) * fix compile Signed-off-by: Hongbin Ma (Mahone) * fix Signed-off-by: Hongbin Ma (Mahone) * clean Signed-off-by: Hongbin Ma (Mahone) * fix doc Signed-off-by: Hongbin Ma (Mahone) * reduce premergeci2 Signed-off-by: Hongbin Ma (Mahone) * reduce premergeci2, 2 Signed-off-by: Hongbin Ma (Mahone) * use test_parallel to workaround flaky array test Signed-off-by: Hongbin Ma (Mahone) * address review comment Signed-off-by: Hongbin Ma (Mahone) * remove comma Signed-off-by: Hongbin Ma (Mahone) * workaround for ci_scala213 Signed-off-by: Hongbin Ma (Mahone) * disable agg ratio heruistic by default Signed-off-by: Hongbin Ma (Mahone) * fix doc Signed-off-by: Hongbin Ma (Mahone) --------- Signed-off-by: Hongbin Ma (Mahone) --- .../advanced_configs.md | 1 + .../src/main/python/hash_aggregate_test.py | 11 +- .../spark/rapids/GpuAggregateExec.scala | 130 ++++++++++++++---- .../com/nvidia/spark/rapids/RapidsConf.scala | 10 ++ 4 files changed, 124 insertions(+), 28 deletions(-) diff --git a/docs/additional-functionality/advanced_configs.md b/docs/additional-functionality/advanced_configs.md index 941ab4046e6..033e332b99c 100644 --- a/docs/additional-functionality/advanced_configs.md +++ b/docs/additional-functionality/advanced_configs.md @@ -60,6 +60,7 @@ Name | Description | Default Value | Applicable at spark.rapids.shuffle.ucx.activeMessages.forceRndv|Set to true to force 'rndv' mode for all UCX Active Messages. This should only be required with UCX 1.10.x. UCX 1.11.x deployments should set to false.|false|Startup spark.rapids.shuffle.ucx.managementServerHost|The host to be used to start the management server|null|Startup spark.rapids.shuffle.ucx.useWakeup|When set to true, use UCX's event-based progress (epoll) in order to wake up the progress thread when needed, instead of a hot loop.|true|Startup +spark.rapids.sql.agg.skipAggPassReductionRatio|In non-final aggregation stages, if the previous pass has a row reduction ratio greater than this value, the next aggregation pass will be skipped.Setting this to 1 essentially disables this feature.|1.0|Runtime spark.rapids.sql.allowMultipleJars|Allow multiple rapids-4-spark, spark-rapids-jni, and cudf jars on the classpath. Spark will take the first one it finds, so the version may not be expected. Possisble values are ALWAYS: allow all jars, SAME_REVISION: only allow jars with the same revision, NEVER: do not allow multiple jars at all.|SAME_REVISION|Startup spark.rapids.sql.castDecimalToFloat.enabled|Casting from decimal to floating point types on the GPU returns results that have tiny difference compared to results returned from CPU.|true|Runtime spark.rapids.sql.castFloatToDecimal.enabled|Casting from floating point types to decimal on the GPU returns results that have tiny difference compared to results returned from CPU.|true|Runtime diff --git a/integration_tests/src/main/python/hash_aggregate_test.py b/integration_tests/src/main/python/hash_aggregate_test.py index e24a34ef3d5..d1cd70aa43c 100644 --- a/integration_tests/src/main/python/hash_aggregate_test.py +++ b/integration_tests/src/main/python/hash_aggregate_test.py @@ -29,12 +29,15 @@ pytestmark = pytest.mark.nightly_resource_consuming_test _float_conf = {'spark.rapids.sql.variableFloatAgg.enabled': 'true', - 'spark.rapids.sql.castStringToFloat.enabled': 'true' - } + 'spark.rapids.sql.castStringToFloat.enabled': 'true' + } _float_smallbatch_conf = copy_and_update(_float_conf, {'spark.rapids.sql.batchSizeBytes' : '250'}) +_float_conf_skipagg = copy_and_update(_float_smallbatch_conf, + {'spark.rapids.sql.agg.skipAggPassReductionRatio': '0'}) + _float_conf_partial = copy_and_update(_float_conf, {'spark.rapids.sql.hashAgg.replaceMode': 'partial'}) @@ -221,8 +224,8 @@ def get_params(init_list, marked_params=[]): return list -# Run these tests with in 4 modes, all on the GPU -_confs = [_float_conf, _float_smallbatch_conf, _float_conf_final, _float_conf_partial] +# Run these tests with in 5 modes, all on the GPU +_confs = [_float_conf, _float_smallbatch_conf, _float_conf_skipagg, _float_conf_final, _float_conf_partial] # Pytest marker for list of operators allowed to run on the CPU, # esp. useful in partial and final only modes. diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuAggregateExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuAggregateExec.scala index c58d9862be1..7e6a1056d01 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuAggregateExec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuAggregateExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019-2023, NVIDIA CORPORATION. + * Copyright (c) 2019-2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -19,6 +19,7 @@ package com.nvidia.spark.rapids import java.util import scala.annotation.tailrec +import scala.collection.JavaConverters.collectionAsScalaIterableConverter import scala.collection.mutable import ai.rapids.cudf @@ -549,7 +550,8 @@ object GpuAggregateIterator extends Logging { object GpuAggFirstPassIterator { def apply(cbIter: Iterator[ColumnarBatch], aggHelper: AggHelper, - metrics: GpuHashAggregateMetrics): Iterator[SpillableColumnarBatch] = { + metrics: GpuHashAggregateMetrics + ): Iterator[SpillableColumnarBatch] = { val preprocessProjectIter = cbIter.map { cb => val sb = SpillableColumnarBatch (cb, SpillPriorities.ACTIVE_ON_DECK_PRIORITY) aggHelper.preStepBound.projectAndCloseWithRetrySingleBatch (sb) @@ -707,6 +709,9 @@ object GpuAggFinalPassIterator { * @param metrics metrics that will be updated during aggregation * @param configuredTargetBatchSize user-specified value for the targeted input batch size * @param useTieredProject user-specified option to enable tiered projections + * @param allowNonFullyAggregatedOutput if allowed to skip third pass Agg + * @param skipAggPassReductionRatio skip if the ratio of rows after a pass is bigger than this value + * @param localInputRowsCount metric to track the number of input rows processed locally */ class GpuMergeAggregateIterator( firstPassIter: Iterator[SpillableColumnarBatch], @@ -718,15 +723,21 @@ class GpuMergeAggregateIterator( modeInfo: AggregateModeInfo, metrics: GpuHashAggregateMetrics, configuredTargetBatchSize: Long, - useTieredProject: Boolean) + useTieredProject: Boolean, + allowNonFullyAggregatedOutput: Boolean, + skipAggPassReductionRatio: Double, + localInputRowsCount: LocalGpuMetric) extends Iterator[ColumnarBatch] with AutoCloseable with Logging { private[this] val isReductionOnly = groupingExpressions.isEmpty private[this] val targetMergeBatchSize = computeTargetMergeBatchSize(configuredTargetBatchSize) private[this] val aggregatedBatches = new util.ArrayDeque[SpillableColumnarBatch] private[this] var outOfCoreIter: Option[GpuOutOfCoreSortIterator] = None - /** Iterator for fetching aggregated batches if a sort-based fallback has occurred */ - private[this] var sortFallbackIter: Option[Iterator[ColumnarBatch]] = None + /** Iterator for fetching aggregated batches either if: + * 1. a sort-based fallback has occurred + * 2. skip third pass agg has occurred + **/ + private[this] var fallbackIter: Option[Iterator[ColumnarBatch]] = None /** Whether a batch is pending for a reduction-only aggregation */ private[this] var hasReductionOnlyBatch: Boolean = isReductionOnly @@ -739,24 +750,61 @@ class GpuMergeAggregateIterator( } override def hasNext: Boolean = { - sortFallbackIter.map(_.hasNext).getOrElse { + fallbackIter.map(_.hasNext).getOrElse { // reductions produce a result even if the input is empty hasReductionOnlyBatch || !aggregatedBatches.isEmpty || firstPassIter.hasNext } } override def next(): ColumnarBatch = { - sortFallbackIter.map(_.next()).getOrElse { + fallbackIter.map(_.next()).getOrElse { + var shouldSkipThirdPassAgg = false + // aggregate and merge all pending inputs if (firstPassIter.hasNext) { - aggregateInputBatches() - tryMergeAggregatedBatches() + // first pass agg + val rowsAfterFirstPassAgg = aggregateInputBatches() + + // by now firstPassIter has been traversed, so localInputRowsCount is finished updating + if (isReductionOnly || + skipAggPassReductionRatio * localInputRowsCount.value >= rowsAfterFirstPassAgg) { + // second pass agg + tryMergeAggregatedBatches() + + val rowsAfterSecondPassAgg = aggregatedBatches.asScala.foldLeft(0L) { + (totalRows, batch) => totalRows + batch.numRows() + } + shouldSkipThirdPassAgg = + rowsAfterSecondPassAgg > skipAggPassReductionRatio * rowsAfterFirstPassAgg + } else { + shouldSkipThirdPassAgg = true + logInfo(s"Rows after first pass aggregation $rowsAfterFirstPassAgg exceeds " + + s"${skipAggPassReductionRatio * 100}% of " + + s"localInputRowsCount ${localInputRowsCount.value}, skip the second pass agg") + } } if (aggregatedBatches.size() > 1) { - // Unable to merge to a single output, so must fall back to a sort-based approach. - sortFallbackIter = Some(buildSortFallbackIterator()) - sortFallbackIter.get.next() + // Unable to merge to a single output, so must fall back + if (allowNonFullyAggregatedOutput && shouldSkipThirdPassAgg) { + // skip third pass agg, return the aggregated batches directly + logInfo(s"Rows after second pass aggregation exceeds " + + s"${skipAggPassReductionRatio * 100}% of " + + s"rows after first pass, skip the third pass agg") + fallbackIter = Some(new Iterator[ColumnarBatch] { + override def hasNext: Boolean = !aggregatedBatches.isEmpty + + override def next(): ColumnarBatch = { + withResource(aggregatedBatches.pop()) { spillableBatch => + spillableBatch.getColumnarBatch() + } + } + }) + } else { + // fallback to sort agg, this is the third pass agg + fallbackIter = Some(buildSortFallbackIterator()) + } + fallbackIter.get.next() } else if (aggregatedBatches.isEmpty) { if (hasReductionOnlyBatch) { hasReductionOnlyBatch = false @@ -779,7 +827,7 @@ class GpuMergeAggregateIterator( aggregatedBatches.clear() outOfCoreIter.foreach(_.close()) outOfCoreIter = None - sortFallbackIter = None + fallbackIter = None hasReductionOnlyBatch = false } @@ -789,11 +837,15 @@ class GpuMergeAggregateIterator( } /** Aggregate all input batches and place the results in the aggregatedBatches queue. */ - private def aggregateInputBatches(): Unit = { + private def aggregateInputBatches(): Long = { + var rowsAfter = 0L // cache everything in the first pass while (firstPassIter.hasNext) { - aggregatedBatches.add(firstPassIter.next()) + val batch = firstPassIter.next() + rowsAfter += batch.numRows() + aggregatedBatches.add(batch) } + rowsAfter } /** @@ -1115,8 +1167,8 @@ abstract class GpuBaseAggregateMeta[INPUT <: SparkPlan]( /* * Type inferencing by the Scala compiler will choose the most specific return type - * something like Array[Set[Product with Serializable with AggregateMode]] or with - * slight differences depending on Scala version. Here we ensure this is + * something like Array[Set[Product with Serializable with AggregateMode]] or with + * slight differences depending on Scala version. Here we ensure this is * Array[Set[AggregateMode]] to perform the subsequent Set and Array operations properly. */ val aggPatternsCanReplace = strPatternToReplace.split("\\|").map { subPattern => @@ -1189,6 +1241,12 @@ abstract class GpuBaseAggregateMeta[INPUT <: SparkPlan]( mode == Partial || mode == PartialMerge } && agg.groupingExpressions.nonEmpty // Don't do this for a reduce... + // for a aggregateExpressions.isEmpty case, we cannot distinguish between final and non-final, + // so don't allow it. + lazy val allowNonFullyAggregatedOutput = aggModes.forall { mode => + mode == Partial || mode == PartialMerge + } && agg.aggregateExpressions.nonEmpty + lazy val groupingCanBeSorted = agg.groupingExpressions.forall { expr => orderable.isSupportedByPlugin(expr.dataType) } @@ -1272,7 +1330,9 @@ abstract class GpuBaseAggregateMeta[INPUT <: SparkPlan]( useTiered, estimatedPreProcessGrowth, conf.forceSinglePassPartialSortAgg, - allowSinglePassAgg) + allowSinglePassAgg, + allowNonFullyAggregatedOutput, + conf.skipAggPassReductionRatio) } } @@ -1358,7 +1418,9 @@ abstract class GpuTypedImperativeSupportedAggregateExecMeta[INPUT <: BaseAggrega // For now we are just going to go with the original hash aggregation 1.0, false, - false) + false, + false, + 1) } else { super.convertToGpu() } @@ -1707,6 +1769,10 @@ object GpuHashAggregateExecBase { * @param child incoming plan (where we get input columns from) * @param configuredTargetBatchSize user-configured maximum device memory size of a batch * @param configuredTieredProjectEnabled configurable optimization to use tiered projections + * @param allowNonFullyAggregatedOutput whether we can skip the third pass of aggregation + * (can omit non fully aggregated data for non-final + * stage of aggregation) + * @param skipAggPassReductionRatio skip if the ratio of rows after a pass is bigger than this value */ case class GpuHashAggregateExec( requiredChildDistributionExpressions: Option[Seq[Expression]], @@ -1719,7 +1785,10 @@ case class GpuHashAggregateExec( configuredTieredProjectEnabled: Boolean, estimatedPreProcessGrowth: Double, forceSinglePassAgg: Boolean, - allowSinglePassAgg: Boolean) extends ShimUnaryExecNode with GpuExec { + allowSinglePassAgg: Boolean, + allowNonFullyAggregatedOutput: Boolean, + skipAggPassReductionRatio: Double +) extends ShimUnaryExecNode with GpuExec { // lifted directly from `BaseAggregateExec.inputAttributes`, edited comment. def inputAttributes: Seq[Attribute] = @@ -1804,7 +1873,7 @@ case class GpuHashAggregateExec( boundGroupExprs, aggregateExprs, aggregateAttrs, resultExprs, modeInfo, localEstimatedPreProcessGrowth, alreadySorted, expectedOrdering, postBoundReferences, targetBatchSize, aggMetrics, useTieredProject, - localForcePre, localAllowPre) + localForcePre, localAllowPre, allowNonFullyAggregatedOutput, skipAggPassReductionRatio) } } @@ -1920,7 +1989,10 @@ class DynamicGpuPartialSortAggregateIterator( metrics: GpuHashAggregateMetrics, useTiered: Boolean, forceSinglePassAgg: Boolean, - allowSinglePassAgg: Boolean) extends Iterator[ColumnarBatch] { + allowSinglePassAgg: Boolean, + allowNonFullyAggregatedOutput: Boolean, + skipAggPassReductionRatio: Double +) extends Iterator[ColumnarBatch] { private var aggIter: Option[Iterator[ColumnarBatch]] = None private[this] val isReductionOnly = boundGroupExprs.outputTypes.isEmpty @@ -1998,7 +2070,14 @@ class DynamicGpuPartialSortAggregateIterator( inputAttrs.map(_.dataType).toArray, preProcessAggHelper.preStepBound, metrics.opTime, metrics.numPreSplits) - val firstPassIter = GpuAggFirstPassIterator(splitInputIter, preProcessAggHelper, metrics) + val localInputRowsMetrics = new LocalGpuMetric + val firstPassIter = GpuAggFirstPassIterator( + splitInputIter.map(cb => { + localInputRowsMetrics += cb.numRows() + cb + }), + preProcessAggHelper, + metrics) val mergeIter = new GpuMergeAggregateIterator( firstPassIter, @@ -2010,7 +2089,10 @@ class DynamicGpuPartialSortAggregateIterator( modeInfo, metrics, configuredTargetBatchSize, - useTiered) + useTiered, + allowNonFullyAggregatedOutput, + skipAggPassReductionRatio, + localInputRowsMetrics) GpuAggFinalPassIterator.makeIter(mergeIter, postBoundReferences, metrics) } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala index 5203e926efa..aad4f05b334 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala @@ -1509,6 +1509,14 @@ val GPU_COREDUMP_PIPE_PATTERN = conf("spark.rapids.gpu.coreDump.pipePattern") .booleanConf .createWithDefault(true) + val SKIP_AGG_PASS_REDUCTION_RATIO = conf("spark.rapids.sql.agg.skipAggPassReductionRatio") + .doc("In non-final aggregation stages, if the previous pass has a row reduction ratio " + + "greater than this value, the next aggregation pass will be skipped." + + "Setting this to 1 essentially disables this feature.") + .doubleConf + .checkValue(v => v >= 0 && v <= 1, "The ratio value must be in [0, 1].") + .createWithDefault(1.0) + val FORCE_SINGLE_PASS_PARTIAL_SORT_AGG: ConfEntryWithDefault[Boolean] = conf("spark.rapids.sql.agg.forceSinglePassPartialSort") .doc("Force a single pass partial sort agg to happen in all cases that it could, " + @@ -3069,6 +3077,8 @@ class RapidsConf(conf: Map[String, String]) extends Logging { lazy val forceSinglePassPartialSortAgg: Boolean = get(FORCE_SINGLE_PASS_PARTIAL_SORT_AGG) + lazy val skipAggPassReductionRatio: Double = get(SKIP_AGG_PASS_REDUCTION_RATIO) + lazy val isRegExpEnabled: Boolean = get(ENABLE_REGEXP) lazy val maxRegExpStateMemory: Long = { From 2498204313dbc2f21cbdd76f84d5b92068949620 Mon Sep 17 00:00:00 2001 From: Haoyang Li Date: Sat, 29 Jun 2024 10:06:05 +0800 Subject: [PATCH 65/68] Support regex patterns with brackets when rewriting to PrefixRange pattern in rlike. (#11088) * Remove bracket when necessary in PrefixRange patten in Regex rewrite Signed-off-by: Haoyang Li * add pytest cases Signed-off-by: Haoyang Li * fix scala 2.13 build Signed-off-by: Haoyang Li --------- Signed-off-by: Haoyang Li --- .../src/main/python/regexp_test.py | 2 + .../com/nvidia/spark/rapids/RegexParser.scala | 45 ++++++++++--------- .../RegularExpressionRewriteSuite.scala | 4 ++ 3 files changed, 30 insertions(+), 21 deletions(-) diff --git a/integration_tests/src/main/python/regexp_test.py b/integration_tests/src/main/python/regexp_test.py index 18a83870d83..c2062605ca1 100644 --- a/integration_tests/src/main/python/regexp_test.py +++ b/integration_tests/src/main/python/regexp_test.py @@ -468,6 +468,8 @@ def test_rlike_rewrite_optimization(): 'rlike(a, "a[a-c]{1,3}")', 'rlike(a, "a[a-c]{1,}")', 'rlike(a, "a[a-c]+")', + 'rlike(a, "(ab)([a-c]{1})")', + 'rlike(a, "(ab[a-c]{1})")', 'rlike(a, "(aaa|bbb|ccc)")', 'rlike(a, ".*.*(aaa|bbb).*.*")', 'rlike(a, "^.*(aaa|bbb|ccc)")', diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala index 1ca155f8a52..362a9cce293 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala @@ -2035,7 +2035,7 @@ object RegexRewrite { @scala.annotation.tailrec private def removeBrackets(astLs: collection.Seq[RegexAST]): collection.Seq[RegexAST] = { astLs match { - case collection.Seq(RegexGroup(_, term, None)) => removeBrackets(term.children()) + case collection.Seq(RegexGroup(_, RegexSequence(terms), None)) => removeBrackets(terms) case _ => astLs } } @@ -2051,28 +2051,31 @@ object RegexRewrite { Option[(String, Int, Int, Int)] = { val haveLiteralPrefix = isLiteralString(astLs.dropRight(1)) val endsWithRange = astLs.lastOption match { - case Some(RegexRepetition( - RegexCharacterClass(false, ListBuffer(RegexCharacterRange(a,b))), - quantifier)) => { - val (start, end) = (a, b) match { - case (RegexChar(start), RegexChar(end)) => (start, end) - case _ => return None - } - val length = quantifier match { - // In Rlike, contains [a-b]{minLen,maxLen} pattern is equivalent to contains - // [a-b]{minLen} because the matching will return the result once it finds the - // minimum match so y here is unnecessary. - case QuantifierVariableLength(minLen, _) => minLen - case QuantifierFixedLength(len) => len - case SimpleQuantifier(ch) => ch match { - case '*' | '?' => 0 - case '+' => 1 + case Some(ast) => removeBrackets(collection.Seq(ast)) match { + case collection.Seq(RegexRepetition( + RegexCharacterClass(false, ListBuffer(RegexCharacterRange(a,b))), + quantifier)) => { + val (start, end) = (a, b) match { + case (RegexChar(start), RegexChar(end)) => (start, end) + case _ => return None + } + val length = quantifier match { + // In Rlike, contains [a-b]{minLen,maxLen} pattern is equivalent to contains + // [a-b]{minLen} because the matching will return the result once it finds the + // minimum match so y here is unnecessary. + case QuantifierVariableLength(minLen, _) => minLen + case QuantifierFixedLength(len) => len + case SimpleQuantifier(ch) => ch match { + case '*' | '?' => 0 + case '+' => 1 + case _ => return None + } case _ => return None } - case _ => return None + // Convert start and end to code points + Some((length, start.toInt, end.toInt)) } - // Convert start and end to code points - Some((length, start.toInt, end.toInt)) + case _ => None } case _ => None } @@ -2153,7 +2156,7 @@ object RegexRewrite { } } - val noStartsWithAst = stripLeadingWildcards(noTailingWildcards) + val noStartsWithAst = removeBrackets(stripLeadingWildcards(noTailingWildcards)) // Check if the pattern is a contains literal pattern if (isLiteralString(noStartsWithAst)) { diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionRewriteSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionRewriteSuite.scala index 7626c1450c1..a55815b95ef 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionRewriteSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionRewriteSuite.scala @@ -52,6 +52,8 @@ class RegularExpressionRewriteSuite extends AnyFunSuite { "(.*)abc[0-9]{1,3}(.*)", "(.*)abc[0-9a-z]{1,3}(.*)", "(.*)abc[0-9]{2}.*", + "((abc))([0-9]{3})", + "(abc[0-9]{3})", "^abc[0-9]{1,3}", "火花急流[\u4e00-\u9fa5]{1}", "^[0-9]{6}", @@ -63,6 +65,8 @@ class RegularExpressionRewriteSuite extends AnyFunSuite { PrefixRange("abc", 1, 48, 57), NoOptimization, // prefix followed by a multi-range not supported PrefixRange("abc", 2, 48, 57), + PrefixRange("abc", 3, 48, 57), + PrefixRange("abc", 3, 48, 57), NoOptimization, // starts with PrefixRange not supported PrefixRange("火花急流", 1, 19968, 40869), NoOptimization, // starts with PrefixRange not supported From f56fe2ccaa14340e33e9261aae18d947a4261d26 Mon Sep 17 00:00:00 2001 From: xieshuaihu Date: Mon, 1 Jul 2024 21:32:39 +0800 Subject: [PATCH 66/68] Fix match error in RapidsShuffleIterator.scala [scala2.13] (#11115) Signed-off-by: xieshuaihu --- .../nvidia/spark/rapids/shuffle/RapidsShuffleIterator.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleIterator.scala b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleIterator.scala index 29f36d6f2f7..72c1f935eed 100644 --- a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleIterator.scala +++ b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleIterator.scala @@ -173,7 +173,7 @@ class RapidsShuffleIterator( val (local, remote) = blocksByAddress.partition(ba => ba._1.host == localHost) (local ++ remote).foreach { - case (blockManagerId: BlockManagerId, blockIds: Seq[(BlockId, Long, Int)]) => { + case (blockManagerId: BlockManagerId, blockIds: collection.Seq[(BlockId, Long, Int)]) => { val shuffleRequestsMapIndex: Seq[BlockIdMapIndex] = blockIds.map { case (blockId, _, mapIndex) => /** @@ -193,7 +193,7 @@ class RapidsShuffleIterator( throw new IllegalArgumentException( s"${blockId.getClass} $blockId is not currently supported") } - } + }.toSeq val client = try { transport.makeClient(blockManagerId) From 850365ceac86d40531fed74f45221a81db7d9132 Mon Sep 17 00:00:00 2001 From: MithunR Date: Mon, 1 Jul 2024 13:58:47 -0700 Subject: [PATCH 67/68] Spark 4: Handle ANSI mode in sort_test.py (#11099) * Spark 4: Handle ANSI mode in sort_test.py Fixes #11027. With ANSI mode enabled (like the default in Spark 4), one sees that some tests in `sort_test.py` fail, because they expect ANSI mode to be off. This commit disables running those tests with ANSI enabled, and add a separate test for ANSI on/off. Signed-off-by: MithunR * Refactored not to use disable_ansi_mode. These tests need not be revisited. They test all combinations of ANSI mode, including overflow failures. Signed-off-by: MithunR --------- Signed-off-by: MithunR --- .../src/main/python/sort_test.py | 66 +++++++++++++++---- 1 file changed, 52 insertions(+), 14 deletions(-) diff --git a/integration_tests/src/main/python/sort_test.py b/integration_tests/src/main/python/sort_test.py index cb905c9fb77..3fe406d180a 100644 --- a/integration_tests/src/main/python/sort_test.py +++ b/integration_tests/src/main/python/sort_test.py @@ -1,4 +1,4 @@ -# Copyright (c) 2020-2023, NVIDIA CORPORATION. +# Copyright (c) 2020-2024, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -14,7 +14,7 @@ import pytest -from asserts import assert_gpu_and_cpu_are_equal_collect, assert_gpu_fallback_collect +from asserts import assert_gpu_and_cpu_are_equal_collect, assert_gpu_and_cpu_error, assert_gpu_fallback_collect from conftest import is_not_utc from data_gen import * from marks import allow_non_gpu @@ -224,29 +224,67 @@ def test_multi_orderby_with_limit_single_part(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark : binary_op_df(spark, data_gen).coalesce(1).orderBy(f.col('a'), f.col('b').desc()).limit(100)) + # We are not trying all possibilities, just doing a few with numbers so the query works. -@pytest.mark.parametrize('data_gen', [byte_gen, long_gen, float_gen], ids=idfn) -def test_orderby_with_processing(data_gen): +@pytest.mark.parametrize('is_ansi_enabled', [False, True]) +@pytest.mark.parametrize('data_gen', [ByteGen, LongGen, FloatGen], ids=idfn) +def test_orderby_with_processing(data_gen, is_ansi_enabled): + """ + Tests the cases where arithmetic overflows don't occur, in ANSI mode. + Overflow exceptions are tested in test_orderby_with_ansi_overflow_exceptions. + """ + conf = {'spark.sql.ansi.enabled': is_ansi_enabled} + gen = data_gen(min_val=0) if (is_ansi_enabled and data_gen != FloatGen) else data_gen() assert_gpu_and_cpu_are_equal_collect( - # avoid ambiguity in the order by statement for floating point by including a as a backup ordering column - lambda spark : unary_op_df(spark, data_gen).orderBy(f.lit(100) - f.col('a'), f.col('a'))) + # avoid ambiguity in the order by statement for floating point by including `a` as a backup ordering column + lambda spark: unary_op_df(spark, gen).orderBy(f.lit(100) - f.col('a'), f.col('a')), + conf=conf) + + +@pytest.mark.parametrize('data_gen', [long_gen], ids=idfn) +def test_orderby_with_ansi_overflow_exceptions(data_gen): + """ + Test to check that ANSI mode is honoured when there's an order-by with a subtraction expression. + With ANSI mode enabled, the subtraction will overflow, causing an ArithmeticException. + """ + def test_function(spark): + return unary_op_df(spark, data_gen).orderBy(f.lit(100) - f.col('a'), f.col('a')) + + assert_gpu_and_cpu_error(lambda spark: test_function(spark).collect(), + conf=ansi_enabled_conf, + error_message='ArithmeticException') + # We are not trying all possibilities, just doing a few with numbers so the query works. -@pytest.mark.parametrize('data_gen', [byte_gen, long_gen, float_gen], ids=idfn) -def test_orderby_with_processing_and_limit(data_gen): +@pytest.mark.parametrize('is_ansi_enabled', [False, True]) +@pytest.mark.parametrize('data_gen', [ByteGen, LongGen, FloatGen], ids=idfn) +def test_orderby_with_processing_and_limit(data_gen, is_ansi_enabled): + """ + Tests the cases where arithmetic overflows don't occur, in ANSI mode. + Overflow exceptions are tested in test_orderby_with_ansi_overflow_exceptions. + """ + conf = {'spark.sql.ansi.enabled': is_ansi_enabled} + gen = data_gen(min_val=0) if (is_ansi_enabled and data_gen != FloatGen) else data_gen() assert_gpu_and_cpu_are_equal_collect( - # avoid ambiguity in the order by statement for floating point by including a as a backup ordering column - lambda spark : unary_op_df(spark, data_gen).orderBy(f.lit(100) - f.col('a'), f.col('a')).limit(100)) + # avoid ambiguity in the order by statement for floating point by including a as a backup ordering column + lambda spark: unary_op_df(spark, gen).orderBy(f.lit(100) - f.col('a'), f.col('a')).limit(100), conf=conf) # We are not trying all possibilities, just doing a few with numbers so the query works. +@pytest.mark.parametrize('is_ansi_enabled', [False, True]) @pytest.mark.parametrize('data_gen', [StructGen([('child0', long_gen)])], ids=idfn) -def test_single_nested_orderby_with_processing_and_limit(data_gen): +def test_single_nested_orderby_with_processing_and_limit(data_gen, is_ansi_enabled): + """ + Tests the cases where arithmetic overflows don't occur, in ANSI mode. + Overflow exceptions are tested in test_orderby_with_ansi_overflow_exceptions. + """ + conf = {'spark.sql.ansi.enabled': is_ansi_enabled} + data_gen = StructGen([('child0', LongGen(min_val=0) if is_ansi_enabled else LongGen())]) assert_gpu_and_cpu_are_equal_collect( # avoid ambiguity in the order by statement for floating point by including a as a backup ordering column - lambda spark : unary_op_df(spark, data_gen)\ - .orderBy(f.struct(f.lit(100) - f.col('a.child0')), f.col('a'))\ - .limit(100)) + lambda spark: unary_op_df(spark, data_gen)\ + .orderBy(f.struct(f.lit(100) - f.col('a.child0')), f.col('a')).limit(100), + conf=conf) # We are not trying all possibilities, just doing a few with numbers so the query works. @pytest.mark.parametrize('data_gen', [byte_gen, long_gen, float_gen], ids=idfn) From 9bb295a21ce7645aa51551ca83716cd33dce722e Mon Sep 17 00:00:00 2001 From: Renjie Liu Date: Tue, 2 Jul 2024 09:40:29 +0800 Subject: [PATCH 68/68] Introduce LORE framework. (#11084) * Introduce lore id * Introduce lore id * Fix type * Fix type * Conf * style * part * Dump * Introduce lore framework * Add tests. * Rename test case Signed-off-by: liurenjie1024 * Fix AQE test * Fix style * Use args to display lore info. * Fix build break * Fix path in loreinfo * Remove path * Fix comments * Update configs * Fix comments * Fix config --------- Signed-off-by: liurenjie1024 --- .../advanced_configs.md | 2 + docs/dev/lore.md | 70 +++++ .../com/nvidia/spark/rapids/DumpUtils.scala | 28 +- .../spark/rapids/GpuAggregateExec.scala | 9 +- .../com/nvidia/spark/rapids/GpuExec.scala | 30 +- .../nvidia/spark/rapids/GpuOverrides.scala | 8 +- .../spark/rapids/GpuTransitionOverrides.scala | 5 + .../com/nvidia/spark/rapids/RapidsConf.scala | 31 ++ .../nvidia/spark/rapids/lore/GpuLore.scala | 295 ++++++++++++++++++ .../spark/rapids/lore/OutputLoreId.scala | 75 +++++ .../com/nvidia/spark/rapids/lore/dump.scala | 106 +++++++ .../nvidia/spark/rapids/lore/package.scala | 35 +++ .../com/nvidia/spark/rapids/lore/replay.scala | 102 ++++++ .../execution/GpuBroadcastExchangeExec.scala | 18 +- .../execution/datasources/GpuWriteFiles.scala | 2 +- .../spark/rapids/lore/GpuLoreSuite.scala | 169 ++++++++++ .../spark/rapids/lore/OutputLoreIdSuite.scala | 55 ++++ 17 files changed, 1029 insertions(+), 11 deletions(-) create mode 100644 docs/dev/lore.md create mode 100644 sql-plugin/src/main/scala/com/nvidia/spark/rapids/lore/GpuLore.scala create mode 100644 sql-plugin/src/main/scala/com/nvidia/spark/rapids/lore/OutputLoreId.scala create mode 100644 sql-plugin/src/main/scala/com/nvidia/spark/rapids/lore/dump.scala create mode 100644 sql-plugin/src/main/scala/com/nvidia/spark/rapids/lore/package.scala create mode 100644 sql-plugin/src/main/scala/com/nvidia/spark/rapids/lore/replay.scala create mode 100644 tests/src/test/scala/com/nvidia/spark/rapids/lore/GpuLoreSuite.scala create mode 100644 tests/src/test/scala/com/nvidia/spark/rapids/lore/OutputLoreIdSuite.scala diff --git a/docs/additional-functionality/advanced_configs.md b/docs/additional-functionality/advanced_configs.md index 033e332b99c..f5d511cbbc5 100644 --- a/docs/additional-functionality/advanced_configs.md +++ b/docs/additional-functionality/advanced_configs.md @@ -136,6 +136,8 @@ Name | Description | Default Value | Applicable at spark.rapids.sql.json.read.decimal.enabled|When reading a quoted string as a decimal Spark supports reading non-ascii unicode digits, and the RAPIDS Accelerator does not.|true|Runtime spark.rapids.sql.json.read.double.enabled|JSON reading is not 100% compatible when reading doubles.|true|Runtime spark.rapids.sql.json.read.float.enabled|JSON reading is not 100% compatible when reading floats.|true|Runtime +spark.rapids.sql.lore.dumpPath|The path to dump the LORE nodes' input data. This must be set if spark.rapids.sql.lore.idsToDump has been set. The data of each LORE node will be dumped to a subfolder with name 'loreId-' under this path. For more details, please refer to [the LORE documentation](../dev/lore.md).|None|Runtime +spark.rapids.sql.lore.idsToDump|Specify the LORE ids of operators to dump. The format is a comma separated list of LORE ids. For example: "1[0]" will dump partition 0 of input of gpu operator with lore id 1. For more details, please refer to [the LORE documentation](../dev/lore.md). If this is not set, no data will be dumped.|None|Runtime spark.rapids.sql.mode|Set the mode for the Rapids Accelerator. The supported modes are explainOnly and executeOnGPU. This config can not be changed at runtime, you must restart the application for it to take affect. The default mode is executeOnGPU, which means the RAPIDS Accelerator plugin convert the Spark operations and execute them on the GPU when possible. The explainOnly mode allows running queries on the CPU and the RAPIDS Accelerator will evaluate the queries as if it was going to run on the GPU. The explanations of what would have run on the GPU and why are output in log messages. When using explainOnly mode, the default explain output is ALL, this can be changed by setting spark.rapids.sql.explain. See that config for more details.|executeongpu|Startup spark.rapids.sql.optimizer.joinReorder.enabled|When enabled, joins may be reordered for improved query performance|true|Runtime spark.rapids.sql.python.gpu.enabled|This is an experimental feature and is likely to change in the future. Enable (true) or disable (false) support for scheduling Python Pandas UDFs with GPU resources. When enabled, pandas UDFs are assumed to share the same GPU that the RAPIDs accelerator uses and will honor the python GPU configs|false|Runtime diff --git a/docs/dev/lore.md b/docs/dev/lore.md new file mode 100644 index 00000000000..d6b28877ae7 --- /dev/null +++ b/docs/dev/lore.md @@ -0,0 +1,70 @@ +--- +layout: page +title: The Local Replay Framework +nav_order: 13 +parent: Developer Overview +--- + +# Local Replay Framework + +## Overview + +LORE (the local replay framework) is a tool that allows developer to replay the execution of a +gpu operator in local environment, so that developer could debug and profile the operator for +performance analysis. In high level it works as follows: + +1. Each gpu operator will be assigned a LORE id, which is a unique identifier for the operator. + This id is guaranteed to be unique within the same query, and guaranteed to be same when two + sql executions have same sql, same configuration, and same data. +2. In the first run of the query, developer could found the LORE id of the operator they are + interested in by checking spark ui, where LORE id usually appears in the arguments of operator. +3. In the second run of the query, developer needs to configure the LORE ids of the operators they + are interested in, and LORE will dump the input data of the operator to given path. +4. Developer could copy the dumped data to local environment, and replay the operator in local + environment. + +## Configuration + +By default, LORE id will always be generated for operators, but user could disable this behavior +by setting `spark.rapids.sql.lore.tag.enabled` to `false`. + +To tell LORE the LORE ids of the operators you are interested in, you need to set +`spark.rapids.sql.lore.idsToDump`. For example, you could set it to "1[*], 2[*], 3[*]" to tell +LORE to dump all partitions of input data of operators with id 1, 2, or 3. You can also only dump +some partition of the operator's input by appending partition numbers to lore ids. For example, +"1[0 4-6 7], 2[*]" tell LORE to dump operator with LORE id 1, but only dump partition 0, 4, 5, 6, +and 7. But for operator with LORE id 2, it will dump all partitions. + +You also need to set `spark.rapids.sql.lore.dumpPath` to tell LORE where to dump the data, the +value of which should point to a directory. All dumped data of a query will live in this +directory. A typical directory hierarchy would look like this: + +```console ++ loreId-10/ + - plan.meta + + input-0/ + - rdd.meta + + partition-0/ + - partition.meta + - batch-0.parquet + - batch-1.parquet + + partition-1/ + - partition.meta + - batch-0.parquet + + input-1/ + - rdd.meta + + partition-0/ + - partition.meta + - batch-0.parquet + - batch-1.parquet + ++ loreId-15/ + - plan.meta + + input-0/ + - rdd.meta + + partition-0/ + - partition.meta + - batch-0.parquet +``` + + diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/DumpUtils.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/DumpUtils.scala index bf949897c78..21d2de6ad68 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/DumpUtils.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/DumpUtils.scala @@ -15,7 +15,7 @@ */ package com.nvidia.spark.rapids -import java.io.{File, FileOutputStream} +import java.io.{File, FileOutputStream, OutputStream} import java.util.Random import scala.collection.mutable @@ -23,7 +23,7 @@ import scala.collection.mutable.ArrayBuffer import ai.rapids.cudf._ import ai.rapids.cudf.ColumnWriterOptions._ -import com.nvidia.spark.rapids.Arm.withResource +import com.nvidia.spark.rapids.Arm.{closeOnExcept, withResource} import org.apache.commons.io.IOUtils import org.apache.hadoop.conf.Configuration @@ -82,6 +82,23 @@ object DumpUtils extends Logging { } } + /** + * Dump columnar batch to output stream in parquet format.
+ * + * @param columnarBatch The columnar batch to be dumped, should be GPU columnar batch. It + * should be closed by caller. + * @param outputStream Will be closed after writing. + */ + def dumpToParquet(columnarBatch: ColumnarBatch, outputStream: OutputStream): Unit = { + closeOnExcept(outputStream) { _ => + withResource(GpuColumnVector.from(columnarBatch)) { table => + withResource(new ParquetDumper(outputStream, table)) { dumper => + dumper.writeTable(table) + } + } + } + } + /** * Debug utility to dump table to parquet file.
* It's running on GPU. Parquet column names are generated from table column type info.
@@ -129,12 +146,15 @@ object DumpUtils extends Logging { } // parquet dumper -class ParquetDumper(path: String, table: Table) extends HostBufferConsumer +class ParquetDumper(private val outputStream: OutputStream, table: Table) extends HostBufferConsumer with AutoCloseable { - private[this] val outputStream = new FileOutputStream(path) private[this] val tempBuffer = new Array[Byte](128 * 1024) private[this] val buffers = mutable.Queue[(HostMemoryBuffer, Long)]() + def this(path: String, table: Table) = { + this(new FileOutputStream(path), table) + } + val tableWriter: TableWriter = { // avoid anything conversion, just dump as it is val builder = ParquetDumper.parquetWriterOptionsFromTable(ParquetWriterOptions.builder(), table) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuAggregateExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuAggregateExec.scala index 7e6a1056d01..b35e687d185 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuAggregateExec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuAggregateExec.scala @@ -1827,6 +1827,7 @@ case class GpuHashAggregateExec( |${ExplainUtils.generateFieldString("Functions", aggregateExpressions)} |${ExplainUtils.generateFieldString("Aggregate Attributes", aggregateAttributes)} |${ExplainUtils.generateFieldString("Results", resultExpressions)} + |Lore: ${loreArgs.mkString(", ")} |""".stripMargin } @@ -1955,10 +1956,12 @@ case class GpuHashAggregateExec( truncatedString(allAggregateExpressions, "[", ", ", "]", maxFields) val outputString = truncatedString(output, "[", ", ", "]", maxFields) if (verbose) { - s"GpuHashAggregate(keys=$keyString, functions=$functionString, output=$outputString)" + s"$nodeName (keys=$keyString, functions=$functionString, output=$outputString) " + + s"""${loreArgs.mkString(", ")}""" } else { - s"GpuHashAggregate(keys=$keyString, functions=$functionString)," + - s" filters=${aggregateExpressions.map(_.filter)})" + s"$nodeName (keys=$keyString, functions=$functionString)," + + s" filters=${aggregateExpressions.map(_.filter)})" + + s""" ${loreArgs.mkString(", ")}""" } } // diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExec.scala index d83f20113b2..e93ac40b5bd 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExec.scala @@ -19,7 +19,10 @@ package com.nvidia.spark.rapids import ai.rapids.cudf.NvtxColor import com.nvidia.spark.rapids.Arm.withResource import com.nvidia.spark.rapids.filecache.FileCacheConf +import com.nvidia.spark.rapids.lore.{GpuLore, GpuLoreDumpRDD} +import com.nvidia.spark.rapids.lore.GpuLore.{loreIdOf, LORE_DUMP_PATH_TAG, LORE_DUMP_RDD_TAG} import com.nvidia.spark.rapids.shims.SparkShimImpl +import org.apache.hadoop.fs.Path import org.apache.spark.internal.Logging import org.apache.spark.rapids.LocationPreservingMapPartitionsRDD @@ -385,7 +388,8 @@ trait GpuExec extends SparkPlan { this.getTagValue(GpuExec.TASK_METRICS_TAG) final override def doExecuteColumnar(): RDD[ColumnarBatch] = { - val orig = internalDoExecuteColumnar() + this.dumpLoreMetaInfo() + val orig = this.dumpLoreRDD(internalDoExecuteColumnar()) val metrics = getTaskMetrics metrics.map { gpuMetrics => // This is ugly, but it reduces the need to change all exec nodes, so we are doing it here @@ -396,5 +400,29 @@ trait GpuExec extends SparkPlan { }.getOrElse(orig) } + override def stringArgs: Iterator[Any] = super.stringArgs ++ loreArgs + + protected def loreArgs: Iterator[String] = { + val loreIdStr = loreIdOf(this).map(id => s"[loreId=$id]") + val lorePathStr = getTagValue(LORE_DUMP_PATH_TAG).map(path => s"[lorePath=$path]") + val loreRDDInfoStr = getTagValue(LORE_DUMP_RDD_TAG).map(info => s"[loreRDDInfo=$info]") + + List(loreIdStr, lorePathStr, loreRDDInfoStr).flatten.iterator + } + + private def dumpLoreMetaInfo(): Unit = { + getTagValue(LORE_DUMP_PATH_TAG).foreach { rootPath => + GpuLore.dumpPlan(this, new Path(rootPath)) + } + } + + protected def dumpLoreRDD(inner: RDD[ColumnarBatch]): RDD[ColumnarBatch] = { + getTagValue(LORE_DUMP_RDD_TAG).map { info => + val rdd = new GpuLoreDumpRDD(info, inner) + rdd.saveMeta() + rdd + }.getOrElse(inner) + } + protected def internalDoExecuteColumnar(): RDD[ColumnarBatch] } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala index 9e26cf751f4..73475ef36f5 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala @@ -25,6 +25,7 @@ import scala.util.control.NonFatal import ai.rapids.cudf.DType import com.nvidia.spark.rapids.RapidsConf.{SUPPRESS_PLANNING_FAILURE, TEST_CONF} import com.nvidia.spark.rapids.jni.GpuTimeZoneDB +import com.nvidia.spark.rapids.lore.GpuLore import com.nvidia.spark.rapids.shims._ import com.nvidia.spark.rapids.window.{GpuDenseRank, GpuLag, GpuLead, GpuPercentRank, GpuRank, GpuRowNumber, GpuSpecialFrameBoundary, GpuWindowExecMeta, GpuWindowSpecDefinitionMeta} import org.apache.hadoop.fs.Path @@ -4708,7 +4709,12 @@ case class GpuOverrides() extends Rule[SparkPlan] with Logging { } } } - GpuOverrides.doConvertPlan(wrap, conf, optimizations) + val convertedPlan = GpuOverrides.doConvertPlan(wrap, conf, optimizations) + if (conf.isTagLoreIdEnabled) { + GpuLore.tagForLore(convertedPlan, conf) + } else { + convertedPlan + } } } } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuTransitionOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuTransitionOverrides.scala index 48f9de5a61a..c8596f983d9 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuTransitionOverrides.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuTransitionOverrides.scala @@ -21,6 +21,7 @@ import java.util.concurrent.atomic.AtomicInteger import scala.annotation.tailrec import scala.collection.mutable +import com.nvidia.spark.rapids.lore.GpuLore import com.nvidia.spark.rapids.shims.{GpuBatchScanExec, SparkShimImpl} import org.apache.spark.SparkContext @@ -823,6 +824,10 @@ class GpuTransitionOverrides extends Rule[SparkPlan] { updatedPlan = fixupAdaptiveExchangeReuse(updatedPlan) } + if (rapidsConf.isTagLoreIdEnabled) { + updatedPlan = GpuLore.tagForLore(updatedPlan, rapidsConf) + } + if (rapidsConf.logQueryTransformations) { logWarning(s"Transformed query:" + s"\nOriginal Plan:\n$plan\nTransformed Plan:\n$updatedPlan") diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala index aad4f05b334..406d09a7a32 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala @@ -23,6 +23,7 @@ import scala.collection.mutable.{HashMap, ListBuffer} import ai.rapids.cudf.Cuda import com.nvidia.spark.rapids.jni.RmmSpark.OomInjectionType +import com.nvidia.spark.rapids.lore.{LoreId, OutputLoreId} import org.apache.spark.SparkConf import org.apache.spark.internal.Logging @@ -2308,6 +2309,28 @@ val SHUFFLE_COMPRESSION_LZ4_CHUNK_SIZE = conf("spark.rapids.shuffle.compression. .booleanConf .createWithDefault(false) + val TAG_LORE_ID_ENABLED = conf("spark.rapids.sql.lore.tag.enabled") + .doc("Enable add a LORE id to each gpu plan node") + .internal() + .booleanConf + .createWithDefault(true) + + val LORE_DUMP_IDS = conf("spark.rapids.sql.lore.idsToDump") + .doc("Specify the LORE ids of operators to dump. The format is a comma separated list of " + + "LORE ids. For example: \"1[0]\" will dump partition 0 of input of gpu operator " + + "with lore id 1. For more details, please refer to " + + "[the LORE documentation](../dev/lore.md). If this is not set, no data will be dumped.") + .stringConf + .createOptional + + val LORE_DUMP_PATH = conf("spark.rapids.sql.lore.dumpPath") + .doc(s"The path to dump the LORE nodes' input data. This must be set if ${LORE_DUMP_IDS.key} " + + "has been set. The data of each LORE node will be dumped to a subfolder with name " + + "'loreId-' under this path. For more details, please refer to " + + "[the LORE documentation](../dev/lore.md).") + .stringConf + .createOptional + private def printSectionHeader(category: String): Unit = println(s"\n### $category") @@ -3121,6 +3144,14 @@ class RapidsConf(conf: Map[String, String]) extends Logging { lazy val isDeltaLowShuffleMergeEnabled: Boolean = get(ENABLE_DELTA_LOW_SHUFFLE_MERGE) + lazy val isTagLoreIdEnabled: Boolean = get(TAG_LORE_ID_ENABLED) + + lazy val loreDumpIds: Map[LoreId, OutputLoreId] = get(LORE_DUMP_IDS) + .map(OutputLoreId.parse) + .getOrElse(Map.empty) + + lazy val loreDumpPath: Option[String] = get(LORE_DUMP_PATH) + private val optimizerDefaults = Map( // this is not accurate because CPU projections do have a cost due to appending values // to each row that is produced, but this needs to be a really small number because diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/lore/GpuLore.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/lore/GpuLore.scala new file mode 100644 index 00000000000..a51a1e13a5e --- /dev/null +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/lore/GpuLore.scala @@ -0,0 +1,295 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed 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 com.nvidia.spark.rapids.lore + +import java.util.concurrent.{ConcurrentHashMap, ConcurrentMap} +import java.util.concurrent.atomic.AtomicInteger + +import scala.collection.mutable +import scala.reflect.ClassTag + +import com.nvidia.spark.rapids.{GpuColumnarToRowExec, GpuExec, RapidsConf} +import com.nvidia.spark.rapids.Arm.withResource +import com.nvidia.spark.rapids.shims.SparkShimImpl +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path + +import org.apache.spark.SparkEnv +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.trees.TreeNodeTag +import org.apache.spark.sql.execution.{BaseSubqueryExec, ExecSubqueryExpression, ReusedSubqueryExec, SparkPlan, SQLExecution} +import org.apache.spark.sql.execution.adaptive.BroadcastQueryStageExec +import org.apache.spark.sql.rapids.execution.{GpuBroadcastExchangeExec, GpuCustomShuffleReaderExec} +import org.apache.spark.sql.types.DataType +import org.apache.spark.util.SerializableConfiguration + +case class LoreRDDMeta(numPartitions: Int, outputPartitions: Seq[Int], attrs: Seq[Attribute]) + +case class LoreRDDPartitionMeta(numBatches: Int, dataType: Seq[DataType]) + +trait GpuLoreRDD { + def rootPath: Path + + def pathOfMeta: Path = new Path(rootPath, "rdd.meta") + + def pathOfPartition(partitionIndex: Int): Path = { + new Path(rootPath, s"partition-$partitionIndex") + } + + def pathOfPartitionMeta(partitionIndex: Int): Path = { + new Path(pathOfPartition(partitionIndex), "partition.meta") + } + + def pathOfBatch(partitionIndex: Int, batchIndex: Int): Path = { + new Path(pathOfPartition(partitionIndex), s"batch-$batchIndex.parquet") + } +} + + +object GpuLore { + /** + * Lore id of a plan node. + */ + val LORE_ID_TAG: TreeNodeTag[String] = new TreeNodeTag[String]("rapids.gpu.lore.id") + /** + * When a [[GpuExec]] node has this tag, it means that this node is a root node whose meta and + * input should be dumped. + */ + val LORE_DUMP_PATH_TAG: TreeNodeTag[String] = new TreeNodeTag[String]("rapids.gpu.lore.dump.path") + /** + * When a [[GpuExec]] node has this tag, it means that this node is a child node whose data + * should be dumped. + */ + val LORE_DUMP_RDD_TAG: TreeNodeTag[LoreDumpRDDInfo] = new TreeNodeTag[LoreDumpRDDInfo]( + "rapids.gpu.lore.dump.rdd.info") + + def pathOfRootPlanMeta(rootPath: Path): Path = { + new Path(rootPath, "plan.meta") + } + + def dumpPlan[T <: SparkPlan : ClassTag](plan: T, rootPath: Path): Unit = { + dumpObject(plan, pathOfRootPlanMeta(rootPath), + SparkShimImpl.sessionFromPlan(plan).sparkContext.hadoopConfiguration) + } + + def dumpObject[T: ClassTag](obj: T, path: Path, hadoopConf: Configuration): Unit = { + withResource(path.getFileSystem(hadoopConf)) { fs => + withResource(fs.create(path, false)) { fout => + val serializerStream = SparkEnv.get.serializer.newInstance().serializeStream(fout) + withResource(serializerStream) { ser => + ser.writeObject(obj) + } + } + } + } + + def loadObject[T: ClassTag](path: Path, hadoopConf: Configuration): T = { + withResource(path.getFileSystem(hadoopConf)) { fs => + withResource(fs.open(path)) { fin => + val serializerStream = SparkEnv.get.serializer.newInstance().deserializeStream(fin) + withResource(serializerStream) { ser => + ser.readObject().asInstanceOf[T] + } + } + } + } + + def pathOfChild(rootPath: Path, childIndex: Int): Path = { + new Path(rootPath, s"input-$childIndex") + } + + def restoreGpuExec(rootPath: Path, spark: SparkSession): GpuExec = { + val rootExec = loadObject[GpuExec](pathOfRootPlanMeta(rootPath), + spark.sparkContext.hadoopConfiguration) + + checkUnsupportedOperator(rootExec) + + val broadcastHadoopConf = { + val sc = spark.sparkContext + sc.broadcast(new SerializableConfiguration(spark.sparkContext.hadoopConfiguration)) + } + + // Load children + val newChildren = rootExec.children.zipWithIndex.map { case (plan, idx) => + val newChild = GpuLoreReplayExec(idx, rootPath.toString, broadcastHadoopConf) + plan match { + case b: GpuBroadcastExchangeExec => + b.withNewChildren(Seq(newChild)) + case b: BroadcastQueryStageExec => + b.broadcast.withNewChildren(Seq(newChild)) + case _ => newChild + } + } + + var nextId = rootExec.children.length + + rootExec.transformExpressionsUp { + case sub: ExecSubqueryExpression => + val newSub = restoreSubqueryPlan(nextId, sub, rootPath, broadcastHadoopConf) + nextId += 1 + newSub + }.withNewChildren(newChildren).asInstanceOf[GpuExec] + } + + private def restoreSubqueryPlan(id: Int, sub: ExecSubqueryExpression, + rootPath: Path, hadoopConf: Broadcast[SerializableConfiguration]): ExecSubqueryExpression = { + val innerPlan = sub.plan.child + + if (innerPlan.isInstanceOf[GpuExec]) { + var newChild: SparkPlan = GpuLoreReplayExec(id, rootPath.toString, hadoopConf) + + if (!innerPlan.supportsColumnar) { + newChild = GpuColumnarToRowExec(newChild) + } + val newSubqueryExec = sub.plan match { + case ReusedSubqueryExec(subqueryExec) => subqueryExec.withNewChildren(Seq(newChild)) + .asInstanceOf[BaseSubqueryExec] + case p: BaseSubqueryExec => p.withNewChildren(Seq(newChild)) + .asInstanceOf[BaseSubqueryExec] + } + sub.withNewPlan(newSubqueryExec) + } else { + throw new IllegalArgumentException(s"Subquery plan ${innerPlan.getClass.getSimpleName} " + + s"is not a GpuExec") + } + } + + /** + * Lore id generator. Key is [[SQLExecution.EXECUTION_ID_KEY]]. + */ + private val idGen: ConcurrentMap[String, AtomicInteger] = + new ConcurrentHashMap[String, AtomicInteger]() + + private def nextLoreIdOf(plan: SparkPlan): Option[Int] = { + // When the execution id is not set, it means there is no actual execution happening, in this + // case we don't need to generate lore id. + Option(SparkShimImpl.sessionFromPlan(plan) + .sparkContext + .getLocalProperty(SQLExecution.EXECUTION_ID_KEY)) + .map { executionId => + idGen.computeIfAbsent(executionId, _ => new AtomicInteger(0)).getAndIncrement() + } + } + + def tagForLore(sparkPlan: SparkPlan, rapidsConf: RapidsConf): SparkPlan = { + val loreDumpIds = rapidsConf.loreDumpIds + + val newPlan = if (loreDumpIds.nonEmpty) { + // We need to dump the output of nodes with the lore id in the dump ids + val loreOutputRootPath = rapidsConf.loreDumpPath.getOrElse(throw + new IllegalArgumentException(s"${RapidsConf.LORE_DUMP_PATH.key} must be set " + + s"when ${RapidsConf.LORE_DUMP_IDS.key} is set.")) + + val spark = SparkShimImpl.sessionFromPlan(sparkPlan) + val hadoopConf = { + val sc = spark.sparkContext + sc.broadcast(new SerializableConfiguration(sc.hadoopConfiguration)) + } + + val subqueries = mutable.Set.empty[SparkPlan] + + sparkPlan.foreachUp { + case g: GpuExec => + nextLoreIdOf(g).foreach { loreId => + g.setTagValue(LORE_ID_TAG, loreId.toString) + + loreDumpIds.get(loreId).foreach { outputLoreIds => + checkUnsupportedOperator(g) + val currentExecRootPath = new Path(loreOutputRootPath, s"loreId-$loreId") + g.setTagValue(LORE_DUMP_PATH_TAG, currentExecRootPath.toString) + val loreOutputInfo = LoreOutputInfo(outputLoreIds, + currentExecRootPath.toString) + + g.children.zipWithIndex.foreach { + case (child, idx) => + val dumpRDDInfo = LoreDumpRDDInfo(idx, loreOutputInfo, child.output, hadoopConf) + child match { + case c: BroadcastQueryStageExec => + c.broadcast.setTagValue(LORE_DUMP_RDD_TAG, dumpRDDInfo) + case o => o.setTagValue(LORE_DUMP_RDD_TAG, dumpRDDInfo) + } + } + + var nextId = g.children.length + g.transformExpressionsUp { + case sub: ExecSubqueryExpression => + if (spark.sessionState.conf.subqueryReuseEnabled) { + if (!subqueries.contains(sub.plan.canonicalized)) { + subqueries += sub.plan.canonicalized + } else { + throw new IllegalArgumentException("Subquery reuse is enabled, and we found" + + " duplicated subqueries, which is currently not supported by LORE.") + } + } + tagSubqueryPlan(nextId, sub, loreOutputInfo, hadoopConf) + nextId += 1 + sub + } + } + } + case _ => + } + + sparkPlan + + } else { + // We don't need to dump the output of the nodes, just tag the lore id + sparkPlan.foreachUp { + case g: GpuExec => + nextLoreIdOf(g).foreach { loreId => + g.setTagValue(LORE_ID_TAG, loreId.toString) + } + case _ => + } + + sparkPlan + } + + newPlan + } + + def loreIdOf(node: SparkPlan): Option[String] = { + node.getTagValue(LORE_ID_TAG) + } + + private def tagSubqueryPlan(id: Int, sub: ExecSubqueryExpression, + loreOutputInfo: LoreOutputInfo, hadoopConf: Broadcast[SerializableConfiguration]) = { + val innerPlan = sub.plan.child + if (innerPlan.isInstanceOf[GpuExec]) { + val dumpRDDInfo = LoreDumpRDDInfo(id, loreOutputInfo, innerPlan.output, + hadoopConf) + innerPlan match { + case p: GpuColumnarToRowExec => p.child.setTagValue(LORE_DUMP_RDD_TAG, dumpRDDInfo) + case c => c.setTagValue(LORE_DUMP_RDD_TAG, dumpRDDInfo) + } + } else { + throw new IllegalArgumentException(s"Subquery plan ${innerPlan.getClass.getSimpleName} " + + s"is not a GpuExec") + } + } + + private def checkUnsupportedOperator(plan: SparkPlan): Unit = { + if (plan.children.isEmpty || + plan.isInstanceOf[GpuCustomShuffleReaderExec] + ) { + throw new UnsupportedOperationException(s"Currently we don't support dumping input of " + + s"${plan.getClass.getSimpleName} operator.") + } + } +} diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/lore/OutputLoreId.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/lore/OutputLoreId.scala new file mode 100644 index 00000000000..28fa0b2dbbf --- /dev/null +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/lore/OutputLoreId.scala @@ -0,0 +1,75 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed 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 com.nvidia.spark.rapids.lore + +import org.apache.hadoop.fs.Path + +case class OutputLoreId(loreId: LoreId, partitionIds: Set[Int]) { + def outputAllParitions: Boolean = partitionIds.isEmpty + + def shouldOutputPartition(partitionId: Int): Boolean = outputAllParitions || + partitionIds.contains(partitionId) +} + +case class LoreOutputInfo(outputLoreId: OutputLoreId, pathStr: String) { + def path: Path = new Path(pathStr) +} + +object OutputLoreId { + private val PARTITION_ID_RANGE_REGEX = raw"(\d+)-(\d+)".r("start", "end") + private val PARTITION_ID_REGEX = raw"(\d+)".r("partitionId") + private val PARTITION_ID_SEP_REGEX = raw" +".r + + private val OUTPUT_LORE_ID_SEP_REGEX = ", *".r + private val OUTPUT_LORE_ID_REGEX = + raw"(?\d+)(\[(?.*)\])?".r + + def apply(loreId: Int): OutputLoreId = OutputLoreId(loreId, Set.empty) + + def apply(inputStr: String): OutputLoreId = { + OUTPUT_LORE_ID_REGEX.findFirstMatchIn(inputStr).map { m => + val loreId = m.group("loreId").toInt + val partitionIds: Set[Int] = m.group("partitionIds") match { + case partitionIdsStr if partitionIdsStr != null => + PARTITION_ID_SEP_REGEX.split(partitionIdsStr).flatMap { + case PARTITION_ID_REGEX(partitionId) => + Seq(partitionId.toInt) + case PARTITION_ID_RANGE_REGEX(start, end) => + start.toInt until end.toInt + case "*" => Set.empty + case partitionIdStr => throw new IllegalArgumentException(s"Invalid partition " + + s"id: $partitionIdStr") + }.toSet + case null => { + throw new IllegalArgumentException(s"Invalid output lore id string: $inputStr, " + + s"partition ids not found!") + } + } + OutputLoreId(loreId, partitionIds) + }.getOrElse(throw new IllegalArgumentException(s"Invalid output lore ids: $inputStr")) + } + + def parse(inputStr: String): OutputLoreIds = { + require(inputStr != null, "inputStr should not be null") + + OUTPUT_LORE_ID_SEP_REGEX.split(inputStr).map(OutputLoreId(_)).map { outputLoreId => + outputLoreId.loreId -> outputLoreId + }.toMap + } +} + + diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/lore/dump.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/lore/dump.scala new file mode 100644 index 00000000000..1b9967e1bf4 --- /dev/null +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/lore/dump.scala @@ -0,0 +1,106 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed 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 com.nvidia.spark.rapids.lore + +import com.nvidia.spark.rapids.{DumpUtils, GpuColumnVector} +import com.nvidia.spark.rapids.GpuCoalesceExec.EmptyPartition +import com.nvidia.spark.rapids.lore.GpuLore.pathOfChild +import org.apache.hadoop.fs.Path + +import org.apache.spark.{Partition, SparkContext, TaskContext} +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.rapids.execution.GpuBroadcastHelper +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.spark.util.SerializableConfiguration + + +case class LoreDumpRDDInfo(idxInParent: Int, loreOutputInfo: LoreOutputInfo, attrs: Seq[Attribute], + hadoopConf: Broadcast[SerializableConfiguration]) + +class GpuLoreDumpRDD(info: LoreDumpRDDInfo, input: RDD[ColumnarBatch]) + extends RDD[ColumnarBatch](input) with GpuLoreRDD { + override def rootPath: Path = pathOfChild(info.loreOutputInfo.path, info.idxInParent) + + def saveMeta(): Unit = { + val meta = LoreRDDMeta(input.getNumPartitions, this.getPartitions.map(_.index), info.attrs) + GpuLore.dumpObject(meta, pathOfMeta, this.context.hadoopConfiguration) + } + + override def compute(split: Partition, context: TaskContext): Iterator[ColumnarBatch] = { + if (info.loreOutputInfo.outputLoreId.shouldOutputPartition(split.index)) { + val originalIter = input.compute(split, context) + new Iterator[ColumnarBatch] { + var batchIdx: Int = -1 + var nextBatch: Option[ColumnarBatch] = None + + override def hasNext: Boolean = { + if (batchIdx == -1) { + loadNextBatch() + } + nextBatch.isDefined + } + + override def next(): ColumnarBatch = { + val ret = dumpCurrentBatch() + loadNextBatch() + if (!hasNext) { + // This is the last batch, save the partition meta + val partitionMeta = LoreRDDPartitionMeta(batchIdx, GpuColumnVector.extractTypes(ret)) + GpuLore.dumpObject(partitionMeta, pathOfPartitionMeta(split.index), + info.hadoopConf.value.value) + } + ret + } + + private def dumpCurrentBatch(): ColumnarBatch = { + val outputPath = pathOfBatch(split.index, batchIdx) + val outputStream = outputPath.getFileSystem(info.hadoopConf.value.value) + .create(outputPath, false) + DumpUtils.dumpToParquet(nextBatch.get, outputStream) + nextBatch.get + } + + private def loadNextBatch(): Unit = { + if (originalIter.hasNext) { + nextBatch = Some(originalIter.next()) + } else { + nextBatch = None + } + batchIdx += 1 + } + } + } else { + input.compute(split, context) + } + } + + override protected def getPartitions: Array[Partition] = { + input.partitions + } +} + +class SimpleRDD(_sc: SparkContext, data: Broadcast[Any], schema: StructType) extends + RDD[ColumnarBatch](_sc, Nil) { + override def compute(split: Partition, context: TaskContext): Iterator[ColumnarBatch] = { + Seq(GpuBroadcastHelper.getBroadcastBatch(data, schema)).iterator + } + + override protected def getPartitions: Array[Partition] = Array(EmptyPartition(0)) +} diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/lore/package.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/lore/package.scala new file mode 100644 index 00000000000..f304ea07d97 --- /dev/null +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/lore/package.scala @@ -0,0 +1,35 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed 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 com.nvidia.spark.rapids + +/** + * Lore framework is used for dumping input data of a gpu executor to disk so that it can be + * replayed in local environment for performance analysis. + *
+ * When [[RapidsConf.TAG_LORE_ID_ENABLED]] is set, during the planning phase we will tag a lore + * id to each gpu operator. Lore id is guaranteed to be unique within a query, and it's supposed + * to be same for operators with same plan. + *
+ * When [[RapidsConf.LORE_DUMP_IDS]] is set, during the execution phase we will dump the input + * data of gpu operators with lore id to disk. The dumped data can be replayed in local + * environment. The dumped data will reside in [[RapidsConf.LORE_DUMP_PATH]]. For more details, + * please refer to `docs/dev/lore.md`. + */ +package object lore { + type LoreId = Int + type OutputLoreIds = Map[LoreId, OutputLoreId] +} diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/lore/replay.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/lore/replay.scala new file mode 100644 index 00000000000..ffbe207646a --- /dev/null +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/lore/replay.scala @@ -0,0 +1,102 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed 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 com.nvidia.spark.rapids.lore + +import ai.rapids.cudf.Table +import com.nvidia.spark.rapids.{GpuColumnVector, GpuExec} +import com.nvidia.spark.rapids.Arm.withResource +import org.apache.commons.io.IOUtils +import org.apache.hadoop.fs.Path + +import org.apache.spark.{Partition, SparkContext, TaskContext} +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.execution.LeafExecNode +import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.spark.util.SerializableConfiguration + +case class GpuLoreReplayExec(idxInParent: Int, parentRootPath: String, + hadoopConf: Broadcast[SerializableConfiguration]) + extends LeafExecNode + with GpuExec { + private lazy val rdd = new GpuLoreReplayRDD(sparkSession.sparkContext, + GpuLore.pathOfChild(new Path(parentRootPath), idxInParent).toString, hadoopConf) + override def output: Seq[Attribute] = rdd.loreRDDMeta.attrs + + override def doExecute(): RDD[InternalRow] = { + throw new UnsupportedOperationException("LoreReplayExec does not support row mode") + } + + override protected def internalDoExecuteColumnar(): RDD[ColumnarBatch] = { + rdd + } +} + +class GpuLoreReplayRDD(sc: SparkContext, rootPathStr: String, + hadoopConf: Broadcast[SerializableConfiguration]) + extends RDD[ColumnarBatch](sc, Nil) with GpuLoreRDD { + + override def rootPath: Path = new Path(rootPathStr) + + private[lore] val loreRDDMeta: LoreRDDMeta = GpuLore.loadObject(pathOfMeta, sc + .hadoopConfiguration) + + override def compute(split: Partition, context: TaskContext): Iterator[ColumnarBatch] = { + val partitionPath = pathOfPartition(split.index) + withResource(partitionPath.getFileSystem(hadoopConf.value.value)) { fs => + if (!fs.exists(partitionPath)) { + Iterator.empty + } else { + val partitionMeta = GpuLore.loadObject[LoreRDDPartitionMeta]( + pathOfPartitionMeta(split.index), hadoopConf.value.value) + new Iterator[ColumnarBatch] { + private var batchIdx: Int = 0 + + override def hasNext: Boolean = { + batchIdx < partitionMeta.numBatches + } + + override def next(): ColumnarBatch = { + val batchPath = pathOfBatch(split.index, batchIdx) + val ret = withResource(batchPath.getFileSystem(hadoopConf.value.value)) { fs => + if (!fs.exists(batchPath)) { + throw new IllegalStateException(s"Batch file $batchPath does not exist") + } + withResource(fs.open(batchPath)) { fin => + val buffer = IOUtils.toByteArray(fin) + withResource(Table.readParquet(buffer)) { restoredTable => + GpuColumnVector.from(restoredTable, partitionMeta.dataType.toArray) + } + } + + } + batchIdx += 1 + ret + } + } + } + } + } + + override protected def getPartitions: Array[Partition] = { + (0 until loreRDDMeta.numPartitions).map(LoreReplayPartition).toArray + } +} + +case class LoreReplayPartition(override val index: Int) extends Partition diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastExchangeExec.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastExchangeExec.scala index 51c6f52d97e..bd30459d63e 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastExchangeExec.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastExchangeExec.scala @@ -31,6 +31,8 @@ import com.nvidia.spark.rapids._ import com.nvidia.spark.rapids.Arm.{closeOnExcept, withResource} import com.nvidia.spark.rapids.GpuMetric._ import com.nvidia.spark.rapids.RapidsPluginImplicits._ +import com.nvidia.spark.rapids.lore.{GpuLoreDumpRDD, SimpleRDD} +import com.nvidia.spark.rapids.lore.GpuLore.LORE_DUMP_RDD_TAG import com.nvidia.spark.rapids.shims.{ShimBroadcastExchangeLike, ShimUnaryExecNode, SparkShimImpl} import org.apache.spark.SparkException @@ -486,7 +488,9 @@ abstract class GpuBroadcastExchangeExecBase( throw new IllegalStateException("A canonicalized plan is not supposed to be executed.") } try { - relationFuture.get(timeout, TimeUnit.SECONDS).asInstanceOf[Broadcast[T]] + val ret = relationFuture.get(timeout, TimeUnit.SECONDS) + doLoreDump(ret) + ret.asInstanceOf[Broadcast[T]] } catch { case ex: TimeoutException => logError(s"Could not execute broadcast in $timeout secs.", ex) @@ -501,6 +505,18 @@ abstract class GpuBroadcastExchangeExecBase( } } + // We have to do this explicitly here rather than similar to the general version one in + // [[GpuExec]] since in adaptive execution, the broadcast value has already been calculated + // before we tag this plan to dump. + private def doLoreDump(result: Broadcast[Any]): Unit = { + val inner = new SimpleRDD(session.sparkContext, result, schema) + getTagValue(LORE_DUMP_RDD_TAG).foreach { info => + val rdd = new GpuLoreDumpRDD(info, inner) + rdd.saveMeta() + rdd.foreach(_.close()) + } + } + override def runtimeStatistics: Statistics = { Statistics( sizeInBytes = metrics("dataSize").value, diff --git a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/execution/datasources/GpuWriteFiles.scala b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/execution/datasources/GpuWriteFiles.scala index 7cc94359daa..f1ffcf4df1f 100644 --- a/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/execution/datasources/GpuWriteFiles.scala +++ b/sql-plugin/src/main/spark332db/scala/org/apache/spark/sql/execution/datasources/GpuWriteFiles.scala @@ -157,7 +157,7 @@ case class GpuWriteFilesExec( s" mismatch:\n$this") } - override protected def stringArgs: Iterator[Any] = Iterator(child) + override def stringArgs: Iterator[Any] = Iterator(child) } object GpuWriteFiles { diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/lore/GpuLoreSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/lore/GpuLoreSuite.scala new file mode 100644 index 00000000000..7db46718e89 --- /dev/null +++ b/tests/src/test/scala/com/nvidia/spark/rapids/lore/GpuLoreSuite.scala @@ -0,0 +1,169 @@ +/* + * Copyright (c) 2020-2024, NVIDIA CORPORATION. + * + * Licensed 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 com.nvidia.spark.rapids.lore + +import com.nvidia.spark.rapids.{FunSuiteWithTempDir, GpuColumnarToRowExec, RapidsConf, SparkQueryCompareTestSuite} +import org.apache.hadoop.fs.Path + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.{functions, DataFrame, SparkSession} +import org.apache.spark.sql.internal.SQLConf + +class GpuLoreSuite extends SparkQueryCompareTestSuite with FunSuiteWithTempDir with Logging { + test("Aggregate") { + doTestReplay("10[*]") { spark => + spark.range(0, 1000, 1, 100) + .selectExpr("id % 10 as key", "id % 100 as value") + .groupBy("key") + .agg(functions.sum("value").as("total")) + } + } + + test("Broadcast join") { + doTestReplay("32[*]") { spark => + val df1 = spark.range(0, 1000, 1, 10) + .selectExpr("id % 10 as key", "id % 100 as value") + .groupBy("key") + .agg(functions.sum("value").as("count")) + + val df2 = spark.range(0, 1000, 1, 10) + .selectExpr("(id % 10 + 5) as key", "id % 100 as value") + .groupBy("key") + .agg(functions.sum("value").as("count")) + + df1.join(df2, Seq("key")) + } + } + + test("Subquery Filter") { + doTestReplay("13[*]") { spark => + spark.range(0, 100, 1, 10) + .createTempView("df1") + + spark.range(50, 1000, 1, 10) + .createTempView("df2") + + spark.sql("select * from df1 where id > (select max(id) from df2)") + } + } + + test("Subquery in projection") { + doTestReplay("11[*]") { spark => + spark.sql( + """ + |CREATE TEMPORARY VIEW t1 + |AS SELECT * FROM VALUES + |(1, "a"), + |(2, "a"), + |(3, "a") t(id, value) + |""".stripMargin) + + spark.sql( + """ + |SELECT *, (SELECT COUNT(*) FROM t1) FROM t1 + |""".stripMargin) + } + } + + test("No broadcast join") { + doTestReplay("30[*]") { spark => + spark.conf.set(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key, "-1") + + val df1 = spark.range(0, 1000, 1, 10) + .selectExpr("id % 10 as key", "id % 100 as value") + .groupBy("key") + .agg(functions.sum("value").as("count")) + + val df2 = spark.range(0, 1000, 1, 10) + .selectExpr("(id % 10 + 5) as key", "id % 100 as value") + .groupBy("key") + .agg(functions.sum("value").as("count")) + + df1.join(df2, Seq("key")) + } + } + + test("AQE broadcast") { + doTestReplay("90[*]") { spark => + spark.conf.set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, "true") + + val df1 = spark.range(0, 1000, 1, 10) + .selectExpr("id % 10 as key", "id % 100 as value") + .groupBy("key") + .agg(functions.sum("value").as("count")) + + val df2 = spark.range(0, 1000, 1, 10) + .selectExpr("(id % 10 + 5) as key", "id % 100 as value") + .groupBy("key") + .agg(functions.sum("value").as("count")) + + df1.join(df2, Seq("key")) + } + } + + test("AQE Exchange") { + doTestReplay("28[*]") { spark => + spark.conf.set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, "true") + + spark.range(0, 1000, 1, 100) + .selectExpr("id % 10 as key", "id % 100 as value") + .groupBy("key") + .agg(functions.sum("value").as("total")) + } + } + + test("Partition only") { + withGpuSparkSession{ spark => + spark.conf.set(RapidsConf.LORE_DUMP_PATH.key, TEST_FILES_ROOT.getAbsolutePath) + spark.conf.set(RapidsConf.LORE_DUMP_IDS.key, "3[0 2]") + + val df = spark.range(0, 1000, 1, 100) + .selectExpr("id % 10 as key", "id % 100 as value") + + val res = df.collect().length + println(s"Length of original: $res") + + + val restoredRes = GpuColumnarToRowExec(GpuLore.restoreGpuExec( + new Path(s"${TEST_FILES_ROOT.getAbsolutePath}/loreId-3"), spark)) + .executeCollect() + .length + + assert(20 == restoredRes) + } + } + + private def doTestReplay(loreDumpIds: String)(dfFunc: SparkSession => DataFrame) = { + val loreId = OutputLoreId.parse(loreDumpIds).head._1 + withGpuSparkSession { spark => + spark.conf.set(RapidsConf.LORE_DUMP_PATH.key, TEST_FILES_ROOT.getAbsolutePath) + spark.conf.set(RapidsConf.LORE_DUMP_IDS.key, loreDumpIds) + + val df = dfFunc(spark) + + val expectedLength = df.collect().length + + val restoredResultLength = GpuColumnarToRowExec(GpuLore.restoreGpuExec( + new Path(s"${TEST_FILES_ROOT.getAbsolutePath}/loreId-$loreId"), + spark)) + .executeCollect() + .length + + assert(expectedLength == restoredResultLength) + } + } +} diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/lore/OutputLoreIdSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/lore/OutputLoreIdSuite.scala new file mode 100644 index 00000000000..aad3d997b9d --- /dev/null +++ b/tests/src/test/scala/com/nvidia/spark/rapids/lore/OutputLoreIdSuite.scala @@ -0,0 +1,55 @@ +/* + * Copyright (c) 2020-2024, NVIDIA CORPORATION. + * + * Licensed 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 com.nvidia.spark.rapids.lore + +import org.scalatest.funsuite.AnyFunSuite + +class OutputLoreIdSuite extends AnyFunSuite { + test("Parse one output lore id") { + val expectedLoreIds = Map(1 -> OutputLoreId(1, Set(1, 2, 4, 8))) + val loreIds = OutputLoreId.parse("1[1 2 4 8]") + + assert(loreIds == expectedLoreIds) + } + + test("Parse multi output lore id") { + val expectedLoreIds = Map( + 1 -> OutputLoreId(1, Set(1, 2, 4, 8)), + 2 -> OutputLoreId(2, Set(1, 4, 5, 6, 7, 8, 100)) + ) + val loreIds = OutputLoreId.parse("1[1 2 4 8], 2[1 4-9 100]") + + assert(loreIds == expectedLoreIds) + } + + test("Parse empty output lore id should fail") { + assertThrows[IllegalArgumentException] { + OutputLoreId.parse(" 1, 2 ") + } + } + + test("Parse mixed") { + val expectedLoreIds = Map( + 1 -> OutputLoreId(1), + 2 -> OutputLoreId(2, Set(4, 5, 8)), + 3 -> OutputLoreId(3, Set(1, 2, 4, 8)) + ) + val loreIds = OutputLoreId.parse("1[*], 2[4-6 8] , 3[1 2 4 8]") + + assert(loreIds == expectedLoreIds) + } +}