From 2d03b003c1b0cf18faae0154e5370a1bd3e829e5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9D=8E=E6=89=AC?= <654010905@qq.com> Date: Tue, 12 Nov 2024 16:16:16 +0800 Subject: [PATCH 01/35] [GLUTEN-7868][CH] Nested column pruning for Project(Filter(Generate)) (#7869) * save test configs * wip * finish dev * fix import * fix style * fix style * fix failed uts * fix failec uts * fix style --- .../backendsapi/clickhouse/CHRuleApi.scala | 1 + ...xtendedGeneratorNestedColumnAliasing.scala | 126 ++++++++++++++++++ .../hive/GlutenClickHouseHiveTableSuite.scala | 51 ++++++- .../org/apache/gluten/GlutenConfig.scala | 10 ++ 4 files changed, 187 insertions(+), 1 deletion(-) create mode 100644 backends-clickhouse/src/main/scala/org/apache/gluten/extension/ExtendedGeneratorNestedColumnAliasing.scala diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala index 4107844f322d..ccb124b61313 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala @@ -57,6 +57,7 @@ private object CHRuleApi { injector.injectResolutionRule(spark => new RewriteToDateExpresstionRule(spark)) injector.injectResolutionRule(spark => new RewriteDateTimestampComparisonRule(spark)) injector.injectOptimizerRule(spark => new CommonSubexpressionEliminateRule(spark)) + injector.injectOptimizerRule(spark => new ExtendedGeneratorNestedColumnAliasing(spark)) injector.injectOptimizerRule(spark => CHAggregateFunctionRewriteRule(spark)) injector.injectOptimizerRule(_ => CountDistinctWithoutExpand) injector.injectOptimizerRule(_ => EqualToRewrite) diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/extension/ExtendedGeneratorNestedColumnAliasing.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/ExtendedGeneratorNestedColumnAliasing.scala new file mode 100644 index 000000000000..a97e625ae618 --- /dev/null +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/ExtendedGeneratorNestedColumnAliasing.scala @@ -0,0 +1,126 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.gluten.extension + +import org.apache.gluten.GlutenConfig + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.optimizer.GeneratorNestedColumnAliasing.canPruneGenerator +import org.apache.spark.sql.catalyst.optimizer.NestedColumnAliasing +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.catalyst.trees.AlwaysProcess +import org.apache.spark.sql.internal.SQLConf + +// ExtendedGeneratorNestedColumnAliasing process Project(Filter(Generate)), +// which is ignored by vanilla spark in optimization rule: ColumnPruning +class ExtendedGeneratorNestedColumnAliasing(spark: SparkSession) + extends Rule[LogicalPlan] + with Logging { + + override def apply(plan: LogicalPlan): LogicalPlan = + plan.transformWithPruning(AlwaysProcess.fn) { + case pj @ Project(projectList, f @ Filter(condition, g: Generate)) + if canPruneGenerator(g.generator) && + GlutenConfig.getConf.enableExtendedGeneratorNestedColumnAliasing && + (SQLConf.get.nestedPruningOnExpressions || SQLConf.get.nestedSchemaPruningEnabled) => + val attrToExtractValues = NestedColumnAliasing.getAttributeToExtractValues( + projectList ++ g.generator.children :+ condition, + Seq.empty) + if (attrToExtractValues.isEmpty) { + pj + } else { + val generatorOutputSet = AttributeSet(g.qualifiedGeneratorOutput) + val (_, attrToExtractValuesNotOnGenerator) = + attrToExtractValues.partition { + case (attr, _) => + attr.references.subsetOf(generatorOutputSet) + } + + val pushedThrough = rewritePlanWithAliases(pj, attrToExtractValuesNotOnGenerator) + pushedThrough + } + case p => + p + } + + private def rewritePlanWithAliases( + plan: LogicalPlan, + attributeToExtractValues: Map[Attribute, Seq[ExtractValue]]): LogicalPlan = { + val attributeToExtractValuesAndAliases = + attributeToExtractValues.map { + case (attr, evSeq) => + val evAliasSeq = evSeq.map { + ev => + val fieldName = ev match { + case g: GetStructField => g.extractFieldName + case g: GetArrayStructFields => g.field.name + } + ev -> Alias(ev, s"_extract_$fieldName")() + } + + attr -> evAliasSeq + } + + val nestedFieldToAlias = attributeToExtractValuesAndAliases.values.flatten.map { + case (field, alias) => field.canonicalized -> alias + }.toMap + + // A reference attribute can have multiple aliases for nested fields. + val attrToAliases = + AttributeMap(attributeToExtractValuesAndAliases.mapValues(_.map(_._2)).toSeq) + + plan match { + // Project(Filter(Generate)) + case p @ Project(projectList, child) + if child + .isInstanceOf[Filter] && child.asInstanceOf[Filter].child.isInstanceOf[Generate] => + val f = child.asInstanceOf[Filter] + val g = f.child.asInstanceOf[Generate] + + val newProjectList = NestedColumnAliasing.getNewProjectList(projectList, nestedFieldToAlias) + val newCondition = getNewExpression(f.condition, nestedFieldToAlias) + val newGenerator = getNewExpression(g.generator, nestedFieldToAlias).asInstanceOf[Generator] + + val tmpG = NestedColumnAliasing + .replaceWithAliases(g, nestedFieldToAlias, attrToAliases) + .asInstanceOf[Generate] + val newG = Generate( + newGenerator, + tmpG.unrequiredChildIndex, + tmpG.outer, + tmpG.qualifier, + tmpG.generatorOutput, + tmpG.children.head) + val newF = Filter(newCondition, newG) + val newP = Project(newProjectList, newF) + newP + case _ => plan + } + } + + private def getNewExpression( + expr: Expression, + nestedFieldToAlias: Map[Expression, Alias]): Expression = { + expr.transform { + case f: ExtractValue if nestedFieldToAlias.contains(f.canonicalized) => + nestedFieldToAlias(f.canonicalized).toAttribute + } + } +} diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseHiveTableSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseHiveTableSuite.scala index 8d311614c7de..ff2d13996dc6 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseHiveTableSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseHiveTableSuite.scala @@ -17,7 +17,7 @@ package org.apache.gluten.execution.hive import org.apache.gluten.GlutenConfig -import org.apache.gluten.execution.{GlutenClickHouseWholeStageTransformerSuite, ProjectExecTransformer, TransformSupport} +import org.apache.gluten.execution.{FileSourceScanExecTransformer, GlutenClickHouseWholeStageTransformerSuite, ProjectExecTransformer, TransformSupport} import org.apache.gluten.test.AllDataTypesWithComplexType import org.apache.gluten.utils.UTSystemParameters @@ -28,6 +28,7 @@ import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.datasources.v2.clickhouse.ClickHouseConfig import org.apache.spark.sql.hive.HiveTableScanExecTransformer import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.StructType import org.apache.hadoop.fs.Path @@ -1450,4 +1451,52 @@ class GlutenClickHouseHiveTableSuite spark.sql("DROP TABLE test_tbl_7054") } + test("Nested column pruning for Project(Filter(Generate))") { + spark.sql("drop table if exists aj") + spark.sql( + """ + |CREATE TABLE if not exists aj ( + | country STRING, + | event STRUCT, event_id:STRING, event_info:MAP> + |) + |USING orc + """.stripMargin) + + spark.sql(""" + |INSERT INTO aj VALUES + | ('USA', named_struct('time', 1622547800, 'lng', -122, 'lat', 37, 'net', + | 'wifi', 'log_extra', map('key1', 'value1'), 'event_id', 'event1', + | 'event_info', map('tab_type', '5', 'action', '13'))), + | ('Canada', named_struct('time', 1622547801, 'lng', -79, 'lat', 43, 'net', + | '4g', 'log_extra', map('key2', 'value2'), 'event_id', 'event2', + | 'event_info', map('tab_type', '4', 'action', '12'))) + """.stripMargin) + + val df = + spark.sql(""" + | SELECT * FROM ( + | SELECT + | game_name, + | CASE WHEN + | event.event_info['tab_type'] IN (5) THEN '1' ELSE '0' END AS entrance + | FROM aj + | LATERAL VIEW explode(split(nvl(event.event_info['game_name'],'0'),',')) + | as game_name + | WHERE event.event_info['action'] IN (13) + |) WHERE game_name = 'xxx' + """.stripMargin) + + val scan = df.queryExecution.executedPlan.collect { + case scan: FileSourceScanExecTransformer => scan + }.head + + val schema = scan.schema + assert(schema.size == 1) + val fieldType = schema.fields.head.dataType.asInstanceOf[StructType] + assert(fieldType.size == 1) + + spark.sql("drop table if exists aj") + } + } diff --git a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala index 3e491eb2753f..6579e30dab8a 100644 --- a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala +++ b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala @@ -107,6 +107,9 @@ class GlutenConfig(conf: SQLConf) extends Logging { def enableCountDistinctWithoutExpand: Boolean = conf.getConf(ENABLE_COUNT_DISTINCT_WITHOUT_EXPAND) + def enableExtendedGeneratorNestedColumnAliasing: Boolean = + conf.getConf(ENABLE_EXTENDED_GENERATOR_NESTED_COLUMN_ALIASING) + def veloxOrcScanEnabled: Boolean = conf.getConf(VELOX_ORC_SCAN_ENABLED) @@ -1929,6 +1932,13 @@ object GlutenConfig { .booleanConf .createWithDefault(false) + val ENABLE_EXTENDED_GENERATOR_NESTED_COLUMN_ALIASING = + buildConf("spark.gluten.sql.extendedGeneratorNestedColumnAliasing") + .internal() + .doc("Do nested column aliasing for Project(Filter(Generator))") + .booleanConf + .createWithDefault(true) + val COLUMNAR_VELOX_BLOOM_FILTER_EXPECTED_NUM_ITEMS = buildConf("spark.gluten.sql.columnar.backend.velox.bloomFilter.expectedNumItems") .internal() From f0d54412e831fc2075fdd89d8281599b61ec1844 Mon Sep 17 00:00:00 2001 From: Hongze Zhang Date: Tue, 12 Nov 2024 18:09:02 +0800 Subject: [PATCH 02/35] [CORE] Consolidate RewriteSparkPlanRulesManager, AddFallbackTagRule, TransformPreOverrides into a single rule (#7918) --- .../backendsapi/clickhouse/CHRuleApi.scala | 21 +- .../clickhouse/CHSparkPlanExecApi.scala | 15 +- .../FallbackBroadcastHashJoinRules.scala | 1 - .../backendsapi/velox/VeloxRuleApi.scala | 18 +- .../org/apache/gluten/backend/Backend.scala | 4 +- .../extension/GlutenSessionExtensions.scala | 4 +- .../{RuleInjector.scala => Injector.scala} | 6 +- .../apache/gluten/backendsapi/RuleApi.scala | 4 +- .../gluten/backendsapi/SubstraitBackend.scala | 4 +- .../apache/gluten/extension/GlutenPlan.scala | 11 + .../EnsureLocalSortRequirements.scala | 20 +- .../extension/columnar/FallbackRules.scala | 402 ------------------ .../extension/columnar/FallbackTag.scala | 140 ++++++ .../columnar/MiscColumnarRules.scala | 29 -- .../enumerated/EnumeratedTransform.scala | 14 +- .../columnar/enumerated/RasOffload.scala | 2 +- .../columnar/heuristic/AddFallbackTags.scala | 298 +++++++++++++ .../heuristic/HeuristicTransform.scala | 55 +++ .../columnar/heuristic/LegacyOffload.scala | 48 +++ .../{ => heuristic}/OffloadSingleNode.scala | 3 +- .../columnar/rewrite/RewriteJoin.scala | 2 +- .../RewriteSparkPlanRulesManager.scala | 13 +- .../GlutenFormatWriterInjectsBase.scala | 14 +- 23 files changed, 624 insertions(+), 504 deletions(-) rename gluten-core/src/main/scala/org/apache/gluten/extension/injector/{RuleInjector.scala => Injector.scala} (89%) create mode 100644 gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/FallbackTag.scala create mode 100644 gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/AddFallbackTags.scala create mode 100644 gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/HeuristicTransform.scala create mode 100644 gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/LegacyOffload.scala rename gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/{ => heuristic}/OffloadSingleNode.scala (99%) diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala index ccb124b61313..a1b8fbdd5bc3 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala @@ -19,10 +19,10 @@ package org.apache.gluten.backendsapi.clickhouse import org.apache.gluten.backendsapi.RuleApi import org.apache.gluten.extension._ import org.apache.gluten.extension.columnar._ -import org.apache.gluten.extension.columnar.MiscColumnarRules.{RemoveGlutenTableCacheColumnarToRow, RemoveTopmostColumnarToRow, RewriteSubqueryBroadcast, TransformPreOverrides} -import org.apache.gluten.extension.columnar.rewrite.RewriteSparkPlanRulesManager +import org.apache.gluten.extension.columnar.MiscColumnarRules.{RemoveGlutenTableCacheColumnarToRow, RemoveTopmostColumnarToRow, RewriteSubqueryBroadcast} +import org.apache.gluten.extension.columnar.heuristic.HeuristicTransform import org.apache.gluten.extension.columnar.transition.{InsertTransitions, RemoveTransitions} -import org.apache.gluten.extension.injector.{RuleInjector, SparkInjector} +import org.apache.gluten.extension.injector.{Injector, SparkInjector} import org.apache.gluten.extension.injector.GlutenInjector.{LegacyInjector, RasInjector} import org.apache.gluten.parser.{GlutenCacheFilesSqlParser, GlutenClickhouseSqlParser} import org.apache.gluten.sql.shims.SparkShimLoader @@ -31,14 +31,14 @@ import org.apache.spark.sql.catalyst.{CHAggregateFunctionRewriteRule, EqualToRew import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.delta.DeltaLogFileIndex import org.apache.spark.sql.delta.rules.CHOptimizeMetadataOnlyDeltaQuery -import org.apache.spark.sql.execution.{ColumnarCollapseTransformStages, CommandResultExec, FileSourceScanExec, GlutenFallbackReporter, RDDScanExec, SparkPlan} +import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.datasources.v2.V2CommandExec import org.apache.spark.util.SparkPlanRules class CHRuleApi extends RuleApi { import CHRuleApi._ - override def injectRules(injector: RuleInjector): Unit = { + override def injectRules(injector: Injector): Unit = { injectSpark(injector.spark) injectLegacy(injector.gluten.legacy) injectRas(injector.gluten.ras) @@ -65,8 +65,7 @@ private object CHRuleApi { } def injectLegacy(injector: LegacyInjector): Unit = { - - // Gluten columnar: Transform rules. + // Legacy: Pre-transform rules. injector.injectTransform(_ => RemoveTransitions) injector.injectTransform(_ => PushDownInputFileExpression.PreOffload) injector.injectTransform(c => FallbackOnANSIMode.apply(c.session)) @@ -74,9 +73,11 @@ private object CHRuleApi { injector.injectTransform(_ => RewriteSubqueryBroadcast()) injector.injectTransform(c => FallbackBroadcastHashJoin.apply(c.session)) injector.injectTransform(c => MergeTwoPhasesHashBaseAggregate.apply(c.session)) - injector.injectTransform(_ => intercept(RewriteSparkPlanRulesManager())) - injector.injectTransform(_ => intercept(AddFallbackTagRule())) - injector.injectTransform(_ => intercept(TransformPreOverrides())) + + // Legacy: The Legacy transform rule. + injector.injectTransform(_ => intercept(HeuristicTransform())) + + // Legacy: Post-transform rules. injector.injectTransform(_ => RemoveNativeWriteFilesSortAndProject()) injector.injectTransform(c => intercept(RewriteTransformer.apply(c.session))) injector.injectTransform(_ => PushDownFilterToScan) diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala index 190fcb13eaeb..f08f21056d14 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala @@ -23,8 +23,7 @@ import org.apache.gluten.execution._ import org.apache.gluten.expression._ import org.apache.gluten.expression.ExpressionNames.MONOTONICALLY_INCREASING_ID import org.apache.gluten.extension.ExpressionExtensionTrait -import org.apache.gluten.extension.columnar.AddFallbackTagRule -import org.apache.gluten.extension.columnar.MiscColumnarRules.TransformPreOverrides +import org.apache.gluten.extension.columnar.heuristic.HeuristicTransform import org.apache.gluten.sql.shims.SparkShimLoader import org.apache.gluten.substrait.expression.{ExpressionBuilder, ExpressionNode, WindowFunctionNode} import org.apache.gluten.utils.{CHJoinValidateUtil, UnknownJoinStrategy} @@ -224,9 +223,9 @@ class CHSparkPlanExecApi extends SparkPlanExecApi with Logging { } // FIXME: The operation happens inside ReplaceSingleNode(). // Caller may not know it adds project on top of the shuffle. - val project = TransformPreOverrides().apply( - AddFallbackTagRule().apply( - ProjectExec(plan.child.output ++ projectExpressions, plan.child))) + // FIXME: HeuristicTransform is costly. Re-applying it may cause performance issues. + val project = + HeuristicTransform()(ProjectExec(plan.child.output ++ projectExpressions, plan.child)) var newExprs = Seq[Expression]() for (i <- exprs.indices) { val pos = newExpressionsPosition(i) @@ -249,9 +248,9 @@ class CHSparkPlanExecApi extends SparkPlanExecApi with Logging { } // FIXME: The operation happens inside ReplaceSingleNode(). // Caller may not know it adds project on top of the shuffle. - val project = TransformPreOverrides().apply( - AddFallbackTagRule().apply( - ProjectExec(plan.child.output ++ projectExpressions, plan.child))) + // FIXME: HeuristicTransform is costly. Re-applying it may cause performance issues. + val project = + HeuristicTransform()(ProjectExec(plan.child.output ++ projectExpressions, plan.child)) var newOrderings = Seq[SortOrder]() for (i <- orderings.indices) { val oldOrdering = orderings(i) diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/extension/FallbackBroadcastHashJoinRules.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/FallbackBroadcastHashJoinRules.scala index 207bb0e3a4d7..d1116c857ab6 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/extension/FallbackBroadcastHashJoinRules.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/FallbackBroadcastHashJoinRules.scala @@ -19,7 +19,6 @@ package org.apache.gluten.extension import org.apache.gluten.GlutenConfig import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.extension.columnar._ -import org.apache.gluten.extension.columnar.FallbackTags.EncodeFallbackTagImplicits import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight} diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxRuleApi.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxRuleApi.scala index 3554bc5c9c01..30e92a18b23c 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxRuleApi.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxRuleApi.scala @@ -20,11 +20,11 @@ import org.apache.gluten.backendsapi.RuleApi import org.apache.gluten.datasource.ArrowConvertorRule import org.apache.gluten.extension._ import org.apache.gluten.extension.columnar._ -import org.apache.gluten.extension.columnar.MiscColumnarRules.{RemoveGlutenTableCacheColumnarToRow, RemoveTopmostColumnarToRow, RewriteSubqueryBroadcast, TransformPreOverrides} +import org.apache.gluten.extension.columnar.MiscColumnarRules.{RemoveGlutenTableCacheColumnarToRow, RemoveTopmostColumnarToRow, RewriteSubqueryBroadcast} import org.apache.gluten.extension.columnar.enumerated.EnumeratedTransform -import org.apache.gluten.extension.columnar.rewrite.RewriteSparkPlanRulesManager +import org.apache.gluten.extension.columnar.heuristic.HeuristicTransform import org.apache.gluten.extension.columnar.transition.{InsertTransitions, RemoveTransitions} -import org.apache.gluten.extension.injector.{RuleInjector, SparkInjector} +import org.apache.gluten.extension.injector.{Injector, SparkInjector} import org.apache.gluten.extension.injector.GlutenInjector.{LegacyInjector, RasInjector} import org.apache.gluten.sql.shims.SparkShimLoader @@ -33,7 +33,7 @@ import org.apache.spark.sql.execution.{ColumnarCollapseTransformStages, GlutenFa class VeloxRuleApi extends RuleApi { import VeloxRuleApi._ - override def injectRules(injector: RuleInjector): Unit = { + override def injectRules(injector: Injector): Unit = { injectSpark(injector.spark) injectLegacy(injector.gluten.legacy) injectRas(injector.gluten.ras) @@ -49,7 +49,7 @@ private object VeloxRuleApi { } def injectLegacy(injector: LegacyInjector): Unit = { - // Gluten columnar: Transform rules. + // Legacy: Pre-transform rules. injector.injectTransform(_ => RemoveTransitions) injector.injectTransform(_ => PushDownInputFileExpression.PreOffload) injector.injectTransform(c => FallbackOnANSIMode.apply(c.session)) @@ -57,9 +57,11 @@ private object VeloxRuleApi { injector.injectTransform(_ => RewriteSubqueryBroadcast()) injector.injectTransform(c => BloomFilterMightContainJointRewriteRule.apply(c.session)) injector.injectTransform(c => ArrowScanReplaceRule.apply(c.session)) - injector.injectTransform(_ => RewriteSparkPlanRulesManager()) - injector.injectTransform(_ => AddFallbackTagRule()) - injector.injectTransform(_ => TransformPreOverrides()) + + // Legacy: The Legacy transform rule. + injector.injectTransform(_ => HeuristicTransform()) + + // Legacy: Post-transform rules. injector.injectTransform(c => PartialProjectRule.apply(c.session)) injector.injectTransform(_ => RemoveNativeWriteFilesSortAndProject()) injector.injectTransform(c => RewriteTransformer.apply(c.session)) diff --git a/gluten-core/src/main/scala/org/apache/gluten/backend/Backend.scala b/gluten-core/src/main/scala/org/apache/gluten/backend/Backend.scala index ef22d97e773b..346181e140ac 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/backend/Backend.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/backend/Backend.scala @@ -17,7 +17,7 @@ package org.apache.gluten.backend import org.apache.gluten.extension.columnar.transition.{Convention, ConventionFunc} -import org.apache.gluten.extension.injector.RuleInjector +import org.apache.gluten.extension.injector.Injector import org.apache.spark.SparkContext import org.apache.spark.api.plugin.PluginContext @@ -51,7 +51,7 @@ trait Backend { def convFuncOverride(): ConventionFunc.Override = ConventionFunc.Override.Empty /** Query planner rules. */ - def injectRules(injector: RuleInjector): Unit + def injectRules(injector: Injector): Unit } object Backend { diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/GlutenSessionExtensions.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/GlutenSessionExtensions.scala index d5afc6b7e7b0..addcad8dd05c 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/GlutenSessionExtensions.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/GlutenSessionExtensions.scala @@ -18,7 +18,7 @@ package org.apache.gluten.extension import org.apache.gluten.GlutenConfig import org.apache.gluten.backend.Backend -import org.apache.gluten.extension.injector.RuleInjector +import org.apache.gluten.extension.injector.Injector import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSessionExtensions @@ -28,7 +28,7 @@ private[gluten] class GlutenSessionExtensions with Logging { import GlutenSessionExtensions._ override def apply(exts: SparkSessionExtensions): Unit = { - val injector = new RuleInjector(exts) + val injector = new Injector(exts) injector.control.disableOn { session => val glutenEnabledGlobally = session.conf diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/injector/RuleInjector.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/injector/Injector.scala similarity index 89% rename from gluten-core/src/main/scala/org/apache/gluten/extension/injector/RuleInjector.scala rename to gluten-core/src/main/scala/org/apache/gluten/extension/injector/Injector.scala index c497a24a07b7..81b5239a8aca 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/injector/RuleInjector.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/injector/Injector.scala @@ -18,8 +18,8 @@ package org.apache.gluten.extension.injector import org.apache.spark.sql.SparkSessionExtensions -/** Injector used to inject query planner rules into Spark and Gluten. */ -class RuleInjector(extensions: SparkSessionExtensions) { +/** Injector used to inject extensible components into Spark and Gluten. */ +class Injector(extensions: SparkSessionExtensions) { val control = new InjectorControl() val spark: SparkInjector = new SparkInjector(control, extensions) val gluten: GlutenInjector = new GlutenInjector(control) @@ -31,4 +31,4 @@ class RuleInjector(extensions: SparkSessionExtensions) { } } -object RuleInjector {} +object Injector {} diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/RuleApi.scala b/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/RuleApi.scala index 7c4c8577f421..745bbcb72d61 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/RuleApi.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/RuleApi.scala @@ -16,9 +16,9 @@ */ package org.apache.gluten.backendsapi -import org.apache.gluten.extension.injector.RuleInjector +import org.apache.gluten.extension.injector.Injector trait RuleApi { // Injects all Spark query planner rules used by the Gluten backend. - def injectRules(injector: RuleInjector): Unit + def injectRules(injector: Injector): Unit } diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/SubstraitBackend.scala b/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/SubstraitBackend.scala index d7785663d577..37be117105e2 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/SubstraitBackend.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/SubstraitBackend.scala @@ -17,7 +17,7 @@ package org.apache.gluten.backendsapi import org.apache.gluten.backend.Backend -import org.apache.gluten.extension.injector.RuleInjector +import org.apache.gluten.extension.injector.Injector import org.apache.spark.SparkContext import org.apache.spark.api.plugin.PluginContext @@ -35,7 +35,7 @@ trait SubstraitBackend extends Backend { final override def onExecutorShutdown(): Unit = { listenerApi().onExecutorShutdown() } - final override def injectRules(injector: RuleInjector): Unit = { + final override def injectRules(injector: Injector): Unit = { ruleApi().injectRules(injector) } def iteratorApi(): IteratorApi diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/GlutenPlan.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/GlutenPlan.scala index 06d798e50fc0..c252bacc9a18 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/GlutenPlan.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/GlutenPlan.scala @@ -21,6 +21,7 @@ import org.apache.gluten.backend.Backend import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.exception.GlutenNotSupportException import org.apache.gluten.expression.TransformerState +import org.apache.gluten.extension.columnar.FallbackTags.add import org.apache.gluten.extension.columnar.transition.Convention import org.apache.gluten.logging.LogLevelUtil import org.apache.gluten.substrait.SubstraitContext @@ -28,6 +29,7 @@ import org.apache.gluten.substrait.plan.PlanBuilder import org.apache.gluten.substrait.rel.RelNode import org.apache.gluten.test.TestStats +import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.execution.SparkPlan import com.google.common.collect.Lists @@ -50,6 +52,15 @@ object ValidationResult { def succeeded: ValidationResult = Succeeded def failed(reason: String): ValidationResult = Failed(reason) + + implicit class EncodeFallbackTagImplicits(result: ValidationResult) { + def tagOnFallback(plan: TreeNode[_]): Unit = { + if (result.ok()) { + return + } + add(plan, result) + } + } } /** Every Gluten Operator should extend this trait. */ diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/EnsureLocalSortRequirements.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/EnsureLocalSortRequirements.scala index ff989d796f8e..73dc617e0853 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/EnsureLocalSortRequirements.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/EnsureLocalSortRequirements.scala @@ -16,9 +16,7 @@ */ package org.apache.gluten.extension.columnar -import org.apache.gluten.GlutenConfig -import org.apache.gluten.extension.columnar.MiscColumnarRules.TransformPreOverrides -import org.apache.gluten.extension.columnar.rewrite.RewriteSparkPlanRulesManager +import org.apache.gluten.extension.columnar.heuristic.HeuristicTransform import org.apache.spark.sql.catalyst.expressions.SortOrder import org.apache.spark.sql.catalyst.rules.Rule @@ -33,24 +31,14 @@ import org.apache.spark.sql.execution.{SortExec, SparkPlan} * SortAggregate with the same key. So, this rule adds local sort back if necessary. */ object EnsureLocalSortRequirements extends Rule[SparkPlan] { - private lazy val offload = TransformPreOverrides.apply() + private lazy val transform = HeuristicTransform() private def addLocalSort( originalChild: SparkPlan, requiredOrdering: Seq[SortOrder]): SparkPlan = { + // FIXME: HeuristicTransform is costly. Re-applying it may cause performance issues. val newChild = SortExec(requiredOrdering, global = false, child = originalChild) - if (!GlutenConfig.getConf.enableColumnarSort) { - FallbackTags.add(newChild, "columnar Sort is not enabled in SortExec") - newChild - } else { - val rewrittenPlan = RewriteSparkPlanRulesManager.apply().apply(newChild) - if (rewrittenPlan.eq(newChild) && FallbackTags.nonEmpty(rewrittenPlan)) { - // The sort can not be offloaded - rewrittenPlan - } else { - offload.apply(rewrittenPlan) - } - } + transform.apply(newChild) } override def apply(plan: SparkPlan): SparkPlan = { diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/FallbackRules.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/FallbackRules.scala index 1268d36d3290..432ecd1584d8 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/FallbackRules.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/FallbackRules.scala @@ -17,147 +17,13 @@ package org.apache.gluten.extension.columnar import org.apache.gluten.GlutenConfig -import org.apache.gluten.backendsapi.BackendsApiManager -import org.apache.gluten.exception.GlutenNotSupportException -import org.apache.gluten.execution._ -import org.apache.gluten.extension.{GlutenPlan, ValidationResult} -import org.apache.gluten.extension.columnar.FallbackTags.EncodeFallbackTagImplicits -import org.apache.gluten.extension.columnar.validator.{Validator, Validators} -import org.apache.gluten.sql.shims.SparkShimLoader -import org.apache.spark.api.python.EvalPythonExecTransformer import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.catalyst.trees.{TreeNode, TreeNodeTag} import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.adaptive.{AQEShuffleReadExec, QueryStageExec} -import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, ObjectHashAggregateExec, SortAggregateExec} -import org.apache.spark.sql.execution.datasources.WriteFilesExec -import org.apache.spark.sql.execution.datasources.v2.BatchScanExec import org.apache.spark.sql.execution.exchange._ import org.apache.spark.sql.execution.joins._ -import org.apache.spark.sql.execution.python.{ArrowEvalPythonExec, BatchEvalPythonExec} -import org.apache.spark.sql.execution.window.{WindowExec, WindowGroupLimitExecShim} -import org.apache.spark.sql.hive.HiveTableScanExecTransformer - -import org.apache.commons.lang3.exception.ExceptionUtils - -sealed trait FallbackTag { - val stacktrace: Option[String] = - if (FallbackTags.DEBUG) { - Some(ExceptionUtils.getStackTrace(new Throwable())) - } else None - - def reason(): String -} - -object FallbackTag { - - /** A tag that stores one reason text of fall back. */ - case class Appendable(override val reason: String) extends FallbackTag - - /** - * A tag that stores reason text of fall back. Other reasons will be discarded when this tag is - * added to plan. - */ - case class Exclusive(override val reason: String) extends FallbackTag - - trait Converter[T] { - def from(obj: T): Option[FallbackTag] - } - - object Converter { - implicit def asIs[T <: FallbackTag]: Converter[T] = (tag: T) => Some(tag) - - implicit object FromString extends Converter[String] { - override def from(reason: String): Option[FallbackTag] = Some(Appendable(reason)) - } - - implicit object FromValidationResult extends Converter[ValidationResult] { - override def from(result: ValidationResult): Option[FallbackTag] = { - if (result.ok()) { - return None - } - Some(Appendable(result.reason())) - } - } - } -} - -object FallbackTags { - val TAG: TreeNodeTag[FallbackTag] = - TreeNodeTag[FallbackTag]("org.apache.gluten.FallbackTag") - - val DEBUG = false - - /** - * If true, the plan node will be guaranteed fallback to Vanilla plan node while being - * implemented. - * - * If false, the plan still has chance to be turned into "non-transformable" in any another - * validation rule. So user should not consider the plan "transformable" unless all validation - * rules are passed. - */ - def nonEmpty(plan: SparkPlan): Boolean = { - getOption(plan).nonEmpty - } - - /** - * If true, it implies the plan maybe transformable during validation phase but not guaranteed, - * since another validation rule could turn it to "non-transformable" before implementing the plan - * within Gluten transformers. If false, the plan node will be guaranteed fallback to Vanilla plan - * node while being implemented. - */ - def maybeOffloadable(plan: SparkPlan): Boolean = !nonEmpty(plan) - - def add[T](plan: TreeNode[_], t: T)(implicit converter: FallbackTag.Converter[T]): Unit = { - val tagOption = getOption(plan) - val newTagOption = converter.from(t) - - val mergedTagOption: Option[FallbackTag] = - (tagOption ++ newTagOption).reduceOption[FallbackTag] { - // New tag comes while the plan was already tagged, merge. - case (_, exclusive: FallbackTag.Exclusive) => - exclusive - case (exclusive: FallbackTag.Exclusive, _) => - exclusive - case (l: FallbackTag.Appendable, r: FallbackTag.Appendable) => - FallbackTag.Appendable(s"${l.reason}; ${r.reason}") - } - mergedTagOption - .foreach(mergedTag => plan.setTagValue(TAG, mergedTag)) - } - - def addRecursively[T](plan: TreeNode[_], t: T)(implicit - converter: FallbackTag.Converter[T]): Unit = { - plan.foreach { - case _: GlutenPlan => // ignore - case other: TreeNode[_] => add(other, t) - } - } - - def untag(plan: TreeNode[_]): Unit = { - plan.unsetTagValue(TAG) - } - - def get(plan: TreeNode[_]): FallbackTag = { - getOption(plan).getOrElse( - throw new IllegalStateException("Transform hint tag not set in plan: " + plan.toString())) - } - - def getOption(plan: TreeNode[_]): Option[FallbackTag] = { - plan.getTagValue(TAG) - } - - implicit class EncodeFallbackTagImplicits(result: ValidationResult) { - def tagOnFallback(plan: TreeNode[_]): Unit = { - if (result.ok()) { - return - } - add(plan, result) - } - } -} case class FallbackOnANSIMode(session: SparkSession) extends Rule[SparkPlan] { override def apply(plan: SparkPlan): SparkPlan = { @@ -237,271 +103,3 @@ case class FallbackMultiCodegens(session: SparkSession) extends Rule[SparkPlan] } else plan } } - -// This rule will try to convert a plan into plan transformer. -// The doValidate function will be called to check if the conversion is supported. -// If false is returned or any unsupported exception is thrown, a row guard will -// be added on the top of that plan to prevent actual conversion. -case class AddFallbackTagRule() extends Rule[SparkPlan] { - import AddFallbackTagRule._ - private val glutenConf: GlutenConfig = GlutenConfig.getConf - private val validator = Validators - .builder() - .fallbackByHint() - .fallbackIfScanOnlyWithFilterPushed(glutenConf.enableScanOnly) - .fallbackComplexExpressions() - .fallbackByBackendSettings() - .fallbackByUserOptions() - .fallbackByTestInjects() - .build() - - def apply(plan: SparkPlan): SparkPlan = { - plan.foreachUp { case p => addFallbackTag(p) } - plan - } - - private def addFallbackTag(plan: SparkPlan): Unit = { - val outcome = validator.validate(plan) - outcome match { - case Validator.Failed(reason) => - FallbackTags.add(plan, reason) - return - case Validator.Passed => - } - - try { - plan match { - case plan: BatchScanExec => - val transformer = - ScanTransformerFactory - .createBatchScanTransformer(plan, validation = true) - .asInstanceOf[BasicScanExecTransformer] - transformer.doValidate().tagOnFallback(plan) - case plan: FileSourceScanExec => - val transformer = - ScanTransformerFactory.createFileSourceScanTransformer(plan) - transformer.doValidate().tagOnFallback(plan) - case plan if HiveTableScanExecTransformer.isHiveTableScan(plan) => - HiveTableScanExecTransformer.validate(plan).tagOnFallback(plan) - case plan: ProjectExec => - val transformer = ProjectExecTransformer(plan.projectList, plan.child) - transformer.doValidate().tagOnFallback(plan) - case plan: FilterExec => - val transformer = BackendsApiManager.getSparkPlanExecApiInstance - .genFilterExecTransformer(plan.condition, plan.child) - transformer.doValidate().tagOnFallback(plan) - case plan: HashAggregateExec => - val transformer = HashAggregateExecBaseTransformer.from(plan) - transformer.doValidate().tagOnFallback(plan) - case plan: SortAggregateExec => - val transformer = HashAggregateExecBaseTransformer.from(plan) - transformer.doValidate().tagOnFallback(plan) - case plan: ObjectHashAggregateExec => - val transformer = HashAggregateExecBaseTransformer.from(plan) - transformer.doValidate().tagOnFallback(plan) - case plan: UnionExec => - val transformer = ColumnarUnionExec(plan.children) - transformer.doValidate().tagOnFallback(plan) - case plan: ExpandExec => - val transformer = ExpandExecTransformer(plan.projections, plan.output, plan.child) - transformer.doValidate().tagOnFallback(plan) - case plan: WriteFilesExec => - val transformer = WriteFilesExecTransformer( - plan.child, - plan.fileFormat, - plan.partitionColumns, - plan.bucketSpec, - plan.options, - plan.staticPartitions) - transformer.doValidate().tagOnFallback(plan) - case plan: SortExec => - val transformer = - SortExecTransformer(plan.sortOrder, plan.global, plan.child, plan.testSpillFrequency) - transformer.doValidate().tagOnFallback(plan) - case plan: ShuffleExchangeExec => - val transformer = ColumnarShuffleExchangeExec(plan, plan.child, plan.child.output) - transformer.doValidate().tagOnFallback(plan) - case plan: ShuffledHashJoinExec => - val transformer = BackendsApiManager.getSparkPlanExecApiInstance - .genShuffledHashJoinExecTransformer( - plan.leftKeys, - plan.rightKeys, - plan.joinType, - OffloadJoin.getShjBuildSide(plan), - plan.condition, - plan.left, - plan.right, - plan.isSkewJoin) - transformer.doValidate().tagOnFallback(plan) - case plan: BroadcastExchangeExec => - val transformer = ColumnarBroadcastExchangeExec(plan.mode, plan.child) - transformer.doValidate().tagOnFallback(plan) - case bhj: BroadcastHashJoinExec => - val transformer = BackendsApiManager.getSparkPlanExecApiInstance - .genBroadcastHashJoinExecTransformer( - bhj.leftKeys, - bhj.rightKeys, - bhj.joinType, - bhj.buildSide, - bhj.condition, - bhj.left, - bhj.right, - isNullAwareAntiJoin = bhj.isNullAwareAntiJoin) - transformer.doValidate().tagOnFallback(plan) - case plan: SortMergeJoinExec => - val transformer = BackendsApiManager.getSparkPlanExecApiInstance - .genSortMergeJoinExecTransformer( - plan.leftKeys, - plan.rightKeys, - plan.joinType, - plan.condition, - plan.left, - plan.right, - plan.isSkewJoin) - transformer.doValidate().tagOnFallback(plan) - case plan: CartesianProductExec => - val transformer = BackendsApiManager.getSparkPlanExecApiInstance - .genCartesianProductExecTransformer(plan.left, plan.right, plan.condition) - transformer.doValidate().tagOnFallback(plan) - case plan: BroadcastNestedLoopJoinExec => - val transformer = BackendsApiManager.getSparkPlanExecApiInstance - .genBroadcastNestedLoopJoinExecTransformer( - plan.left, - plan.right, - plan.buildSide, - plan.joinType, - plan.condition) - transformer.doValidate().tagOnFallback(plan) - case plan: WindowExec => - val transformer = WindowExecTransformer( - plan.windowExpression, - plan.partitionSpec, - plan.orderSpec, - plan.child) - transformer.doValidate().tagOnFallback(plan) - case plan if SparkShimLoader.getSparkShims.isWindowGroupLimitExec(plan) => - val windowGroupLimitPlan = SparkShimLoader.getSparkShims - .getWindowGroupLimitExecShim(plan) - .asInstanceOf[WindowGroupLimitExecShim] - val transformer = WindowGroupLimitExecTransformer( - windowGroupLimitPlan.partitionSpec, - windowGroupLimitPlan.orderSpec, - windowGroupLimitPlan.rankLikeFunction, - windowGroupLimitPlan.limit, - windowGroupLimitPlan.mode, - windowGroupLimitPlan.child - ) - transformer.doValidate().tagOnFallback(plan) - case plan: CoalesceExec => - ColumnarCoalesceExec(plan.numPartitions, plan.child) - .doValidate() - .tagOnFallback(plan) - case plan: GlobalLimitExec => - val (limit, offset) = - SparkShimLoader.getSparkShims.getLimitAndOffsetFromGlobalLimit(plan) - val transformer = LimitExecTransformer(plan.child, offset, limit) - transformer.doValidate().tagOnFallback(plan) - case plan: LocalLimitExec => - val transformer = LimitExecTransformer(plan.child, 0L, plan.limit) - transformer.doValidate().tagOnFallback(plan) - case plan: GenerateExec => - val transformer = BackendsApiManager.getSparkPlanExecApiInstance.genGenerateTransformer( - plan.generator, - plan.requiredChildOutput, - plan.outer, - plan.generatorOutput, - plan.child) - transformer.doValidate().tagOnFallback(plan) - case plan: BatchEvalPythonExec => - val transformer = EvalPythonExecTransformer(plan.udfs, plan.resultAttrs, plan.child) - transformer.doValidate().tagOnFallback(plan) - case plan: ArrowEvalPythonExec => - // When backend doesn't support ColumnarArrow or colunmnar arrow configuration not - // enabled, we will try offloading through EvalPythonExecTransformer - if ( - !BackendsApiManager.getSettings.supportColumnarArrowUdf() || - !GlutenConfig.getConf.enableColumnarArrowUDF - ) { - // Both CH and Velox will try using backend's built-in functions for calculate - val transformer = EvalPythonExecTransformer(plan.udfs, plan.resultAttrs, plan.child) - transformer.doValidate().tagOnFallback(plan) - } - case plan: TakeOrderedAndProjectExec => - val (limit, offset) = - SparkShimLoader.getSparkShims.getLimitAndOffsetFromTopK(plan) - val transformer = TakeOrderedAndProjectExecTransformer( - limit, - plan.sortOrder, - plan.projectList, - plan.child, - offset) - transformer.doValidate().tagOnFallback(plan) - case plan: SampleExec => - val transformer = - BackendsApiManager.getSparkPlanExecApiInstance.genSampleExecTransformer( - plan.lowerBound, - plan.upperBound, - plan.withReplacement, - plan.seed, - plan.child) - transformer.doValidate().tagOnFallback(plan) - case _ => - // Currently we assume a plan to be offload-able by default. - } - } catch { - case e @ (_: GlutenNotSupportException | _: UnsupportedOperationException) => - FallbackTags.add( - plan, - s"${e.getMessage}, original Spark plan is " + - s"${plan.getClass}(${plan.children.toList.map(_.getClass)})") - if (!e.isInstanceOf[GlutenNotSupportException]) { - logDebug("Just a warning. This exception perhaps needs to be fixed.", e) - } - } - } -} - -object AddFallbackTagRule { - implicit private class ValidatorBuilderImplicits(builder: Validators.Builder) { - - /** - * Fails validation on non-scan plan nodes if Gluten is running as scan-only mode. Also, passes - * validation on filter for the exception that filter + scan is detected. Because filters can be - * pushed into scan then the filter conditions will be processed only in scan. - */ - def fallbackIfScanOnlyWithFilterPushed(scanOnly: Boolean): Validators.Builder = { - builder.add(new FallbackIfScanOnlyWithFilterPushed(scanOnly)) - builder - } - } - - private class FallbackIfScanOnlyWithFilterPushed(scanOnly: Boolean) extends Validator { - override def validate(plan: SparkPlan): Validator.OutCome = { - if (!scanOnly) { - return pass() - } - // Scan-only mode - plan match { - case _: BatchScanExec => pass() - case _: FileSourceScanExec => pass() - case p if HiveTableScanExecTransformer.isHiveTableScan(p) => pass() - case filter: FilterExec => - val childIsScan = filter.child.isInstanceOf[FileSourceScanExec] || - filter.child.isInstanceOf[BatchScanExec] - if (childIsScan) { - pass() - } else { - fail(filter) - } - case other => fail(other) - } - } - } -} - -case class RemoveFallbackTagRule() extends Rule[SparkPlan] { - override def apply(plan: SparkPlan): SparkPlan = { - plan.foreach(FallbackTags.untag) - plan - } -} diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/FallbackTag.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/FallbackTag.scala new file mode 100644 index 000000000000..c2d844c19bc0 --- /dev/null +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/FallbackTag.scala @@ -0,0 +1,140 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.gluten.extension.columnar + +import org.apache.gluten.extension.{GlutenPlan, ValidationResult} + +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.catalyst.trees.{TreeNode, TreeNodeTag} +import org.apache.spark.sql.execution.SparkPlan + +import org.apache.commons.lang3.exception.ExceptionUtils + +sealed trait FallbackTag { + val stacktrace: Option[String] = + if (FallbackTags.DEBUG) { + Some(ExceptionUtils.getStackTrace(new Throwable())) + } else None + + def reason(): String +} + +object FallbackTag { + + /** A tag that stores one reason text of fall back. */ + case class Appendable(override val reason: String) extends FallbackTag + + /** + * A tag that stores reason text of fall back. Other reasons will be discarded when this tag is + * added to plan. + */ + case class Exclusive(override val reason: String) extends FallbackTag + + trait Converter[T] { + def from(obj: T): Option[FallbackTag] + } + + object Converter { + implicit def asIs[T <: FallbackTag]: Converter[T] = (tag: T) => Some(tag) + + implicit object FromString extends Converter[String] { + override def from(reason: String): Option[FallbackTag] = Some(Appendable(reason)) + } + + implicit object FromValidationResult extends Converter[ValidationResult] { + override def from(result: ValidationResult): Option[FallbackTag] = { + if (result.ok()) { + return None + } + Some(Appendable(result.reason())) + } + } + } +} + +object FallbackTags { + val TAG: TreeNodeTag[FallbackTag] = + TreeNodeTag[FallbackTag]("org.apache.gluten.FallbackTag") + + val DEBUG = false + + /** + * If true, the plan node will be guaranteed fallback to Vanilla plan node while being + * implemented. + * + * If false, the plan still has chance to be turned into "non-transformable" in any another + * validation rule. So user should not consider the plan "transformable" unless all validation + * rules are passed. + */ + def nonEmpty(plan: SparkPlan): Boolean = { + getOption(plan).nonEmpty + } + + /** + * If true, it implies the plan maybe transformable during validation phase but not guaranteed, + * since another validation rule could turn it to "non-transformable" before implementing the plan + * within Gluten transformers. If false, the plan node will be guaranteed fallback to Vanilla plan + * node while being implemented. + */ + def maybeOffloadable(plan: SparkPlan): Boolean = !nonEmpty(plan) + + def add[T](plan: TreeNode[_], t: T)(implicit converter: FallbackTag.Converter[T]): Unit = { + val tagOption = getOption(plan) + val newTagOption = converter.from(t) + + val mergedTagOption: Option[FallbackTag] = + (tagOption ++ newTagOption).reduceOption[FallbackTag] { + // New tag comes while the plan was already tagged, merge. + case (_, exclusive: FallbackTag.Exclusive) => + exclusive + case (exclusive: FallbackTag.Exclusive, _) => + exclusive + case (l: FallbackTag.Appendable, r: FallbackTag.Appendable) => + FallbackTag.Appendable(s"${l.reason}; ${r.reason}") + } + mergedTagOption + .foreach(mergedTag => plan.setTagValue(TAG, mergedTag)) + } + + def addRecursively[T](plan: TreeNode[_], t: T)(implicit + converter: FallbackTag.Converter[T]): Unit = { + plan.foreach { + case _: GlutenPlan => // ignore + case other: TreeNode[_] => add(other, t) + } + } + + def untag(plan: TreeNode[_]): Unit = { + plan.unsetTagValue(TAG) + } + + def get(plan: TreeNode[_]): FallbackTag = { + getOption(plan).getOrElse( + throw new IllegalStateException("Transform hint tag not set in plan: " + plan.toString())) + } + + def getOption(plan: TreeNode[_]): Option[FallbackTag] = { + plan.getTagValue(TAG) + } +} + +case class RemoveFallbackTagRule() extends Rule[SparkPlan] { + override def apply(plan: SparkPlan): SparkPlan = { + plan.foreach(FallbackTags.untag) + plan + } +} diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/MiscColumnarRules.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/MiscColumnarRules.scala index 11b4b8650842..a199b5920ca4 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/MiscColumnarRules.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/MiscColumnarRules.scala @@ -17,7 +17,6 @@ package org.apache.gluten.extension.columnar import org.apache.gluten.extension.columnar.transition.{ColumnarToRowLike, Transitions} -import org.apache.gluten.logging.LogLevelUtil import org.apache.gluten.utils.PlanUtil import org.apache.spark.sql.SparkSession @@ -32,34 +31,6 @@ import org.apache.spark.sql.execution.joins.BroadcastNestedLoopJoinExec import org.apache.spark.sql.internal.SQLConf object MiscColumnarRules { - object TransformPreOverrides { - def apply(): TransformPreOverrides = { - TransformPreOverrides( - List(), - List( - OffloadOthers(), - OffloadExchange(), - OffloadJoin() - ) - ) - } - } - - // This rule will conduct the conversion from Spark plan to the plan transformer. - case class TransformPreOverrides( - topDownRules: Seq[OffloadSingleNode], - bottomUpRules: Seq[OffloadSingleNode]) - extends Rule[SparkPlan] - with LogLevelUtil { - - def apply(plan: SparkPlan): SparkPlan = { - val plan0 = - topDownRules.foldLeft(plan)((p, rule) => p.transformDown { case p => rule.offload(p) }) - val plan1 = - bottomUpRules.foldLeft(plan0)((p, rule) => p.transformUp { case p => rule.offload(p) }) - plan1 - } - } // Replaces all SubqueryBroadcastExec used by sub-queries with ColumnarSubqueryBroadcastExec. // This prevents query execution from being failed by fallen-back SubqueryBroadcastExec with diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedTransform.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedTransform.scala index e6d1c4859ed4..5abc64ec374f 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedTransform.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedTransform.scala @@ -16,7 +16,7 @@ */ package org.apache.gluten.extension.columnar.enumerated -import org.apache.gluten.extension.columnar.{OffloadExchange, OffloadJoin, OffloadOthers} +import org.apache.gluten.extension.columnar.heuristic.{OffloadExchange, OffloadJoin, OffloadOthers} import org.apache.gluten.extension.columnar.transition.ConventionReq import org.apache.gluten.extension.columnar.validator.{Validator, Validators} import org.apache.gluten.logging.LogLevelUtil @@ -38,6 +38,18 @@ import org.apache.spark.sql.execution.python.EvalPythonExec import org.apache.spark.sql.execution.window.WindowExec import org.apache.spark.sql.hive.HiveTableScanExecTransformer +/** + * Rule to offload Spark query plan to Gluten query plan using a search algorithm and a defined cost + * model. + * + * The effect of this rule is similar to + * [[org.apache.gluten.extension.columnar.heuristic.HeuristicTransform]], except that the 3 stages + * in the heuristic version, known as rewrite, validate, offload, will take place together + * individually for each Spark query plan node in RAS rule + * [[org.apache.gluten.extension.columnar.enumerated.RasOffload]]. + * + * The feature requires enabling RAS to function. + */ case class EnumeratedTransform(session: SparkSession, outputsColumnar: Boolean) extends Rule[SparkPlan] with LogLevelUtil { diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/RasOffload.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/RasOffload.scala index 52798f712c12..1e97ebfbab85 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/RasOffload.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/RasOffload.scala @@ -17,7 +17,7 @@ package org.apache.gluten.extension.columnar.enumerated import org.apache.gluten.extension.GlutenPlan -import org.apache.gluten.extension.columnar.OffloadSingleNode +import org.apache.gluten.extension.columnar.heuristic.OffloadSingleNode import org.apache.gluten.extension.columnar.rewrite.RewriteSingleNode import org.apache.gluten.extension.columnar.validator.Validator import org.apache.gluten.ras.path.Pattern diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/AddFallbackTags.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/AddFallbackTags.scala new file mode 100644 index 000000000000..538132c81a2e --- /dev/null +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/AddFallbackTags.scala @@ -0,0 +1,298 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.gluten.extension.columnar.heuristic + +import org.apache.gluten.GlutenConfig +import org.apache.gluten.backendsapi.BackendsApiManager +import org.apache.gluten.exception.GlutenNotSupportException +import org.apache.gluten.execution.{BasicScanExecTransformer, ColumnarCoalesceExec, ColumnarUnionExec, ExpandExecTransformer, HashAggregateExecBaseTransformer, LimitExecTransformer, ProjectExecTransformer, ScanTransformerFactory, SortExecTransformer, TakeOrderedAndProjectExecTransformer, WindowExecTransformer, WindowGroupLimitExecTransformer, WriteFilesExecTransformer} +import org.apache.gluten.extension.columnar.FallbackTags +import org.apache.gluten.extension.columnar.validator.{Validator, Validators} +import org.apache.gluten.sql.shims.SparkShimLoader + +import org.apache.spark.api.python.EvalPythonExecTransformer +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.{CoalesceExec, ColumnarBroadcastExchangeExec, ColumnarShuffleExchangeExec, ExpandExec, FileSourceScanExec, FilterExec, GenerateExec, GlobalLimitExec, LocalLimitExec, ProjectExec, SampleExec, SortExec, SparkPlan, TakeOrderedAndProjectExec, UnionExec} +import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, ObjectHashAggregateExec, SortAggregateExec} +import org.apache.spark.sql.execution.datasources.WriteFilesExec +import org.apache.spark.sql.execution.datasources.v2.BatchScanExec +import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ShuffleExchangeExec} +import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BroadcastNestedLoopJoinExec, CartesianProductExec, ShuffledHashJoinExec, SortMergeJoinExec} +import org.apache.spark.sql.execution.python.{ArrowEvalPythonExec, BatchEvalPythonExec} +import org.apache.spark.sql.execution.window.{WindowExec, WindowGroupLimitExecShim} +import org.apache.spark.sql.hive.HiveTableScanExecTransformer + +// This rule will try to convert a plan into plan transformer. +// The doValidate function will be called to check if the conversion is supported. +// If false is returned or any unsupported exception is thrown, a row guard will +// be added on the top of that plan to prevent actual conversion. +case class AddFallbackTags() extends Rule[SparkPlan] { + import AddFallbackTags._ + private val glutenConf: GlutenConfig = GlutenConfig.getConf + private val validator = Validators + .builder() + .fallbackByHint() + .fallbackIfScanOnlyWithFilterPushed(glutenConf.enableScanOnly) + .fallbackComplexExpressions() + .fallbackByBackendSettings() + .fallbackByUserOptions() + .fallbackByTestInjects() + .build() + + def apply(plan: SparkPlan): SparkPlan = { + plan.foreachUp { case p => addFallbackTag(p) } + plan + } + + private def addFallbackTag(plan: SparkPlan): Unit = { + val outcome = validator.validate(plan) + outcome match { + case Validator.Failed(reason) => + FallbackTags.add(plan, reason) + return + case Validator.Passed => + } + + try { + plan match { + case plan: BatchScanExec => + val transformer = + ScanTransformerFactory + .createBatchScanTransformer(plan, validation = true) + .asInstanceOf[BasicScanExecTransformer] + transformer.doValidate().tagOnFallback(plan) + case plan: FileSourceScanExec => + val transformer = + ScanTransformerFactory.createFileSourceScanTransformer(plan) + transformer.doValidate().tagOnFallback(plan) + case plan if HiveTableScanExecTransformer.isHiveTableScan(plan) => + HiveTableScanExecTransformer.validate(plan).tagOnFallback(plan) + case plan: ProjectExec => + val transformer = ProjectExecTransformer(plan.projectList, plan.child) + transformer.doValidate().tagOnFallback(plan) + case plan: FilterExec => + val transformer = BackendsApiManager.getSparkPlanExecApiInstance + .genFilterExecTransformer(plan.condition, plan.child) + transformer.doValidate().tagOnFallback(plan) + case plan: HashAggregateExec => + val transformer = HashAggregateExecBaseTransformer.from(plan) + transformer.doValidate().tagOnFallback(plan) + case plan: SortAggregateExec => + val transformer = HashAggregateExecBaseTransformer.from(plan) + transformer.doValidate().tagOnFallback(plan) + case plan: ObjectHashAggregateExec => + val transformer = HashAggregateExecBaseTransformer.from(plan) + transformer.doValidate().tagOnFallback(plan) + case plan: UnionExec => + val transformer = ColumnarUnionExec(plan.children) + transformer.doValidate().tagOnFallback(plan) + case plan: ExpandExec => + val transformer = ExpandExecTransformer(plan.projections, plan.output, plan.child) + transformer.doValidate().tagOnFallback(plan) + case plan: WriteFilesExec => + val transformer = WriteFilesExecTransformer( + plan.child, + plan.fileFormat, + plan.partitionColumns, + plan.bucketSpec, + plan.options, + plan.staticPartitions) + transformer.doValidate().tagOnFallback(plan) + case plan: SortExec => + val transformer = + SortExecTransformer(plan.sortOrder, plan.global, plan.child, plan.testSpillFrequency) + transformer.doValidate().tagOnFallback(plan) + case plan: ShuffleExchangeExec => + val transformer = ColumnarShuffleExchangeExec(plan, plan.child, plan.child.output) + transformer.doValidate().tagOnFallback(plan) + case plan: ShuffledHashJoinExec => + val transformer = BackendsApiManager.getSparkPlanExecApiInstance + .genShuffledHashJoinExecTransformer( + plan.leftKeys, + plan.rightKeys, + plan.joinType, + OffloadJoin.getShjBuildSide(plan), + plan.condition, + plan.left, + plan.right, + plan.isSkewJoin) + transformer.doValidate().tagOnFallback(plan) + case plan: BroadcastExchangeExec => + val transformer = ColumnarBroadcastExchangeExec(plan.mode, plan.child) + transformer.doValidate().tagOnFallback(plan) + case bhj: BroadcastHashJoinExec => + val transformer = BackendsApiManager.getSparkPlanExecApiInstance + .genBroadcastHashJoinExecTransformer( + bhj.leftKeys, + bhj.rightKeys, + bhj.joinType, + bhj.buildSide, + bhj.condition, + bhj.left, + bhj.right, + isNullAwareAntiJoin = bhj.isNullAwareAntiJoin) + transformer.doValidate().tagOnFallback(plan) + case plan: SortMergeJoinExec => + val transformer = BackendsApiManager.getSparkPlanExecApiInstance + .genSortMergeJoinExecTransformer( + plan.leftKeys, + plan.rightKeys, + plan.joinType, + plan.condition, + plan.left, + plan.right, + plan.isSkewJoin) + transformer.doValidate().tagOnFallback(plan) + case plan: CartesianProductExec => + val transformer = BackendsApiManager.getSparkPlanExecApiInstance + .genCartesianProductExecTransformer(plan.left, plan.right, plan.condition) + transformer.doValidate().tagOnFallback(plan) + case plan: BroadcastNestedLoopJoinExec => + val transformer = BackendsApiManager.getSparkPlanExecApiInstance + .genBroadcastNestedLoopJoinExecTransformer( + plan.left, + plan.right, + plan.buildSide, + plan.joinType, + plan.condition) + transformer.doValidate().tagOnFallback(plan) + case plan: WindowExec => + val transformer = WindowExecTransformer( + plan.windowExpression, + plan.partitionSpec, + plan.orderSpec, + plan.child) + transformer.doValidate().tagOnFallback(plan) + case plan if SparkShimLoader.getSparkShims.isWindowGroupLimitExec(plan) => + val windowGroupLimitPlan = SparkShimLoader.getSparkShims + .getWindowGroupLimitExecShim(plan) + .asInstanceOf[WindowGroupLimitExecShim] + val transformer = WindowGroupLimitExecTransformer( + windowGroupLimitPlan.partitionSpec, + windowGroupLimitPlan.orderSpec, + windowGroupLimitPlan.rankLikeFunction, + windowGroupLimitPlan.limit, + windowGroupLimitPlan.mode, + windowGroupLimitPlan.child + ) + transformer.doValidate().tagOnFallback(plan) + case plan: CoalesceExec => + ColumnarCoalesceExec(plan.numPartitions, plan.child) + .doValidate() + .tagOnFallback(plan) + case plan: GlobalLimitExec => + val (limit, offset) = + SparkShimLoader.getSparkShims.getLimitAndOffsetFromGlobalLimit(plan) + val transformer = LimitExecTransformer(plan.child, offset, limit) + transformer.doValidate().tagOnFallback(plan) + case plan: LocalLimitExec => + val transformer = LimitExecTransformer(plan.child, 0L, plan.limit) + transformer.doValidate().tagOnFallback(plan) + case plan: GenerateExec => + val transformer = BackendsApiManager.getSparkPlanExecApiInstance.genGenerateTransformer( + plan.generator, + plan.requiredChildOutput, + plan.outer, + plan.generatorOutput, + plan.child) + transformer.doValidate().tagOnFallback(plan) + case plan: BatchEvalPythonExec => + val transformer = EvalPythonExecTransformer(plan.udfs, plan.resultAttrs, plan.child) + transformer.doValidate().tagOnFallback(plan) + case plan: ArrowEvalPythonExec => + // When backend doesn't support ColumnarArrow or colunmnar arrow configuration not + // enabled, we will try offloading through EvalPythonExecTransformer + if ( + !BackendsApiManager.getSettings.supportColumnarArrowUdf() || + !GlutenConfig.getConf.enableColumnarArrowUDF + ) { + // Both CH and Velox will try using backend's built-in functions for calculate + val transformer = EvalPythonExecTransformer(plan.udfs, plan.resultAttrs, plan.child) + transformer.doValidate().tagOnFallback(plan) + } + case plan: TakeOrderedAndProjectExec => + val (limit, offset) = + SparkShimLoader.getSparkShims.getLimitAndOffsetFromTopK(plan) + val transformer = TakeOrderedAndProjectExecTransformer( + limit, + plan.sortOrder, + plan.projectList, + plan.child, + offset) + transformer.doValidate().tagOnFallback(plan) + case plan: SampleExec => + val transformer = + BackendsApiManager.getSparkPlanExecApiInstance.genSampleExecTransformer( + plan.lowerBound, + plan.upperBound, + plan.withReplacement, + plan.seed, + plan.child) + transformer.doValidate().tagOnFallback(plan) + case _ => + // Currently we assume a plan to be offload-able by default. + } + } catch { + case e @ (_: GlutenNotSupportException | _: UnsupportedOperationException) => + FallbackTags.add( + plan, + s"${e.getMessage}, original Spark plan is " + + s"${plan.getClass}(${plan.children.toList.map(_.getClass)})") + if (!e.isInstanceOf[GlutenNotSupportException]) { + logDebug("Just a warning. This exception perhaps needs to be fixed.", e) + } + } + } +} + +object AddFallbackTags { + implicit private class ValidatorBuilderImplicits(builder: Validators.Builder) { + + /** + * Fails validation on non-scan plan nodes if Gluten is running as scan-only mode. Also, passes + * validation on filter for the exception that filter + scan is detected. Because filters can be + * pushed into scan then the filter conditions will be processed only in scan. + */ + def fallbackIfScanOnlyWithFilterPushed(scanOnly: Boolean): Validators.Builder = { + builder.add(new FallbackIfScanOnlyWithFilterPushed(scanOnly)) + builder + } + } + + private class FallbackIfScanOnlyWithFilterPushed(scanOnly: Boolean) extends Validator { + override def validate(plan: SparkPlan): Validator.OutCome = { + if (!scanOnly) { + return pass() + } + // Scan-only mode + plan match { + case _: BatchScanExec => pass() + case _: FileSourceScanExec => pass() + case p if HiveTableScanExecTransformer.isHiveTableScan(p) => pass() + case filter: FilterExec => + val childIsScan = filter.child.isInstanceOf[FileSourceScanExec] || + filter.child.isInstanceOf[BatchScanExec] + if (childIsScan) { + pass() + } else { + fail(filter) + } + case other => fail(other) + } + } + } +} diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/HeuristicTransform.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/HeuristicTransform.scala new file mode 100644 index 000000000000..8a026326e5de --- /dev/null +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/HeuristicTransform.scala @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.gluten.extension.columnar.heuristic + +import org.apache.gluten.extension.columnar.rewrite.RewriteSparkPlanRulesManager +import org.apache.gluten.logging.LogLevelUtil + +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.SparkPlan + +/** + * Rule to offload Spark query plan to Gluten query plan using programed heuristics. + * + * The procedure consists of 3 stages: + * + * 1. Rewrite 2. Validate 3. Offload + * + * In the rewrite stage, planner will try converting the Spark query plan to various forms of + * possible alternative Spark query plans, then choose the optimal one to send to next stage. During + * which, the same validation code that is about to be used in stage 2 might be invoked early to + * predict on the estimate "cost" of an alternative Spark query plan. + * + * Once the plan is rewritten, query planner will call native validation code in stage 2 to + * determine which part of the plan is offload-able or not, then add fallback tags to the + * non-offload-able parts. + * + * In stage 3, query planner will convert the offload-able Spark plan nodes into Gluten plan nodes. + */ +case class HeuristicTransform() extends Rule[SparkPlan] with LogLevelUtil { + + private val validate = AddFallbackTags() + private val rewrite = RewriteSparkPlanRulesManager(validate) + private val offload = LegacyOffload() + + override def apply(plan: SparkPlan): SparkPlan = { + Seq(rewrite, validate, offload).foldLeft(plan) { + case (plan, stage) => + stage(plan) + } + } +} diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/LegacyOffload.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/LegacyOffload.scala new file mode 100644 index 000000000000..9cf08ff303cd --- /dev/null +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/LegacyOffload.scala @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.gluten.extension.columnar.heuristic + +import org.apache.gluten.logging.LogLevelUtil + +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.SparkPlan + +class LegacyOffload(topDownRules: Seq[OffloadSingleNode], bottomUpRules: Seq[OffloadSingleNode]) + extends Rule[SparkPlan] + with LogLevelUtil { + + def apply(plan: SparkPlan): SparkPlan = { + val plan0 = + topDownRules.foldLeft(plan)((p, rule) => p.transformDown { case p => rule.offload(p) }) + val plan1 = + bottomUpRules.foldLeft(plan0)((p, rule) => p.transformUp { case p => rule.offload(p) }) + plan1 + } +} + +object LegacyOffload { + def apply(): LegacyOffload = { + new LegacyOffload( + List(), + List( + OffloadOthers(), + OffloadExchange(), + OffloadJoin() + ) + ) + } +} diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/OffloadSingleNode.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/OffloadSingleNode.scala similarity index 99% rename from gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/OffloadSingleNode.scala rename to gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/OffloadSingleNode.scala index 220a45f7b35c..a8c200e9be44 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/OffloadSingleNode.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/OffloadSingleNode.scala @@ -14,13 +14,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.gluten.extension.columnar +package org.apache.gluten.extension.columnar.heuristic import org.apache.gluten.GlutenConfig import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.exception.GlutenNotSupportException import org.apache.gluten.execution._ import org.apache.gluten.extension.GlutenPlan +import org.apache.gluten.extension.columnar.FallbackTags import org.apache.gluten.logging.LogLevelUtil import org.apache.gluten.sql.shims.SparkShimLoader diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteJoin.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteJoin.scala index d0cac0b29deb..5fcf37439772 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteJoin.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteJoin.scala @@ -17,7 +17,7 @@ package org.apache.gluten.extension.columnar.rewrite import org.apache.gluten.GlutenConfig -import org.apache.gluten.extension.columnar.OffloadJoin +import org.apache.gluten.extension.columnar.heuristic.OffloadJoin import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide, JoinSelectionHelper} import org.apache.spark.sql.catalyst.plans.logical.Join diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteSparkPlanRulesManager.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteSparkPlanRulesManager.scala index c6d5e1cca1f7..3b8ca0df3001 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteSparkPlanRulesManager.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteSparkPlanRulesManager.scala @@ -16,7 +16,7 @@ */ package org.apache.gluten.extension.columnar.rewrite -import org.apache.gluten.extension.columnar.{AddFallbackTagRule, FallbackTag, FallbackTags} +import org.apache.gluten.extension.columnar.{FallbackTag, FallbackTags} import org.apache.gluten.sql.shims.SparkShimLoader import org.apache.spark.rdd.RDD @@ -45,7 +45,9 @@ case class RewrittenNodeWall(originalChild: SparkPlan) extends LeafExecNode { * * Note that, this rule does not touch and tag these operators who does not need to rewrite. */ -class RewriteSparkPlanRulesManager private (rewriteRules: Seq[RewriteSingleNode]) +class RewriteSparkPlanRulesManager private ( + validateRule: Rule[SparkPlan], + rewriteRules: Seq[RewriteSingleNode]) extends Rule[SparkPlan] { private def mayNeedRewrite(plan: SparkPlan): Boolean = { @@ -96,7 +98,6 @@ class RewriteSparkPlanRulesManager private (rewriteRules: Seq[RewriteSingleNode] } override def apply(plan: SparkPlan): SparkPlan = { - val addHint = AddFallbackTagRule() plan.transformUp { case origin if mayNeedRewrite(origin) => // Add a wall to avoid transforming unnecessary nodes. @@ -114,7 +115,7 @@ class RewriteSparkPlanRulesManager private (rewriteRules: Seq[RewriteSingleNode] // We do not add tag and leave it to the outside `AddFallbackTagRule`. origin } else { - addHint.apply(rewrittenPlan) + validateRule.apply(rewrittenPlan) val tag = getFallbackTagBack(rewrittenPlan) if (tag.isDefined) { // If the rewritten plan is still not transformable, return the original plan. @@ -136,7 +137,7 @@ class RewriteSparkPlanRulesManager private (rewriteRules: Seq[RewriteSingleNode] } object RewriteSparkPlanRulesManager { - def apply(): Rule[SparkPlan] = { - new RewriteSparkPlanRulesManager(RewriteSingleNode.allRules()) + def apply(validateRule: Rule[SparkPlan]): Rule[SparkPlan] = { + new RewriteSparkPlanRulesManager(validateRule, RewriteSingleNode.allRules()) } } diff --git a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/datasources/GlutenFormatWriterInjectsBase.scala b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/datasources/GlutenFormatWriterInjectsBase.scala index 9ec75aa209d1..11ac62ef5823 100644 --- a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/datasources/GlutenFormatWriterInjectsBase.scala +++ b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/datasources/GlutenFormatWriterInjectsBase.scala @@ -18,9 +18,7 @@ package org.apache.spark.sql.execution.datasources import org.apache.gluten.execution.{ProjectExecTransformer, SortExecTransformer, TransformSupport, WholeStageTransformer} import org.apache.gluten.execution.datasource.GlutenFormatWriterInjects -import org.apache.gluten.extension.columnar.AddFallbackTagRule -import org.apache.gluten.extension.columnar.MiscColumnarRules.TransformPreOverrides -import org.apache.gluten.extension.columnar.rewrite.RewriteSparkPlanRulesManager +import org.apache.gluten.extension.columnar.heuristic.HeuristicTransform import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow @@ -42,12 +40,10 @@ trait GlutenFormatWriterInjectsBase extends GlutenFormatWriterInjects { return plan.execute() } - val rules = List( - RewriteSparkPlanRulesManager(), - AddFallbackTagRule(), - TransformPreOverrides() - ) - val transformed = rules.foldLeft(plan) { case (latestPlan, rule) => rule.apply(latestPlan) } + // FIXME: HeuristicTransform is costly. Re-applying it may cause performance issues. + val transform = HeuristicTransform() + val transformed = transform(plan) + if (!transformed.isInstanceOf[TransformSupport]) { throw new IllegalStateException( "Cannot transform the SparkPlans wrapped by FileFormatWriter, " + From 5d5a963f46cf05bc2b0c3e1da7b8b07ffbb36d20 Mon Sep 17 00:00:00 2001 From: Chang chen Date: Tue, 12 Nov 2024 18:51:41 +0800 Subject: [PATCH 03/35] [CH] Revert "read data from orc file format - ignore reading except date32" by cherry pick https://github.com/ClickHouse/ClickHouse/pull/71500 (#7917) This reverts commit f498fe74850348e4ac3120da41138fe4dda122b1. --- .../gluten/execution/GlutenClickHouseExcelFormatSuite.scala | 5 ++--- cpp-ch/clickhouse.version | 2 +- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseExcelFormatSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseExcelFormatSuite.scala index a5b866cb448d..0fd868a8a878 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseExcelFormatSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseExcelFormatSuite.scala @@ -1038,13 +1038,12 @@ class GlutenClickHouseExcelFormatSuite ) } - test("read data from orc file format - except date32") { + test("read data from orc file format") { val filePath = s"$orcDataPath/all_data_types_with_non_primitive_type.snappy.orc" val orcFileFormat = "orc" val sql = s""" - | select string_field, int_field, long_field, float_field, double_field, short_field, - | byte_field, boolean_field, decimal_field + | select * | from $orcFileFormat.`$filePath` | where long_field > 30 |""".stripMargin diff --git a/cpp-ch/clickhouse.version b/cpp-ch/clickhouse.version index b5e11ca13f06..e51caf2aa788 100644 --- a/cpp-ch/clickhouse.version +++ b/cpp-ch/clickhouse.version @@ -1,3 +1,3 @@ CH_ORG=Kyligence CH_BRANCH=rebase_ch/20241111 -CH_COMMIT=c988219798d \ No newline at end of file +CH_COMMIT=06ffc32a255 From 0a89a6ac2a6636dd0348ebcf0251d2a5ab1bfa98 Mon Sep 17 00:00:00 2001 From: Yuan Date: Tue, 12 Nov 2024 20:01:20 +0800 Subject: [PATCH 04/35] [GLUTEN-6887][VL] Daily Update Velox Version (2024_11_12) (#7899) * [VL] rebase velox to 2024_11_12 Velox changes: afa657237 (HEAD, upstream/main) Enable common dictionary layer generation in expression fuzzers (#11502) 7a64fa1ff Legacy code cleanup (#11503) 592ac67a7 Extracts pipeline IDs automatically in Trace Replayer (#11492) dfc737d74 CMake: Move resolve dependency macros under velox_ prefix (#11392) cc48ac9a6 Fix MockSharedArbitrationTest.ensureMemoryPoolMaxCapacity (#11470) c8c4707f2 Fix memory arbitration fuzzer failure (#11493) 57040fd78 Fix flaky test ssdWriteOptions (#11491) f69b852c4 Make default abort time ratio small in MockSharedArbitrationTest (#11464) bed3184ba Prefix sort code cleanup and fix benchmark targets (#11490) This patch also removed the unnecessary changes in Velox patch. --------- Signed-off-by: Yuan Zhou --- ep/build-velox/src/get_velox.sh | 2 +- ep/build-velox/src/modify_velox.patch | 99 ++------------------------- 2 files changed, 8 insertions(+), 93 deletions(-) diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh index 44679ae6ee2a..3d72b950bf93 100755 --- a/ep/build-velox/src/get_velox.sh +++ b/ep/build-velox/src/get_velox.sh @@ -17,7 +17,7 @@ set -exu VELOX_REPO=https://github.com/oap-project/velox.git -VELOX_BRANCH=2024_11_11 +VELOX_BRANCH=2024_11_12 VELOX_HOME="" OS=`uname -s` diff --git a/ep/build-velox/src/modify_velox.patch b/ep/build-velox/src/modify_velox.patch index 3d45ff4b48a3..bf801ead83ef 100644 --- a/ep/build-velox/src/modify_velox.patch +++ b/ep/build-velox/src/modify_velox.patch @@ -1,66 +1,5 @@ -diff --git a/CMake/resolve_dependency_modules/arrow/CMakeLists.txt b/CMake/resolve_dependency_modules/arrow/CMakeLists.txt -index ddf1fac71..ef48ae9d9 100644 ---- a/CMake/resolve_dependency_modules/arrow/CMakeLists.txt -+++ b/CMake/resolve_dependency_modules/arrow/CMakeLists.txt -@@ -14,17 +14,13 @@ - project(Arrow) - - if(VELOX_ENABLE_ARROW) -- find_package(Thrift) -- if(Thrift_FOUND) -- set(THRIFT_SOURCE "SYSTEM") -- else() -- set(THRIFT_SOURCE "BUNDLED") -- endif() -- - set(ARROW_PREFIX "${CMAKE_CURRENT_BINARY_DIR}/arrow_ep") - set(ARROW_CMAKE_ARGS -- -DARROW_PARQUET=OFF -- -DARROW_DEPENDENCY_SOURCE=AUTO -+ -DARROW_PARQUET=ON -+ -DARROW_FILESYSTEM=ON -+ -DARROW_PROTOBUF_USE_SHARED=OFF -+ -DARROW_DEPENDENCY_USE_SHARED=OFF -+ -DARROW_DEPENDENCY_SOURCE=BUNDLED - -DARROW_WITH_THRIFT=ON - -DARROW_WITH_LZ4=ON - -DARROW_WITH_SNAPPY=ON -@@ -38,18 +34,16 @@ if(VELOX_ENABLE_ARROW) - -DCMAKE_INSTALL_PREFIX=${ARROW_PREFIX}/install - -DCMAKE_BUILD_TYPE=${CMAKE_BUILD_TYPE} - -DARROW_BUILD_STATIC=ON -- -DThrift_SOURCE=${THRIFT_SOURCE} -+ -DARROW_BUILD_SHARED=OFF - -DCMAKE_PREFIX_PATH=${CMAKE_PREFIX_PATH}) - set(ARROW_LIBDIR ${ARROW_PREFIX}/install/${CMAKE_INSTALL_LIBDIR}) - - add_library(thrift STATIC IMPORTED GLOBAL) -- if(NOT Thrift_FOUND) -- set(THRIFT_ROOT ${ARROW_PREFIX}/src/arrow_ep-build/thrift_ep-install) -- set(THRIFT_LIB ${THRIFT_ROOT}/lib/libthrift.a) -+ set(THRIFT_ROOT ${ARROW_PREFIX}/src/arrow_ep-build/thrift_ep-install) -+ set(THRIFT_LIB ${THRIFT_ROOT}/lib/libthrift.a) - -- file(MAKE_DIRECTORY ${THRIFT_ROOT}/include) -- set(THRIFT_INCLUDE_DIR ${THRIFT_ROOT}/include) -- endif() -+ file(MAKE_DIRECTORY ${THRIFT_ROOT}/include) -+ set(THRIFT_INCLUDE_DIR ${THRIFT_ROOT}/include) - - set_property(TARGET thrift PROPERTY INTERFACE_INCLUDE_DIRECTORIES - ${THRIFT_INCLUDE_DIR}) -@@ -68,6 +62,9 @@ if(VELOX_ENABLE_ARROW) - arrow_ep - PREFIX ${ARROW_PREFIX} - URL ${VELOX_ARROW_SOURCE_URL} -+ PATCH_COMMAND patch -p1 < ${CMAKE_CURRENT_SOURCE_DIR}/modify_arrow.patch -+ COMMAND patch -p1 < -+ ${CMAKE_CURRENT_SOURCE_DIR}/modify_arrow_dataset_scan_option.patch - URL_HASH ${VELOX_ARROW_BUILD_SHA256_CHECKSUM} - SOURCE_SUBDIR cpp - CMAKE_ARGS ${ARROW_CMAKE_ARGS} diff --git a/CMake/resolve_dependency_modules/simdjson.cmake b/CMake/resolve_dependency_modules/simdjson.cmake -index 69e7f2044..777eb5ec1 100644 +index ddd232050..b6ed2432b 100644 --- a/CMake/resolve_dependency_modules/simdjson.cmake +++ b/CMake/resolve_dependency_modules/simdjson.cmake @@ -29,4 +29,6 @@ FetchContent_Declare( @@ -71,40 +10,16 @@ index 69e7f2044..777eb5ec1 100644 + FetchContent_MakeAvailable(simdjson) diff --git a/CMakeLists.txt b/CMakeLists.txt -index 7f7cbc92f..52adb1250 100644 +index 7fd99b6dc..e7e03a800 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt -@@ -386,7 +391,7 @@ resolve_dependency(Boost 1.77.0 COMPONENTS ${BOOST_INCLUDE_LIBRARIES}) +@@ -428,7 +428,7 @@ velox_resolve_dependency(Boost 1.77.0 COMPONENTS ${BOOST_INCLUDE_LIBRARIES}) # for reference. find_package(range-v3) - set_source(gflags) --resolve_dependency(gflags COMPONENTS ${VELOX_GFLAGS_TYPE}) -+resolve_dependency(gflags) + velox_set_source(gflags) +-velox_resolve_dependency(gflags COMPONENTS ${VELOX_GFLAGS_TYPE}) ++velox_resolve_dependency(gflags) if(NOT TARGET gflags::gflags) # This is a bit convoluted, but we want to be able to use gflags::gflags as a # target even when velox is built as a subproject which uses -diff --git a/velox/connectors/hive/storage_adapters/abfs/RegisterAbfsFileSystem.cpp b/velox/connectors/hive/storage_adapters/abfs/RegisterAbfsFileSystem.cpp -index a8b0df52a..254c56b70 100644 ---- a/velox/connectors/hive/storage_adapters/abfs/RegisterAbfsFileSystem.cpp -+++ b/velox/connectors/hive/storage_adapters/abfs/RegisterAbfsFileSystem.cpp -@@ -38,7 +38,6 @@ std::shared_ptr abfsFileSystemGenerator( - - void registerAbfsFileSystem() { - #ifdef VELOX_ENABLE_ABFS -- LOG(INFO) << "Register ABFS"; - registerFileSystem(isAbfsFile, std::function(abfsFileSystemGenerator)); - #endif - } -diff --git a/velox/connectors/hive/storage_adapters/hdfs/tests/HdfsMiniCluster.cpp b/velox/connectors/hive/storage_adapters/hdfs/tests/HdfsMiniCluster.cpp -index 10ee508ba..027a58ecc 100644 ---- a/velox/connectors/hive/storage_adapters/hdfs/tests/HdfsMiniCluster.cpp -+++ b/velox/connectors/hive/storage_adapters/hdfs/tests/HdfsMiniCluster.cpp -@@ -72,7 +72,7 @@ HdfsMiniCluster::HdfsMiniCluster() { - "Failed to find minicluster executable {}'", miniClusterExecutableName); - } - boost::filesystem::path hadoopHomeDirectory = exePath_; -- hadoopHomeDirectory.remove_leaf().remove_leaf(); -+ hadoopHomeDirectory.remove_filename().remove_filename(); - setupEnvironment(hadoopHomeDirectory.string()); - } - + \ No newline at end of file From c1a3f7b502ceafd7b5ca11117a93a295ea5d69ec Mon Sep 17 00:00:00 2001 From: Chang chen Date: Wed, 13 Nov 2024 09:32:32 +0800 Subject: [PATCH 05/35] [GLUTEN-7028][CH][Part-8] Support one pipeline write for partition mergetree (#7924) * [Refactor] simple refactor * [Refactor] Remove setStats * [Refactor] SparkPartitionedBaseSink and WriteStatsBase * [Refactor] Add explicit SparkMergeTreeWriteSettings(const DB::ContextPtr & context); * [New] Support writing partition mergetree in one pipeline --- .../GlutenClickHouseMergeTreeWriteSuite.scala | 180 +++++---- .../Parser/RelParsers/WriteRelParser.cpp | 39 +- .../Parser/SerializedPlanParser.cpp | 8 +- .../Parser/SerializedPlanParser.h | 4 +- .../Storages/MergeTree/SparkMergeTreeSink.cpp | 19 +- .../Storages/MergeTree/SparkMergeTreeSink.h | 160 ++++---- .../MergeTree/SparkMergeTreeWriteSettings.cpp | 3 +- .../MergeTree/SparkMergeTreeWriteSettings.h | 2 +- .../MergeTree/SparkStorageMergeTree.cpp | 8 +- .../Storages/Output/NormalFileWriter.cpp | 2 +- .../Storages/Output/NormalFileWriter.h | 122 +++--- .../tests/gtest_write_pipeline_mergetree.cpp | 42 +- .../tests/json/mergetree/4_one_pipeline.json | 377 ++++++++++++++++++ ...input.json => lineitem_parquet_input.json} | 0 14 files changed, 701 insertions(+), 265 deletions(-) create mode 100644 cpp-ch/local-engine/tests/json/mergetree/4_one_pipeline.json rename cpp-ch/local-engine/tests/json/mergetree/{3_one_pipeline_input.json => lineitem_parquet_input.json} (100%) diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteSuite.scala index b33a2065f665..85c8c2d92a52 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteSuite.scala @@ -162,103 +162,107 @@ class GlutenClickHouseMergeTreeWriteSuite } test("test mergetree insert overwrite") { - spark.sql(s""" - |DROP TABLE IF EXISTS lineitem_mergetree_insertoverwrite; - |""".stripMargin) + withSQLConf((CHConf.ENABLE_ONEPIPELINE_MERGETREE_WRITE.key, spark35.toString)) { + spark.sql(s""" + |DROP TABLE IF EXISTS lineitem_mergetree_insertoverwrite; + |""".stripMargin) - spark.sql(s""" - |CREATE TABLE IF NOT EXISTS lineitem_mergetree_insertoverwrite - |( - | l_orderkey bigint, - | l_partkey bigint, - | l_suppkey bigint, - | l_linenumber bigint, - | l_quantity double, - | l_extendedprice double, - | l_discount double, - | l_tax double, - | l_returnflag string, - | l_linestatus string, - | l_shipdate date, - | l_commitdate date, - | l_receiptdate date, - | l_shipinstruct string, - | l_shipmode string, - | l_comment string - |) - |USING clickhouse - |LOCATION '$basePath/lineitem_mergetree_insertoverwrite' - |""".stripMargin) + spark.sql(s""" + |CREATE TABLE IF NOT EXISTS lineitem_mergetree_insertoverwrite + |( + | l_orderkey bigint, + | l_partkey bigint, + | l_suppkey bigint, + | l_linenumber bigint, + | l_quantity double, + | l_extendedprice double, + | l_discount double, + | l_tax double, + | l_returnflag string, + | l_linestatus string, + | l_shipdate date, + | l_commitdate date, + | l_receiptdate date, + | l_shipinstruct string, + | l_shipmode string, + | l_comment string + |) + |USING clickhouse + |LOCATION '$basePath/lineitem_mergetree_insertoverwrite' + |""".stripMargin) - spark.sql(s""" - | insert into table lineitem_mergetree_insertoverwrite - | select * from lineitem - |""".stripMargin) + spark.sql(s""" + | insert into table lineitem_mergetree_insertoverwrite + | select * from lineitem + |""".stripMargin) - spark.sql(s""" - | insert overwrite table lineitem_mergetree_insertoverwrite - | select * from lineitem where mod(l_orderkey,2) = 1 - |""".stripMargin) - val sql2 = - s""" - | select count(*) from lineitem_mergetree_insertoverwrite - | - |""".stripMargin - assertResult(300001)( - // total rows should remain unchanged - spark.sql(sql2).collect().apply(0).get(0) - ) + spark.sql(s""" + | insert overwrite table lineitem_mergetree_insertoverwrite + | select * from lineitem where mod(l_orderkey,2) = 1 + |""".stripMargin) + val sql2 = + s""" + | select count(*) from lineitem_mergetree_insertoverwrite + | + |""".stripMargin + assertResult(300001)( + // total rows should remain unchanged + spark.sql(sql2).collect().apply(0).get(0) + ) + } } test("test mergetree insert overwrite partitioned table with small table, static") { - spark.sql(s""" - |DROP TABLE IF EXISTS lineitem_mergetree_insertoverwrite2; - |""".stripMargin) + withSQLConf((CHConf.ENABLE_ONEPIPELINE_MERGETREE_WRITE.key, "false")) { + spark.sql(s""" + |DROP TABLE IF EXISTS lineitem_mergetree_insertoverwrite2; + |""".stripMargin) - spark.sql(s""" - |CREATE TABLE IF NOT EXISTS lineitem_mergetree_insertoverwrite2 - |( - | l_orderkey bigint, - | l_partkey bigint, - | l_suppkey bigint, - | l_linenumber bigint, - | l_quantity double, - | l_extendedprice double, - | l_discount double, - | l_tax double, - | l_returnflag string, - | l_linestatus string, - | l_shipdate date, - | l_commitdate date, - | l_receiptdate date, - | l_shipinstruct string, - | l_shipmode string, - | l_comment string - |) - |USING clickhouse - |PARTITIONED BY (l_shipdate) - |LOCATION '$basePath/lineitem_mergetree_insertoverwrite2' - |""".stripMargin) + spark.sql(s""" + |CREATE TABLE IF NOT EXISTS lineitem_mergetree_insertoverwrite2 + |( + | l_orderkey bigint, + | l_partkey bigint, + | l_suppkey bigint, + | l_linenumber bigint, + | l_quantity double, + | l_extendedprice double, + | l_discount double, + | l_tax double, + | l_returnflag string, + | l_linestatus string, + | l_shipdate date, + | l_commitdate date, + | l_receiptdate date, + | l_shipinstruct string, + | l_shipmode string, + | l_comment string + |) + |USING clickhouse + |PARTITIONED BY (l_shipdate) + |LOCATION '$basePath/lineitem_mergetree_insertoverwrite2' + |""".stripMargin) - spark.sql(s""" - | insert into table lineitem_mergetree_insertoverwrite2 - | select * from lineitem - |""".stripMargin) + spark.sql(s""" + | insert into table lineitem_mergetree_insertoverwrite2 + | select * from lineitem + |""".stripMargin) - spark.sql( - s""" - | insert overwrite table lineitem_mergetree_insertoverwrite2 - | select * from lineitem where l_shipdate BETWEEN date'1993-02-01' AND date'1993-02-10' - |""".stripMargin) - val sql2 = - s""" - | select count(*) from lineitem_mergetree_insertoverwrite2 - | - |""".stripMargin - assertResult(2418)( - // total rows should remain unchanged - spark.sql(sql2).collect().apply(0).get(0) - ) + spark.sql( + s""" + | insert overwrite table lineitem_mergetree_insertoverwrite2 + | select * from lineitem where l_shipdate BETWEEN date'1993-02-01' AND date'1993-02-10' + |""".stripMargin) + val sql2 = + s""" + | select count(*) from lineitem_mergetree_insertoverwrite2 + | + |""".stripMargin + assertResult(2418)( + // total rows should remain unchanged + spark.sql(sql2).collect().apply(0).get(0) + ) + } } test("test mergetree insert overwrite partitioned table with small table, dynamic") { diff --git a/cpp-ch/local-engine/Parser/RelParsers/WriteRelParser.cpp b/cpp-ch/local-engine/Parser/RelParsers/WriteRelParser.cpp index ea1239a8093a..25f786a77465 100644 --- a/cpp-ch/local-engine/Parser/RelParsers/WriteRelParser.cpp +++ b/cpp-ch/local-engine/Parser/RelParsers/WriteRelParser.cpp @@ -58,15 +58,11 @@ DB::ProcessorPtr make_sink( { if (partition_by.empty()) { - auto file_sink = std::make_shared(context, base_path, "", filename, format_hint, input_header); - file_sink->setStats(stats); - return file_sink; + return std::make_shared(context, base_path, "", filename, format_hint, input_header, stats); } - auto file_sink = std::make_shared( - context, partition_by, input_header, output_header, base_path, filename, format_hint); - file_sink->setStats(stats); - return file_sink; + return std::make_shared( + context, partition_by, input_header, output_header, base_path, filename, format_hint, stats); } DB::ExpressionActionsPtr create_rename_action(const DB::Block & input, const DB::Block & output) @@ -148,25 +144,30 @@ void addMergeTreeSinkTransform( const DB::ContextPtr & context, const DB::QueryPipelineBuilderPtr & builder, const MergeTreeTable & merge_tree_table, - const DB::Block & output, - const DB::Names & /*partitionCols*/) + const DB::Block & header, + const DB::Names & partition_by) { - const DB::Settings & settings = context->getSettingsRef(); - const auto dest_storage = merge_tree_table.getStorage(context->getGlobalContext()); - StorageMetadataPtr metadata_snapshot = dest_storage->getInMemoryMetadataPtr(); - ASTPtr none; - auto sink = dest_storage->write(none, metadata_snapshot, context, false); + Chain chain; - chain.addSink(sink); - const SinkHelper & sink_helper = assert_cast(*sink).sinkHelper(); // - auto stats = std::make_shared(output, sink_helper); + auto stats = std::make_shared(header); chain.addSink(stats); // + + SparkMergeTreeWriteSettings write_settings{context}; + if (partition_by.empty()) + write_settings.partition_settings.partition_dir = SubstraitFileSink::NO_PARTITION_ID; + + auto sink = partition_by.empty() ? + SparkMergeTreeSink::create(merge_tree_table, write_settings, context->getGlobalContext(), {stats}) : + std::make_shared(header, partition_by, merge_tree_table, write_settings, context, stats); + + chain.addSource(sink); + const DB::Settings & settings = context->getSettingsRef(); chain.addSource(std::make_shared( - output, settings[Setting::min_insert_block_size_rows], settings[Setting::min_insert_block_size_bytes])); + header, settings[Setting::min_insert_block_size_rows], settings[Setting::min_insert_block_size_bytes])); chain.addSource(std::make_shared( - output, settings[Setting::min_insert_block_size_rows], settings[Setting::min_insert_block_size_bytes])); + header, settings[Setting::min_insert_block_size_rows], settings[Setting::min_insert_block_size_bytes])); builder->addChain(std::move(chain)); } diff --git a/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp b/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp index 4e49c577bf86..748ff88acbd1 100644 --- a/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp +++ b/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp @@ -283,7 +283,7 @@ QueryPlanPtr SerializedPlanParser::parseOp(const substrait::Rel & rel, std::list return query_plan; } -DB::QueryPipelineBuilderPtr SerializedPlanParser::buildQueryPipeline(DB::QueryPlan & query_plan) +DB::QueryPipelineBuilderPtr SerializedPlanParser::buildQueryPipeline(DB::QueryPlan & query_plan) const { const Settings & settings = parser_context->queryContext()->getSettingsRef(); QueryPriorities priorities; @@ -355,11 +355,7 @@ NonNullableColumnsResolver::NonNullableColumnsResolver( expression_parser = std::make_unique(parser_context); } -NonNullableColumnsResolver::~NonNullableColumnsResolver() -{ -} - -// make it simple at present, if the condition contains or, return empty for both side. +// make it simple at present if the condition contains or, return empty for both side. std::set NonNullableColumnsResolver::resolve() { collected_columns.clear(); diff --git a/cpp-ch/local-engine/Parser/SerializedPlanParser.h b/cpp-ch/local-engine/Parser/SerializedPlanParser.h index 2bcc09a8ed9e..f0ec608a330f 100644 --- a/cpp-ch/local-engine/Parser/SerializedPlanParser.h +++ b/cpp-ch/local-engine/Parser/SerializedPlanParser.h @@ -48,7 +48,7 @@ class NonNullableColumnsResolver public: explicit NonNullableColumnsResolver( const DB::Block & header_, std::shared_ptr parser_context_, const substrait::Expression & cond_rel_); - ~NonNullableColumnsResolver(); + ~NonNullableColumnsResolver() = default; // return column names std::set resolve(); @@ -76,7 +76,7 @@ class SerializedPlanParser /// visible for UT DB::QueryPlanPtr parse(const substrait::Plan & plan); std::unique_ptr createExecutor(const substrait::Plan & plan); - DB::QueryPipelineBuilderPtr buildQueryPipeline(DB::QueryPlan & query_plan); + DB::QueryPipelineBuilderPtr buildQueryPipeline(DB::QueryPlan & query_plan) const; /// std::unique_ptr createExecutor(const std::string_view plan); diff --git a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeSink.cpp b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeSink.cpp index 43a3a78295d2..de0c244f3e53 100644 --- a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeSink.cpp +++ b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeSink.cpp @@ -62,15 +62,24 @@ void SparkMergeTreeSink::onStart() void SparkMergeTreeSink::onFinish() { sink_helper->finish(context); + if (stats_.has_value()) + (*stats_)->collectStats(sink_helper->unsafeGet(), sink_helper->write_settings.partition_settings.partition_dir); } ///// -SinkHelperPtr SparkMergeTreeSink::create( - const MergeTreeTable & merge_tree_table, const SparkMergeTreeWriteSettings & write_settings_, const DB::ContextMutablePtr & context) +SinkToStoragePtr SparkMergeTreeSink::create( + const MergeTreeTable & merge_tree_table, + const SparkMergeTreeWriteSettings & write_settings_, + const DB::ContextMutablePtr & context, + const SinkStatsOption & stats) { + if (write_settings_.partition_settings.part_name_prefix.empty()) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "empty part_name_prefix is not allowed."); + auto dest_storage = merge_tree_table.getStorage(context); bool isRemoteStorage = dest_storage->getStoragePolicy()->getAnyDisk()->isRemote(); bool insert_with_local_storage = !write_settings_.insert_without_local_storage; + SinkHelperPtr sink_helper; if (insert_with_local_storage && isRemoteStorage) { auto temp = merge_tree_table.copyToDefaultPolicyStorage(context); @@ -78,10 +87,10 @@ SinkHelperPtr SparkMergeTreeSink::create( &Poco::Logger::get("SparkMergeTreeWriter"), "Create temp table {} for local merge.", temp->getStorageID().getFullNameNotQuoted()); - return std::make_shared(temp, dest_storage, write_settings_); + sink_helper = std::make_shared(temp, dest_storage, write_settings_); } - - return std::make_shared(dest_storage, write_settings_, isRemoteStorage); + sink_helper = std::make_shared(dest_storage, write_settings_, isRemoteStorage); + return std::make_shared(sink_helper, context, stats); } SinkHelper::SinkHelper(const SparkStorageMergeTreePtr & data_, const SparkMergeTreeWriteSettings & write_settings_, bool isRemoteStorage_) diff --git a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeSink.h b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeSink.h index d19e5dc4e969..065b147462f9 100644 --- a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeSink.h +++ b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeSink.h @@ -21,6 +21,7 @@ #include #include #include +#include #include namespace local_engine @@ -152,39 +153,18 @@ class CopyToRemoteSinkHelper : public SinkHelper } }; -class SparkMergeTreeSink : public DB::SinkToStorage +class MergeTreeStats : public WriteStatsBase { -public: - static SinkHelperPtr create( - const MergeTreeTable & merge_tree_table, - const SparkMergeTreeWriteSettings & write_settings_, - const DB::ContextMutablePtr & context); + DB::MutableColumns columns_; - explicit SparkMergeTreeSink(const SinkHelperPtr & sink_helper_, const ContextPtr & context_) - : SinkToStorage(sink_helper_->metadata_snapshot->getSampleBlock()), context(context_), sink_helper(sink_helper_) + enum ColumnIndex { - } - ~SparkMergeTreeSink() override = default; - - String getName() const override { return "SparkMergeTreeSink"; } - void consume(Chunk & chunk) override; - void onStart() override; - void onFinish() override; - - const SinkHelper & sinkHelper() const { return *sink_helper; } - -private: - ContextPtr context; - SinkHelperPtr sink_helper; - - int part_num = 1; -}; - - -class MergeTreeStats : public DB::ISimpleTransform -{ - bool all_chunks_processed_ = false; /// flag to determine if we have already processed all chunks - const SinkHelper & sink_helper; + part_name, + partition_id, + record_count, + marks_count, + size_in_bytes + }; static DB::Block statsHeader() { @@ -196,69 +176,109 @@ class MergeTreeStats : public DB::ISimpleTransform {BIGINT(), "size_in_bytes"}}); } - DB::Chunk final_result() const + DB::Chunk final_result() override + { + size_t rows = columns_[part_name]->size(); + return DB::Chunk(std::move(columns_), rows); + } + +public: + explicit MergeTreeStats(const DB::Block & input_header_) + : WriteStatsBase(input_header_, statsHeader()), columns_(statsHeader().cloneEmptyColumns()) { - // TODO: remove it - const std::string NO_PARTITION_ID{"__NO_PARTITION_ID__"}; + } - auto parts = sink_helper.unsafeGet(); + String getName() const override { return "MergeTreeStats"; } - const size_t size = parts.size(); - auto file_col = STRING()->createColumn(); - file_col->reserve(size); + void collectStats(const std::deque & parts, const std::string & partition) const + { + const size_t size = parts.size() + columns_[part_name]->size(); - auto partition_col = STRING()->createColumn(); - partition_col->reserve(size); + columns_[part_name]->reserve(size); + columns_[partition_id]->reserve(size); - auto countCol = BIGINT()->createColumn(); - countCol->reserve(size); - auto & countColData = static_cast &>(*countCol).getData(); + columns_[record_count]->reserve(size); + auto & countColData = static_cast &>(*columns_[record_count]).getData(); - auto marksCol = BIGINT()->createColumn(); - marksCol->reserve(size); - auto & marksColData = static_cast &>(*marksCol).getData(); + columns_[marks_count]->reserve(size); + auto & marksColData = static_cast &>(*columns_[marks_count]).getData(); - auto bytesCol = BIGINT()->createColumn(); - bytesCol->reserve(size); - auto & bytesColData = static_cast &>(*bytesCol).getData(); + columns_[size_in_bytes]->reserve(size); + auto & bytesColData = static_cast &>(*columns_[size_in_bytes]).getData(); for (const auto & part : parts) { - file_col->insertData(part->name.c_str(), part->name.size()); - partition_col->insertData(NO_PARTITION_ID.c_str(), NO_PARTITION_ID.size()); + columns_[part_name]->insertData(part->name.c_str(), part->name.size()); + columns_[partition_id]->insertData(partition.c_str(), partition.size()); + countColData.emplace_back(part->rows_count); marksColData.emplace_back(part->getMarksCount()); bytesColData.emplace_back(part->getBytesOnDisk()); } - const DB::Columns res_columns{ - std::move(file_col), std::move(partition_col), std::move(countCol), std::move(marksCol), std::move(bytesCol)}; - return DB::Chunk(res_columns, size); } +}; +class SparkMergeTreeSink : public DB::SinkToStorage +{ public: - explicit MergeTreeStats(const DB::Block & input_header_, const SinkHelper & sink_helper_) - : ISimpleTransform(input_header_, statsHeader(), true), sink_helper(sink_helper_) + using SinkStatsOption = std::optional>; + static SinkToStoragePtr create( + const MergeTreeTable & merge_tree_table, + const SparkMergeTreeWriteSettings & write_settings_, + const DB::ContextMutablePtr & context, + const SinkStatsOption & stats = {}); + + explicit SparkMergeTreeSink(const SinkHelperPtr & sink_helper_, const ContextPtr & context_, const SinkStatsOption & stats) + : SinkToStorage(sink_helper_->metadata_snapshot->getSampleBlock()), context(context_), sink_helper(sink_helper_), stats_(stats) { } - Status prepare() override - { - if (input.isFinished() && !output.isFinished() && !has_input && !all_chunks_processed_) - { - all_chunks_processed_ = true; - /// return Ready to call transform() for generating filling rows after latest chunk was processed - return Status::Ready; - } + ~SparkMergeTreeSink() override = default; + + String getName() const override { return "SparkMergeTreeSink"; } + void consume(Chunk & chunk) override; + void onStart() override; + void onFinish() override; + + const SinkHelper & sinkHelper() const { return *sink_helper; } - return ISimpleTransform::prepare(); +private: + ContextPtr context; + SinkHelperPtr sink_helper; + std::optional> stats_; + int part_num = 1; +}; + + +class SparkMergeTreePartitionedFileSink final : public SparkPartitionedBaseSink +{ + const SparkMergeTreeWriteSettings write_settings_; + MergeTreeTable table; + +public: + SparkMergeTreePartitionedFileSink( + const DB::Block & input_header, + const DB::Names & partition_by, + const MergeTreeTable & merge_tree_table, + const SparkMergeTreeWriteSettings & write_settings, + const DB::ContextPtr & context, + const std::shared_ptr & stats) + : SparkPartitionedBaseSink(context, partition_by, input_header, stats), write_settings_(write_settings), table(merge_tree_table) + { } - String getName() const override { return "MergeTreeStats"; } - void transform(DB::Chunk & chunk) override + SinkPtr createSinkForPartition(const String & partition_id) override { - if (all_chunks_processed_) - chunk = final_result(); - else - chunk = {}; + SparkMergeTreeWriteSettings write_settings{write_settings_}; + + assert(write_settings.partition_settings.partition_dir.empty()); + assert(write_settings.partition_settings.bucket_dir.empty()); + write_settings.partition_settings.part_name_prefix + = fmt::format("{}/{}", partition_id, write_settings.partition_settings.part_name_prefix); + write_settings.partition_settings.partition_dir = partition_id; + + return SparkMergeTreeSink::create( + table, write_settings, context_->getGlobalContext(), {std::dynamic_pointer_cast(stats_)}); } }; + } diff --git a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriteSettings.cpp b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriteSettings.cpp index 75472d410c97..e584b003d2c6 100644 --- a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriteSettings.cpp +++ b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriteSettings.cpp @@ -26,7 +26,8 @@ namespace local_engine IMPLEMENT_GLUTEN_SETTINGS(SparkMergeTreeWritePartitionSettings, MERGE_TREE_WRITE_RELATED_SETTINGS) -void SparkMergeTreeWriteSettings::load(const DB::ContextPtr & context) +SparkMergeTreeWriteSettings::SparkMergeTreeWriteSettings(const DB::ContextPtr & context) + : partition_settings(SparkMergeTreeWritePartitionSettings::get(context)) { const DB::Settings & settings = context->getSettingsRef(); merge_after_insert = settings.get(MERGETREE_MERGE_AFTER_INSERT).safeGet(); diff --git a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriteSettings.h b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriteSettings.h index 1fbbdbe3461d..e89b2aaf5e44 100644 --- a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriteSettings.h +++ b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriteSettings.h @@ -35,6 +35,6 @@ struct SparkMergeTreeWriteSettings size_t merge_min_size = 1024 * 1024 * 1024; size_t merge_limit_parts = 10; - void load(const DB::ContextPtr & context); + explicit SparkMergeTreeWriteSettings(const DB::ContextPtr & context); }; } \ No newline at end of file diff --git a/cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.cpp b/cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.cpp index a6a252be3d63..0be7e0d892b7 100644 --- a/cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.cpp +++ b/cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.cpp @@ -513,17 +513,11 @@ MergeTreeDataWriter::TemporaryPart SparkMergeTreeDataWriter::writeTempPart( SinkToStoragePtr SparkWriteStorageMergeTree::write( const ASTPtr &, const StorageMetadataPtr & /*storage_in_memory_metadata*/, ContextPtr context, bool /*async_insert*/) { - SparkMergeTreeWriteSettings settings{.partition_settings{SparkMergeTreeWritePartitionSettings::get(context)}}; - if (settings.partition_settings.part_name_prefix.empty()) - throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "empty part_name_prefix is not allowed."); - settings.load(context); - SinkHelperPtr sink_helper = SparkMergeTreeSink::create(table, settings, getContext()); #ifndef NDEBUG auto dest_storage = table.getStorage(getContext()); assert(dest_storage.get() == this); #endif - - return std::make_shared(sink_helper, context); + return SparkMergeTreeSink::create(table, SparkMergeTreeWriteSettings{context}, getContext()); } } diff --git a/cpp-ch/local-engine/Storages/Output/NormalFileWriter.cpp b/cpp-ch/local-engine/Storages/Output/NormalFileWriter.cpp index 8c904ab20539..d572b8538591 100644 --- a/cpp-ch/local-engine/Storages/Output/NormalFileWriter.cpp +++ b/cpp-ch/local-engine/Storages/Output/NormalFileWriter.cpp @@ -23,7 +23,7 @@ namespace local_engine { const std::string SubstraitFileSink::NO_PARTITION_ID{"__NO_PARTITION_ID__"}; -const std::string SubstraitPartitionedFileSink::DEFAULT_PARTITION_NAME{"__HIVE_DEFAULT_PARTITION__"}; +const std::string SparkPartitionedBaseSink::DEFAULT_PARTITION_NAME{"__HIVE_DEFAULT_PARTITION__"}; NormalFileWriter::NormalFileWriter(const OutputFormatFilePtr & file_, const DB::ContextPtr & context_) : file(file_), context(context_) { diff --git a/cpp-ch/local-engine/Storages/Output/NormalFileWriter.h b/cpp-ch/local-engine/Storages/Output/NormalFileWriter.h index 8a25d7b21fa6..cd9c848dd418 100644 --- a/cpp-ch/local-engine/Storages/Output/NormalFileWriter.h +++ b/cpp-ch/local-engine/Storages/Output/NormalFileWriter.h @@ -61,20 +61,51 @@ class NormalFileWriter : public NativeOutputWriter OutputFormatFilePtr createOutputFormatFile( const DB::ContextPtr & context, const std::string & file_uri, const DB::Block & preferred_schema, const std::string & format_hint); -class WriteStats : public DB::ISimpleTransform +class WriteStatsBase : public DB::ISimpleTransform { +protected: bool all_chunks_processed_ = false; /// flag to determine if we have already processed all chunks - DB::Arena partition_keys_arena_; - std::string filename_; + virtual DB::Chunk final_result() = 0; - absl::flat_hash_map file_to_count_; +public: + WriteStatsBase(const DB::Block & input_header_, const DB::Block & output_header_) + : ISimpleTransform(input_header_, output_header_, true) + { + } + + Status prepare() override + { + if (input.isFinished() && !output.isFinished() && !has_input && !all_chunks_processed_) + { + all_chunks_processed_ = true; + /// return Ready to call transform() for generating filling rows after latest chunk was processed + return Status::Ready; + } + return ISimpleTransform::prepare(); + } + + void transform(DB::Chunk & chunk) override + { + if (all_chunks_processed_) + chunk = final_result(); + else + chunk = {}; + } +}; + +class WriteStats : public WriteStatsBase +{ static DB::Block statsHeader() { return makeBlockHeader({{STRING(), "filename"}, {STRING(), "partition_id"}, {BIGINT(), "record_count"}}); } + DB::Arena partition_keys_arena_; + std::string filename_; + absl::flat_hash_map file_to_count_; - DB::Chunk final_result() const +protected: + DB::Chunk final_result() override { const size_t size = file_to_count_.size(); @@ -102,30 +133,9 @@ class WriteStats : public DB::ISimpleTransform } public: - explicit WriteStats(const DB::Block & input_header_) : ISimpleTransform(input_header_, statsHeader(), true) { } - - Status prepare() override - { - if (input.isFinished() && !output.isFinished() && !has_input && !all_chunks_processed_) - { - all_chunks_processed_ = true; - /// return Ready to call transform() for generating filling rows after latest chunk was processed - return Status::Ready; - } - - return ISimpleTransform::prepare(); - } - + explicit WriteStats(const DB::Block & input_header_) : WriteStatsBase(input_header_, statsHeader()) { } String getName() const override { return "WriteStats"; } - void transform(DB::Chunk & chunk) override - { - if (all_chunks_processed_) - chunk = final_result(); - else - chunk = {}; - } - - void addFilePath(const String & patition_id, const String & filename) + void addFilePath(const String & partition_id, const String & filename) { assert(!filename.empty()); @@ -134,9 +144,9 @@ class WriteStats : public DB::ISimpleTransform assert(filename_ == filename); - if (patition_id.empty()) + if (partition_id.empty()) return; - file_to_count_.emplace(copyStringInArena(partition_keys_arena_, patition_id), 0); + file_to_count_.emplace(copyStringInArena(partition_keys_arena_, partition_id), 0); } void collectStats(const String & file_path, size_t rows) @@ -155,7 +165,7 @@ class SubstraitFileSink final : public DB::SinkToStorage const std::string partition_id_; const std::string relative_path_; OutputFormatFile::OutputFormatPtr output_format_; - std::shared_ptr stats_{nullptr}; + std::shared_ptr stats_; static std::string makeFilename(const std::string & base_path, const std::string & partition_id, const std::string & relative) { @@ -174,22 +184,18 @@ class SubstraitFileSink final : public DB::SinkToStorage const std::string & partition_id, const std::string & relative, const std::string & format_hint, - const DB::Block & header) + const DB::Block & header, + const std::shared_ptr & stats) : SinkToStorage(header) , partition_id_(partition_id.empty() ? NO_PARTITION_ID : partition_id) , relative_path_(relative) , output_format_(createOutputFormatFile(context, makeFilename(base_path, partition_id, relative), header, format_hint) ->createOutputFormat(header)) + , stats_(std::dynamic_pointer_cast(stats)) { - } - String getName() const override { return "SubstraitFileSink"; } - - ///TODO: remove this function - void setStats(const std::shared_ptr & stats) - { - stats_ = stats; stats_->addFilePath(partition_id_, relative_path_); } + String getName() const override { return "SubstraitFileSink"; } protected: void consume(DB::Chunk & chunk) override @@ -208,7 +214,7 @@ class SubstraitFileSink final : public DB::SinkToStorage } }; -class SubstraitPartitionedFileSink final : public DB::PartitionedSink +class SparkPartitionedBaseSink : public DB::PartitionedSink { static const std::string DEFAULT_PARTITION_NAME; @@ -237,13 +243,27 @@ class SubstraitPartitionedFileSink final : public DB::PartitionedSink return DB::makeASTFunction("concat", std::move(arguments)); } -private: +protected: + DB::ContextPtr context_; + std::shared_ptr stats_; + +public: + SparkPartitionedBaseSink( + const DB::ContextPtr & context, + const DB::Names & partition_by, + const DB::Block & input_header, + const std::shared_ptr & stats) + : PartitionedSink(make_partition_expression(partition_by), context, input_header), context_(context), stats_(stats) + { + } +}; + +class SubstraitPartitionedFileSink final : public SparkPartitionedBaseSink +{ const std::string base_path_; const std::string filename_; - DB::ContextPtr context_; const DB::Block sample_block_; const std::string format_hint_; - std::shared_ptr stats_{nullptr}; public: SubstraitPartitionedFileSink( @@ -253,27 +273,23 @@ class SubstraitPartitionedFileSink final : public DB::PartitionedSink const DB::Block & sample_block, const std::string & base_path, const std::string & filename, - const std::string & format_hint) - : PartitionedSink(make_partition_expression(partition_by), context, input_header) + const std::string & format_hint, + const std::shared_ptr & stats) + : SparkPartitionedBaseSink(context, partition_by, input_header, stats) , base_path_(base_path) , filename_(filename) - , context_(context) , sample_block_(sample_block) , format_hint_(format_hint) { } + DB::SinkPtr createSinkForPartition(const String & partition_id) override { assert(stats_); const auto partition_path = fmt::format("{}/{}", partition_id, filename_); - PartitionedSink::validatePartitionKey(partition_path, true); - auto file_sink = std::make_shared(context_, base_path_, partition_id, filename_, format_hint_, sample_block_); - file_sink->setStats(stats_); - return file_sink; + validatePartitionKey(partition_path, true); + return std::make_shared(context_, base_path_, partition_id, filename_, format_hint_, sample_block_, stats_); } String getName() const override { return "SubstraitPartitionedFileSink"; } - - ///TODO: remove this function - void setStats(const std::shared_ptr & stats) { stats_ = stats; } }; } diff --git a/cpp-ch/local-engine/tests/gtest_write_pipeline_mergetree.cpp b/cpp-ch/local-engine/tests/gtest_write_pipeline_mergetree.cpp index ddbf113353b1..1ad90060f475 100644 --- a/cpp-ch/local-engine/tests/gtest_write_pipeline_mergetree.cpp +++ b/cpp-ch/local-engine/tests/gtest_write_pipeline_mergetree.cpp @@ -255,24 +255,42 @@ TEST(MergeTree, SparkMergeTree) } } -INCBIN(_3_mergetree_plan_, SOURCE_DIR "/utils/extern-local-engine/tests/json/mergetree/3_one_pipeline.json"); -INCBIN(_3_mergetree_plan_input_, SOURCE_DIR "/utils/extern-local-engine/tests/json/mergetree/3_one_pipeline_input.json"); - -TEST(MergeTree, Pipeline) +INCBIN(_3_mergetree_plan_input_, SOURCE_DIR "/utils/extern-local-engine/tests/json/mergetree/lineitem_parquet_input.json"); +namespace +{ +void writeMerge(std::string_view json_plan, + const std::string & outputPath , + const std::function & callback, std::optional input = std::nullopt) { const auto context = DB::Context::createCopy(QueryContext::globalContext()); - GlutenWriteSettings settings{.task_write_tmp_dir = "tmp/lineitem_mergetree"}; + GlutenWriteSettings settings{.task_write_tmp_dir = outputPath}; settings.set(context); SparkMergeTreeWritePartitionSettings partition_settings{.part_name_prefix = "pipline_prefix"}; partition_settings.set(context); - auto [_, local_executor] = test::create_plan_and_executor( - EMBEDDED_PLAN(_3_mergetree_plan_), replaceLocalFilesWithTPCH(EMBEDDED_PLAN(_3_mergetree_plan_input_)), context); - size_t sum = 0; + auto input_json = input.value_or(replaceLocalFilesWithTPCH(EMBEDDED_PLAN(_3_mergetree_plan_input_))); + auto [_, local_executor] = test::create_plan_and_executor(json_plan, input_json, context); + while (local_executor->hasNext()) + callback(*local_executor->nextColumnar()); +} +} +INCBIN(_3_mergetree_plan_, SOURCE_DIR "/utils/extern-local-engine/tests/json/mergetree/3_one_pipeline.json"); +INCBIN(_4_mergetree_plan_, SOURCE_DIR "/utils/extern-local-engine/tests/json/mergetree/4_one_pipeline.json"); +TEST(MergeTree, Pipeline) +{ + writeMerge(EMBEDDED_PLAN(_3_mergetree_plan_),"tmp/lineitem_mergetree",[&](const DB::Block & block) { - const Block & x = *local_executor->nextColumnar(); - EXPECT_EQ(1, x.rows()); - //-debug::headBlock("pipeline write", x); - } + EXPECT_EQ(1, block.rows()); + debug::headBlock(block); + }); +} + +TEST(MergeTree, PipelineWithPartition) +{ + writeMerge(EMBEDDED_PLAN(_4_mergetree_plan_),"tmp/lineitem_mergetree_p",[&](const DB::Block & block) + { + EXPECT_EQ(2525, block.rows()); + debug::headBlock(block); + }); } \ No newline at end of file diff --git a/cpp-ch/local-engine/tests/json/mergetree/4_one_pipeline.json b/cpp-ch/local-engine/tests/json/mergetree/4_one_pipeline.json new file mode 100644 index 000000000000..14a9b3dda2ad --- /dev/null +++ b/cpp-ch/local-engine/tests/json/mergetree/4_one_pipeline.json @@ -0,0 +1,377 @@ +{ + "relations": [ + { + "root": { + "input": { + "write": { + "namedTable": { + "advancedExtension": { + "optimization": { + "@type": "type.googleapis.com/local_engine.Write", + "common": { + "format": "mergetree" + }, + "mergetree": { + "database": "default", + "table": "lineitem_mergetree_insertoverwrite2", + "snapshotId": "1731309448915_0", + "orderByKey": "tuple()", + "storagePolicy": "default" + } + }, + "enhancement": { + "@type": "type.googleapis.com/substrait.Type", + "struct": { + "types": [ + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "fp64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "fp64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "fp64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "fp64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "date": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "date": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "date": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + } + ], + "nullability": "NULLABILITY_REQUIRED" + } + } + } + }, + "tableSchema": { + "names": [ + "l_orderkey", + "l_partkey", + "l_suppkey", + "l_linenumber", + "l_quantity", + "l_extendedprice", + "l_discount", + "l_tax", + "l_returnflag", + "l_linestatus", + "l_shipdate", + "l_commitdate", + "l_receiptdate", + "l_shipinstruct", + "l_shipmode", + "l_comment" + ], + "struct": { + "types": [ + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "fp64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "fp64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "fp64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "fp64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "date": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "date": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "date": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + } + ] + }, + "columnTypes": [ + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "PARTITION_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL" + ] + }, + "input": { + "read": { + "common": { + "direct": {} + }, + "baseSchema": { + "names": [ + "l_orderkey", + "l_partkey", + "l_suppkey", + "l_linenumber", + "l_quantity", + "l_extendedprice", + "l_discount", + "l_tax", + "l_returnflag", + "l_linestatus", + "l_shipdate", + "l_commitdate", + "l_receiptdate", + "l_shipinstruct", + "l_shipmode", + "l_comment" + ], + "struct": { + "types": [ + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "fp64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "fp64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "fp64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "fp64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "date": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "date": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "date": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + } + ] + }, + "columnTypes": [ + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL" + ] + }, + "advancedExtension": { + "optimization": { + "@type": "type.googleapis.com/google.protobuf.StringValue", + "value": "isMergeTree=0\n" + } + } + } + } + } + }, + "outputSchema": { + "nullability": "NULLABILITY_REQUIRED" + } + } + } + ] +} \ No newline at end of file diff --git a/cpp-ch/local-engine/tests/json/mergetree/3_one_pipeline_input.json b/cpp-ch/local-engine/tests/json/mergetree/lineitem_parquet_input.json similarity index 100% rename from cpp-ch/local-engine/tests/json/mergetree/3_one_pipeline_input.json rename to cpp-ch/local-engine/tests/json/mergetree/lineitem_parquet_input.json From 8b8c5b3f553b711c7e4c044195f5995d0a83119f Mon Sep 17 00:00:00 2001 From: PHILO-HE Date: Wed, 13 Nov 2024 13:09:47 +0800 Subject: [PATCH 06/35] [Minor] Fix a typo in Gluten config (#7931) --- .../scala/org/apache/gluten/execution/FallbackSuite.scala | 4 ++-- .../scala/org/apache/gluten/execution/MiscOperatorSuite.scala | 2 +- .../scala/org/apache/gluten/execution/VeloxMetricsSuite.scala | 2 +- .../sql/execution/benchmark/StreamingAggregateBenchmark.scala | 4 ++-- .../spark/sql/execution/joins/GlutenInnerJoinSuite.scala | 4 ++-- .../spark/sql/execution/joins/GlutenOuterJoinSuite.scala | 4 ++-- .../src/main/scala/org/apache/gluten/GlutenConfig.scala | 4 ++-- 7 files changed, 12 insertions(+), 12 deletions(-) diff --git a/backends-velox/src/test/scala/org/apache/gluten/execution/FallbackSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/execution/FallbackSuite.scala index d8b0ee8981e2..0f7b70bf0032 100644 --- a/backends-velox/src/test/scala/org/apache/gluten/execution/FallbackSuite.scala +++ b/backends-velox/src/test/scala/org/apache/gluten/execution/FallbackSuite.scala @@ -253,7 +253,7 @@ class FallbackSuite extends VeloxWholeStageTransformerSuite with AdaptiveSparkPl test("fallback with smj") { val sql = "SELECT /*+ SHUFFLE_MERGE(tmp1) */ * FROM tmp1 join tmp2 on tmp1.c1 = tmp2.c1" withSQLConf( - GlutenConfig.COLUMNAR_FPRCE_SHUFFLED_HASH_JOIN_ENABLED.key -> "true", + GlutenConfig.COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED.key -> "true", GlutenConfig.COLUMNAR_SHUFFLED_HASH_JOIN_ENABLED.key -> "false") { runQueryAndCompare(sql) { df => @@ -262,7 +262,7 @@ class FallbackSuite extends VeloxWholeStageTransformerSuite with AdaptiveSparkPl } } withSQLConf( - GlutenConfig.COLUMNAR_FPRCE_SHUFFLED_HASH_JOIN_ENABLED.key -> "false", + GlutenConfig.COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED.key -> "false", GlutenConfig.COLUMNAR_SORTMERGEJOIN_ENABLED.key -> "false") { runQueryAndCompare(sql) { df => diff --git a/backends-velox/src/test/scala/org/apache/gluten/execution/MiscOperatorSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/execution/MiscOperatorSuite.scala index da5184f752b3..2b42d7fb8d64 100644 --- a/backends-velox/src/test/scala/org/apache/gluten/execution/MiscOperatorSuite.scala +++ b/backends-velox/src/test/scala/org/apache/gluten/execution/MiscOperatorSuite.scala @@ -845,7 +845,7 @@ class MiscOperatorSuite extends VeloxWholeStageTransformerSuite with AdaptiveSpa withSQLConf( GlutenConfig.COLUMNAR_PREFER_STREAMING_AGGREGATE.key -> "true", - GlutenConfig.COLUMNAR_FPRCE_SHUFFLED_HASH_JOIN_ENABLED.key -> "false", + GlutenConfig.COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED.key -> "false", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1" ) { val query = diff --git a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxMetricsSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxMetricsSuite.scala index 0b74824832f2..e41f00821c27 100644 --- a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxMetricsSuite.scala +++ b/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxMetricsSuite.scala @@ -63,7 +63,7 @@ class VeloxMetricsSuite extends VeloxWholeStageTransformerSuite with AdaptiveSpa test("test sort merge join metrics") { withSQLConf( - GlutenConfig.COLUMNAR_FPRCE_SHUFFLED_HASH_JOIN_ENABLED.key -> "false", + GlutenConfig.COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED.key -> "false", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { // without preproject runQueryAndCompare( diff --git a/backends-velox/src/test/scala/org/apache/spark/sql/execution/benchmark/StreamingAggregateBenchmark.scala b/backends-velox/src/test/scala/org/apache/spark/sql/execution/benchmark/StreamingAggregateBenchmark.scala index 7f11fd043db4..77370d059caa 100644 --- a/backends-velox/src/test/scala/org/apache/spark/sql/execution/benchmark/StreamingAggregateBenchmark.scala +++ b/backends-velox/src/test/scala/org/apache/spark/sql/execution/benchmark/StreamingAggregateBenchmark.scala @@ -51,7 +51,7 @@ object StreamingAggregateBenchmark extends SqlBasedBenchmark { withSQLConf( GlutenConfig.COLUMNAR_PREFER_STREAMING_AGGREGATE.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", - GlutenConfig.COLUMNAR_FPRCE_SHUFFLED_HASH_JOIN_ENABLED.key -> "false" + GlutenConfig.COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED.key -> "false" ) { spark.sql(query).noop() } @@ -62,7 +62,7 @@ object StreamingAggregateBenchmark extends SqlBasedBenchmark { withSQLConf( GlutenConfig.COLUMNAR_PREFER_STREAMING_AGGREGATE.key -> "false", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", - GlutenConfig.COLUMNAR_FPRCE_SHUFFLED_HASH_JOIN_ENABLED.key -> "false" + GlutenConfig.COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED.key -> "false" ) { spark.sql(query).noop() } diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/joins/GlutenInnerJoinSuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/joins/GlutenInnerJoinSuite.scala index d31936211b72..8d9e82df195a 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/joins/GlutenInnerJoinSuite.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/joins/GlutenInnerJoinSuite.scala @@ -24,13 +24,13 @@ import org.apache.spark.sql.GlutenSQLTestsBaseTrait class GlutenInnerJoinSuiteForceShjOn extends InnerJoinSuite with GlutenSQLTestsBaseTrait { override def sparkConf: SparkConf = { super.sparkConf - .set(GlutenConfig.COLUMNAR_FPRCE_SHUFFLED_HASH_JOIN_ENABLED.key, "true") + .set(GlutenConfig.COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED.key, "true") } } class GlutenInnerJoinSuiteForceShjOff extends InnerJoinSuite with GlutenSQLTestsBaseTrait { override def sparkConf: SparkConf = { super.sparkConf - .set(GlutenConfig.COLUMNAR_FPRCE_SHUFFLED_HASH_JOIN_ENABLED.key, "false") + .set(GlutenConfig.COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED.key, "false") } } diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/joins/GlutenOuterJoinSuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/joins/GlutenOuterJoinSuite.scala index 9c45eae15b0a..f48939c3d012 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/joins/GlutenOuterJoinSuite.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/joins/GlutenOuterJoinSuite.scala @@ -24,13 +24,13 @@ import org.apache.spark.sql.GlutenSQLTestsBaseTrait class GlutenOuterJoinSuiteForceShjOn extends OuterJoinSuite with GlutenSQLTestsBaseTrait { override def sparkConf: SparkConf = { super.sparkConf - .set(GlutenConfig.COLUMNAR_FPRCE_SHUFFLED_HASH_JOIN_ENABLED.key, "true") + .set(GlutenConfig.COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED.key, "true") } } class GlutenOuterJoinSuiteForceShjOff extends OuterJoinSuite with GlutenSQLTestsBaseTrait { override def sparkConf: SparkConf = { super.sparkConf - .set(GlutenConfig.COLUMNAR_FPRCE_SHUFFLED_HASH_JOIN_ENABLED.key, "false") + .set(GlutenConfig.COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED.key, "false") } } diff --git a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala index 6579e30dab8a..0bfe16270e5f 100644 --- a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala +++ b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala @@ -75,7 +75,7 @@ class GlutenConfig(conf: SQLConf) extends Logging { def enableNativeColumnarToRow: Boolean = conf.getConf(COLUMNAR_COLUMNAR_TO_ROW_ENABLED) - def forceShuffledHashJoin: Boolean = conf.getConf(COLUMNAR_FPRCE_SHUFFLED_HASH_JOIN_ENABLED) + def forceShuffledHashJoin: Boolean = conf.getConf(COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED) def enableColumnarSortMergeJoin: Boolean = conf.getConf(COLUMNAR_SORTMERGEJOIN_ENABLED) @@ -942,7 +942,7 @@ object GlutenConfig { .booleanConf .createWithDefault(true) - val COLUMNAR_FPRCE_SHUFFLED_HASH_JOIN_ENABLED = + val COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED = buildConf("spark.gluten.sql.columnar.forceShuffledHashJoin") .internal() .booleanConf From f44e4c0c1f1503d1f6ce026b2f9b6f4272123e13 Mon Sep 17 00:00:00 2001 From: Rui Mo Date: Wed, 13 Nov 2024 13:12:55 +0800 Subject: [PATCH 07/35] [VL] Clean up unused variables in cpp source files (#7929) --- cpp/velox/jni/VeloxJniWrapper.cc | 1 - cpp/velox/shuffle/VeloxRssSortShuffleWriter.cc | 1 - 2 files changed, 2 deletions(-) diff --git a/cpp/velox/jni/VeloxJniWrapper.cc b/cpp/velox/jni/VeloxJniWrapper.cc index b8d2b0c3c2ff..6ea60d651a73 100644 --- a/cpp/velox/jni/VeloxJniWrapper.cc +++ b/cpp/velox/jni/VeloxJniWrapper.cc @@ -385,7 +385,6 @@ JNIEXPORT void JNICALL Java_org_apache_gluten_datasource_VeloxDataSourceJniWrapp jlong dsHandle, jlong batchHandle) { JNI_METHOD_START - auto ctx = gluten::getRuntime(env, wrapper); auto datasource = ObjectStore::retrieve(dsHandle); auto batch = ObjectStore::retrieve(batchHandle); datasource->write(batch); diff --git a/cpp/velox/shuffle/VeloxRssSortShuffleWriter.cc b/cpp/velox/shuffle/VeloxRssSortShuffleWriter.cc index 34796e378e1d..e17ad5e2f713 100644 --- a/cpp/velox/shuffle/VeloxRssSortShuffleWriter.cc +++ b/cpp/velox/shuffle/VeloxRssSortShuffleWriter.cc @@ -117,7 +117,6 @@ arrow::Status VeloxRssSortShuffleWriter::write(std::shared_ptr cb } arrow::Status VeloxRssSortShuffleWriter::evictBatch(uint32_t partitionId) { - int64_t rawSize = batch_->size(); bufferOutputStream_->seekp(0); batch_->flush(bufferOutputStream_.get()); auto buffer = bufferOutputStream_->getBuffer(); From a96a09eebdc3ef26e141edb519c9eb8dbf8de0a0 Mon Sep 17 00:00:00 2001 From: LiuNeng <1398775315@qq.com> Date: Wed, 13 Nov 2024 13:49:45 +0800 Subject: [PATCH 08/35] [CH] Fix SIGSEGV on jstring2string #7928 What changes were proposed in this pull request? replace jstring2string implementation, avoid SIGSEGV on getObjectClass How was this patch tested? (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) manual tests --- cpp-ch/local-engine/local_engine_jni.cpp | 24 ++++++++---------------- 1 file changed, 8 insertions(+), 16 deletions(-) diff --git a/cpp-ch/local-engine/local_engine_jni.cpp b/cpp-ch/local-engine/local_engine_jni.cpp index 24c5950065ea..2ec0622311be 100644 --- a/cpp-ch/local-engine/local_engine_jni.cpp +++ b/cpp-ch/local-engine/local_engine_jni.cpp @@ -74,22 +74,14 @@ static DB::ColumnWithTypeAndName getColumnFromColumnVector(JNIEnv * /*env*/, job return block->getByPosition(column_position); } -static std::string jstring2string(JNIEnv * env, jstring jStr) -{ - if (!jStr) - return ""; - - const jclass string_class = env->GetObjectClass(jStr); - const jmethodID get_bytes = env->GetMethodID(string_class, "getBytes", "(Ljava/lang/String;)[B"); - auto * const string_jbytes - = static_cast(local_engine::safeCallObjectMethod(env, jStr, get_bytes, env->NewStringUTF("UTF-8"))); - SCOPE_EXIT({ - env->DeleteLocalRef(string_jbytes); - env->DeleteLocalRef(string_class); - }); - - const auto string_jbytes_a = local_engine::getByteArrayElementsSafe(env, string_jbytes); - return {reinterpret_cast(string_jbytes_a.elems()), static_cast(string_jbytes_a.length())}; +static std::string jstring2string(JNIEnv * env, jstring string) +{ + if (string == nullptr) + return std::string(); + const char * chars = env->GetStringUTFChars(string, nullptr); + std::string ret(chars); + env->ReleaseStringUTFChars(string, chars); + return ret; } extern "C" { From e63b7d5828133e2a4c085b87d7039ac68997b27e Mon Sep 17 00:00:00 2001 From: lgbo Date: Wed, 13 Nov 2024 14:06:57 +0800 Subject: [PATCH 09/35] fixed bug in finding attributes in replacement map (#7927) --- .../apache/gluten/extension/LazyAggregateExpandRule.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/extension/LazyAggregateExpandRule.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/LazyAggregateExpandRule.scala index 86b28ab1f717..d21cbfa2505a 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/extension/LazyAggregateExpandRule.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/LazyAggregateExpandRule.scala @@ -335,7 +335,9 @@ case class LazyAggregateExpandRule(session: SparkSession) extends Rule[SparkPlan val groupingExpressions = partialAggregate.groupingExpressions .filter( - e => e.toAttribute != groupIdAttribute && attributesToReplace.contains(e.toAttribute)) + e => + !e.toAttribute.semanticEquals(groupIdAttribute) && + attributesToReplace.find(kv => kv._1.semanticEquals(e.toAttribute)).isDefined) .map(e => getReplaceAttribute(e.toAttribute, attributesToReplace)) .distinct logDebug( @@ -344,7 +346,7 @@ case class LazyAggregateExpandRule(session: SparkSession) extends Rule[SparkPlan // Remove group id column from result expressions val resultExpressions = partialAggregate.resultExpressions - .filter(_.toAttribute != groupIdAttribute) + .filter(!_.toAttribute.semanticEquals(groupIdAttribute)) .map(e => getReplaceAttribute(e.toAttribute, attributesToReplace)) logDebug( s"xxx newResultExpressions: $resultExpressions\n" + From d6c6bba877b677e00ff37e086522a492d9e64a30 Mon Sep 17 00:00:00 2001 From: Yang Zhang Date: Wed, 13 Nov 2024 14:45:24 +0800 Subject: [PATCH 10/35] Revert "[GLUTEN-7336][CORE] Bump Spark version to v3.5.3 (#7537) (#7930) Revert Spark release version from v3.5.3 to v3.5.2 due to datalake issue. --- .../workflows/util/install_spark_resources.sh | 20 ++++++------- .github/workflows/velox_backend.yml | 30 +++++++++---------- pom.xml | 6 ++-- .../sql/shims/spark35/SparkShimProvider.scala | 2 +- tools/gluten-it/pom.xml | 2 +- 5 files changed, 30 insertions(+), 30 deletions(-) diff --git a/.github/workflows/util/install_spark_resources.sh b/.github/workflows/util/install_spark_resources.sh index d5f413ee2116..dd2afec821d4 100755 --- a/.github/workflows/util/install_spark_resources.sh +++ b/.github/workflows/util/install_spark_resources.sh @@ -63,26 +63,26 @@ case "$1" in 3.5) # Spark-3.5 cd ${INSTALL_DIR} && \ - wget -nv https://archive.apache.org/dist/spark/spark-3.5.3/spark-3.5.3-bin-hadoop3.tgz && \ - tar --strip-components=1 -xf spark-3.5.3-bin-hadoop3.tgz spark-3.5.3-bin-hadoop3/jars/ && \ - rm -rf spark-3.5.3-bin-hadoop3.tgz && \ + wget -nv https://archive.apache.org/dist/spark/spark-3.5.2/spark-3.5.2-bin-hadoop3.tgz && \ + tar --strip-components=1 -xf spark-3.5.2-bin-hadoop3.tgz spark-3.5.2-bin-hadoop3/jars/ && \ + rm -rf spark-3.5.2-bin-hadoop3.tgz && \ mkdir -p ${INSTALL_DIR}/shims/spark35/spark_home/assembly/target/scala-2.12 && \ mv jars ${INSTALL_DIR}/shims/spark35/spark_home/assembly/target/scala-2.12 && \ - wget -nv https://github.com/apache/spark/archive/refs/tags/v3.5.3.tar.gz && \ - tar --strip-components=1 -xf v3.5.3.tar.gz spark-3.5.3/sql/core/src/test/resources/ && \ + wget -nv https://github.com/apache/spark/archive/refs/tags/v3.5.2.tar.gz && \ + tar --strip-components=1 -xf v3.5.2.tar.gz spark-3.5.2/sql/core/src/test/resources/ && \ mkdir -p shims/spark35/spark_home/ && \ mv sql shims/spark35/spark_home/ ;; 3.5-scala2.13) # Spark-3.5, scala 2.13 cd ${INSTALL_DIR} && \ - wget -nv https://archive.apache.org/dist/spark/spark-3.5.3/spark-3.5.3-bin-hadoop3.tgz && \ - tar --strip-components=1 -xf spark-3.5.3-bin-hadoop3.tgz spark-3.5.3-bin-hadoop3/jars/ && \ - rm -rf spark-3.5.3-bin-hadoop3.tgz && \ + wget -nv https://archive.apache.org/dist/spark/spark-3.5.2/spark-3.5.2-bin-hadoop3.tgz && \ + tar --strip-components=1 -xf spark-3.5.2-bin-hadoop3.tgz spark-3.5.2-bin-hadoop3/jars/ && \ + rm -rf spark-3.5.2-bin-hadoop3.tgz && \ mkdir -p ${INSTALL_DIR}/shims/spark35/spark_home/assembly/target/scala-2.13 && \ mv jars ${INSTALL_DIR}/shims/spark35/spark_home/assembly/target/scala-2.13 && \ - wget -nv https://github.com/apache/spark/archive/refs/tags/v3.5.3.tar.gz && \ - tar --strip-components=1 -xf v3.5.3.tar.gz spark-3.5.3/sql/core/src/test/resources/ && \ + wget -nv https://github.com/apache/spark/archive/refs/tags/v3.5.2.tar.gz && \ + tar --strip-components=1 -xf v3.5.2.tar.gz spark-3.5.2/sql/core/src/test/resources/ && \ mkdir -p shims/spark35/spark_home/ && \ mv sql shims/spark35/spark_home/ ;; diff --git a/.github/workflows/velox_backend.yml b/.github/workflows/velox_backend.yml index c316dba6f887..4b7f5836e745 100644 --- a/.github/workflows/velox_backend.yml +++ b/.github/workflows/velox_backend.yml @@ -754,19 +754,19 @@ jobs: with: name: arrow-jars-centos-7-${{github.sha}} path: /root/.m2/repository/org/apache/arrow/ - - name: Prepare spark.test.home for Spark 3.5.3 (other tests) + - name: Prepare spark.test.home for Spark 3.5.2 (other tests) run: | bash .github/workflows/util/install_spark_resources.sh 3.5 dnf module -y install python39 && \ alternatives --set python3 /usr/bin/python3.9 && \ pip3 install setuptools && \ - pip3 install pyspark==3.5.3 cython && \ + pip3 install pyspark==3.5.2 cython && \ pip3 install pandas pyarrow - - name: Build and Run unit test for Spark 3.5.3 (other tests) + - name: Build and Run unit test for Spark 3.5.2 (other tests) run: | cd $GITHUB_WORKSPACE/ export SPARK_SCALA_VERSION=2.12 - $MVN_CMD clean test -Pspark-3.5 -Pbackends-velox -Pceleborn -Piceberg -Pdelta -Pspark-ut \ + $MVN_CMD clean test -Pspark-3.5 -Pbackends-velox -Pceleborn -Piceberg -Pdelta -Phudi -Pspark-ut \ -DargLine="-Dspark.test.home=$GITHUB_WORKSPACE//shims/spark35/spark_home/" \ -DtagsToExclude=org.apache.spark.tags.ExtendedSQLTest,org.apache.gluten.tags.UDFTest,org.apache.gluten.tags.SkipTestTags - name: Upload test report @@ -792,15 +792,15 @@ jobs: with: name: arrow-jars-centos-7-${{github.sha}} path: /root/.m2/repository/org/apache/arrow/ - - name: Prepare spark.test.home for Spark 3.5.3 (other tests) + - name: Prepare spark.test.home for Spark 3.5.2 (other tests) run: | bash .github/workflows/util/install_spark_resources.sh 3.5-scala2.13 dnf module -y install python39 && \ alternatives --set python3 /usr/bin/python3.9 && \ pip3 install setuptools && \ - pip3 install pyspark==3.5.3 cython && \ + pip3 install pyspark==3.5.2 cython && \ pip3 install pandas pyarrow - - name: Build and Run unit test for Spark 3.5.3 with scala-2.13 (other tests) + - name: Build and Run unit test for Spark 3.5.2 with scala-2.13 (other tests) run: | cd $GITHUB_WORKSPACE/ export SPARK_SCALA_VERSION=2.13 @@ -830,13 +830,13 @@ jobs: with: name: arrow-jars-centos-7-${{github.sha}} path: /root/.m2/repository/org/apache/arrow/ - - name: Prepare spark.test.home for Spark 3.5.3 (slow tests) + - name: Prepare spark.test.home for Spark 3.5.2 (slow tests) run: | bash .github/workflows/util/install_spark_resources.sh 3.5 - - name: Build and Run unit test for Spark 3.5.3 (slow tests) + - name: Build and Run unit test for Spark 3.5.2 (slow tests) run: | cd $GITHUB_WORKSPACE/ - $MVN_CMD clean test -Pspark-3.5 -Pbackends-velox -Pceleborn -Piceberg -Pdelta -Pspark-ut \ + $MVN_CMD clean test -Pspark-3.5 -Pbackends-velox -Pceleborn -Piceberg -Pdelta -Phudi -Pspark-ut \ -DargLine="-Dspark.test.home=$GITHUB_WORKSPACE//shims/spark35/spark_home/" \ -DtagsToInclude=org.apache.spark.tags.ExtendedSQLTest - name: Upload test report @@ -862,15 +862,15 @@ jobs: with: name: arrow-jars-centos-7-${{github.sha}} path: /root/.m2/repository/org/apache/arrow/ - - name: Prepare spark.test.home for Spark 3.5.3 (other tests) + - name: Prepare spark.test.home for Spark 3.5.2 (other tests) run: | bash .github/workflows/util/install_spark_resources.sh 3.5 dnf module -y install python39 && \ alternatives --set python3 /usr/bin/python3.9 && \ pip3 install setuptools && \ - pip3 install pyspark==3.5.3 cython && \ + pip3 install pyspark==3.5.2 cython && \ pip3 install pandas pyarrow - - name: Build and Run unit test for Spark 3.5.3 (other tests) + - name: Build and Run unit test for Spark 3.5.2 (other tests) run: | cd $GITHUB_WORKSPACE/ export SPARK_SCALA_VERSION=2.12 @@ -899,10 +899,10 @@ jobs: with: name: arrow-jars-centos-7-${{github.sha}} path: /root/.m2/repository/org/apache/arrow/ - - name: Prepare spark.test.home for Spark 3.5.3 (slow tests) + - name: Prepare spark.test.home for Spark 3.5.2 (slow tests) run: | bash .github/workflows/util/install_spark_resources.sh 3.5 - - name: Build and Run unit test for Spark 3.5.3 (slow tests) + - name: Build and Run unit test for Spark 3.5.2 (slow tests) run: | cd $GITHUB_WORKSPACE/ $MVN_CMD clean test -Pspark-3.5 -Pbackends-velox -Pceleborn -Piceberg -Pdelta -Pspark-ut \ diff --git a/pom.xml b/pom.xml index d3c67f231e9e..030828a327d6 100644 --- a/pom.xml +++ b/pom.xml @@ -336,11 +336,11 @@ 3.5 spark-sql-columnar-shims-spark35 - 3.5.3 + 3.5.2 1.5.0 delta-spark - 3.2.1 - 32 + 3.2.0 + 32 0.15.0 2.15.1 3.3.4 diff --git a/shims/spark35/src/main/scala/org/apache/gluten/sql/shims/spark35/SparkShimProvider.scala b/shims/spark35/src/main/scala/org/apache/gluten/sql/shims/spark35/SparkShimProvider.scala index ad99f7be2547..eab32ab9d0b9 100644 --- a/shims/spark35/src/main/scala/org/apache/gluten/sql/shims/spark35/SparkShimProvider.scala +++ b/shims/spark35/src/main/scala/org/apache/gluten/sql/shims/spark35/SparkShimProvider.scala @@ -20,7 +20,7 @@ import org.apache.gluten.sql.shims.{SparkShimDescriptor, SparkShims} import org.apache.gluten.sql.shims.spark35.SparkShimProvider.DESCRIPTOR object SparkShimProvider { - val DESCRIPTOR = SparkShimDescriptor(3, 5, 3) + val DESCRIPTOR = SparkShimDescriptor(3, 5, 2) } class SparkShimProvider extends org.apache.gluten.sql.shims.SparkShimProvider { diff --git a/tools/gluten-it/pom.xml b/tools/gluten-it/pom.xml index 70b8d20c5ba4..bad4d6087f11 100644 --- a/tools/gluten-it/pom.xml +++ b/tools/gluten-it/pom.xml @@ -164,7 +164,7 @@ spark-3.5 - 3.5.3 + 3.5.2 2.12.18 From 73a1a01872779d5e454cef83da28c68455021b75 Mon Sep 17 00:00:00 2001 From: Mingliang Zhu Date: Wed, 13 Nov 2024 15:11:11 +0800 Subject: [PATCH 11/35] [VL][CI] Fix back upload golden files (#7880) --- .github/workflows/velox_backend.yml | 24 ++++++++++++++++++++++++ 1 file changed, 24 insertions(+) diff --git a/.github/workflows/velox_backend.yml b/.github/workflows/velox_backend.yml index 4b7f5836e745..fafceb1aa06c 100644 --- a/.github/workflows/velox_backend.yml +++ b/.github/workflows/velox_backend.yml @@ -564,6 +564,12 @@ jobs: with: name: test-report-spark32 path: '**/surefire-reports/TEST-*.xml' + - name: Upload golden files + if: failure() + uses: actions/upload-artifact@v4 + with: + name: golden-files-spark32 + path: /tmp/tpch-approved-plan/** run-spark-test-spark32-slow: needs: build-native-lib-centos-7 @@ -633,6 +639,12 @@ jobs: with: name: test-report-spark33 path: '**/surefire-reports/TEST-*.xml' + - name: Upload golden files + if: failure() + uses: actions/upload-artifact@v4 + with: + name: golden-files-spark33 + path: /tmp/tpch-approved-plan/** run-spark-test-spark33-slow: @@ -704,6 +716,12 @@ jobs: with: name: test-report-spark34 path: '**/surefire-reports/TEST-*.xml' + - name: Upload golden files + if: failure() + uses: actions/upload-artifact@v4 + with: + name: golden-files-spark34 + path: /tmp/tpch-approved-plan/** run-spark-test-spark34-slow: @@ -775,6 +793,12 @@ jobs: with: name: test-report-spark35 path: '**/surefire-reports/TEST-*.xml' + - name: Upload golden files + if: failure() + uses: actions/upload-artifact@v4 + with: + name: golden-files-spark35 + path: /tmp/tpch-approved-plan/** run-spark-test-spark35-scala213: needs: build-native-lib-centos-7 From 5a2814363b19064561eed656b5161c3eef6dcfe1 Mon Sep 17 00:00:00 2001 From: Yuan Date: Wed, 13 Nov 2024 19:24:02 +0800 Subject: [PATCH 12/35] [GLUTEN-6887][VL] Daily Update Velox Version (2024_11_13) (#7926) Signed-off-by: Yuan Zhou Co-authored-by: Hongze Zhang --- .../gluten/backendsapi/velox/VeloxSparkPlanExecApi.scala | 3 ++- cpp/velox/memory/VeloxMemoryManager.cc | 4 +++- cpp/velox/operators/functions/RegistrationAllFunctions.cc | 8 -------- ep/build-velox/src/get_velox.sh | 2 +- 4 files changed, 6 insertions(+), 11 deletions(-) diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxSparkPlanExecApi.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxSparkPlanExecApi.scala index 5925666d2c00..2df2e2718eaa 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxSparkPlanExecApi.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxSparkPlanExecApi.scala @@ -153,7 +153,8 @@ class VeloxSparkPlanExecApi extends SparkPlanExecApi { left.dataType.isInstanceOf[DecimalType] && right.dataType .isInstanceOf[DecimalType] && !SQLConf.get.decimalOperationsAllowPrecisionLoss ) { - val newName = "not_allow_precision_loss_" + // https://github.com/facebookincubator/velox/pull/10383 + val newName = substraitExprName + "_deny_precision_loss" GenericExpressionTransformer(newName, Seq(left, right), original) } else { GenericExpressionTransformer(substraitExprName, Seq(left, right), original) diff --git a/cpp/velox/memory/VeloxMemoryManager.cc b/cpp/velox/memory/VeloxMemoryManager.cc index 19a5d458049b..1d281b22dc86 100644 --- a/cpp/velox/memory/VeloxMemoryManager.cc +++ b/cpp/velox/memory/VeloxMemoryManager.cc @@ -99,7 +99,9 @@ class ListenableArbitrator : public velox::memory::MemoryArbitrator { } bool growCapacity(velox::memory::MemoryPool* pool, uint64_t targetBytes) override { - velox::memory::ScopedMemoryArbitrationContext ctx(pool); + // Set arbitration context to allow memory over-use during recursive arbitration. + // See MemoryPoolImpl::maybeIncrementReservation. + velox::memory::ScopedMemoryArbitrationContext ctx{}; velox::memory::MemoryPool* candidate; { std::unique_lock guard{mutex_}; diff --git a/cpp/velox/operators/functions/RegistrationAllFunctions.cc b/cpp/velox/operators/functions/RegistrationAllFunctions.cc index 6e2f90f0105b..ea9d3f4202f7 100644 --- a/cpp/velox/operators/functions/RegistrationAllFunctions.cc +++ b/cpp/velox/operators/functions/RegistrationAllFunctions.cc @@ -76,13 +76,6 @@ void registerFunctionOverwrite() { velox::functions::registerPrestoVectorFunctions(); } -void registerFunctionForConfig() { - const std::string prefix = "not_allow_precision_loss_"; - velox::functions::sparksql::registerDecimalAdd(prefix, false); - velox::functions::sparksql::registerDecimalSubtract(prefix, false); - velox::functions::sparksql::registerDecimalMultiply(prefix, false); - velox::functions::sparksql::registerDecimalDivide(prefix, false); -} } // namespace void registerAllFunctions() { @@ -96,7 +89,6 @@ void registerAllFunctions() { // Using function overwrite to handle function names mismatch between Spark // and Velox. registerFunctionOverwrite(); - registerFunctionForConfig(); } } // namespace gluten diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh index 3d72b950bf93..d802838b4b70 100755 --- a/ep/build-velox/src/get_velox.sh +++ b/ep/build-velox/src/get_velox.sh @@ -17,7 +17,7 @@ set -exu VELOX_REPO=https://github.com/oap-project/velox.git -VELOX_BRANCH=2024_11_12 +VELOX_BRANCH=2024_11_13 VELOX_HOME="" OS=`uname -s` From 61033b719f8494ef80dd0c46fa6e738afa7e732d Mon Sep 17 00:00:00 2001 From: Hongze Zhang Date: Wed, 13 Nov 2024 19:33:08 +0800 Subject: [PATCH 13/35] [GLUTEN-7243][VL] A follow-up fix for #7748 (#7935) --- cpp/velox/operators/plannodes/RowVectorStream.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/cpp/velox/operators/plannodes/RowVectorStream.h b/cpp/velox/operators/plannodes/RowVectorStream.h index ce26305fa054..e5a469afee3d 100644 --- a/cpp/velox/operators/plannodes/RowVectorStream.h +++ b/cpp/velox/operators/plannodes/RowVectorStream.h @@ -46,7 +46,7 @@ class RowVectorStream { // As of now, non-zero running threads usually happens when: // 1. Task A spills task B; // 2. Task A trys to grow buffers created by task B, during which spill is requested on task A again. - facebook::velox::exec::SuspendedSection(driverCtx_->driver); + facebook::velox::exec::SuspendedSection ss(driverCtx_->driver); hasNext = iterator_->hasNext(); } if (!hasNext) { @@ -64,7 +64,7 @@ class RowVectorStream { { // We are leaving Velox task execution and are probably entering Spark code through JNI. Suspend the current // driver to make the current task open to spilling. - facebook::velox::exec::SuspendedSection(driverCtx_->driver); + facebook::velox::exec::SuspendedSection ss(driverCtx_->driver); cb = iterator_->next(); } const std::shared_ptr& vb = VeloxColumnarBatch::from(pool_, cb); From d6326f06c8158de2eb08bb9ba7e3e4eb6e3964ef Mon Sep 17 00:00:00 2001 From: Rui Mo Date: Wed, 13 Nov 2024 23:39:08 +0800 Subject: [PATCH 14/35] [GLUTEN-6666][VL] Use custom SparkExprToSubfieldFilterParser (#6754) Removes separating filter relevant code from Gluten. With a custom filter parser registered, we are able to use Velox provided filter extraction. --- cpp/velox/CMakeLists.txt | 1 + cpp/velox/compute/VeloxBackend.cc | 3 + .../SparkExprToSubfieldFilterParser.cc | 103 ++ .../SparkExprToSubfieldFilterParser.h | 37 + cpp/velox/substrait/SubstraitToVeloxPlan.cc | 1174 +---------------- cpp/velox/substrait/SubstraitToVeloxPlan.h | 341 ----- .../Substrait2VeloxPlanConversionTest.cc | 10 +- ep/build-velox/src/get_velox.sh | 2 +- 8 files changed, 151 insertions(+), 1520 deletions(-) create mode 100644 cpp/velox/operators/functions/SparkExprToSubfieldFilterParser.cc create mode 100644 cpp/velox/operators/functions/SparkExprToSubfieldFilterParser.h diff --git a/cpp/velox/CMakeLists.txt b/cpp/velox/CMakeLists.txt index 586c08653dfc..329da49497f3 100644 --- a/cpp/velox/CMakeLists.txt +++ b/cpp/velox/CMakeLists.txt @@ -168,6 +168,7 @@ set(VELOX_SRCS memory/VeloxMemoryManager.cc operators/functions/RegistrationAllFunctions.cc operators/functions/RowConstructorWithNull.cc + operators/functions/SparkExprToSubfieldFilterParser.cc operators/serializer/VeloxColumnarToRowConverter.cc operators/serializer/VeloxColumnarBatchSerializer.cc operators/serializer/VeloxRowToColumnarConverter.cc diff --git a/cpp/velox/compute/VeloxBackend.cc b/cpp/velox/compute/VeloxBackend.cc index c1e907bea1d5..889f979b2869 100644 --- a/cpp/velox/compute/VeloxBackend.cc +++ b/cpp/velox/compute/VeloxBackend.cc @@ -33,6 +33,7 @@ #include "compute/VeloxRuntime.h" #include "config/VeloxConfig.h" #include "jni/JniFileSystem.h" +#include "operators/functions/SparkExprToSubfieldFilterParser.h" #include "udf/UdfLoader.h" #include "utils/Exception.h" #include "velox/common/caching/SsdCache.h" @@ -155,6 +156,8 @@ void VeloxBackend::init(const std::unordered_map& conf velox::parquet::registerParquetReaderFactory(); velox::parquet::registerParquetWriterFactory(); velox::orc::registerOrcReaderFactory(); + velox::exec::ExprToSubfieldFilterParser::registerParserFactory( + []() { return std::make_shared(); }); // Register Velox functions registerAllFunctions(); diff --git a/cpp/velox/operators/functions/SparkExprToSubfieldFilterParser.cc b/cpp/velox/operators/functions/SparkExprToSubfieldFilterParser.cc new file mode 100644 index 000000000000..8ad537d8168a --- /dev/null +++ b/cpp/velox/operators/functions/SparkExprToSubfieldFilterParser.cc @@ -0,0 +1,103 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#include "operators/functions/SparkExprToSubfieldFilterParser.h" + +namespace gluten { + +using namespace facebook::velox; + +bool SparkExprToSubfieldFilterParser::toSparkSubfield(const core::ITypedExpr* field, common::Subfield& subfield) { + std::vector> path; + for (auto* current = field;;) { + if (auto* fieldAccess = dynamic_cast(current)) { + path.push_back(std::make_unique(fieldAccess->name())); + } else if (dynamic_cast(current)) { + return false; + } else if (dynamic_cast(current) == nullptr) { + return false; + } else { + break; + } + + if (!current->inputs().empty()) { + return false; + } else { + break; + } + } + std::reverse(path.begin(), path.end()); + subfield = common::Subfield(std::move(path)); + return true; +} + +std::unique_ptr SparkExprToSubfieldFilterParser::leafCallToSubfieldFilter( + const core::CallTypedExpr& call, + common::Subfield& subfield, + core::ExpressionEvaluator* evaluator, + bool negated) { + if (call.inputs().empty()) { + return nullptr; + } + + const auto* leftSide = call.inputs()[0].get(); + + if (call.name() == "equalto") { + if (toSparkSubfield(leftSide, subfield)) { + return negated ? makeNotEqualFilter(call.inputs()[1], evaluator) : makeEqualFilter(call.inputs()[1], evaluator); + } + } else if (call.name() == "lessthanorequal") { + if (toSparkSubfield(leftSide, subfield)) { + return negated ? makeGreaterThanFilter(call.inputs()[1], evaluator) + : makeLessThanOrEqualFilter(call.inputs()[1], evaluator); + } + } else if (call.name() == "lessthan") { + if (toSparkSubfield(leftSide, subfield)) { + return negated ? makeGreaterThanOrEqualFilter(call.inputs()[1], evaluator) + : makeLessThanFilter(call.inputs()[1], evaluator); + } + } else if (call.name() == "greaterthanorequal") { + if (toSparkSubfield(leftSide, subfield)) { + return negated ? makeLessThanFilter(call.inputs()[1], evaluator) + : makeGreaterThanOrEqualFilter(call.inputs()[1], evaluator); + } + } else if (call.name() == "greaterthan") { + if (toSparkSubfield(leftSide, subfield)) { + return negated ? makeLessThanOrEqualFilter(call.inputs()[1], evaluator) + : makeGreaterThanFilter(call.inputs()[1], evaluator); + } + } else if (call.name() == "in") { + if (toSparkSubfield(leftSide, subfield)) { + return makeInFilter(call.inputs()[1], evaluator, negated); + } + } else if (call.name() == "isnull") { + if (toSparkSubfield(leftSide, subfield)) { + if (negated) { + return exec::isNotNull(); + } + return exec::isNull(); + } + } else if (call.name() == "isnotnull") { + if (toSparkSubfield(leftSide, subfield)) { + if (negated) { + return exec::isNull(); + } + return exec::isNotNull(); + } + } + return nullptr; +} +} // namespace gluten diff --git a/cpp/velox/operators/functions/SparkExprToSubfieldFilterParser.h b/cpp/velox/operators/functions/SparkExprToSubfieldFilterParser.h new file mode 100644 index 000000000000..d050091c0793 --- /dev/null +++ b/cpp/velox/operators/functions/SparkExprToSubfieldFilterParser.h @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#include "velox/expression/ExprToSubfieldFilter.h" + +namespace gluten { + +/// Parses Spark expression into subfield filter. Differences from Presto's parser include: +/// 1) Some Spark functions are registered under different names. +/// 2) The supported functions vary. +/// 3) Filter push-down on nested fields is disabled. +class SparkExprToSubfieldFilterParser : public facebook::velox::exec::ExprToSubfieldFilterParser { + public: + std::unique_ptr leafCallToSubfieldFilter( + const facebook::velox::core::CallTypedExpr& call, + facebook::velox::common::Subfield& subfield, + facebook::velox::core::ExpressionEvaluator* evaluator, + bool negated) override; + + private: + // Compared to the upstream 'toSubfield', the push-down of filter on nested field is disabled. + bool toSparkSubfield(const facebook::velox::core::ITypedExpr* field, facebook::velox::common::Subfield& subfield); +}; +} // namespace gluten diff --git a/cpp/velox/substrait/SubstraitToVeloxPlan.cc b/cpp/velox/substrait/SubstraitToVeloxPlan.cc index 9e2959043334..cdd9269e1494 100644 --- a/cpp/velox/substrait/SubstraitToVeloxPlan.cc +++ b/cpp/velox/substrait/SubstraitToVeloxPlan.cc @@ -73,45 +73,6 @@ EmitInfo getEmitInfo(const ::substrait::RelCommon& relCommon, const core::PlanNo return emitInfo; } -template -// Get the lowest value for numeric type. -T getLowest() { - return std::numeric_limits::lowest(); -} - -// Get the lowest value for string. -template <> -std::string getLowest() { - return ""; -} - -// Get the max value for numeric type. -template -T getMax() { - return std::numeric_limits::max(); -} - -// The max value will be used in BytesRange. Return empty string here instead. -template <> -std::string getMax() { - return ""; -} - -// Substrait function names. -const std::string sIsNotNull = "is_not_null"; -const std::string sIsNull = "is_null"; -const std::string sGte = "gte"; -const std::string sGt = "gt"; -const std::string sLte = "lte"; -const std::string sLt = "lt"; -const std::string sEqual = "equal"; -const std::string sOr = "or"; -const std::string sNot = "not"; - -// Substrait types. -const std::string sI32 = "i32"; -const std::string sI64 = "i64"; - /// @brief Get the input type from both sides of join. /// @param leftNode the plan node of left side. /// @param rightNode the plan node of right side. @@ -1190,37 +1151,10 @@ core::PlanNodePtr SubstraitToVeloxPlanConverter::toVeloxPlan(const ::substrait:: tableHandle = std::make_shared( kHiveConnectorId, "hive_table", filterPushdownEnabled, connector::hive::SubfieldFilters{}, nullptr); } else { - // Flatten the conditions connected with 'and'. - std::vector<::substrait::Expression_ScalarFunction> scalarFunctions; - std::vector<::substrait::Expression_SingularOrList> singularOrLists; - std::vector<::substrait::Expression_IfThen> ifThens; - flattenConditions(readRel.filter(), scalarFunctions, singularOrLists, ifThens); - - // The vector's subscript stands for the column index. - std::vector rangeRecorders(veloxTypeList.size()); - - // Separate the filters to be two parts. The subfield part can be - // pushed down. - std::vector<::substrait::Expression_ScalarFunction> subfieldFunctions; - std::vector<::substrait::Expression_ScalarFunction> remainingFunctions; - std::vector<::substrait::Expression_SingularOrList> subfieldOrLists; - std::vector<::substrait::Expression_SingularOrList> remainingOrLists; - - separateFilters( - rangeRecorders, - scalarFunctions, - subfieldFunctions, - remainingFunctions, - singularOrLists, - subfieldOrLists, - remainingOrLists, - veloxTypeList, - splitInfo->format); - - // Create subfield filters based on the constructed filter info map. - auto subfieldFilters = createSubfieldFilters(colNameList, veloxTypeList, subfieldFunctions, subfieldOrLists); - // Connect the remaining filters with 'and'. - auto remainingFilter = connectWithAnd(colNameList, veloxTypeList, remainingFunctions, remainingOrLists, ifThens); + connector::hive::SubfieldFilters subfieldFilters; + auto names = colNameList; + auto types = veloxTypeList; + auto remainingFilter = exprConverter_->toVeloxExpr(readRel.filter(), ROW(std::move(names), std::move(types))); tableHandle = std::make_shared( kHiveConnectorId, "hive_table", filterPushdownEnabled, std::move(subfieldFilters), remainingFilter); @@ -1386,39 +1320,6 @@ void SubstraitToVeloxPlanConverter::constructFunctionMap(const ::substrait::Plan exprConverter_ = std::make_unique(pool_, functionMap_); } -void SubstraitToVeloxPlanConverter::flattenConditions( - const ::substrait::Expression& substraitFilter, - std::vector<::substrait::Expression_ScalarFunction>& scalarFunctions, - std::vector<::substrait::Expression_SingularOrList>& singularOrLists, - std::vector<::substrait::Expression_IfThen>& ifThens) { - auto typeCase = substraitFilter.rex_type_case(); - switch (typeCase) { - case ::substrait::Expression::RexTypeCase::kScalarFunction: { - const auto& sFunc = substraitFilter.scalar_function(); - auto filterNameSpec = SubstraitParser::findFunctionSpec(functionMap_, sFunc.function_reference()); - // TODO: Only and relation is supported here. - if (SubstraitParser::getNameBeforeDelimiter(filterNameSpec) == "and") { - for (const auto& sCondition : sFunc.arguments()) { - flattenConditions(sCondition.value(), scalarFunctions, singularOrLists, ifThens); - } - } else { - scalarFunctions.emplace_back(sFunc); - } - break; - } - case ::substrait::Expression::RexTypeCase::kSingularOrList: { - singularOrLists.emplace_back(substraitFilter.singular_or_list()); - break; - } - case ::substrait::Expression::RexTypeCase::kIfThen: { - ifThens.emplace_back(substraitFilter.if_then()); - break; - } - default: - VELOX_NYI("GetFlatConditions not supported for type '{}'", std::to_string(typeCase)); - } -} - std::string SubstraitToVeloxPlanConverter::findFuncSpec(uint64_t id) { return SubstraitParser::findFunctionSpec(functionMap_, id); } @@ -1481,878 +1382,6 @@ void SubstraitToVeloxPlanConverter::extractJoinKeys( } } -connector::hive::SubfieldFilters SubstraitToVeloxPlanConverter::createSubfieldFilters( - const std::vector& inputNameList, - const std::vector& inputTypeList, - const std::vector<::substrait::Expression_ScalarFunction>& scalarFunctions, - const std::vector<::substrait::Expression_SingularOrList>& singularOrLists) { - // The vector's subscript stands for the column index. - std::vector columnToFilterInfo(inputTypeList.size()); - - // Process scalarFunctions. - for (const auto& scalarFunction : scalarFunctions) { - auto filterNameSpec = SubstraitParser::findFunctionSpec(functionMap_, scalarFunction.function_reference()); - auto filterName = SubstraitParser::getNameBeforeDelimiter(filterNameSpec); - - if (filterName == sNot) { - VELOX_CHECK(scalarFunction.arguments().size() == 1); - auto expr = scalarFunction.arguments()[0].value(); - if (expr.has_scalar_function()) { - // Set its child to filter info with reverse enabled. - setFilterInfo(expr.scalar_function(), inputTypeList, columnToFilterInfo, true); - } else if (expr.has_singular_or_list()) { - auto singularOrList = expr.singular_or_list(); - setFilterInfo(singularOrList, columnToFilterInfo, true); - } else { - VELOX_NYI("Only support push down Not with scalar function or In."); - } - } else if (filterName == sOr) { - VELOX_CHECK(scalarFunction.arguments().size() == 2); - VELOX_CHECK(std::all_of( - scalarFunction.arguments().cbegin(), - scalarFunction.arguments().cend(), - [](const ::substrait::FunctionArgument& arg) { - return arg.value().has_scalar_function() || arg.value().has_singular_or_list(); - })); - - // Set the children functions to filter info. They should be - // effective to the same field. - for (const auto& arg : scalarFunction.arguments()) { - const auto& expr = arg.value(); - if (expr.has_scalar_function()) { - setFilterInfo(arg.value().scalar_function(), inputTypeList, columnToFilterInfo); - } else if (expr.has_singular_or_list()) { - setFilterInfo(expr.singular_or_list(), columnToFilterInfo); - } else { - VELOX_NYI("Scalar function or SingularOrList expected."); - } - } - } else { - setFilterInfo(scalarFunction, inputTypeList, columnToFilterInfo); - } - } - - // Process singularOrLists. - for (const auto& list : singularOrLists) { - setFilterInfo(list, columnToFilterInfo); - } - - return mapToFilters(inputNameList, inputTypeList, columnToFilterInfo); -} - -bool SubstraitToVeloxPlanConverter::fieldOrWithLiteral( - const ::google::protobuf::RepeatedPtrField<::substrait::FunctionArgument>& arguments, - uint32_t& fieldIndex) { - if (arguments.size() == 1) { - if (arguments[0].value().has_selection()) { - // Only field exists. - return SubstraitParser::parseReferenceSegment(arguments[0].value().selection().direct_reference(), fieldIndex); - } else { - return false; - } - } - - if (arguments.size() != 2) { - // Not the field and literal combination. - return false; - } - bool fieldExists = false; - bool literalExists = false; - for (const auto& param : arguments) { - auto typeCase = param.value().rex_type_case(); - switch (typeCase) { - case ::substrait::Expression::RexTypeCase::kSelection: { - if (!SubstraitParser::parseReferenceSegment(param.value().selection().direct_reference(), fieldIndex)) { - return false; - } - fieldExists = true; - break; - } - case ::substrait::Expression::RexTypeCase::kLiteral: { - literalExists = true; - break; - } - default: - break; - } - } - // Whether the field and literal both exist. - return fieldExists && literalExists; -} - -bool SubstraitToVeloxPlanConverter::childrenFunctionsOnSameField( - const ::substrait::Expression_ScalarFunction& function) { - // Get the column indices of the children functions. - std::vector colIndices; - for (const auto& arg : function.arguments()) { - if (arg.value().has_scalar_function()) { - const auto& scalarFunction = arg.value().scalar_function(); - for (const auto& param : scalarFunction.arguments()) { - if (param.value().has_selection()) { - const auto& field = param.value().selection(); - VELOX_CHECK(field.has_direct_reference()); - uint32_t colIdx; - if (!SubstraitParser::parseReferenceSegment(field.direct_reference(), colIdx)) { - return false; - } - colIndices.emplace_back(colIdx); - } - } - } else if (arg.value().has_singular_or_list()) { - const auto& singularOrList = arg.value().singular_or_list(); - colIndices.emplace_back(getColumnIndexFromSingularOrList(singularOrList)); - } else { - return false; - } - } - - if (std::all_of(colIndices.begin(), colIndices.end(), [&](uint32_t idx) { return idx == colIndices[0]; })) { - // All indices are the same. - return true; - } - return false; -} - -bool SubstraitToVeloxPlanConverter::canPushdownFunction( - const ::substrait::Expression_ScalarFunction& scalarFunction, - const std::string& filterName, - uint32_t& fieldIdx) { - // Condtions can be pushed down. - static const std::unordered_set supportedFunctions = {sIsNotNull, sIsNull, sGte, sGt, sLte, sLt, sEqual}; - - bool canPushdown = false; - if (supportedFunctions.find(filterName) != supportedFunctions.end() && - fieldOrWithLiteral(scalarFunction.arguments(), fieldIdx)) { - // The arg should be field or field with literal. - canPushdown = true; - } - return canPushdown; -} - -bool SubstraitToVeloxPlanConverter::canPushdownNot( - const ::substrait::Expression_ScalarFunction& scalarFunction, - std::vector& rangeRecorders) { - VELOX_CHECK(scalarFunction.arguments().size() == 1, "Only one arg is expected for Not."); - const auto& notArg = scalarFunction.arguments()[0]; - if (notArg.value().has_singular_or_list()) { - auto singularOrList = notArg.value().singular_or_list(); - if (!canPushdownSingularOrList(singularOrList)) { - return false; - } - uint32_t colIdx = getColumnIndexFromSingularOrList(singularOrList); - return rangeRecorders.at(colIdx).setInRange(); - } else if (notArg.value().has_scalar_function()) { - auto argFunction = - SubstraitParser::findFunctionSpec(functionMap_, notArg.value().scalar_function().function_reference()); - auto functionName = SubstraitParser::getNameBeforeDelimiter(argFunction); - - static const std::unordered_set supportedNotFunctions = {sGte, sGt, sLte, sLt, sEqual}; - - uint32_t fieldIdx; - bool isFieldOrWithLiteral = fieldOrWithLiteral(notArg.value().scalar_function().arguments(), fieldIdx); - - return ( - supportedNotFunctions.find(functionName) != supportedNotFunctions.end() && isFieldOrWithLiteral && - rangeRecorders.at(fieldIdx).setCertainRangeForFunction(functionName, true /*reverse*/)); - } - return false; -} - -bool SubstraitToVeloxPlanConverter::canPushdownOr( - const ::substrait::Expression_ScalarFunction& scalarFunction, - std::vector& rangeRecorders) { - // OR Conditon whose children functions are on different columns is not - // supported to be pushed down. - if (!childrenFunctionsOnSameField(scalarFunction)) { - return false; - } - - static const std::unordered_set supportedOrFunctions = {sIsNotNull, sGte, sGt, sLte, sLt, sEqual}; - - for (const auto& arg : scalarFunction.arguments()) { - if (arg.value().has_scalar_function()) { - auto nameSpec = - SubstraitParser::findFunctionSpec(functionMap_, arg.value().scalar_function().function_reference()); - auto functionName = SubstraitParser::getNameBeforeDelimiter(nameSpec); - - uint32_t fieldIdx; - bool isFieldOrWithLiteral = fieldOrWithLiteral(arg.value().scalar_function().arguments(), fieldIdx); - if (supportedOrFunctions.find(functionName) == supportedOrFunctions.end() || !isFieldOrWithLiteral || - !rangeRecorders.at(fieldIdx).setCertainRangeForFunction( - functionName, false /*reverse*/, true /*forOrRelation*/)) { - // The arg should be field or field with literal. - return false; - } - } else if (arg.value().has_singular_or_list()) { - const auto& singularOrList = arg.value().singular_or_list(); - if (!canPushdownSingularOrList(singularOrList, true)) { - return false; - } - uint32_t fieldIdx = getColumnIndexFromSingularOrList(singularOrList); - // Disable IN pushdown for int-like types. - if (!rangeRecorders.at(fieldIdx).setInRange(true /*forOrRelation*/)) { - return false; - } - } else { - // Or relation betweeen other expressions is not supported to be pushded - // down currently. - return false; - } - } - return true; -} - -void SubstraitToVeloxPlanConverter::separateFilters( - std::vector& rangeRecorders, - const std::vector<::substrait::Expression_ScalarFunction>& scalarFunctions, - std::vector<::substrait::Expression_ScalarFunction>& subfieldFunctions, - std::vector<::substrait::Expression_ScalarFunction>& remainingFunctions, - const std::vector<::substrait::Expression_SingularOrList>& singularOrLists, - std::vector<::substrait::Expression_SingularOrList>& subfieldOrLists, - std::vector<::substrait::Expression_SingularOrList>& remainingOrLists, - const std::vector& veloxTypeList, - const dwio::common::FileFormat& format) { - for (const auto& singularOrList : singularOrLists) { - if (!canPushdownSingularOrList(singularOrList)) { - remainingOrLists.emplace_back(singularOrList); - continue; - } - uint32_t colIdx = getColumnIndexFromSingularOrList(singularOrList); - if (rangeRecorders.at(colIdx).setInRange()) { - subfieldOrLists.emplace_back(singularOrList); - } else { - remainingOrLists.emplace_back(singularOrList); - } - } - - for (const auto& scalarFunction : scalarFunctions) { - auto filterNameSpec = SubstraitParser::findFunctionSpec(functionMap_, scalarFunction.function_reference()); - auto filterName = SubstraitParser::getNameBeforeDelimiter(filterNameSpec); - // Add all decimal filters to remaining functions because their pushdown are not supported. - if (format == dwio::common::FileFormat::ORC && scalarFunction.arguments().size() > 0) { - auto value = scalarFunction.arguments().at(0).value(); - if (value.has_selection()) { - uint32_t fieldIndex; - bool parsed = SubstraitParser::parseReferenceSegment(value.selection().direct_reference(), fieldIndex); - if (!parsed || (!veloxTypeList.empty() && veloxTypeList.at(fieldIndex)->isDecimal())) { - remainingFunctions.emplace_back(scalarFunction); - continue; - } - } - } - - // Check whether NOT and OR functions can be pushed down. - // If yes, the scalar function will be added into the subfield functions. - if (filterName == sNot) { - if (canPushdownNot(scalarFunction, rangeRecorders)) { - subfieldFunctions.emplace_back(scalarFunction); - } else { - remainingFunctions.emplace_back(scalarFunction); - } - } else if (filterName == sOr) { - if (canPushdownOr(scalarFunction, rangeRecorders)) { - subfieldFunctions.emplace_back(scalarFunction); - } else { - remainingFunctions.emplace_back(scalarFunction); - } - } else { - // Check if the condition is supported to be pushed down. - uint32_t fieldIdx; - if (canPushdownFunction(scalarFunction, filterName, fieldIdx) && - rangeRecorders.at(fieldIdx).setCertainRangeForFunction(filterName)) { - subfieldFunctions.emplace_back(scalarFunction); - } else { - remainingFunctions.emplace_back(scalarFunction); - } - } - } -} - -bool SubstraitToVeloxPlanConverter::RangeRecorder::setCertainRangeForFunction( - const std::string& functionName, - bool reverse, - bool forOrRelation) { - if (functionName == sLt || functionName == sLte) { - if (reverse) { - return setLeftBound(forOrRelation); - } else { - return setRightBound(forOrRelation); - } - } else if (functionName == sGt || functionName == sGte) { - if (reverse) { - return setRightBound(forOrRelation); - } else { - return setLeftBound(forOrRelation); - } - } else if (functionName == sEqual) { - if (reverse) { - // Not equal means lt or gt. - return setMultiRange(); - } else { - return setLeftBound(forOrRelation) && setRightBound(forOrRelation); - } - } else if (functionName == sOr) { - if (reverse) { - // Not supported. - return false; - } else { - return setMultiRange(); - } - } else if (functionName == sIsNotNull) { - if (reverse) { - // Not supported. - return false; - } else { - // Is not null can always coexist with the other range. - return true; - } - } else if (functionName == sIsNull) { - if (reverse) { - return setCertainRangeForFunction(sIsNotNull, false, forOrRelation); - } else { - return setIsNull(); - } - } else { - return false; - } -} - -void SubstraitToVeloxPlanConverter::setColumnFilterInfo( - const std::string& filterName, - std::optional literalVariant, - FilterInfo& columnFilterInfo, - bool reverse) { - if (filterName == sIsNotNull) { - if (reverse) { - columnFilterInfo.setNull(); - } else { - columnFilterInfo.forbidsNull(); - } - } else if (filterName == sIsNull) { - if (reverse) { - columnFilterInfo.forbidsNull(); - } else { - columnFilterInfo.setNull(); - } - } else if (filterName == sGte) { - if (reverse) { - columnFilterInfo.setUpper(literalVariant, true); - } else { - columnFilterInfo.setLower(literalVariant, false); - } - } else if (filterName == sGt) { - if (reverse) { - columnFilterInfo.setUpper(literalVariant, false); - } else { - columnFilterInfo.setLower(literalVariant, true); - } - } else if (filterName == sLte) { - if (reverse) { - columnFilterInfo.setLower(literalVariant, true); - } else { - columnFilterInfo.setUpper(literalVariant, false); - } - } else if (filterName == sLt) { - if (reverse) { - columnFilterInfo.setLower(literalVariant, false); - } else { - columnFilterInfo.setUpper(literalVariant, true); - } - } else if (filterName == sEqual) { - if (reverse) { - columnFilterInfo.setNotValue(literalVariant); - } else { - columnFilterInfo.setLower(literalVariant, false); - columnFilterInfo.setUpper(literalVariant, false); - } - } else { - VELOX_NYI("setColumnFilterInfo not supported for filter name '{}'", filterName); - } -} - -template -variant getVariantFromLiteral(const ::substrait::Expression::Literal& literal) { - using LitT = typename facebook::velox::TypeTraits::NativeType; - return variant(SubstraitParser::getLiteralValue(literal)); -} - -void SubstraitToVeloxPlanConverter::setFilterInfo( - const ::substrait::Expression_ScalarFunction& scalarFunction, - const std::vector& inputTypeList, - std::vector& columnToFilterInfo, - bool reverse) { - auto nameSpec = SubstraitParser::findFunctionSpec(functionMap_, scalarFunction.function_reference()); - auto functionName = SubstraitParser::getNameBeforeDelimiter(nameSpec); - - // Extract the column index and column bound from the scalar function. - std::optional colIdx; - std::optional<::substrait::Expression_Literal> substraitLit; - std::vector typeCases; - - for (const auto& param : scalarFunction.arguments()) { - auto typeCase = param.value().rex_type_case(); - switch (typeCase) { - case ::substrait::Expression::RexTypeCase::kSelection: { - typeCases.emplace_back("kSelection"); - uint32_t index; - VELOX_CHECK( - SubstraitParser::parseReferenceSegment(param.value().selection().direct_reference(), index), - "Failed to parse the column index from the selection."); - colIdx = index; - break; - } - case ::substrait::Expression::RexTypeCase::kLiteral: { - typeCases.emplace_back("kLiteral"); - substraitLit = param.value().literal(); - break; - } - default: - VELOX_NYI("Substrait conversion not supported for arg type '{}'", std::to_string(typeCase)); - } - } - - static const std::unordered_map functionRevertMap = { - {sLt, sGt}, {sGt, sLt}, {sGte, sLte}, {sLte, sGte}}; - - // Handle the case where literal is before the variable in a binary function, e.g. "123 < q1". - if (typeCases.size() > 1 && (typeCases[0] == "kLiteral" && typeCases[1] == "kSelection")) { - auto x = functionRevertMap.find(functionName); - if (x != functionRevertMap.end()) { - // Change the function name: lt => gt, gt => lt, gte => lte, lte => gte. - functionName = x->second; - } - } - - if (!colIdx.has_value()) { - VELOX_NYI("Column index is expected in subfield filters creation."); - } - - // Set the extracted bound to the specific column. - uint32_t colIdxVal = colIdx.value(); - std::optional val; - - auto inputType = inputTypeList[colIdxVal]; - switch (inputType->kind()) { - case TypeKind::TINYINT: - case TypeKind::SMALLINT: - case TypeKind::INTEGER: - case TypeKind::BIGINT: - case TypeKind::REAL: - case TypeKind::DOUBLE: - case TypeKind::BOOLEAN: - case TypeKind::VARCHAR: - case TypeKind::HUGEINT: - if (substraitLit) { - auto kind = inputType->kind(); - val = VELOX_DYNAMIC_SCALAR_TYPE_DISPATCH(getVariantFromLiteral, kind, substraitLit.value()); - } - break; - case TypeKind::ARRAY: - case TypeKind::MAP: - case TypeKind::ROW: - // Doing nothing here can let filter IsNotNull still work. - break; - default: - VELOX_NYI("Subfield filters creation not supported for input type '{}' in setFilterInfo", inputType->toString()); - } - - setColumnFilterInfo(functionName, val, columnToFilterInfo[colIdxVal], reverse); -} - -template -void SubstraitToVeloxPlanConverter::createNotEqualFilter( - variant notVariant, - bool nullAllowed, - std::vector>& colFilters) { - using NativeType = typename RangeTraits::NativeType; - using RangeType = typename RangeTraits::RangeType; - // Value > lower - std::unique_ptr lowerFilter; - if constexpr (std::is_same_v) { - if (notVariant.value() < getMax()) { - lowerFilter = std::make_unique( - notVariant.value() + 1 /*lower*/, getMax() /*upper*/, nullAllowed); - } - } else { - lowerFilter = std::make_unique( - notVariant.value() /*lower*/, - false /*lowerUnbounded*/, - true /*lowerExclusive*/, - getMax() /*upper*/, - true /*upperUnbounded*/, - false /*upperExclusive*/, - nullAllowed); - } - - // Value < upper - std::unique_ptr upperFilter; - if constexpr (std::is_same_v) { - if (getLowest() < notVariant.value()) { - upperFilter = std::make_unique( - getLowest() /*lower*/, notVariant.value() - 1 /*upper*/, nullAllowed); - } - } else { - upperFilter = std::make_unique( - getLowest() /*lower*/, - true /*lowerUnbounded*/, - false /*lowerExclusive*/, - notVariant.value() /*upper*/, - false /*upperUnbounded*/, - true /*upperExclusive*/, - nullAllowed); - } - - // To avoid overlap of BigintMultiRange, keep this appending order to make sure lower bound of one range is less than - // the upper bounds of others. - if (upperFilter) { - colFilters.emplace_back(std::move(upperFilter)); - } - if (lowerFilter) { - colFilters.emplace_back(std::move(lowerFilter)); - } -} - -template -void SubstraitToVeloxPlanConverter::setInFilter( - const std::vector& variants, - bool nullAllowed, - bool negated, - const std::string& inputName, - connector::hive::SubfieldFilters& filters) {} - -template <> -void SubstraitToVeloxPlanConverter::setInFilter( - const std::vector& variants, - bool nullAllowed, - bool negated, - const std::string& inputName, - connector::hive::SubfieldFilters& filters) { - std::vector values; - values.reserve(variants.size()); - for (const auto& variant : variants) { - int64_t value = variant.value(); - values.emplace_back(value); - } - if (negated) { - filters[common::Subfield(std::move(getPath(inputName)))] = common::createNegatedBigintValues(values, nullAllowed); - } else { - filters[common::Subfield(std::move(getPath(inputName)))] = common::createBigintValues(values, nullAllowed); - } -} - -template <> -void SubstraitToVeloxPlanConverter::setInFilter( - const std::vector& variants, - bool nullAllowed, - bool negated, - const std::string& inputName, - connector::hive::SubfieldFilters& filters) { - // Use bigint values for int type. - std::vector values; - values.reserve(variants.size()); - for (const auto& variant : variants) { - // Use the matched type to get value from variant. - int64_t value = variant.value(); - values.emplace_back(value); - } - if (negated) { - filters[common::Subfield(std::move(getPath(inputName)))] = common::createNegatedBigintValues(values, nullAllowed); - } else { - filters[common::Subfield(std::move(getPath(inputName)))] = common::createBigintValues(values, nullAllowed); - } -} - -template <> -void SubstraitToVeloxPlanConverter::setInFilter( - const std::vector& variants, - bool nullAllowed, - bool negated, - const std::string& inputName, - connector::hive::SubfieldFilters& filters) { - // Use bigint values for small int type. - std::vector values; - values.reserve(variants.size()); - for (const auto& variant : variants) { - // Use the matched type to get value from variant. - int64_t value = variant.value(); - values.emplace_back(value); - } - if (negated) { - filters[common::Subfield(std::move(getPath(inputName)))] = common::createNegatedBigintValues(values, nullAllowed); - } else { - filters[common::Subfield(std::move(getPath(inputName)))] = common::createBigintValues(values, nullAllowed); - } -} - -template <> -void SubstraitToVeloxPlanConverter::setInFilter( - const std::vector& variants, - bool nullAllowed, - bool negated, - const std::string& inputName, - connector::hive::SubfieldFilters& filters) { - // Use bigint values for tiny int type. - std::vector values; - values.reserve(variants.size()); - for (const auto& variant : variants) { - // Use the matched type to get value from variant. - int64_t value = variant.value(); - values.emplace_back(value); - } - if (negated) { - filters[common::Subfield(std::move(getPath(inputName)))] = common::createNegatedBigintValues(values, nullAllowed); - } else { - filters[common::Subfield(std::move(getPath(inputName)))] = common::createBigintValues(values, nullAllowed); - } -} - -template <> -void SubstraitToVeloxPlanConverter::setInFilter( - const std::vector& variants, - bool nullAllowed, - bool negated, - const std::string& inputName, - connector::hive::SubfieldFilters& filters) { - std::vector values; - values.reserve(variants.size()); - for (const auto& variant : variants) { - std::string value = variant.value(); - values.emplace_back(value); - } - if (negated) { - filters[common::Subfield(std::move(getPath(inputName)))] = - std::make_unique(values, nullAllowed); - } else { - filters[common::Subfield(std::move(getPath(inputName)))] = - std::make_unique(values, nullAllowed); - } -} - -template -void SubstraitToVeloxPlanConverter::setSubfieldFilter( - std::vector> colFilters, - const std::string& inputName, - bool nullAllowed, - connector::hive::SubfieldFilters& filters) { - using MultiRangeType = typename RangeTraits::MultiRangeType; - - if (colFilters.size() == 1) { - filters[common::Subfield(std::move(getPath(inputName)))] = std::move(colFilters[0]); - } else if (colFilters.size() > 1) { - // BigintMultiRange should have been sorted - if (colFilters[0]->kind() == common::FilterKind::kBigintRange) { - std::sort(colFilters.begin(), colFilters.end(), [](const auto& a, const auto& b) { - return dynamic_cast(a.get())->lower() < - dynamic_cast(b.get())->lower(); - }); - } - if constexpr (std::is_same_v) { - filters[common::Subfield(std::move(getPath(inputName)))] = - std::make_unique(std::move(colFilters), nullAllowed, true /*nanAllowed*/); - } else { - filters[common::Subfield(std::move(getPath(inputName)))] = - std::make_unique(std::move(colFilters), nullAllowed); - } - } -} - -template -void SubstraitToVeloxPlanConverter::constructSubfieldFilters( - uint32_t colIdx, - const std::string& inputName, - const TypePtr& inputType, - const FilterInfo& filterInfo, - connector::hive::SubfieldFilters& filters) { - if (!filterInfo.isInitialized()) { - return; - } - - bool nullAllowed = filterInfo.nullAllowed_; - bool isNull = filterInfo.isNull_; - bool existIsNullAndIsNotNull = filterInfo.forbidsNullSet_ && filterInfo.isNullSet_; - uint32_t rangeSize = std::max(filterInfo.lowerBounds_.size(), filterInfo.upperBounds_.size()); - - if constexpr (KIND == facebook::velox::TypeKind::HUGEINT) { - // TODO: open it when the Velox's modification is ready. - VELOX_NYI("constructSubfieldFilters not support for HUGEINT type"); - } else if constexpr (KIND == facebook::velox::TypeKind::BOOLEAN) { - // Handle bool type filters. - // Not equal. - if (filterInfo.notValue_) { - filters[common::Subfield(std::move(getPath(inputName)))] = - std::make_unique(!filterInfo.notValue_.value().value(), nullAllowed); - } else if (filterInfo.notValues_.size() > 0) { - std::set notValues; - for (auto v : filterInfo.notValues_) { - notValues.emplace(v.value()); - } - if (notValues.size() == 1) { - filters[common::Subfield(std::move(getPath(inputName)))] = - std::make_unique(!(*notValues.begin()), nullAllowed); - } else { - // if there are more than one distinct value in NOT IN list, the filter should be AlwaysFalse - filters[common::Subfield(std::move(getPath(inputName)))] = std::make_unique(); - } - } else if (rangeSize == 0) { - // IsNull/IsNotNull. - if (!nullAllowed) { - filters[common::Subfield(std::move(getPath(inputName)))] = std::make_unique(); - } else if (isNull) { - filters[common::Subfield(std::move(getPath(inputName)))] = std::make_unique(); - } else { - VELOX_NYI("Only IsNotNull and IsNull are supported in constructSubfieldFilters when no other filter ranges."); - } - return; - } else { - // Equal. - auto value = filterInfo.lowerBounds_[0].value().value(); - VELOX_CHECK(value == filterInfo.upperBounds_[0].value().value(), "invalid state of bool equal"); - filters[common::Subfield(std::move(getPath(inputName)))] = - std::make_unique(value, nullAllowed); - } - } else if constexpr ( - KIND == facebook::velox::TypeKind::ARRAY || KIND == facebook::velox::TypeKind::MAP || - KIND == facebook::velox::TypeKind::ROW) { - // Only IsNotNull and IsNull are supported for complex types. - VELOX_CHECK_EQ(rangeSize, 0, "Only IsNotNull and IsNull are supported for complex type."); - if (!nullAllowed) { - filters[common::Subfield(std::move(getPath(inputName)))] = std::make_unique(); - } else if (isNull) { - filters[common::Subfield(std::move(getPath(inputName)))] = std::make_unique(); - } else { - VELOX_NYI("Only IsNotNull and IsNull are supported for input type '{}'.", inputType->toString()); - } - } else { - using NativeType = typename RangeTraits::NativeType; - using RangeType = typename RangeTraits::RangeType; - using MultiRangeType = typename RangeTraits::MultiRangeType; - - // Handle 'in' filter. - if (filterInfo.values_.size() > 0) { - // To filter out null is a default behaviour of Spark IN expression. - nullAllowed = false; - setInFilter(filterInfo.values_, nullAllowed, false, inputName, filters); - // Currently, In cannot coexist with other filter conditions - // due to multirange is in 'OR' relation but 'AND' is needed. - VELOX_CHECK(rangeSize == 0, "LowerBounds or upperBounds conditons cannot be supported after IN filter."); - VELOX_CHECK(!filterInfo.notValue_.has_value(), "Not equal cannot be supported after IN filter."); - VELOX_CHECK(filterInfo.notValues_.size() == 0, "Not in cannot be supported after IN filter."); - return; - } - - // Handle not in filter. - if (filterInfo.notValues_.size() > 0) { - setInFilter(filterInfo.notValues_, filterInfo.nullAllowed_, true, inputName, filters); - // Currently, NOT In cannot coexist with other filter conditions - // due to multirange is in 'OR' relation but 'AND' is needed. - VELOX_CHECK(rangeSize == 0, "LowerBounds or upperBounds conditons cannot be supported after NOT IN filter."); - VELOX_CHECK(!filterInfo.notValue_.has_value(), "Not equal cannot be supported after NOT IN filter."); - return; - } - - // Construct the Filters. - std::vector> colFilters; - - // Handle not(equal) filter. - if (filterInfo.notValue_) { - variant notVariant = filterInfo.notValue_.value(); - createNotEqualFilter(notVariant, filterInfo.nullAllowed_, colFilters); - // Currently, Not-equal cannot coexist with other filter conditions - // due to multirange is in 'OR' relation but 'AND' is needed. - VELOX_CHECK(rangeSize == 0, "LowerBounds or upperBounds conditons cannot be supported after not-equal filter."); - if constexpr (std::is_same_v) { - if (colFilters.size() == 1) { - filters[common::Subfield(std::move(getPath(inputName)))] = std::move(colFilters.front()); - } else { - filters[common::Subfield(std::move(getPath(inputName)))] = - std::make_unique(std::move(colFilters), nullAllowed, true /*nanAllowed*/); - } - } else { - if (colFilters.size() == 1) { - filters[common::Subfield(std::move(getPath(inputName)))] = std::move(colFilters.front()); - } else { - filters[common::Subfield(std::move(getPath(inputName)))] = - std::make_unique(std::move(colFilters), nullAllowed); - } - } - return; - } - - // Handle null filtering. - if (rangeSize == 0) { - // handle is not null and is null exists at same time - if (existIsNullAndIsNotNull) { - filters[common::Subfield(std::move(getPath(inputName)))] = std::move(std::make_unique()); - } else if (!nullAllowed) { - filters[common::Subfield(std::move(getPath(inputName)))] = std::make_unique(); - } else if (isNull) { - filters[common::Subfield(std::move(getPath(inputName)))] = std::make_unique(); - } else { - VELOX_NYI("Only IsNotNull and IsNull are supported in constructSubfieldFilters when no other filter ranges."); - } - return; - } - - NativeType lowerBound; - if constexpr (KIND == facebook::velox::TypeKind::BIGINT) { - if (inputType->isShortDecimal()) { - lowerBound = DecimalUtil::kShortDecimalMin; - } else { - lowerBound = getLowest(); - } - } else { - lowerBound = getLowest(); - } - - NativeType upperBound; - if constexpr (KIND == facebook::velox::TypeKind::BIGINT) { - if (inputType->isShortDecimal()) { - upperBound = DecimalUtil::kShortDecimalMax; - } else { - upperBound = getMax(); - } - } else { - upperBound = getMax(); - } - - [[maybe_unused]] bool lowerUnbounded = true; - [[maybe_unused]] bool upperUnbounded = true; - bool lowerExclusive = false; - bool upperExclusive = false; - - // Handle other filter ranges. - for (uint32_t idx = 0; idx < rangeSize; idx++) { - if (idx < filterInfo.lowerBounds_.size() && filterInfo.lowerBounds_[idx]) { - lowerUnbounded = false; - variant lowerVariant = filterInfo.lowerBounds_[idx].value(); - lowerBound = lowerVariant.value(); - lowerExclusive = filterInfo.lowerExclusives_[idx]; - } - - if (idx < filterInfo.upperBounds_.size() && filterInfo.upperBounds_[idx]) { - upperUnbounded = false; - variant upperVariant = filterInfo.upperBounds_[idx].value(); - upperBound = upperVariant.value(); - upperExclusive = filterInfo.upperExclusives_[idx]; - } - - std::unique_ptr filter; - if constexpr (std::is_same_v) { - filter = std::move(std::make_unique( - lowerExclusive ? lowerBound + 1 : lowerBound, upperExclusive ? upperBound - 1 : upperBound, nullAllowed)); - } else { - filter = std::move(std::make_unique( - lowerBound, lowerUnbounded, lowerExclusive, upperBound, upperUnbounded, upperExclusive, nullAllowed)); - } - - colFilters.emplace_back(std::move(filter)); - } - - // Set the SubfieldFilter. - setSubfieldFilter(std::move(colFilters), inputName, filterInfo.nullAllowed_, filters); - } -} - bool SubstraitToVeloxPlanConverter::checkTypeExtension(const ::substrait::Plan& substraitPlan) { for (const auto& sExtension : substraitPlan.extensions()) { if (!sExtension.has_extension_type()) { @@ -2367,199 +1396,4 @@ bool SubstraitToVeloxPlanConverter::checkTypeExtension(const ::substrait::Plan& return true; } -connector::hive::SubfieldFilters SubstraitToVeloxPlanConverter::mapToFilters( - const std::vector& inputNameList, - const std::vector& inputTypeList, - std::vector& columnToFilterInfo) { - // Construct the subfield filters based on the filter info map. - connector::hive::SubfieldFilters filters; - for (uint32_t colIdx = 0; colIdx < inputNameList.size(); colIdx++) { - if (columnToFilterInfo[colIdx].isInitialized()) { - auto inputType = inputTypeList[colIdx]; - if (inputType->isDate()) { - constructSubfieldFilters( - colIdx, inputNameList[colIdx], inputType, columnToFilterInfo[colIdx], filters); - continue; - } - switch (inputType->kind()) { - case TypeKind::TINYINT: - constructSubfieldFilters( - colIdx, inputNameList[colIdx], inputType, columnToFilterInfo[colIdx], filters); - break; - case TypeKind::SMALLINT: - constructSubfieldFilters( - colIdx, inputNameList[colIdx], inputType, columnToFilterInfo[colIdx], filters); - break; - case TypeKind::INTEGER: - constructSubfieldFilters( - colIdx, inputNameList[colIdx], inputType, columnToFilterInfo[colIdx], filters); - break; - case TypeKind::BIGINT: - constructSubfieldFilters( - colIdx, inputNameList[colIdx], inputType, columnToFilterInfo[colIdx], filters); - break; - case TypeKind::REAL: - constructSubfieldFilters( - colIdx, inputNameList[colIdx], inputType, columnToFilterInfo[colIdx], filters); - break; - case TypeKind::DOUBLE: - constructSubfieldFilters( - colIdx, inputNameList[colIdx], inputType, columnToFilterInfo[colIdx], filters); - break; - case TypeKind::BOOLEAN: - constructSubfieldFilters( - colIdx, inputNameList[colIdx], inputType, columnToFilterInfo[colIdx], filters); - break; - case TypeKind::VARCHAR: - constructSubfieldFilters( - colIdx, inputNameList[colIdx], inputType, columnToFilterInfo[colIdx], filters); - break; - case TypeKind::HUGEINT: - constructSubfieldFilters( - colIdx, inputNameList[colIdx], inputType, columnToFilterInfo[colIdx], filters); - break; - case TypeKind::ARRAY: - constructSubfieldFilters( - colIdx, inputNameList[colIdx], inputType, columnToFilterInfo[colIdx], filters); - break; - case TypeKind::MAP: - constructSubfieldFilters( - colIdx, inputNameList[colIdx], inputType, columnToFilterInfo[colIdx], filters); - break; - case TypeKind::ROW: - constructSubfieldFilters( - colIdx, inputNameList[colIdx], inputType, columnToFilterInfo[colIdx], filters); - break; - default: - VELOX_NYI( - "Subfield filters creation not supported for input type '{}' in mapToFilters", inputType->toString()); - } - } - } - - return filters; -} - -core::TypedExprPtr SubstraitToVeloxPlanConverter::connectWithAnd( - std::vector inputNameList, - std::vector inputTypeList, - const std::vector<::substrait::Expression_ScalarFunction>& scalarFunctions, - const std::vector<::substrait::Expression_SingularOrList>& singularOrLists, - const std::vector<::substrait::Expression_IfThen>& ifThens) { - if (scalarFunctions.size() == 0 && singularOrLists.size() == 0 && ifThens.size() == 0) { - return nullptr; - } - auto inputType = ROW(std::move(inputNameList), std::move(inputTypeList)); - - // Filter for scalar functions. - std::vector allFilters; - for (auto scalar : scalarFunctions) { - auto filter = exprConverter_->toVeloxExpr(scalar, inputType); - if (filter != nullptr) { - allFilters.emplace_back(filter); - } - } - - for (auto orList : singularOrLists) { - auto filter = exprConverter_->toVeloxExpr(orList, inputType); - if (filter != nullptr) { - allFilters.emplace_back(filter); - } - } - - for (auto ifThen : ifThens) { - auto filter = exprConverter_->toVeloxExpr(ifThen, inputType); - if (filter != nullptr) { - allFilters.emplace_back(filter); - } - } - VELOX_CHECK_GT(allFilters.size(), 0, "One filter should be valid."); - core::TypedExprPtr andFilter = allFilters[0]; - for (auto i = 1; i < allFilters.size(); i++) { - andFilter = connectWithAnd(andFilter, allFilters[i]); - } - - return andFilter; -} - -core::TypedExprPtr SubstraitToVeloxPlanConverter::connectWithAnd( - core::TypedExprPtr leftExpr, - core::TypedExprPtr rightExpr) { - std::vector params; - params.reserve(2); - params.emplace_back(leftExpr); - params.emplace_back(rightExpr); - return std::make_shared(BOOLEAN(), std::move(params), "and"); -} - -bool SubstraitToVeloxPlanConverter::canPushdownSingularOrList( - const ::substrait::Expression_SingularOrList& singularOrList, - bool disableIntLike) { - VELOX_CHECK(singularOrList.options_size() > 0, "At least one option is expected."); - // Check whether the value is field. - bool hasField = singularOrList.value().has_selection(); - const auto& options = singularOrList.options(); - for (const auto& option : options) { - VELOX_CHECK(option.has_literal(), "Literal is expected as option."); - auto type = option.literal().literal_type_case(); - // Only BigintValues and BytesValues are supported. - if (type != ::substrait::Expression_Literal::LiteralTypeCase::kI32 && - type != ::substrait::Expression_Literal::LiteralTypeCase::kI64 && - type != ::substrait::Expression_Literal::LiteralTypeCase::kString) { - return false; - } - - // BigintMultiRange can only accept BigintRange, so disableIntLike is set to - // true for OR pushdown of int-like types. - if (disableIntLike && - (type == ::substrait::Expression_Literal::LiteralTypeCase::kI32 || - type == ::substrait::Expression_Literal::LiteralTypeCase::kI64)) { - return false; - } - } - return hasField; -} - -uint32_t SubstraitToVeloxPlanConverter::getColumnIndexFromSingularOrList( - const ::substrait::Expression_SingularOrList& singularOrList) { - // Get the column index. - ::substrait::Expression_FieldReference selection; - if (singularOrList.value().has_scalar_function()) { - selection = singularOrList.value().scalar_function().arguments()[0].value().selection(); - } else if (singularOrList.value().has_selection()) { - selection = singularOrList.value().selection(); - } else { - VELOX_FAIL("Unsupported type in IN pushdown."); - } - uint32_t index; - VELOX_CHECK( - SubstraitParser::parseReferenceSegment(selection.direct_reference(), index), - "Failed to parse column index from SingularOrList."); - return index; -} - -void SubstraitToVeloxPlanConverter::setFilterInfo( - const ::substrait::Expression_SingularOrList& singularOrList, - std::vector& columnToFilterInfo, - bool reverse) { - VELOX_CHECK(singularOrList.options_size() > 0, "At least one option is expected."); - // Get the column index. - uint32_t colIdx = getColumnIndexFromSingularOrList(singularOrList); - - // Get the value list. - const auto& options = singularOrList.options(); - std::vector variants; - variants.reserve(options.size()); - for (const auto& option : options) { - VELOX_CHECK(option.has_literal(), "Literal is expected as option."); - variants.emplace_back(exprConverter_->toVeloxExpr(option.literal())->value()); - } - // Set the value list to filter info. - if (!reverse) { - columnToFilterInfo[colIdx].setValues(variants); - } else { - columnToFilterInfo[colIdx].setNotValues(variants); - } -} - } // namespace gluten diff --git a/cpp/velox/substrait/SubstraitToVeloxPlan.h b/cpp/velox/substrait/SubstraitToVeloxPlan.h index 0e892469d098..51e50ce34767 100644 --- a/cpp/velox/substrait/SubstraitToVeloxPlan.h +++ b/cpp/velox/substrait/SubstraitToVeloxPlan.h @@ -215,354 +215,13 @@ class SubstraitToVeloxPlanConverter { /// if output order is 'kDriect'. core::PlanNodePtr processEmit(const ::substrait::RelCommon& relCommon, const core::PlanNodePtr& noEmitNode); - /// Multiple conditions are connected to a binary tree structure with - /// the relation key words, including AND, OR, and etc. Currently, only - /// AND is supported. This function is used to extract all the Substrait - /// conditions in the binary tree structure into a vector. - void flattenConditions( - const ::substrait::Expression& sFilter, - std::vector<::substrait::Expression_ScalarFunction>& scalarFunctions, - std::vector<::substrait::Expression_SingularOrList>& singularOrLists, - std::vector<::substrait::Expression_IfThen>& ifThens); - /// Check the Substrait type extension only has one unknown extension. static bool checkTypeExtension(const ::substrait::Plan& substraitPlan); - /// Range filter recorder for a field is used to make sure only the conditions - /// that can coexist for this field being pushed down with a range filter. - class RangeRecorder { - public: - /// Set the existence of values range and returns whether this condition can - /// coexist with existing conditions for one field. Conditions in OR - /// relation can coexist with each other. - bool setInRange(bool forOrRelation = false) { - if (forOrRelation) { - return true; - } - if (inRange_ || multiRange_ || leftBound_ || rightBound_ || isNull_) { - return false; - } - inRange_ = true; - return true; - } - - /// Set the existence of left bound and returns whether it can coexist with - /// existing conditions for this field. - bool setLeftBound(bool forOrRelation = false) { - if (forOrRelation) { - if (!rightBound_) - leftBound_ = true; - return !rightBound_; - } - if (leftBound_ || inRange_ || multiRange_ || isNull_) { - return false; - } - leftBound_ = true; - return true; - } - - /// Set the existence of right bound and returns whether it can coexist with - /// existing conditions for this field. - bool setRightBound(bool forOrRelation = false) { - if (forOrRelation) { - if (!leftBound_) - rightBound_ = true; - return !leftBound_; - } - if (rightBound_ || inRange_ || multiRange_ || isNull_) { - return false; - } - rightBound_ = true; - return true; - } - - /// Set the existence of multi-range and returns whether it can coexist with - /// existing conditions for this field. - bool setMultiRange() { - if (inRange_ || multiRange_ || leftBound_ || rightBound_ || isNull_) { - return false; - } - multiRange_ = true; - return true; - } - - /// Set the existence of IsNull and returns whether it can coexist with - /// existing conditions for this field. - bool setIsNull() { - if (inRange_ || multiRange_ || leftBound_ || rightBound_) { - return false; - } - isNull_ = true; - return true; - } - - /// Set certain existence according to function name and returns whether it - /// can coexist with existing conditions for this field. - bool setCertainRangeForFunction(const std::string& functionName, bool reverse = false, bool forOrRelation = false); - - private: - /// The existence of values range. - bool inRange_ = false; - - /// The existence of left bound. - bool leftBound_ = false; - - /// The existence of right bound. - bool rightBound_ = false; - - /// The existence of multi-range. - bool multiRange_ = false; - - /// The existence of IsNull. - bool isNull_ = false; - }; - - /// Filter info for a column used in filter push down. - class FilterInfo { - public: - // Null is not allowed. - void forbidsNull() { - nullAllowed_ = false; - if (!initialized_) { - initialized_ = true; - } - forbidsNullSet_ = true; - } - - // Only null is allowed. - void setNull() { - isNull_ = true; - nullAllowed_ = true; - if (!initialized_) { - initialized_ = true; - } - isNullSet_ = true; - } - - // Return the initialization status. - bool isInitialized() const { - return initialized_; - } - - // Add a lower bound to the range. Multiple lower bounds are - // regarded to be in 'or' relation. - void setLower(const std::optional& left, bool isExclusive) { - lowerBounds_.emplace_back(left); - lowerExclusives_.emplace_back(isExclusive); - if (!initialized_) { - initialized_ = true; - } - } - - // Add a upper bound to the range. Multiple upper bounds are - // regarded to be in 'or' relation. - void setUpper(const std::optional& right, bool isExclusive) { - upperBounds_.emplace_back(right); - upperExclusives_.emplace_back(isExclusive); - if (!initialized_) { - initialized_ = true; - } - } - - // Set a list of values to be used in the push down of 'in' expression. - void setValues(const std::vector& values) { - for (const auto& value : values) { - values_.emplace_back(value); - } - if (!initialized_) { - initialized_ = true; - } - } - - // Set a value for the not(equal) condition. - void setNotValue(const std::optional& notValue) { - notValue_ = notValue; - if (!initialized_) { - initialized_ = true; - } - } - - // Set a list of values to be used in the push down of 'not in' expression. - void setNotValues(const std::vector& notValues) { - for (const auto& value : notValues) { - notValues_.emplace_back(value); - } - if (!initialized_) { - initialized_ = true; - } - } - - // Whether this filter map is initialized. - bool initialized_ = false; - - bool nullAllowed_ = false; - bool isNull_ = false; - bool forbidsNullSet_ = false; - bool isNullSet_ = false; - - // If true, left bound will be exclusive. - std::vector lowerExclusives_; - - // If true, right bound will be exclusive. - std::vector upperExclusives_; - - // A value should not be equal to. - std::optional notValue_ = std::nullopt; - - // The lower bounds in 'or' relation. - std::vector> lowerBounds_; - - // The upper bounds in 'or' relation. - std::vector> upperBounds_; - - // The list of values used in 'in' expression. - std::vector values_; - - // The list of values should not be equal to. - std::vector notValues_; - }; - /// Returns unique ID to use for plan node. Produces sequential numbers /// starting from zero. std::string nextPlanNodeId(); - /// Returns whether the args of a scalar function being field or - /// field with literal. If yes, extract and set the field index. - static bool fieldOrWithLiteral( - const ::google::protobuf::RepeatedPtrField<::substrait::FunctionArgument>& arguments, - uint32_t& fieldIndex); - - /// Separate the functions to be two parts: - /// subfield functions to be handled by the subfieldFilters in HiveConnector, - /// and remaining functions to be handled by the remainingFilter in - /// HiveConnector. - void separateFilters( - std::vector& rangeRecorders, - const std::vector<::substrait::Expression_ScalarFunction>& scalarFunctions, - std::vector<::substrait::Expression_ScalarFunction>& subfieldFunctions, - std::vector<::substrait::Expression_ScalarFunction>& remainingFunctions, - const std::vector<::substrait::Expression_SingularOrList>& singularOrLists, - std::vector<::substrait::Expression_SingularOrList>& subfieldrOrLists, - std::vector<::substrait::Expression_SingularOrList>& remainingrOrLists, - const std::vector& veloxTypeList, - const dwio::common::FileFormat& format); - - /// Returns whether a function can be pushed down. - static bool canPushdownFunction( - const ::substrait::Expression_ScalarFunction& scalarFunction, - const std::string& filterName, - uint32_t& fieldIdx); - - /// Returns whether a NOT function can be pushed down. - bool canPushdownNot( - const ::substrait::Expression_ScalarFunction& scalarFunction, - std::vector& rangeRecorders); - - /// Returns whether a OR function can be pushed down. - bool canPushdownOr( - const ::substrait::Expression_ScalarFunction& scalarFunction, - std::vector& rangeRecorders); - - /// Returns whether a SingularOrList can be pushed down. - static bool canPushdownSingularOrList( - const ::substrait::Expression_SingularOrList& singularOrList, - bool disableIntLike = false); - - /// Check whether the children functions of this scalar function have the same - /// column index. Curretly used to check whether the two chilren functions of - /// 'or' expression are effective on the same column. - static bool childrenFunctionsOnSameField(const ::substrait::Expression_ScalarFunction& function); - - /// Extract the scalar function, and set the filter info for different types - /// of columns. If reverse is true, the opposite filter info will be set. - void setFilterInfo( - const ::substrait::Expression_ScalarFunction& scalarFunction, - const std::vector& inputTypeList, - std::vector& columnToFilterInfo, - bool reverse = false); - - /// Extract SingularOrList and set it to the filter info map. - /// If reverse is true, the opposite filter info will be set. - void setFilterInfo( - const ::substrait::Expression_SingularOrList& singularOrList, - std::vector& columnToFilterInfo, - bool reverse = false); - - /// Extract SingularOrList and returns the field index. - static uint32_t getColumnIndexFromSingularOrList(const ::substrait::Expression_SingularOrList&); - - /// Set the filter info for a column base on the information - /// extracted from filter condition. - static void setColumnFilterInfo( - const std::string& filterName, - std::optional literalVariant, - FilterInfo& columnToFilterInfo, - bool reverse); - - /// Create a multirange to specify the filter 'x != notValue' with: - /// x > notValue or x < notValue. - template - void createNotEqualFilter(variant notVariant, bool nullAllowed, std::vector>& colFilters); - - /// Create a values range to handle (not) in filter. - /// variants: the list of values extracted from the (not) in expression. - // negated: false for IN filter, true for NOT IN filter. - /// inputName: the column input name. - template - void setInFilter( - const std::vector& variants, - bool nullAllowed, - bool negated, - const std::string& inputName, - connector::hive::SubfieldFilters& filters); - - /// Set the constructed filters into SubfieldFilters. - /// The FilterType is used to distinguish BigintRange and - /// Filter (the base class). This is needed because BigintMultiRange - /// can only accept the unique ptr of BigintRange as parameter. - template - void setSubfieldFilter( - std::vector> colFilters, - const std::string& inputName, - bool nullAllowed, - connector::hive::SubfieldFilters& filters); - - /// Create the subfield filter based on the constructed filter info. - /// inputName: the input name of a column. - template - void constructSubfieldFilters( - uint32_t colIdx, - const std::string& inputName, - const TypePtr& inputType, - const FilterInfo& filterInfo, - connector::hive::SubfieldFilters& filters); - - /// Construct subfield filters according to the pre-set map of filter info. - connector::hive::SubfieldFilters mapToFilters( - const std::vector& inputNameList, - const std::vector& inputTypeList, - std::vector& columnToFilterInfo); - - /// Convert subfield functions into subfieldFilters to - /// be used in Hive Connector. - connector::hive::SubfieldFilters createSubfieldFilters( - const std::vector& inputNameList, - const std::vector& inputTypeList, - const std::vector<::substrait::Expression_ScalarFunction>& subfieldFunctions, - const std::vector<::substrait::Expression_SingularOrList>& singularOrLists); - - /// Connect all remaining functions with 'and' relation - /// for the use of remaingFilter in Hive Connector. - core::TypedExprPtr connectWithAnd( - std::vector inputNameList, - std::vector inputTypeList, - const std::vector<::substrait::Expression_ScalarFunction>& remainingFunctions, - const std::vector<::substrait::Expression_SingularOrList>& singularOrLists, - const std::vector<::substrait::Expression_IfThen>& ifThens); - - /// Connect the left and right expressions with 'and' relation. - core::TypedExprPtr connectWithAnd(core::TypedExprPtr leftExpr, core::TypedExprPtr rightExpr); - /// Used to convert AggregateRel into Velox plan node. /// The output of child node will be used as the input of Aggregation. std::shared_ptr toVeloxAgg( diff --git a/cpp/velox/tests/Substrait2VeloxPlanConversionTest.cc b/cpp/velox/tests/Substrait2VeloxPlanConversionTest.cc index 06d4ea019572..3193f53bfbf1 100644 --- a/cpp/velox/tests/Substrait2VeloxPlanConversionTest.cc +++ b/cpp/velox/tests/Substrait2VeloxPlanConversionTest.cc @@ -256,12 +256,7 @@ TEST_F(Substrait2VeloxPlanConversionTest, ifthenTest) { // Convert to Velox PlanNode. auto planNode = planConverter_->toVeloxPlan(substraitPlan, std::vector<::substrait::ReadRel_LocalFiles>{split}); ASSERT_EQ( - "-- Project[1][expressions: ] -> \n " - "-- TableScan[0][table: hive_table, range filters: [(hd_demo_sk, Filter(IsNotNull, deterministic, null not allowed))," - " (hd_vehicle_count, BigintRange: [1, 9223372036854775807] no nulls)], remaining filter: " - "(and(or(equalto(\"hd_buy_potential\",\">10000\"),equalto(\"hd_buy_potential\",\"unknown\"))," - "if(greaterthan(\"hd_vehicle_count\",0),greaterthan(divide(cast \"hd_dep_count\" as DOUBLE," - "cast \"hd_vehicle_count\" as DOUBLE),1.2))))] -> n0_0:BIGINT, n0_1:VARCHAR, n0_2:BIGINT, n0_3:BIGINT\n", + "-- Project[1][expressions: ] -> \n -- TableScan[0][table: hive_table, remaining filter: (and(and(and(and(isnotnull(\"hd_vehicle_count\"),or(equalto(\"hd_buy_potential\",\">10000\"),equalto(\"hd_buy_potential\",\"unknown\"))),greaterthan(\"hd_vehicle_count\",0)),if(greaterthan(\"hd_vehicle_count\",0),greaterthan(divide(cast \"hd_dep_count\" as DOUBLE,cast \"hd_vehicle_count\" as DOUBLE),1.2))),isnotnull(\"hd_demo_sk\")))] -> n0_0:BIGINT, n0_1:VARCHAR, n0_2:BIGINT, n0_3:BIGINT\n", planNode->toString(true, true)); } @@ -277,8 +272,7 @@ TEST_F(Substrait2VeloxPlanConversionTest, filterUpper) { // Convert to Velox PlanNode. auto planNode = planConverter_->toVeloxPlan(substraitPlan, std::vector<::substrait::ReadRel_LocalFiles>{split}); ASSERT_EQ( - "-- Project[1][expressions: ] -> \n -- TableScan[0][table: hive_table, range filters: " - "[(key, BigintRange: [-2147483648, 2] no nulls)]] -> n0_0:INTEGER\n", + "-- Project[1][expressions: ] -> \n -- TableScan[0][table: hive_table, remaining filter: (and(isnotnull(\"key\"),lessthan(\"key\",3)))] -> n0_0:INTEGER\n", planNode->toString(true, true)); } } // namespace gluten diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh index d802838b4b70..2be1481fc05d 100755 --- a/ep/build-velox/src/get_velox.sh +++ b/ep/build-velox/src/get_velox.sh @@ -17,7 +17,7 @@ set -exu VELOX_REPO=https://github.com/oap-project/velox.git -VELOX_BRANCH=2024_11_13 +VELOX_BRANCH=2024_11_13_new VELOX_HOME="" OS=`uname -s` From 0b899c046f2f0b9c141518f4ef43d2075540d93c Mon Sep 17 00:00:00 2001 From: Rong Ma Date: Thu, 14 Nov 2024 10:28:16 +0800 Subject: [PATCH 15/35] [GLUTEN-7641][VL] Add Gluten benchmark scripts (#7642) --- tools/workload/benchmark_velox/README.md | 38 + tools/workload/benchmark_velox/init_disks.py | 103 + .../workload/benchmark_velox/initialize.ipynb | 2918 +++++++++++++++++ .../native_sql_initialize.ipynb | 1388 ++++++++ .../benchmark_velox/params.yaml.template | 71 + .../benchmark_velox/run_tpc_workload.sh | 86 + .../benchmark_velox/tpc_workload.ipynb | 381 +++ .../parquet_dataset/tpcds_datagen_parquet.sh | 10 +- .../tpch_convert_parquet_dwrf.scala | 62 - .../dwrf_dataset/tpch_convert_parquet_dwrf.sh | 47 - .../parquet_dataset/tpch_datagen_parquet.sh | 8 +- 11 files changed, 4992 insertions(+), 120 deletions(-) create mode 100644 tools/workload/benchmark_velox/README.md create mode 100644 tools/workload/benchmark_velox/init_disks.py create mode 100644 tools/workload/benchmark_velox/initialize.ipynb create mode 100644 tools/workload/benchmark_velox/native_sql_initialize.ipynb create mode 100644 tools/workload/benchmark_velox/params.yaml.template create mode 100755 tools/workload/benchmark_velox/run_tpc_workload.sh create mode 100644 tools/workload/benchmark_velox/tpc_workload.ipynb delete mode 100644 tools/workload/tpch/gen_data/dwrf_dataset/tpch_convert_parquet_dwrf.scala delete mode 100644 tools/workload/tpch/gen_data/dwrf_dataset/tpch_convert_parquet_dwrf.sh diff --git a/tools/workload/benchmark_velox/README.md b/tools/workload/benchmark_velox/README.md new file mode 100644 index 000000000000..5f080077f606 --- /dev/null +++ b/tools/workload/benchmark_velox/README.md @@ -0,0 +1,38 @@ +# Setup, Build and Benchmark Spark/Gluten with Jupyter Notebook + +This guide provides notebooks and scripts for conducting performance testing in Gluten. The standard approach involves setting up the test environment on a bare-metal machine or cloud instance and running performance tests with TPC-H/TPC-DS workloads. These scripts enable users to reproduce our performance results in their own environment. + +## Environment Setup + +The recommended OS is ubuntu22.04 with kernel 5.15. To prepare the environment, run [initialize.ipynb](./initialize.ipynb), which will: + +- Install system dependencies and set up jupyter notebook +- Configure Hadoop and Spark +- Configure kernel parameters +- Build Gluten using Docker +- Generate TPC-H/TPC-DS tables + +## Running TPC-H/TPC-DS Benchmarks + +To run TPC-H/TPC-DS benchmarks, use [tpc_workload.ipynb](./tpc_workload.ipynb). You can create a copy of the notebook and modify the parameters defined in this notebook to run different workloads. However, creating and modifying a copy each time you change workloads can be inconvenient. Instead, it's recommended to use Papermill to pass parameters via the command line for greater flexibility. + +The required parameters are specified in [params.yaml.template](./params.yaml.template). To use it, create your own YAML file by copying and modifying the template. The command to run the notebook is: + +```bash +papermill tpc_workload.ipynb --inject-output-path -f params.yaml gluten_tpch.ipynb +``` +After execution, the output notebook will be saved as `gluten_tpch.ipynb`. + +If you want to use different parameters, you can specify them via the `-p` option. It will overwrite the previously defined parameters in `params.yaml`. e.g. To switch to the TPC-DS workload, run: + +```bash +papermill tpc_workload.ipynb --inject-output-path -f params.yaml -p workoad tpcds gluten_tpcds.ipynb +``` + +Please refer to the Papermill documentation for additional usage details. + +We also provide a script [run_tpc_workload.sh](./run_tpc_workload.sh). This script wraps the Papermill command, automatically renaming the output notebook with a timestamp and application ID to prevent overwriting existing output files. + +## Analyzing Performance Results + +You can check the **Show Performance** section in the output notebook after execution. It shows the cpu% per query, and draws charts for the cpu%, memory throughput, disk throughput/util%, network throughput and pagefaults. diff --git a/tools/workload/benchmark_velox/init_disks.py b/tools/workload/benchmark_velox/init_disks.py new file mode 100644 index 000000000000..8e47e16e3078 --- /dev/null +++ b/tools/workload/benchmark_velox/init_disks.py @@ -0,0 +1,103 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +# To set up the virtual environment required to run this script, +# refer to the `Format and mount disks` subsection under `System Setup` in initialize.ipynb. +import sys +import subprocess +import questionary +import json + +def yes_or_no(question): + while True: + user_input = input(question + '(yes/no/quit): ') + if user_input.lower() == 'yes': + return True + elif user_input.lower() == 'no': + return False + elif user_input.lower() == 'quit': + sys.exit(1) + else: + continue + +def filter_empty_str(l): + return [x for x in l if x] + +def run_and_log(cmd): + # Print command in yellow + print('\033[93m' + '>>> Running command: ' + repr(cmd) + '\033[0m') + result = subprocess.run(cmd, check=True, shell=True, capture_output=True, text=True) + # Print stdout in green + print('\033[92m' + '==========stdout==========' + '\033[0m') + print(result.stdout) + # Print stderr in red + print('\033[91m' + '==========stderr==========' + '\033[0m') + print(result.stderr) + +def init_disks(): + all_disks = filter_empty_str(subprocess.run("lsblk -I 7,8,259 -npd --output NAME".split(' '), capture_output=True, text=True).stdout.split('\n')) + if not all_disks: + print("No disks found on system. Exit.") + sys.exit(0) + + answer = False + disks = [] + while not answer: + disks = questionary.checkbox('Select disks to initialize:', choices=all_disks).ask() + answer = yes_or_no('Confirm selected:\n' + '\n'.join(disks) + '\n') + + if not disks: + print('No disks are selected.') + return + + for d in disks: + print('Initializing {} ...'.format(d)) + run_and_log('wipefs -a {}'.format(d)) + run_and_log('echo "g\nw\n" | fdisk {}'.format(d)) + run_and_log('echo "n\n\n\n\nw\n" | fdisk {}'.format(d)) + run_and_log('mkfs.ext4 {}p1'.format(d)) + +def mount_partitions(): + subprocess.run('lsblk -pf --json > lsblk.json', shell=True) + partitions = [] + with open('lsblk.json', 'r') as f: + data = json.load(f) + for d in data['blockdevices']: + if 'children' in d: + for c in d['children']: + if c['fstype'] == 'ext4': + partitions.append(c['name']) + answer = False + while not answer: + partitions = questionary.checkbox('Select partitions to create mount points:', choices=partitions).ask() + answer = yes_or_no('Confirm selected:\n' + '\n'.join(partitions) + '\n') + + for i, p in enumerate(partitions): + d = 'data{}'.format(i) + run_and_log('e2label {} ""'.format(p)) + run_and_log('e2label {} {}'.format(p, d)) + run_and_log('mkdir -p /{}'.format(d)) + run_and_log('mount -L {} /{}'.format(d, d)) + +def choose(): + choice = questionary.select('Select operation:', choices=['Format disks', 'Mount partitions']).ask() + print(choice) + if choice == 'Format disks': + init_disks() + elif choice == 'Mount partitions': + mount_partitions() + +if __name__ == '__main__': + choose() diff --git a/tools/workload/benchmark_velox/initialize.ipynb b/tools/workload/benchmark_velox/initialize.ipynb new file mode 100644 index 000000000000..cbbc27686951 --- /dev/null +++ b/tools/workload/benchmark_velox/initialize.ipynb @@ -0,0 +1,2918 @@ +{ + "cells": [ + { + "cell_type": "markdown", + "metadata": { + "heading_collapsed": true + }, + "source": [ + "# System Setup" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "**1. Install system dependencies and python packages. Prepare the environment.**" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "First, install all dependencies and python packages as `root`. Run commands and make sure the installations are successful.\n", + "\n", + "```bash\n", + "apt update\n", + "\n", + "apt install -y sudo locales wget tar tzdata git ccache cmake ninja-build build-essential llvm-11-dev clang-11 libiberty-dev libdwarf-dev libre2-dev libz-dev libssl-dev libboost-all-dev libcurl4-openssl-dev openjdk-8-jdk maven vim pip sysstat gcc-9 libjemalloc-dev nvme-cli curl zip unzip bison flex\n", + "\n", + "python3 -m pip install notebook==6.5.2\n", + "python3 -m pip install jupyter_server==1.23.4\n", + "python3 -m pip install jupyter_highlight_selected_word\n", + "python3 -m pip install jupyter_contrib_nbextensions\n", + "python3 -m pip install virtualenv==20.21.1\n", + "python3 -m pip uninstall -y ipython\n", + "python3 -m pip install ipython==8.21.0\n", + "python3 -m pip uninstall -y traitlets\n", + "python3 -m pip install traitlets==5.9.0\n", + "```" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "***Required for Ubuntu***\n", + "\n", + "Check that there isn't an entry for your hostname mapped to 127.0.0.1 or 127.0.1.1 in /etc/hosts (Ubuntu is notorious for this). If there is, delete it.\n", + "Then add `` and `` for master and worker nodes.\n", + "\n", + "Example /etc/hosts:\n", + " \n", + "```\n", + "127.0.0.1 localhost\n", + "\n", + "# The following lines are desirable for IPv6 capable hosts\n", + "::1 ip6-localhost ip6-loopback\n", + "fe00::0 ip6-localnet\n", + "ff00::0 ip6-mcastprefix\n", + "ff02::1 ip6-allnodes\n", + "ff02::2 ip6-allrouters\n", + "\n", + "10.0.0.117 sr217\n", + "10.0.0.113 sr213\n", + "```" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "**2. Format and mount disks**" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "Create a python virtual environment to finish the system setup process:\n", + "\n", + "```bash\n", + "virtualenv -p python3 -v venv\n", + "source venv/bin/activate\n", + "```\n", + "\n", + "And install packages under `venv`:\n", + "```bash\n", + "(venv) python3 -m pip install questionary\n", + "```" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "Run script [init_disks.py](./init_disks.py) to format and mount disks. **Be careful when choosing the disks to format.** If you see errors like `device or resource busy`, perhaps the partition has been mounted, you should unmount it first. If you still see this error, reboot the system and try again." + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "Exit `venv`:\n", + "```bash\n", + "(venv) deactivate\n", + "```" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "**3. Create user `sparkuser`**" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "Create user `sparkuser` without password and with sudo priviledge. It's recommended to use one of the disks as the home directory instead of the system drive.\n", + "\n", + "```bash\n", + "mkdir -p /data0/home/sparkuser\n", + "ln -s /data0/home/sparkuser /home/sparkuser\n", + "cp -r /etc/skel/. /home/sparkuser/\n", + "adduser --home /home/sparkuser --disabled-password --gecos \"\" sparkuser\n", + "\n", + "chown -R sparkuser:sparkuser /data*\n", + "\n", + "echo 'sparkuser ALL=(ALL:ALL) NOPASSWD:ALL' | EDITOR='tee -a' visudo\n", + "```" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "Generate ssh keys for `sparkuser`\n", + "\n", + "```bashrc\n", + "su - sparkuser\n", + "```\n", + "\n", + "```bashrc\n", + "rm -rf ~/.ssh\n", + "ssh-keygen -q -t rsa -N '' -f ~/.ssh/id_rsa <</dev/null 2>&1\n", + "cat ~/.ssh/id_rsa.pub >> ~/.ssh/authorized_keys\n", + "\n", + "exit\n", + "```" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "Generate ssh keys for `root`, and enable no password ssh from `sparkuser`\n", + "\n", + "```bash\n", + "rm -rf /root/.ssh\n", + "ssh-keygen -q -t rsa -N '' -f ~/.ssh/id_rsa <</dev/null 2>&1\n", + "cat /root/.ssh/id_rsa.pub >> /root/.ssh/authorized_keys\n", + "cat /home/sparkuser/.ssh/id_rsa.pub >> /root/.ssh/authorized_keys\n", + "```" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "Login to `sparkuser` and run the first-time ssh to the `root`\n", + "\n", + "```bash\n", + "su - sparkuser\n", + "```\n", + "\n", + "```bash\n", + "ssh -o StrictHostKeyChecking=no root@localhost ls\n", + "ssh -o StrictHostKeyChecking=no root@127.0.0.1 ls\n", + "ssh -o StrictHostKeyChecking=no root@`hostname` ls\n", + "```" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "***Required for Ubuntu***\n", + "\n", + "Run below command to comment out lines starting from `If not running interactively, don't do anything` in ~/.bashrc\n", + "\n", + "```bash\n", + "sed -i '5,9 s/^/# /' ~/.bashrc\n", + "```" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "**4. Configure jupyter notebook**" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "As `sparkuser`, install python packages\n", + "\n", + "```bash\n", + "cd /home/sparkuser/.local/lib/ && rm -rf python*\n", + "\n", + "python3 -m pip install --upgrade jsonschema\n", + "python3 -m pip install jsonschema[format]\n", + "python3 -m pip install sqlalchemy==1.4.46\n", + "python3 -m pip install papermill Black\n", + "python3 -m pip install NotebookScripter\n", + "python3 -m pip install findspark spylon-kernel matplotlib pandasql pyhdfs\n", + "python3 -m pip install ipywidgets jupyter_nbextensions_configurator ipyparallel\n", + "```\n" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "Configure jupyter notebook. Setup password when it prompts\n", + "\n", + "```bash\n", + "jupyter notebook --generate-config\n", + "\n", + "jupyter notebook password\n", + "\n", + "mkdir -p ~/.jupyter/custom/\n", + "\n", + "echo '.container { width:100% !important; }' >> ~/.jupyter/custom/custom.css\n", + "\n", + "echo 'div.output_stderr { background: #ffdd; display: none; }' >> ~/.jupyter/custom/custom.css\n", + "\n", + "jupyter nbextension install --py jupyter_highlight_selected_word --user\n", + "\n", + "jupyter nbextension enable highlight_selected_word/main\n", + "\n", + "jupyter nbextension install --py widgetsnbextension --user\n", + "\n", + "jupyter contrib nbextension install --user\n", + "\n", + "jupyter nbextension enable codefolding/main\n", + "\n", + "jupyter nbextension enable code_prettify/code_prettify\n", + "\n", + "jupyter nbextension enable codefolding/edit\n", + "\n", + "jupyter nbextension enable code_font_size/code_font_size\n", + "\n", + "jupyter nbextension enable collapsible_headings/main\n", + "\n", + "jupyter nbextension enable highlight_selected_word/main\n", + "\n", + "jupyter nbextension enable ipyparallel/main\n", + "\n", + "jupyter nbextension enable move_selected_cells/main\n", + "\n", + "jupyter nbextension enable nbTranslate/main\n", + "\n", + "jupyter nbextension enable scratchpad/main\n", + "\n", + "jupyter nbextension enable tree-filter/index\n", + "\n", + "jupyter nbextension enable comment-uncomment/main\n", + "\n", + "jupyter nbextension enable export_embedded/main\n", + "\n", + "jupyter nbextension enable hide_header/main\n", + "\n", + "jupyter nbextension enable highlighter/highlighter\n", + "\n", + "jupyter nbextension enable scroll_down/main\n", + "\n", + "jupyter nbextension enable snippets/main\n", + "\n", + "jupyter nbextension enable toc2/main\n", + "\n", + "jupyter nbextension enable varInspector/main\n", + "\n", + "jupyter nbextension enable codefolding/edit\n", + "\n", + "jupyter nbextension enable contrib_nbextensions_help_item/main\n", + "\n", + "jupyter nbextension enable freeze/main\n", + "\n", + "jupyter nbextension enable hide_input/main\n", + "\n", + "jupyter nbextension enable jupyter-js-widgets/extension\n", + "\n", + "jupyter nbextension enable snippets_menu/main\n", + "\n", + "jupyter nbextension enable table_beautifier/main\n", + "\n", + "jupyter nbextension enable hide_input_all/main\n", + "\n", + "jupyter nbextension enable spellchecker/main\n", + "\n", + "jupyter nbextension enable toggle_all_line_numbers/main\n", + "\n", + "jupyter nbextensions_configurator enable --user\n", + "```" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "Clone Gluten\n", + "\n", + "```bash\n", + "cd ~\n", + "git clone https://github.com/apache/incubator-gluten.git gluten\n", + "```" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "Start jupyter notebook\n", + "\n", + "```bash\n", + "mkdir -p ~/ipython\n", + "cd ~/ipython\n", + "\n", + "nohup jupyter notebook --ip=0.0.0.0 --port=8888 &\n", + "\n", + "cp ~/gluten/tools/workload/benchmark_velox ~/ipython/\n", + "```" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "heading_collapsed": true + }, + "source": [ + "# Initialize\n", + " Run this section after notebook restart! " + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "Specify datadir. The directories are used for spark.local.dirs and hadoop namenode/datanode." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "datadir=[f'/data{i}' for i in range(0, 8)]\n", + "datadir" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "Specify clients(workers). Leave it empty if the cluster is setup on the local machine." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "clients=''''''.split()\n", + "print(clients)" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "Specify JAVA_HOME" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "java_home = '/usr/lib/jvm/java-8-openjdk-amd64'" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "import os\n", + "import socket\n", + "import platform\n", + "\n", + "user=os.getenv('USER')\n", + "print(f\"user: {user}\")\n", + "print()\n", + "\n", + "masterip=socket.gethostbyname(socket.gethostname())\n", + "hostname=socket.gethostname() \n", + "print(f\"masterip: {masterip} hostname: {hostname}\")\n", + "print()\n", + "\n", + "hclients=clients.copy()\n", + "hclients.append(hostname)\n", + "print(f\"master and workers: {hclients}\")\n", + "print()\n", + "\n", + "\n", + "if clients:\n", + " cmd = f\"ssh {clients[0]} \" + \"\\\"lscpu | grep '^CPU(s)'\\\"\" + \" | awk '{print $2}'\"\n", + " client_cpu = !{cmd}\n", + " cpu_num = client_cpu[0]\n", + "\n", + " cmd = f\"ssh {clients[0]} \" + \"\\\"cat /proc/meminfo | grep MemTotal\\\"\" + \" | awk '{print $2}'\"\n", + " totalmemory = !{cmd}\n", + " totalmemory = int(totalmemory[0])\n", + "else:\n", + " cpu_num = os.cpu_count()\n", + " totalmemory = !cat /proc/meminfo | grep MemTotal | awk '{print $2}'\n", + " totalmemory = int(totalmemory[0])\n", + " \n", + "print(f\"cpu_num: {cpu_num}\")\n", + "print()\n", + "\n", + "print(\"total memory: \", totalmemory, \"KB\")\n", + "print()\n", + "\n", + "mem_mib = int(totalmemory/1024)-1024\n", + "print(f\"mem_mib: {mem_mib}\")\n", + "print()\n", + "\n", + "is_arm = platform.machine() == 'aarch64'\n", + "print(\"is_arm: \",is_arm)\n", + "print()\n", + "\n", + "sparklocals=\",\".join([f'{l}/{user}/yarn/local' for l in datadir])\n", + "print(f\"SPARK_LOCAL_DIR={sparklocals}\")\n", + "print()\n", + "\n", + "%cd ~" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "heading_collapsed": true + }, + "source": [ + "# Set up clients\n", + " SKIP for single node " + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "## Install dependencies" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "Manually configure ssh login without password to all clients\n", + "\n", + "```bash\n", + "ssh-copy-id -o StrictHostKeyChecking=no root@\n", + "```" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "for l in clients:\n", + " !ssh root@{l} apt update > /dev/null 2>&1\n", + " !ssh root@{l} apt install -y sudo locales wget tar tzdata git ccache cmake ninja-build build-essential llvm-11-dev clang-11 libiberty-dev libdwarf-dev libre2-dev libz-dev libssl-dev libboost-all-dev libcurl4-openssl-dev openjdk-8-jdk maven vim pip sysstat gcc-9 libjemalloc-dev nvme-cli curl zip unzip bison flex > /dev/null 2>&1" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "## Create user" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "for l in clients:\n", + " !ssh -o StrictHostKeyChecking=no root@{l} ls" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "for l in clients:\n", + " !ssh root@{l} adduser --disabled-password --gecos '\"\"' {user}" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "for l in clients:\n", + " !ssh root@{l} cp -r .ssh /home/{user}/\n", + " !ssh root@{l} chown -R {user}:{user} /home/{user}/.ssh" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "for l in clients:\n", + " !ssh root@{l} \"echo -e 'sparkuser ALL=(ALL:ALL) NOPASSWD:ALL' | EDITOR='tee -a' visudo\"" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "***Required for Ubuntu***\n", + "\n", + "Run below command to comment out lines starting from If not running interactively, don't do anything in ~/.bashrc" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "for l in clients:\n", + " !ssh {l} sed -i \"'5,9 s/^/# /'\" ~/.bashrc" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "Use /etc/hosts on master node" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "for l in clients:\n", + " !scp /etc/hosts root@{l}:/etc/hosts" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "## Setup disks" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "for l in clients:\n", + " !ssh root@{l} apt update > /dev/null 2>&1\n", + " !ssh root@{l} apt install -y pip > /dev/null 2>&1\n", + " !ssh root@{l} python3 -m pip install virtualenv" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "Manually run **2. Format and mount disks** section under [System Setup](#System-Setup)" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "heading_collapsed": true + }, + "source": [ + "# Configure Spark, Hadoop" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "## Download packages" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!wget https://archive.apache.org/dist/spark/spark-3.3.1/spark-3.3.1-bin-hadoop3.tgz > /dev/null 2>&1" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!wget https://archive.apache.org/dist/hadoop/common/hadoop-3.2.4/hadoop-3.2.4.tar.gz > /dev/null 2>&1\n", + "# backup url: !wget https://dlcdn.apache.org/hadoop/common/hadoop-3.2.4/hadoop-3.2.4.tar.gz > /dev/null 2>&1\n", + "if is_arm:\n", + " # download both versions\n", + " !wget https://dlcdn.apache.org/hadoop/common/hadoop-3.3.5/hadoop-3.3.5-aarch64.tar.gz > /dev/null 2>&1" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "## Create directories" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "cmd=\";\".join([f\"chown -R {user}:{user} \" + l for l in datadir])\n", + "for l in hclients:\n", + " !ssh root@{l} '{cmd}'" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "cmd=\";\".join([f\"rm -rf {l}/tmp; mkdir -p {l}/tmp\" for l in datadir])\n", + "for l in hclients:\n", + " !ssh {l} '{cmd}'" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "cmd=\";\".join([f\"mkdir -p {l}/{user}/hdfs/data; mkdir -p {l}/{user}/yarn/local\" for l in datadir])\n", + "for l in hclients:\n", + " !ssh {l} '{cmd}'" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!mkdir -p {datadir[0]}/{user}/hdfs/name\n", + "!mkdir -p {datadir[0]}/{user}/hdfs/namesecondary" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "for l in hclients:\n", + " !scp hadoop-3.2.4.tar.gz {l}:~/\n", + " !scp spark-3.3.1-bin-hadoop3.tgz {l}:~/\n", + " !ssh {l} \"mv -f hadoop hadoop.bak; mv -f spark spark.bak\"\n", + " !ssh {l} \"tar zxvf hadoop-3.2.4.tar.gz > /dev/null 2>&1\"\n", + " !ssh {l} \"tar -zxvf spark-3.3.1-bin-hadoop3.tgz > /dev/null 2>&1\"\n", + " !ssh root@{l} \"apt install -y openjdk-8-jdk > /dev/null 2>&1\"\n", + " !ssh {l} \"ln -s hadoop-3.2.4 hadoop; ln -s spark-3.3.1-bin-hadoop3 spark\"\n", + " if is_arm:\n", + " !ssh {l} \"tar zxvf hadoop-3.3.5-aarch64.tar.gz > /dev/null 2>&1\"\n", + " !ssh {l} \"cd hadoop && mv lib lib.bak && cp -rf ~/hadoop-3.3.5/lib ~/hadoop\"" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "## Configure bashrc" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "\n", + "cfg=f'''export HADOOP_HOME=~/hadoop\n", + "export PATH=$PATH:$HADOOP_HOME/bin:$HADOOP_HOME/sbin\n", + "\n", + "export HADOOP_CONF_DIR=$HADOOP_HOME/etc/hadoop\n", + "export YARN_CONF_DIR=$HADOOP_HOME/etc/hadoop\n", + "\n", + "export SPARK_HOME=~/spark\n", + "export PYTHONPATH=$SPARK_HOME/python:$SPARK_HOME/python/lib/py4j-0.10.9.5-src.zip:$PYTHONPATH\n", + "export PATH=$SPARK_HOME/bin:$PATH\n", + "\n", + "'''" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "if is_arm:\n", + " cfg += 'export CPU_TARGET=\"aarch64\"\\nexport JAVA_HOME=/usr/lib/jvm/java-8-openjdk-arm64\\nexport PATH=$JAVA_HOME/bin:$PATH\\n'\n", + "else:\n", + " cfg += f'export JAVA_HOME={java_home}\\nexport PATH=$JAVA_HOME/bin:$PATH\\n'" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "with open(\"tmpcfg\",'w') as f:\n", + " f.writelines(cfg)" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "for l in hclients:\n", + " !scp tmpcfg {l}:~/tmpcfg.in\n", + " !ssh {l} \"cat ~/tmpcfg.in >> ~/.bashrc\"" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "for l in hclients:\n", + " !ssh {l} tail -n10 ~/.bashrc" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "## Configure Hadoop" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "for l in hclients:\n", + " !ssh root@{l} \"apt install -y libiberty-dev libxml2-dev libkrb5-dev libgsasl7-dev libuuid1 uuid-dev > /dev/null 2>&1\"" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "### setup short-circuit " + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "for l in hclients:\n", + " !ssh root@{l} \"mkdir -p /var/lib/hadoop-hdfs/\"\n", + " !ssh root@{l} 'chown {user}:{user} /var/lib/hadoop-hdfs/'" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "### enable security.authorization" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "coresite='''\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + " \n", + " fs.default.name\n", + " hdfs://{:s}:8020\n", + " true\n", + " \n", + " \n", + " hadoop.security.authentication\n", + " simple\n", + " \n", + " \n", + " hadoop.security.authorization\n", + " true\n", + " \n", + "\n", + "'''.format(hostname)\n", + "\n", + "with open(f'/home/{user}/hadoop/etc/hadoop/core-site.xml','w') as f:\n", + " f.writelines(coresite)\n", + " \n", + "for l in clients:\n", + " !scp ~/hadoop/etc/hadoop/core-site.xml {l}:~/hadoop/etc/hadoop/core-site.xml >/dev/null 2>&1" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "### set IP check, note the command \", \".join" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "hadooppolicy='''\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + " \n", + " security.service.authorization.default.hosts\n", + " {:s}\n", + " \n", + " \n", + " security.service.authorization.default.acl\n", + " {:s} {:s}\n", + " \n", + " \n", + " \n", + " \n", + " security.client.protocol.acl\n", + " *\n", + " ACL for ClientProtocol, which is used by user code\n", + " via the DistributedFileSystem.\n", + " The ACL is a comma-separated list of user and group names. The user and\n", + " group list is separated by a blank. For e.g. \"alice,bob users,wheel\".\n", + " A special value of \"*\" means all users are allowed.\n", + " \n", + "\n", + " \n", + " security.client.datanode.protocol.acl\n", + " *\n", + " ACL for ClientDatanodeProtocol, the client-to-datanode protocol\n", + " for block recovery.\n", + " The ACL is a comma-separated list of user and group names. The user and\n", + " group list is separated by a blank. For e.g. \"alice,bob users,wheel\".\n", + " A special value of \"*\" means all users are allowed.\n", + " \n", + "\n", + " \n", + " security.datanode.protocol.acl\n", + " *\n", + " ACL for DatanodeProtocol, which is used by datanodes to\n", + " communicate with the namenode.\n", + " The ACL is a comma-separated list of user and group names. The user and\n", + " group list is separated by a blank. For e.g. \"alice,bob users,wheel\".\n", + " A special value of \"*\" means all users are allowed.\n", + " \n", + "\n", + " \n", + " security.inter.datanode.protocol.acl\n", + " *\n", + " ACL for InterDatanodeProtocol, the inter-datanode protocol\n", + " for updating generation timestamp.\n", + " The ACL is a comma-separated list of user and group names. The user and\n", + " group list is separated by a blank. For e.g. \"alice,bob users,wheel\".\n", + " A special value of \"*\" means all users are allowed.\n", + " \n", + "\n", + " \n", + " security.namenode.protocol.acl\n", + " *\n", + " ACL for NamenodeProtocol, the protocol used by the secondary\n", + " namenode to communicate with the namenode.\n", + " The ACL is a comma-separated list of user and group names. The user and\n", + " group list is separated by a blank. For e.g. \"alice,bob users,wheel\".\n", + " A special value of \"*\" means all users are allowed.\n", + " \n", + "\n", + " \n", + " security.admin.operations.protocol.acl\n", + " *\n", + " ACL for AdminOperationsProtocol. Used for admin commands.\n", + " The ACL is a comma-separated list of user and group names. The user and\n", + " group list is separated by a blank. For e.g. \"alice,bob users,wheel\".\n", + " A special value of \"*\" means all users are allowed.\n", + " \n", + "\n", + " \n", + " security.refresh.user.mappings.protocol.acl\n", + " *\n", + " ACL for RefreshUserMappingsProtocol. Used to refresh\n", + " users mappings. The ACL is a comma-separated list of user and\n", + " group names. The user and group list is separated by a blank. For\n", + " e.g. \"alice,bob users,wheel\". A special value of \"*\" means all\n", + " users are allowed.\n", + " \n", + "\n", + " \n", + " security.refresh.policy.protocol.acl\n", + " *\n", + " ACL for RefreshAuthorizationPolicyProtocol, used by the\n", + " dfsadmin and mradmin commands to refresh the security policy in-effect.\n", + " The ACL is a comma-separated list of user and group names. The user and\n", + " group list is separated by a blank. For e.g. \"alice,bob users,wheel\".\n", + " A special value of \"*\" means all users are allowed.\n", + " \n", + "\n", + " \n", + " security.ha.service.protocol.acl\n", + " *\n", + " ACL for HAService protocol used by HAAdmin to manage the\n", + " active and stand-by states of namenode.\n", + " \n", + "\n", + " \n", + " security.zkfc.protocol.acl\n", + " *\n", + " ACL for access to the ZK Failover Controller\n", + " \n", + " \n", + "\n", + " \n", + " security.qjournal.service.protocol.acl\n", + " *\n", + " ACL for QJournalProtocol, used by the NN to communicate with\n", + " JNs when using the QuorumJournalManager for edit logs.\n", + " \n", + "\n", + " \n", + " security.mrhs.client.protocol.acl\n", + " *\n", + " ACL for HSClientProtocol, used by job clients to\n", + " communciate with the MR History Server job status etc.\n", + " The ACL is a comma-separated list of user and group names. The user and\n", + " group list is separated by a blank. For e.g. \"alice,bob users,wheel\".\n", + " A special value of \"*\" means all users are allowed.\n", + " \n", + "\n", + " \n", + "\n", + " \n", + " security.resourcetracker.protocol.acl\n", + " *\n", + " ACL for ResourceTrackerProtocol, used by the\n", + " ResourceManager and NodeManager to communicate with each other.\n", + " The ACL is a comma-separated list of user and group names. The user and\n", + " group list is separated by a blank. For e.g. \"alice,bob users,wheel\".\n", + " A special value of \"*\" means all users are allowed.\n", + " \n", + "\n", + " \n", + " security.resourcemanager-administration.protocol.acl\n", + " *\n", + " ACL for ResourceManagerAdministrationProtocol, for admin commands.\n", + " The ACL is a comma-separated list of user and group names. The user and\n", + " group list is separated by a blank. For e.g. \"alice,bob users,wheel\".\n", + " A special value of \"*\" means all users are allowed.\n", + " \n", + "\n", + " \n", + " security.applicationclient.protocol.acl\n", + " *\n", + " ACL for ApplicationClientProtocol, used by the ResourceManager\n", + " and applications submission clients to communicate with each other.\n", + " The ACL is a comma-separated list of user and group names. The user and\n", + " group list is separated by a blank. For e.g. \"alice,bob users,wheel\".\n", + " A special value of \"*\" means all users are allowed.\n", + " \n", + "\n", + " \n", + " security.applicationmaster.protocol.acl\n", + " *\n", + " ACL for ApplicationMasterProtocol, used by the ResourceManager\n", + " and ApplicationMasters to communicate with each other.\n", + " The ACL is a comma-separated list of user and group names. The user and\n", + " group list is separated by a blank. For e.g. \"alice,bob users,wheel\".\n", + " A special value of \"*\" means all users are allowed.\n", + " \n", + "\n", + " \n", + " security.containermanagement.protocol.acl\n", + " *\n", + " ACL for ContainerManagementProtocol protocol, used by the NodeManager\n", + " and ApplicationMasters to communicate with each other.\n", + " The ACL is a comma-separated list of user and group names. The user and\n", + " group list is separated by a blank. For e.g. \"alice,bob users,wheel\".\n", + " A special value of \"*\" means all users are allowed.\n", + " \n", + "\n", + " \n", + " security.resourcelocalizer.protocol.acl\n", + " *\n", + " ACL for ResourceLocalizer protocol, used by the NodeManager\n", + " and ResourceLocalizer to communicate with each other.\n", + " The ACL is a comma-separated list of user and group names. The user and\n", + " group list is separated by a blank. For e.g. \"alice,bob users,wheel\".\n", + " A special value of \"*\" means all users are allowed.\n", + " \n", + "\n", + " \n", + " security.job.task.protocol.acl\n", + " *\n", + " ACL for TaskUmbilicalProtocol, used by the map and reduce\n", + " tasks to communicate with the parent tasktracker.\n", + " The ACL is a comma-separated list of user and group names. The user and\n", + " group list is separated by a blank. For e.g. \"alice,bob users,wheel\".\n", + " A special value of \"*\" means all users are allowed.\n", + " \n", + "\n", + " \n", + " security.job.client.protocol.acl\n", + " *\n", + " ACL for MRClientProtocol, used by job clients to\n", + " communciate with the MR ApplicationMaster to query job status etc.\n", + " The ACL is a comma-separated list of user and group names. The user and\n", + " group list is separated by a blank. For e.g. \"alice,bob users,wheel\".\n", + " A special value of \"*\" means all users are allowed.\n", + " \n", + "\n", + " \n", + " security.applicationhistory.protocol.acl\n", + " *\n", + " ACL for ApplicationHistoryProtocol, used by the timeline\n", + " server and the generic history service client to communicate with each other.\n", + " The ACL is a comma-separated list of user and group names. The user and\n", + " group list is separated by a blank. For e.g. \"alice,bob users,wheel\".\n", + " A special value of \"*\" means all users are allowed.\n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + "\n", + "'''.format((\",\").join(hclients),user,user)\n", + "\n", + "with open(f'/home/{user}/hadoop/etc/hadoop/hadoop-policy.xml','w') as f:\n", + " f.writelines(hadooppolicy)\n", + " \n", + "for l in clients:\n", + " !scp ~/hadoop/etc/hadoop/hadoop-policy.xml {l}:~/hadoop/etc/hadoop/hadoop-policy.xml >/dev/null 2>&1\n" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "### hdfs config, set replication to 1 to cache all the data" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "code_folding": [], + "hidden": true + }, + "outputs": [], + "source": [ + "hdfs_data=\",\".join([f'{l}/{user}/hdfs/data' for l in datadir])\n", + "\n", + "hdfs_site=f'''\n", + "\n", + "\n", + "\n", + "\n", + "\n", + " \n", + " dfs.namenode.secondary.http-address\n", + " {hostname}:50090\n", + " \n", + " \n", + " dfs.namenode.name.dir\n", + " {datadir[0]}/{user}/hdfs/name\n", + " true\n", + " \n", + "\n", + " \n", + " dfs.datanode.data.dir\n", + " {hdfs_data}\n", + " true\n", + " \n", + "\n", + " \n", + " dfs.namenode.checkpoint.dir\n", + " {datadir[0]}/{user}/hdfs/namesecondary\n", + " true\n", + " \n", + " \n", + " dfs.name.handler.count\n", + " 100\n", + " \n", + " \n", + " dfs.blocksize\n", + " 128m\n", + "\n", + " \n", + " dfs.replication\n", + " 1\n", + "\n", + "\n", + "\n", + " dfs.client.read.shortcircuit\n", + " true\n", + "\n", + "\n", + "\n", + " dfs.domain.socket.path\n", + " /var/lib/hadoop-hdfs/dn_socket\n", + "\n", + "\n", + "\n", + "'''\n", + "\n", + "\n", + "with open(f'/home/{user}/hadoop/etc/hadoop/hdfs-site.xml','w') as f:\n", + " f.writelines(hdfs_site)\n", + " \n", + "for l in clients:\n", + " !scp ~/hadoop/etc/hadoop/hdfs-site.xml {l}:~/hadoop/etc/hadoop/hdfs-site.xml >/dev/null 2>&1\n" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "### mapreduce config" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "mapreduce='''\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + " \n", + " mapreduce.framework.name\n", + " yarn\n", + " \n", + "\n", + " \n", + " mapreduce.job.maps\n", + " 288\n", + " \n", + " \n", + " mapreduce.job.reduces\n", + " 64\n", + " \n", + "\n", + " \n", + " mapreduce.map.java.opts\n", + " -Xmx5120M -DpreferIPv4Stack=true\n", + " \n", + " \n", + " mapreduce.map.memory.mb\n", + " 6144\n", + " \n", + "\n", + " \n", + " mapreduce.reduce.java.opts\n", + " -Xmx5120M -DpreferIPv4Stack=true\n", + " \n", + " \n", + " mapreduce.reduce.memory.mb\n", + " 6144\n", + " \n", + " \n", + " yarn.app.mapreduce.am.staging-dir\n", + " /user\n", + " \n", + " \n", + " mapreduce.task.io.sort.mb\n", + " 2000\n", + " \n", + " \n", + " mapreduce.task.timeout\n", + " 3600000\n", + " \n", + "\n", + "\n", + " mapreduce.jobhistory.address\n", + " {:s}:10020\n", + "\n", + "\n", + "\n", + "'''.format(hostname)\n", + "\n", + "\n", + "with open(f'/home/{user}/hadoop/etc/hadoop/mapred-site.xml','w') as f:\n", + " f.writelines(mapreduce)\n", + " \n", + "for l in clients:\n", + " !scp ~/hadoop/etc/hadoop/mapred-site.xml {l}:~/hadoop/etc/hadoop/mapred-site.xml >/dev/null 2>&1\n" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "### yarn config" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "code_folding": [], + "hidden": true + }, + "outputs": [], + "source": [ + "yarn_site=f'''\n", + "\n", + "\n", + " \n", + " yarn.resourcemanager.hostname\n", + " {hostname}\n", + " \n", + " \n", + " yarn.resourcemanager.address\n", + " {hostname}:8032\n", + " \n", + " \n", + " yarn.resourcemanager.webapp.address\n", + " {hostname}:8088\n", + " \n", + " \n", + " yarn.nodemanager.resource.memory-mb\n", + " {mem_mib}\n", + " \n", + " \n", + " yarn.nodemanager.resource.cpu-vcores\n", + " {cpu_num}\n", + " \n", + " \n", + " yarn.nodemanager.pmem-check-enabled\n", + " false\n", + " \n", + "\n", + " \n", + " yarn.nodemanager.vmem-check-enabled\n", + " false\n", + " \n", + " \n", + " yarn.nodemanager.vmem-pmem-ratio\n", + " 4.1\n", + " \n", + " \n", + " yarn.nodemanager.aux-services\n", + " mapreduce_shuffle,spark_shuffle\n", + " \n", + "\n", + " \n", + " yarn.scheduler.minimum-allocation-mb\n", + " 1024\n", + " \n", + " \n", + " yarn.scheduler.maximum-allocation-mb\n", + " {mem_mib}\n", + " \n", + " \n", + " yarn.scheduler.minimum-allocation-vcores\n", + " 1\n", + " \n", + " \n", + " yarn.scheduler.maximum-allocation-vcores\n", + " {cpu_num}\n", + " \n", + "\n", + " \n", + " yarn.log-aggregation-enable\n", + " false\n", + " \n", + " \n", + " yarn.nodemanager.log.retain-seconds\n", + " 36000\n", + " \n", + " \n", + " yarn.nodemanager.delete.debug-delay-sec\n", + " 3600\n", + " \n", + " \n", + " yarn.log.server.url\n", + " http://{hostname}:19888/jobhistory/logs/\n", + " \n", + "\n", + " \n", + " yarn.nodemanager.log-dirs\n", + " /home/{user}/hadoop/logs/userlogs\n", + " \n", + " \n", + " yarn.nodemanager.local-dirs\n", + " {sparklocals}\n", + " \n", + " \n", + " \n", + " yarn.nodemanager.aux-services.spark_shuffle.class\n", + " org.apache.spark.network.yarn.YarnShuffleService\n", + " \n", + "\n", + "'''\n", + "\n", + "\n", + "with open(f'/home/{user}/hadoop/etc/hadoop/yarn-site.xml','w') as f:\n", + " f.writelines(yarn_site)\n", + " \n", + "for l in clients:\n", + " !scp ~/hadoop/etc/hadoop/yarn-site.xml {l}:~/hadoop/etc/hadoop/yarn-site.xml >/dev/null 2>&1\n" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "### hadoop-env" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "#config java home\n", + "if is_arm:\n", + " !echo \"export JAVA_HOME=/usr/lib/jvm/java-8-openjdk-arm64\" >> ~/hadoop/etc/hadoop/hadoop-env.sh\n", + "else:\n", + " !echo \"export JAVA_HOME={java_home}\" >> ~/hadoop/etc/hadoop/hadoop-env.sh\n", + "\n", + "for l in clients:\n", + " !scp hadoop/etc/hadoop/hadoop-env.sh {l}:~/hadoop/etc/hadoop/ >/dev/null 2>&1\n" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "### workers config" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "if clients:\n", + " with open(f'/home/{user}/hadoop/etc/hadoop/workers','w') as f:\n", + " f.writelines(\"\\n\".join(clients))\n", + " for l in clients:\n", + " !scp hadoop/etc/hadoop/workers {l}:~/hadoop/etc/hadoop/ >/dev/null 2>&1\n", + "else:\n", + " !echo {hostname} > ~/hadoop/etc/hadoop/workers" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "### Copy jar from Spark for external shuffle service" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "for l in hclients:\n", + " !scp spark/yarn/spark-3.3.1-yarn-shuffle.jar {l}:~/hadoop/share/hadoop/common/lib/" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "## Configure Spark" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "eventlog_dir=f'hdfs://{hostname}:8020/tmp/sparkEventLog'" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "sparkconf=f'''\n", + "spark.eventLog.enabled true\n", + "spark.eventLog.dir {eventlog_dir}\n", + "spark.history.fs.logDirectory {eventlog_dir}\n", + "'''\n", + "\n", + "with open(f'/home/{user}/spark/conf/spark-defaults.conf','w+') as f:\n", + " f.writelines(sparkconf)\n", + " \n", + "for l in clients:\n", + " !scp ~/spark/conf/spark-defaults.conf {l}:~/spark/conf/spark-defaults.conf >/dev/null 2>&1" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "sparkenv = f'export SPARK_LOCAL_DIRS={sparklocals}\\n'\n", + "with open(f'/home/{user}/.bashrc', 'a+') as f:\n", + " f.writelines(sparkenv)\n", + "for l in clients:\n", + " !scp ~/.bashrc {l}:~/.bashrc >/dev/null 2>&1" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "for l in hclients:\n", + " !ssh {l} tail -n10 ~/.bashrc" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "heading_collapsed": true + }, + "source": [ + "# Configure startup" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "startup=f'''#!/bin/bash\n", + "echo -1 > /proc/sys/kernel/perf_event_paranoid\n", + "echo 0 > /proc/sys/kernel/kptr_restrict\n", + "echo madvise >/sys/kernel/mm/transparent_hugepage/enabled\n", + "echo 1 > /proc/sys/kernel/numa_balancing\n", + "end=$(($(nproc) - 1))\n", + "for i in $(seq 0 $end); do echo performance > /sys/devices/system/cpu/cpu$i/cpufreq/scaling_governor; done\n", + "for file in $(find /sys/devices/system/cpu/cpu*/power/energy_perf_bias); do echo \"0\" > $file; done\n", + "'''\n", + "\n", + "with open('/tmp/tmpstartup', 'w') as f:\n", + " f.writelines(startup)\n", + "\n", + "startup_service=f'''[Unit]\n", + "Description=Configure Transparent Hugepage, Auto NUMA Balancing, CPU Freq Scaling Governor\n", + "\n", + "[Service]\n", + "ExecStart=/usr/local/bin/mystartup.sh\n", + "\n", + "[Install]\n", + "WantedBy=multi-user.target\n", + "'''\n", + "\n", + "with open('/tmp/tmpstartup_service', 'w') as f:\n", + " f.writelines(startup_service)\n", + " \n", + "for l in hclients:\n", + " !scp /tmp/tmpstartup $l:/tmp/tmpstartup\n", + " !scp /tmp/tmpstartup_service $l:/tmp/tmpstartup_service\n", + " !ssh root@$l \"cat /tmp/tmpstartup > /usr/local/bin/mystartup.sh\"\n", + " !ssh root@$l \"chmod +x /usr/local/bin/mystartup.sh\"\n", + " !ssh root@$l \"cat /tmp/tmpstartup_service > /etc/systemd/system/mystartup.service\"\n", + " !ssh $l \"sudo systemctl enable mystartup.service\"\n", + " !ssh $l \"sudo systemctl start mystartup.service\"\n", + " !ssh $l \"sudo systemctl status mystartup.service\"" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "deletable": false, + "editable": false, + "heading_collapsed": true, + "hidden": true, + "run_control": { + "frozen": true + } + }, + "source": [ + "## Inspect CPU Freq & HT" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "deletable": false, + "editable": false, + "hidden": true, + "run_control": { + "frozen": true + } + }, + "outputs": [], + "source": [ + "if is_arm:\n", + " t = r'''\n", + " #include \n", + " #include \n", + " #include \n", + " #include \n", + " #include \n", + " #include \n", + " #include \n", + " #include \n", + " #include //used for parsing the command line arguments\n", + " #include //used for opening the memory device file\n", + " #include //used for rounding functions\n", + " #include \n", + " #include \n", + " #include \n", + "\n", + " static inline uint64_t GetTickCount()\n", + " {//Return ns counts\n", + " struct timeval tp;\n", + " gettimeofday(&tp,NULL);\n", + " return tp.tv_sec*1000+tp.tv_usec/1000;\n", + " }\n", + "\n", + " uint64_t CNT=CNT_DEF;\n", + "\n", + " int main()\n", + " {\n", + "\n", + " uint64_t start, end;\n", + " start=end=GetTickCount();\n", + "\n", + " asm volatile (\n", + " \"1:\\n\"\n", + " \"SUBS %0,%0,#1\\n\"\n", + " \"bne 1b\\n\"\n", + " ::\"r\"(CNT)\n", + " );\n", + "\n", + " end=GetTickCount();\n", + "\n", + " printf(\" total time = %lu, freq = %lu \\n\", end-start, CNT/(end-start)/1000);\n", + "\n", + " return 0;\n", + " }\n", + " '''\n", + "else:\n", + " t=r'''\n", + " #include \n", + " #include \n", + " #include \n", + " #include \n", + " #include \n", + " #include \n", + " #include \n", + " #include \n", + " #include //used for parsing the command line arguments\n", + " #include //used for opening the memory device file\n", + " #include //used for rounding functions\n", + " #include \n", + " #include \n", + " #include \n", + "\n", + " static inline uint64_t GetTickCount()\n", + " {//Return ns counts\n", + " struct timeval tp;\n", + " gettimeofday(&tp,NULL);\n", + " return tp.tv_sec*1000+tp.tv_usec/1000;\n", + " }\n", + "\n", + " uint64_t CNT=CNT_DEF;\n", + "\n", + " int main()\n", + " {\n", + "\n", + " uint64_t start, end;\n", + " start=end=GetTickCount();\n", + "\n", + " asm volatile (\n", + " \"1:\\n\"\n", + " \"dec %0\\n\"\n", + " \"jnz 1b\\n\"\n", + " ::\"r\"(CNT)\n", + " );\n", + "\n", + " end=GetTickCount();\n", + "\n", + " printf(\" total time = %lu, freq = %lu \\n\", end-start, CNT/(end-start)/1000);\n", + "\n", + " return 0;\n", + " }\n", + " '''" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "deletable": false, + "editable": false, + "hidden": true, + "run_control": { + "frozen": true + } + }, + "outputs": [], + "source": [ + "%cd ~\n", + "with open(\"t.c\", 'w') as f:\n", + " f.writelines(t)\n", + "!gcc -O3 -DCNT_DEF=10000000000LL -o t t.c; gcc -O3 -DCNT_DEF=1000000000000LL -o t.delay t.c;\n", + "!for j in `seq 1 $(nproc)`; do echo -n $j; (for i in `seq 1 $j`; do taskset -c $i ./t.delay & done); sleep 1; ./t; killall t.delay; sleep 2; done" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "heading_collapsed": true + }, + "source": [ + "# Shutdown Jupyter; source ~/.bashrc; reboot Jupyter; run section [Initialize](#Initialize)" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "heading_collapsed": true + }, + "source": [ + "# Build gluten" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "## Install docker" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "# Instructions from https://docs.docker.com/engine/install/ubuntu/\n", + "\n", + "# Add Docker's official GPG key:\n", + "!sudo -E apt-get update\n", + "!sudo -E apt-get install ca-certificates curl\n", + "!sudo -E install -m 0755 -d /etc/apt/keyrings\n", + "!sudo -E curl -fsSL https://download.docker.com/linux/ubuntu/gpg -o /etc/apt/keyrings/docker.asc\n", + "!sudo chmod a+r /etc/apt/keyrings/docker.asc\n", + "\n", + "# Add the repository to Apt sources:\n", + "!echo \\\n", + " \"deb [arch=$(dpkg --print-architecture) signed-by=/etc/apt/keyrings/docker.asc] https://download.docker.com/linux/ubuntu \\\n", + " $(. /etc/os-release && echo \"$VERSION_CODENAME\") stable\" | \\\n", + " sudo -E tee /etc/apt/sources.list.d/docker.list > /dev/null\n", + "!sudo -E apt-get update" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!sudo -E apt-get install -y docker-ce docker-ce-cli containerd.io docker-buildx-plugin docker-compose-plugin >/dev/null 2>&1" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "import os\n", + "http_proxy=os.getenv('http_proxy')\n", + "https_proxy=os.getenv('https_proxy')\n", + "\n", + "if http_proxy or https_proxy:\n", + " !sudo mkdir -p /etc/systemd/system/docker.service.d\n", + " with open('/tmp/http-proxy.conf', 'w') as f:\n", + " s = '''\n", + "[Service]\n", + "{}\n", + "{}\n", + "'''.format(f'Environment=\"HTTP_PROXY={http_proxy}\"' if http_proxy else '', f'Environment=\"HTTPS_PROXY={https_proxy}\"' if https_proxy else '')\n", + " f.writelines(s)\n", + " !sudo cp /tmp/http-proxy.conf /etc/systemd/system/docker.service.d" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!mkdir -p ~/.m2\n", + "\n", + "def get_proxy(proxy):\n", + " pos0 = proxy.rfind('/')\n", + " pos = proxy.rfind(':')\n", + " host = http_proxy[pos0+1:pos]\n", + " port = http_proxy[pos+1:]\n", + " return host, port\n", + "\n", + "if http_proxy or https_proxy:\n", + " with open(f\"/home/{user}/.m2/settings.xml\",\"w+\") as f:\n", + " f.write('''\n", + "\n", + " ''')\n", + " if http_proxy:\n", + " host, port = get_proxy(http_proxy)\n", + " f.write(f'''\n", + " \n", + " http_proxy\n", + " true\n", + " http\n", + " {host}\n", + " {port}\n", + " ''')\n", + " if https_proxy:\n", + " host, port = get_proxy(http_proxy)\n", + " f.write(f'''\n", + " \n", + " https_proxy\n", + " true\n", + " https\n", + " {host}\n", + " {port}\n", + " ''')\n", + " f.write('''\n", + " \n", + "\n", + "''')" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!sudo systemctl daemon-reload" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!sudo systemctl restart docker.service" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "## Build gluten" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!sudo docker pull apache/gluten:vcpkg-centos-7" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "import os\n", + "http_proxy=os.getenv('http_proxy')\n", + "https_proxy=os.getenv('https_proxy')\n", + "\n", + "container=!sudo docker run -e http_proxy={http_proxy} -e https_proxy={https_proxy} -itd apache/gluten:vcpkg-centos-7\n", + "containerid = container[0]\n", + "containerid" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!sudo docker exec {containerid} bash -c \"cd /opt && git clone https://github.com/apache/incubator-gluten.git gluten\"" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!sudo docker exec {containerid} bash -c \"cd /opt && source /opt/rh/devtoolset-9/enable && cd gluten && ./dev/builddeps-veloxbe.sh --enable_vcpkg=ON --enable_hdfs=ON > build.log\"" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "import os\n", + "if os.path.exists(f'/home/{user}/.m2/settings.xml'):\n", + " !sudo docker exec {containerid} bash -c \"mkdir -p ~/.m2\"\n", + " !sudo docker cp /home/{user}/.m2/settings.xml {containerid}:/root/.m2/settings.xml" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!sudo docker exec {containerid} bash -c \"cd /opt/gluten && mvn clean package -DskipTests -Pspark-3.3 -Pbackends-velox\"" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!sudo docker cp {containerid}:/opt/gluten/package/target/gluten-velox-bundle-spark3.3_2.12-centos_7_x86_64-1.3.0-SNAPSHOT.jar ~/" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "for l in clients:\n", + " !scp ~/gluten-velox-bundle-spark3.3_2.12-centos_7_x86_64-1.3.0-SNAPSHOT.jar {l}:~/" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "heading_collapsed": true + }, + "source": [ + "# Generate data" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "## Build spark-sql-perf" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!echo \"deb https://repo.scala-sbt.org/scalasbt/debian all main\" | sudo tee /etc/apt/sources.list.d/sbt.list\n", + "!echo \"deb https://repo.scala-sbt.org/scalasbt/debian /\" | sudo tee /etc/apt/sources.list.d/sbt_old.list\n", + "!curl -sL \"https://keyserver.ubuntu.com/pks/lookup?op=get&search=0x2EE0EA64E40A89B84B2DF73499E82A75642AC823\" | sudo apt-key add\n", + "!sudo -E apt-get update > /dev/null 2>&1\n", + "!sudo -E apt-get install sbt > /dev/null 2>&1" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "import os\n", + "http_proxy=os.getenv('http_proxy')\n", + "https_proxy=os.getenv('https_proxy')\n", + "\n", + "def get_proxy(proxy):\n", + " pos0 = proxy.rfind('/')\n", + " pos = proxy.rfind(':')\n", + " host = http_proxy[pos0+1:pos]\n", + " port = http_proxy[pos+1:]\n", + " return host, port\n", + "\n", + "sbt_opts=''\n", + "\n", + "if http_proxy:\n", + " host, port = get_proxy(http_proxy)\n", + " sbt_opts = f'{sbt_opts} -Dhttp.proxyHost={host} -Dhttp.proxyPort={port}'\n", + "if https_proxy:\n", + " host, port = get_proxy(https_proxy)\n", + " sbt_opts = f'{sbt_opts} -Dhttps.proxyHost={host} -Dhttps.proxyPort={port}'\n", + " \n", + "if sbt_opts:\n", + " %env SBT_OPTS={sbt_opts}" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!git clone https://github.com/databricks/spark-sql-perf.git ~/spark-sql-perf" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!cd ~/spark-sql-perf && sbt package" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!cp ~/spark-sql-perf/target/scala-2.12/spark-sql-perf_2.12-0.5.1-SNAPSHOT.jar ~/ipython/" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "## Start Hadoop/Spark cluster, Spark history server" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!~/hadoop/bin/hadoop namenode -format" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!~/hadoop/bin/hadoop datanode -format " + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!~/hadoop/sbin/start-dfs.sh" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!hadoop dfsadmin -safemode leave" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!hadoop fs -mkdir -p /tmp/sparkEventLog" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!cd ~/spark && sbin/start-history-server.sh" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "import os\n", + "\n", + "master=''\n", + "if clients:\n", + " !~/hadoop/sbin/start-yarn.sh\n", + " master='yarn'\n", + "else:\n", + " # If we run on single node, we use standalone mode\n", + " !{os.environ['SPARK_HOME']}/sbin/stop-slave.sh\n", + " !{os.environ['SPARK_HOME']}/sbin/stop-master.sh\n", + " !{os.environ['SPARK_HOME']}/sbin/start-master.sh\n", + " !{os.environ['SPARK_HOME']}/sbin/start-worker.sh spark://{hostname}:7077 -c {cpu_num}\n", + " master=f'spark://{hostname}:7077'" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!jps\n", + "for l in clients:\n", + " !ssh {l} jps" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "## TPCH" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!rm -rf ~/tpch-dbgen\n", + "!git clone https://github.com/databricks/tpch-dbgen.git ~/tpch-dbgen" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "for l in clients:\n", + " !scp -r ~/tpch-dbgen {l}:~/" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "for l in hclients:\n", + " !ssh {l} cd ~/tpch-dbgen && git checkout 0469309147b42abac8857fa61b4cf69a6d3128a8 && make clean && make OS=LINUX" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "%cd ~/gluten/tools/workload/tpch/gen_data/parquet_dataset" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "# Suggest 2x cpu# partitions.\n", + "scaleFactor = 1500\n", + "numPartitions = 2*cpu_num if len(clients)==0 else len(clients)*2*cpu_num\n", + "dataformat = \"parquet\" # data format of data source\n", + "dataSourceCodec = \"snappy\"\n", + "rootDir = f\"/tpch_sf{scaleFactor}_{dataformat}_{dataSourceCodec}\" # root directory of location to create data in." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "# Verify parameters\n", + "print(f'scaleFactor = {scaleFactor}')\n", + "print(f'numPartitions = {numPartitions}')\n", + "print(f'dataformat = {dataformat}')\n", + "print(f'rootDir = {rootDir}')" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "scala=f'''import com.databricks.spark.sql.perf.tpch._\n", + "\n", + "\n", + "val scaleFactor = \"{scaleFactor}\" // scaleFactor defines the size of the dataset to generate (in GB).\n", + "val numPartitions = {numPartitions} // how many dsdgen partitions to run - number of input tasks.\n", + "\n", + "val format = \"{dataformat}\" // valid spark format like parquet \"parquet\".\n", + "val rootDir = \"{rootDir}\" // root directory of location to create data in.\n", + "val dbgenDir = \"/home/{user}/tpch-dbgen\" // location of dbgen\n", + "\n", + "val tables = new TPCHTables(spark.sqlContext,\n", + " dbgenDir = dbgenDir,\n", + " scaleFactor = scaleFactor,\n", + " useDoubleForDecimal = false, // true to replace DecimalType with DoubleType\n", + " useStringForDate = false) // true to replace DateType with StringType\n", + "\n", + "\n", + "tables.genData(\n", + " location = rootDir,\n", + " format = format,\n", + " overwrite = true, // overwrite the data that is already there\n", + " partitionTables = false, // do not create the partitioned fact tables\n", + " clusterByPartitionColumns = false, // shuffle to get partitions coalesced into single files.\n", + " filterOutNullPartitionValues = false, // true to filter out the partition with NULL key value\n", + " tableFilter = \"\", // \"\" means generate all tables\n", + " numPartitions = numPartitions) // how many dsdgen partitions to run - number of input tasks.\n", + "'''\n", + "\n", + "with open(\"tpch_datagen_parquet.scala\",\"w\") as f:\n", + " f.writelines(scala)" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "executor_cores = 8\n", + "num_executors=cpu_num/executor_cores\n", + "executor_memory = (totalmemory - 10*1024*1024)/num_executors - 1*1024*1024\n", + "\n", + "# Verify parameters\n", + "print(f'--master {master}')\n", + "print(f'--num-executors {int(num_executors)}')\n", + "print(f'--executor-cores {int(executor_cores)}')\n", + "print(f'--executor-memory {int(executor_memory)}k')\n", + "print(f'--conf spark.sql.shuffle.partitions={numPartitions}')\n", + "print(f'--conf spark.sql.parquet.compression.codec={dataSourceCodec}')" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "tpch_datagen_parquet=f'''\n", + "cat tpch_datagen_parquet.scala | {os.environ['SPARK_HOME']}/bin/spark-shell \\\n", + " --master {master} \\\n", + " --name tpch_gen_parquet \\\n", + " --driver-memory 10g \\\n", + " --num-executors {int(num_executors)} \\\n", + " --executor-cores {int(executor_cores)} \\\n", + " --executor-memory {int(executor_memory)}k \\\n", + " --conf spark.executor.memoryOverhead=1g \\\n", + " --conf spark.sql.broadcastTimeout=4800 \\\n", + " --conf spark.driver.maxResultSize=4g \\\n", + " --conf spark.sql.shuffle.partitions={numPartitions} \\\n", + " --conf spark.sql.parquet.compression.codec={dataSourceCodec} \\\n", + " --conf spark.network.timeout=800s \\\n", + " --conf spark.executor.heartbeatInterval=200s \\\n", + " --jars /home/{user}/ipython/spark-sql-perf_2.12-0.5.1-SNAPSHOT.jar \\\n", + "'''\n", + "\n", + "with open(\"tpch_datagen_parquet.sh\",\"w\") as f:\n", + " f.writelines(tpch_datagen_parquet)\n" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!nohup bash tpch_datagen_parquet.sh" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "## TPCDS" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!rm -rf ~/tpcds-kit\n", + "!git clone https://github.com/databricks/tpcds-kit.git ~/tpcds-kit" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "for l in clients:\n", + " !scp -r ~/tpcds-kit {l}:~/" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "for l in hclients:\n", + " !ssh {l} \"cd ~/tpcds-kit/tools && make clean && make OS=LINUX CC=gcc-9\"" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "%cd ~/gluten/tools/workload/tpcds/gen_data/parquet_dataset" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "# Suggest 2x cpu# partitions\n", + "scaleFactor = 1500\n", + "numPartitions = 2*cpu_num if len(clients)==0 else len(clients)*2*cpu_num\n", + "dataformat = \"parquet\" # data format of data source\n", + "dataSourceCodec = \"snappy\"\n", + "rootDir = f\"/tpcds_sf{scaleFactor}_{dataformat}_{dataSourceCodec}\" # root directory of location to create data in." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "# Verify parameters\n", + "print(f'scaleFactor = {scaleFactor}')\n", + "print(f'numPartitions = {numPartitions}')\n", + "print(f'dataformat = {dataformat}')\n", + "print(f'rootDir = {rootDir}')" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "scala=f'''import com.databricks.spark.sql.perf.tpcds._\n", + "\n", + "val scaleFactor = \"{scaleFactor}\" // scaleFactor defines the size of the dataset to generate (in GB).\n", + "val numPartitions = {numPartitions} // how many dsdgen partitions to run - number of input tasks.\n", + "\n", + "val format = \"{dataformat}\" // valid spark format like parquet \"parquet\".\n", + "val rootDir = \"{rootDir}\" // root directory of location to create data in.\n", + "val dsdgenDir = \"/home/{user}/tpcds-kit/tools/\" // location of dbgen\n", + "\n", + "val tables = new TPCDSTables(spark.sqlContext,\n", + " dsdgenDir = dsdgenDir,\n", + " scaleFactor = scaleFactor,\n", + " useDoubleForDecimal = false, // true to replace DecimalType with DoubleType\n", + " useStringForDate = false) // true to replace DateType with StringType\n", + "\n", + "\n", + "tables.genData(\n", + " location = rootDir,\n", + " format = format,\n", + " overwrite = true, // overwrite the data that is already there\n", + " partitionTables = true, // create the partitioned fact tables\n", + " clusterByPartitionColumns = true, // shuffle to get partitions coalesced into single files.\n", + " filterOutNullPartitionValues = false, // true to filter out the partition with NULL key value\n", + " tableFilter = \"\", // \"\" means generate all tables\n", + " numPartitions = numPartitions) // how many dsdgen partitions to run - number of input tasks.\n", + "'''\n", + "\n", + "with open(\"tpcds_datagen_parquet.scala\",\"w\") as f:\n", + " f.writelines(scala)" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "executor_cores = 8\n", + "num_executors=cpu_num/executor_cores\n", + "executor_memory = (totalmemory - 10*1024*1024)/num_executors - 1*1024*1024\n", + "\n", + "# Verify parameters\n", + "print(f'--master {master}')\n", + "print(f'--num-executors {int(num_executors)}')\n", + "print(f'--executor-cores {int(executor_cores)}')\n", + "print(f'--executor-memory {int(executor_memory)}k')\n", + "print(f'--conf spark.sql.shuffle.partitions={numPartitions}')\n", + "print(f'--conf spark.sql.parquet.compression.codec={dataSourceCodec}')" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "tpcds_datagen_parquet=f'''\n", + "cat tpcds_datagen_parquet.scala | {os.environ['SPARK_HOME']}/bin/spark-shell \\\n", + " --master {master} \\\n", + " --name tpcds_gen_parquet \\\n", + " --driver-memory 10g \\\n", + " --num-executors {int(num_executors)} \\\n", + " --executor-cores {int(executor_cores)} \\\n", + " --executor-memory {int(executor_memory)}k \\\n", + " --conf spark.executor.memoryOverhead=1g \\\n", + " --conf spark.sql.broadcastTimeout=4800 \\\n", + " --conf spark.driver.maxResultSize=4g \\\n", + " --conf spark.sql.shuffle.partitions={numPartitions} \\\n", + " --conf spark.sql.parquet.compression.codec={dataSourceCodec} \\\n", + " --conf spark.network.timeout=800s \\\n", + " --conf spark.executor.heartbeatInterval=200s \\\n", + " --jars /home/{user}/ipython/spark-sql-perf_2.12-0.5.1-SNAPSHOT.jar \\\n", + "'''\n", + "\n", + "with open(\"tpcds_datagen_parquet.sh\",\"w\") as f:\n", + " f.writelines(tpcds_datagen_parquet)" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!nohup bash tpcds_datagen_parquet.sh" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "heading_collapsed": true + }, + "source": [ + "# Install Trace-Viewer (optional)" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "Clone the master branch\n", + "```\n", + "cd ~\n", + "git clone https://github.com/catapult-project/catapult.git -b master\n", + "```" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "Trace-Viewer requires python version 2.7. Create a virtualenv for python2.7\n", + "```\n", + "sudo apt install -y python2.7\n", + "virtualenv -p /usr/bin/python2.7 py27\n", + "source py27/bin/activate\n", + "```" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "Apply patch\n", + "\n", + "```\n", + "cd catapult\n", + "```\n", + "```\n", + "git apply <\n", + "" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "%%html\n", + "\n", + "" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "%%html\n", + "\n", + "" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "%%html\n", + "\n", + "" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "# System Settings" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "import os\n", + "import pandas as pd\n", + "\n", + "pd.set_option('display.max_rows', None)\n", + "\n", + "# Convert the os.environ object to a dictionary and then to a DataFrame\n", + "env_df = pd.DataFrame(list(dict(os.environ).items()), columns=['Environment Variable', 'Value'])\n", + "\n", + "# Display the DataFrame\n", + "from IPython.display import display\n", + "\n", + "display(env_df)" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "import socket\n", + "localhost=socket.gethostname()\n", + "local_ip=socket.gethostbyname(localhost)\n", + "\n", + "print(f'localhost: {localhost}')\n", + "print(f'ip: {local_ip}')" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "spark_version=!head -n1 $SPARK_HOME/RELEASE | awk '{print $2}'\n", + "spark_version = spark_version[0]\n", + "\n", + "print(f\"Spark version from SPARK_HOME: {spark_version}\")" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "import logging\n", + "import sys\n", + "\n", + "logging.basicConfig(format='%(levelname)s : %(message)s', level=logging.ERROR, stream=sys.stdout)\n", + "logger = logging.getLogger()" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "import os\n", + "\n", + "hdfs_event_dir=''\n", + "local_event_dir=''\n", + "\n", + "def get_spark_eventlog_dir(path):\n", + " eventlog_dir = None\n", + " eventlog_enabled = False\n", + " try:\n", + " with open(path, 'r') as f:\n", + " for line in f:\n", + " if line.startswith('spark.eventLog.dir'):\n", + " eventlog_dir = line.split(' ')[-1].strip()\n", + " elif line.startswith('spark.eventLog.enabled'):\n", + " eventlog_enabled = line.split(' ')[-1].strip().lower() == 'true'\n", + " except FileNotFoundError:\n", + " raise SystemExit(f\"'spark-defaults.conf' not found: {path}\")\n", + " if not eventlog_enabled:\n", + " raise SystemExit(\"'spark.eventLog.enabled' must be enabled.\")\n", + " return eventlog_dir\n", + "\n", + "spark_defaults_conf = None\n", + "\n", + "if 'SPARK_CONF_DIR' in os.environ:\n", + " spark_defaults_conf = os.path.join(os.environ['SPARK_CONF_DIR'], 'spark-defaults.conf')\n", + "elif 'SPARK_HOME' in os.environ:\n", + " spark_defaults_conf = os.path.join(os.environ['SPARK_HOME'], 'conf', 'spark-defaults.conf')\n", + "\n", + "if spark_defaults_conf:\n", + " event_log_dir = get_spark_eventlog_dir(spark_defaults_conf)\n", + " if event_log_dir:\n", + " print(f\"spark.eventLog.dir: {event_log_dir}\")\n", + " if event_log_dir[:7] == 'hdfs://':\n", + " hdfs_event_dir = event_log_dir\n", + " elif event_log_dir[:6] == 'file:/':\n", + " local_event_dir = event_log_dir[6:]\n", + " else:\n", + " raise SystemExit(f\"'spark.eventLog.dir' is not configured in {spark_defaults_conf}\")\n", + "else:\n", + " raise SystemExit(\"Cannot get `spark.eventLog.dir`. Neither SPARK_CONF_DIR nor SPARK_HOME defined in envrionment variables.\")\n", + " " + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "# Monitor" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "import findspark\n", + "import os\n", + "\n", + "findspark.init(os.environ['SPARK_HOME'])\n", + "os.environ.setdefault('SPARK_SUBMIT_OPTS', '-Dscala.usejavacp=true')" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "import warnings\n", + "warnings.filterwarnings('ignore')\n", + "\n", + "import atexit\n", + "import collections\n", + "import gzip\n", + "import importlib\n", + "import json\n", + "import logging\n", + "import math\n", + "import os\n", + "import pathlib\n", + "import shutil\n", + "import signal\n", + "import subprocess\n", + "import tempfile\n", + "import threading\n", + "import time\n", + "import timeit\n", + "import traceback\n", + "\n", + "import matplotlib\n", + "import matplotlib.colors as colors\n", + "import matplotlib.pyplot as plt\n", + "import matplotlib.ticker as mtick\n", + "import numpy as np\n", + "import pandas as pd\n", + "import platform\n", + "import pyspark\n", + "import pyspark.sql.functions as F\n", + "import pyspark.sql.types as T\n", + "import spylon_kernel\n", + "from collections import namedtuple\n", + "from concurrent.futures import ThreadPoolExecutor\n", + "from datetime import date\n", + "from functools import reduce\n", + "from IPython.display import display, HTML\n", + "from matplotlib import rcParams\n", + "from pyspark import SparkConf, SparkContext\n", + "from pyspark.ml import Pipeline\n", + "from pyspark.ml.feature import StringIndexer, VectorAssembler\n", + "from pyspark.sql import SparkSession, SQLContext, Window\n", + "from pyspark.sql.functions import col, floor, lit, rank, to_date\n", + "from pyspark.sql.types import (DoubleType, FloatType, IntegerType,\n", + " StringType, StructField, StructType,\n", + " TimestampType)\n", + "\n", + "from spylon_kernel import register_ipython_magics\n", + "from spylon.spark.utils import SparkJVMHelpers\n", + "\n", + "register_ipython_magics()\n", + "\n", + "rcParams['font.sans-serif'] = 'Courier New'\n", + "rcParams['font.family'] = 'Courier New'\n", + "rcParams['font.size'] = '12'\n", + "\n", + "%matplotlib inline\n" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "code_folding": [] + }, + "outputs": [], + "source": [ + "import socket\n", + "import os\n", + "import sys\n", + "\n", + "from pathlib import Path\n", + "home = str(Path.home())\n", + "\n", + "def upload_profile(server, base_dir, appid):\n", + " local_profile_dir = os.path.join(home, 'profile')\n", + " !mkdir -p {local_profile_dir}\n", + " !cd {local_profile_dir}; rm -f {appid}.tar.gz; tar zcvf {appid}.tar.gz {appid} >/dev/null 2>&1\n", + " \n", + " server_local_dir=os.path.join('PAUS', base_dir)\n", + " server_local_profile_dir=os.path.join(server_local_dir, 'profile')\n", + " server_hdfs_dir=f'/{base_dir}/'\n", + "\n", + " !ssh {server} \"mkdir -p {server_local_profile_dir}\"\n", + " !ssh {server} \"cd {server_local_profile_dir}; rm {appid}.tar.gz; rm -r {appid} >/dev/null 2>&1\"\n", + " !scp {local_profile_dir}/{appid}.tar.gz {server}:{server_local_profile_dir}/\n", + " !ssh {server} \"cd {server_local_profile_dir} && tar zxf {appid}.tar.gz\"\n", + " !ssh {server} \"hdfs dfs -mkdir -p {server_hdfs_dir}; hdfs dfs -rm -r {server_hdfs_dir}{appid}; hdfs dfs -put {server_local_profile_dir}/{appid} {server_hdfs_dir}\"\n", + " !ssh {server} \"cd {server_local_profile_dir}; rm {appid}.tar.gz; rm -r {appid}\"\n", + "\n", + "def killsar(clients):\n", + " for l in clients:\n", + " out=!ssh $l \"ps aux | grep -w sar | grep -v grep | tr -s ' ' | cut -d' ' -f2\"\n", + " for x in out:\n", + " !ssh $l \"kill $x > /dev/null 2>&1\"\n", + " for l in clients:\n", + " out=!ssh $l \"ps aux | grep -w pidstat | grep -v grep | tr -s ' ' | cut -d' ' -f2\"\n", + " for x in out:\n", + " !ssh $l \"kill $x > /dev/null 2>&1\"\n", + " for l in clients:\n", + " out=!ssh $l \"ps aux | grep -w perf | grep -v grep | tr -s ' ' | cut -d' ' -f2\"\n", + " for x in out:\n", + " !ssh root@$l \"kill $x > /dev/null 2>&1\"\n", + "\n", + "def killnumactl(clients):\n", + " for l in clients:\n", + " out =!ssh $l \"ps aux | grep numactl | grep bash | tr -s ' ' | cut -d' ' -f2\"\n", + " for x in out:\n", + " !ssh $l \"kill $x > /dev/null 2>&1\"\n", + "\n", + "def startmonitor(clients,appid,**kwargs):\n", + " local_profile_dir=os.path.join(home, 'profile')\n", + " prof=os.path.join(local_profile_dir, appid)\n", + " !mkdir -p {prof}\n", + " \n", + " for l in clients:\n", + " !ssh root@{l} date\n", + " \n", + " killsar(clients)\n", + " \n", + " perfsyscalls=kwargs.get(\"collect_perf_syscall\",None)\n", + " \n", + " for l in clients:\n", + " prof_client=os.path.join(prof, l)\n", + " !mkdir -p {prof_client}\n", + " !ssh {l} mkdir -p {prof_client}\n", + " !ssh {l} \"sar -o {prof_client}/sar.bin -r -u -d -B -n DEV 1 >/dev/null 2>&1 &\"\n", + " !ssh root@{l} \"jps | grep CoarseGrainedExecutorBackend | cut -d' ' -f 1 | xargs -I % bash -c '(cat /proc/%/status >> {prof_client}/%.stat; cat /proc/%/io >> {prof_client}/%.stat)'\"\n", + " if kwargs.get(\"collect_pid\",False):\n", + " !ssh {l} \"jps | grep CoarseGrainedExecutorBackend | head -n 1 | cut -d' ' -f 1 | xargs -I % pidstat -h -t -p % 1 > {prof_client}/pidstat.out 2>/dev/null &\"\n", + " !ssh root@{l} 'cat /proc/uptime | cut -d\" \" -f 1 | xargs -I ^ date -d \"- ^ seconds\" +%s.%N' > $prof/$l/uptime.txt\n", + " if kwargs.get(\"collect_sched\",False):\n", + " !ssh root@{l} 'perf trace -e \"sched:sched_switch\" -C 8-15 -o {prof_client}/sched.txt -T -- sleep 10000 >/dev/null 2>/dev/null &'\n", + " if perfsyscalls is not None:\n", + " !ssh root@{l} \"perf stat -e 'syscalls:sys_exit_poll,syscalls:sys_exit_epoll_wait' -a -I 1000 -o {prof_client}/perfstat.txt >/dev/null 2>&1 & \"\n", + " if kwargs.get(\"collect_hbm\",False):\n", + " hbm_nodes = kwargs.get(\"hbm_nodes\")\n", + " if hbm_nodes is not None:\n", + " print(\"collect_hbm\")\n", + " hbm_nodes = '\\|'.join([\"node \" + str(i) for i in hbm_nodes])\n", + " %env hbm_numa_nodes={hbm_nodes}\n", + " %env hbm_l = {l}\n", + " %env hbm_prof = {prof}\n", + " !ssh $hbm_l \"echo timestamp, size, free > $hbm_prof/$hbm_l/numactl.csv\"\n", + " !ssh $hbm_l \"while :; do echo \\$(numactl -H | grep '$hbm_numa_nodes' | grep 'size' | awk '{ print \\$4 }' | awk '{ s += \\$1 } END { print s }'), \\$(numactl -H | grep '$hbm_numa_nodes' | grep 'free' | awk '{ print \\$4 }' | awk '{ s += \\$1 } END { print s }') | ts '%Y-%m-%d %H:%M:%S,' >> $hbm_prof/$hbm_l/numactl.csv; sleep 1; done >/dev/null 2>&1 &\"\n", + " else:\n", + " print(\"Missing argument: hbm_nodes. e.g. hbm_nodes = list(range(8,16))\")\n", + " return prof\n", + "\n", + "def stopmonitor(clients, sc, appid, **kwargs):\n", + " %cd ~\n", + " \n", + " local_profile_dir=os.path.join(home, 'profile')\n", + " prof=os.path.join(local_profile_dir, appid)\n", + " !mkdir -p {prof}\n", + "\n", + " killsar(clients)\n", + " killnumactl(clients) \n", + " \n", + " with open(f\"{prof}/starttime\",\"w\") as f:\n", + " f.write(\"{:d}\".format(int(time.time()*1000)))\n", + " \n", + " for l in clients:\n", + " prof_client=os.path.join(prof, l)\n", + " !ssh {l} \"sar -f {prof_client}/sar.bin -r > {prof_client}/sar_mem.sar;sar -f {prof_client}/sar.bin -u > {prof_client}/sar_cpu.sar;sar -f {prof_client}/sar.bin -d -p > {prof_client}/sar_disk.sar;sar -f {prof_client}/sar.bin -n DEV > {prof_client}/sar_nic.sar;sar -f {prof_client}/sar.bin -B > {prof_client}/sar_page.sar;\" \n", + " !ssh root@{l} \"jps | grep CoarseGrainedExecutorBackend | cut -d' ' -f 1 | xargs -I % bash -c '(cat /proc/%/status >> {prof_client}/%.stat; cat /proc/%/io >> {prof_client}/%.stat)'\"\n", + " !ssh {l} \"sar -V \" > {prof_client}/sarv.txt\n", + " !test -f {prof_client}/perfstat.txt && head -n 1 {prof_client}/perfstat.txt > {prof_client}/perfstarttime\n", + " if l!= socket.gethostname():\n", + " !scp -r {l}:{prof_client} {prof}/ > /dev/null 2>&1\n", + " \n", + " if sc is not None:\n", + " sc.stop()\n", + " \n", + " if hdfs_event_dir != '':\n", + " !hadoop fs -copyToLocal {hdfs_event_dir}/{appid} {prof}/app.log\n", + " elif local_event_dir != '':\n", + " !cp {local_event_dir}/{appid} {prof}/app.log" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "def pinexecutor_numa(clients):\n", + " cpunum = !ssh {clients[0]} \"grep 'processor' /proc/cpuinfo | wc -l\"\n", + " cpunum = int(cpunum[0])\n", + " \n", + " numanodes=!ssh {clients[0]} \"cat /sys/devices/system/node/node*/cpulist\"\n", + " numanodes = list(filter(lambda x: x != '', numanodes))\n", + " print(numanodes)\n", + " for client in clients:\n", + " pids=!ssh {client} \"jps | grep CoarseGrainedExecutorBackend | cut -d' ' -f1\"\n", + " print(client,\":\",len(pids),\" \",\"\\t\".join(map(str,pids)))\n", + " \n", + " cpunum_c = !ssh {client} \"grep 'processor' /proc/cpuinfo | wc -l\"\n", + " cpunum_c = int(cpunum_c[0])\n", + " if cpunum_c != cpunum:\n", + " print(f\"client {client} cpunum not match!\")\n", + " return\n", + " numanodes_c=!ssh {client} \"cat /sys/devices/system/node/node*/cpulist\"\n", + " numanodes_c = list(filter(lambda x: x != '', numanodes))\n", + " time.sleep(1)\n", + " print(numanodes_c)\n", + " if numanodes_c != numanodes:\n", + " print(f\"client {client} numanodes not match!\")\n", + " return\n", + " \n", + " idx = 0\n", + " nodes=len(numanodes)\n", + " for i in range(nodes):\n", + " cpus = numanodes[i]\n", + " for l in pids[idx:idx+int(len(pids)/nodes)]: # executors on 1 numanode\n", + " print(f\" {cpus} {l}\")\n", + " !ssh {client} \"taskset -a -p -c $cpus $l > /dev/null 2>&1 \"\n", + " idx += int(len(pids)/nodes)" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "def config_pagecache(clients, run_gluten=True):\n", + " for l in clients:\n", + " if run_gluten:\n", + " !ssh root@$l \"echo 80 > /proc/sys/vm/dirty_ratio\"\n", + " !ssh root@$l \"echo 50 > /proc/sys/vm/dirty_background_ratio\"\n", + " !ssh root@$l \"echo 360000 > /proc/sys/vm/dirty_expire_centisecs\"\n", + " !ssh root@$l \"echo 3000 > /proc/sys/vm/dirty_writeback_centisecs\"\n", + "\n", + " else:\n", + " !ssh root@$l \"echo 10 > /proc/sys/vm/dirty_ratio\"\n", + " !ssh root@$l \"echo 20 > /proc/sys/vm/dirty_background_ratio\"\n", + " !ssh root@$l \"echo 3000 > /proc/sys/vm/dirty_expire_centisecs\"\n", + " !ssh root@$l \"echo 500 > /proc/sys/vm/dirty_writeback_centisecs\"" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "def print_kernel_params(clietns):\n", + " params = {\n", + " 'transparent hugepage': '/sys/kernel/mm/transparent_hugepage/enabled',\n", + " 'auto numa balancing': '/proc/sys/kernel/numa_balancing',\n", + " 'scaling governor': '/sys/devices/system/cpu/cpu*/cpufreq/scaling_governor',\n", + " 'scaling max freq': '/sys/devices/system/cpu/cpu*/cpufreq/scaling_max_freq',\n", + " 'scaling cur freq': '/sys/devices/system/cpu/cpu*/cpufreq/scaling_cur_freq',\n", + " 'power & perf policy': '/sys/devices/system/cpu/cpu*/power/energy_perf_bias',\n", + " 'dirty_ratio': '/proc/sys/vm/dirty_ratio',\n", + " 'dirty_background_ratio': '/proc/sys/vm/dirty_background_ratio',\n", + " 'dirty_expire_centisecs': '/proc/sys/vm/dirty_expire_centisecs',\n", + " 'dirty_writeback_centisecs': '/proc/sys/vm/dirty_writeback_centisecs'\n", + " }\n", + " for k, param in params.items():\n", + " print()\n", + " print(f'{k} ({param})')\n", + " for l in clients:\n", + " print(l + \": \", end='')\n", + " res = !ssh root@$l \"cat {param}\"\n", + " print(*res)\n", + " # print numactl\n", + " print()\n", + " print(\"numactl -H\")\n", + " for l in clients:\n", + " print(l + \":\")\n", + " res = !ssh $l \"numactl -H\"\n", + " print('\\n'.join(res))\n", + " # print memory freq\n", + " print()\n", + " print(\"Memory Frequency\")\n", + " for l in clients:\n", + " print(l + \":\")\n", + " res= !ssh root@$l \"dmidecode -t memory | grep Speed\"\n", + " print('\\n'.join(res))" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "code_folding": [] + }, + "outputs": [], + "source": [ + "def dropcache(clients):\n", + " for l in clients:\n", + " !ssh root@$l \"sync && echo 3 > /proc/sys/vm/drop_caches; echo 1 >/proc/sys/vm/compact_memory; free -h\"" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "def config_mem_cgroup(clients):\n", + " mem_cgroup = \"\"\"\n", + "CGROUP_ROOT=/sys/fs/cgroup/gluten\n", + "\n", + "if [ ! -d $CGROUP_ROOT ] ; then\n", + " sudo mkdir $CGROUP_ROOT\n", + " # enable memory for subtree\n", + " sudo bash -c \"echo '+memory' >> $CGROUP_ROOT/cgroup.subtree_control\"\n", + "fi\n", + "\n", + "# move each process to sub memory group\n", + "index=0\n", + "for pid in `jps | grep Coarse | awk '{print $1}'` ; do\n", + " target_cgroup=$CGROUP_ROOT/mem-${index}\n", + " if [ ! -d $target_cgroup ] ; then\n", + " sudo mkdir $target_cgroup\n", + " fi\n", + " proc_file=$target_cgroup/cgroup.procs\n", + " sudo bash -c \"echo $pid >> $proc_file\"\n", + " index=`expr $index + 1`\n", + "done\n", + " \"\"\"\n", + " with open(f'{home}/mem-cgroup.sh', 'w+') as f:\n", + " f.writelines(mem_cgroup)\n", + " for l in clients:\n", + " !scp {home}/mem-cgroup.sh {l}:{home}/ >/dev/null 2>&1\n", + " !ssh {l} \"bash {home}/mem-cgroup.sh >/dev/null 2>&1 &\"" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "import pandas as pd\n", + "import matplotlib.pyplot as plt\n", + "import os\n", + "\n", + "\n", + "from IPython.display import display, HTML\n", + "\n", + "def get_io_stats(appid, client):\n", + " file_path = os.path.join(home,'profile',appid,client)\n", + " statf = [f for f in os.listdir(file_path) if f.endswith('.stat')]\n", + " statmap=[]\n", + " for f in statf:\n", + " statmap.append({'pid':f[:-len(\".stat\")]})\n", + " with open(os.path.join(file_path, f),\"r\") as fi:\n", + " cnts=fi.readlines()\n", + " for l in cnts:\n", + " for fld in ['rchar','wchar','syscr','syscw','read_bytes','write_bytes','cancelled_write_bytes']:\n", + " if l.startswith(fld):\n", + " if not fld in statmap[-1]:\n", + " statmap[-1][fld]=int(l.split(\" \")[-1].strip())\n", + " else:\n", + " statmap[-1][fld]=(int(l.split(\" \")[-1].strip())-statmap[-1][fld])/1024/1024/1024\n", + "\n", + " df = pd.DataFrame(statmap).drop('pid', axis=1).sum().to_frame()\n", + " df.columns = ['sum']\n", + " return df\n", + "\n", + "# Preprocess 'time' column\n", + "def process_time(dataframes):\n", + " for df in dataframes:\n", + " df.columns=['time']+list(df.columns[1:])\n", + " df = df[df.time != 'Average:']\n", + " df['time'] = pd.to_datetime(df['time'], format='%H:%M:%S').dt.time\n", + " df['time'] = df['time'].apply(lambda dt: dt.hour*3600 + dt.minute*60 + dt.second)\n", + "\n", + " offset = 12 * 3600 # half-day seconds\n", + " for i in range(1, len(df)):\n", + " if df['time'].iloc[i] < df['time'].iloc[i-1]: # Detect AM->PM or PM->AM\n", + " for j in range(i, len(df)): # Apply offset until end\n", + " df['time'].iloc[j] += offset\n", + "\n", + " df['time'] = df['time'].astype(int)\n", + " yield df\n", + "\n", + "def draw_sar(appid, qtime=None, disk_dev=None, nic_dev=None, client=None):\n", + " if client is None:\n", + " client = clients[0]\n", + "\n", + " display(HTML('{:s}'.format(client)))\n", + "\n", + " display(get_io_stats(appid, client))\n", + "\n", + " # Read data\n", + " profile_dir = os.path.join(home,'profile',appid,client)\n", + " datafiles = [os.path.join(profile_dir, datafile) for datafile in ['sar_cpu.sar', 'sar_mem.sar', 'sar_disk.sar', 'sar_nic.sar', 'sar_page.sar']]\n", + " dataframes = [pd.read_csv(datafile, header=1, delim_whitespace=True, parse_dates=True) for datafile in datafiles]\n", + " \n", + " num_figs=5\n", + " fig, axs=plt.subplots(num_figs,1,sharex=True,figsize=(30,5*4))\n", + "\n", + " [cpu_df, mem_df, disk_df, nic_df, page_df] = process_time(dataframes)\n", + "\n", + " # CPU usage\n", + " cpu_df['total'] = cpu_df['%user'] + cpu_df['%system'] + cpu_df['%iowait']\n", + "\n", + " starttime = cpu_df[cpu_df['total'] > 50]['time'].min() - 1\n", + " cpu_df['time'] -= starttime\n", + "\n", + " axs[4].stackplot(cpu_df['time'], cpu_df['%user'], cpu_df['%system'], cpu_df['%iowait'], labels=['user','system','iowait'])\n", + " axs[4].legend(loc='upper left')\n", + "\n", + " # Memory usage\n", + " mem_df['dirty_cached'] = mem_df['kbdirty'] * mem_df['%memused'] / mem_df['kbmemused']\n", + " mem_df['clean_cached'] = (mem_df['kbcached'] - mem_df['kbdirty']) * mem_df['%memused'] / mem_df['kbmemused']\n", + " mem_df['used'] = mem_df['kbmemused'] * mem_df['%memused'] / mem_df['kbmemused']\n", + "# mem_df['used'] = (mem_df['kbmemused'] - mem_df['kbbuffers'] - mem_df['kbcached'])* mem_df['%memused'] / mem_df['kbmemused']\n", + "\n", + " mem_df['time'] -= starttime\n", + "\n", + " axs[0].stackplot(mem_df['time'], mem_df['used'], mem_df['clean_cached'], mem_df['dirty_cached'], labels=['used','clean cached','dirty cached'])\n", + " axs[0].legend(loc='upper left')\n", + " axs[0].grid(axis = 'y')\n", + "\n", + " # Disk usage\n", + " if disk_dev is not None:\n", + " disk_df = disk_df[disk_df['DEV'].isin(disk_dev)]\n", + " disk_df['rkB/s'] = disk_df['rkB/s'].astype(float)\n", + " disk_df['wkB/s'] = disk_df['wkB/s'].astype(float)\n", + " disk_df['%util'] = disk_df['%util'].astype(float)\n", + "\n", + "\n", + " disk_df = disk_df.groupby('time').agg({'rkB/s': 'sum', 'wkB/s': 'sum', '%util':'mean'}).reset_index()\n", + " disk_df['read'] = disk_df['rkB/s'] / 1024\n", + " disk_df['write'] = disk_df['wkB/s'] / 1024\n", + "\n", + " disk_df['time'] -= starttime\n", + "\n", + " axs[1].stackplot(disk_df['time'], disk_df['read'], disk_df['write'], labels=['read MB/s','write MB/s'])\n", + " axs[1].grid(axis = 'y')\n", + "\n", + " ax2 = axs[1].twinx()\n", + "\n", + " ax2.plot(disk_df['time'], disk_df['%util'],'g-')\n", + " axs[1].legend(loc='upper left')\n", + "\n", + " \n", + " # Nic usage\n", + " if nic_dev is not None:\n", + " nic_df = nic_df[nic_df['IFACE'].isin(nic_dev)]\n", + " nic_df['rxkB/s'] = nic_df['rxkB/s'].astype(float)\n", + " nic_df['txkB/s'] = nic_df['txkB/s'].astype(float)\n", + " \n", + " nic_df = nic_df.groupby('time').agg({'rxkB/s': 'sum', 'txkB/s': \"sum\"}).reset_index()\n", + " nic_df['rx'] = nic_df['rxkB/s'] / 1024\n", + " nic_df['tx'] = nic_df['txkB/s'] / 1024\n", + " \n", + " nic_df['time'] -= starttime\n", + " \n", + " axs[2].stackplot(nic_df['time'], nic_df['rx'], nic_df['tx'], labels=['rx MB/s','tx MB/s'])\n", + " axs[2].legend(loc='upper left')\n", + " axs[2].grid(axis = 'y')\n", + "\n", + " # Pagefaults\n", + " page_df['minflt/s'] = page_df['fault/s'] - page_df['majflt/s']\n", + " \n", + " page_df['time'] -= starttime\n", + "\n", + " axs[3].stackplot(page_df['time'], page_df['minflt/s'], page_df['majflt/s'], labels=['minor_fault/s','major_fault/s'])\n", + " axs[3].legend(loc='upper left')\n", + " axs[3].grid(axis = 'y')\n", + "\n", + " # Add vertical lines and text for qtime, and calculate per query cpu%\n", + " if qtime is not None:\n", + " for ax in axs:\n", + " x = 0\n", + " ax.axvline(x = x, color = 'b')\n", + " for k, v in qtime.items():\n", + " x += v\n", + " ax.axvline(x = x, color = 'b')\n", + "\n", + " tx = 0\n", + " for k, v in qtime.items():\n", + " if v / x > 15 / 772:\n", + " ax.text(tx + v / 2 - 6 * x / 772, ax.get_ylim()[1] * 1.05, k)\n", + " tx += v\n", + "\n", + " x = 0\n", + " qtime_se = {}\n", + " cols = ['%user','%system','%iowait']\n", + " for k, v in qtime.items():\n", + " filtered_df = cpu_df[(cpu_df['time'] >= x) & (cpu_df['time'] <= x+v)]\n", + " averages = filtered_df[cols].mean()\n", + " qtime_se[k] = averages.tolist()\n", + " x += v\n", + " if qtime_se:\n", + " perqcpu = pd.DataFrame(qtime_se).T\n", + " perqcpu.columns = cols\n", + " display(perqcpu)\n", + "\n", + " plt.show()\n" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "def convert_to_etc_gmt(tz_offset=None):\n", + " # Run the 'date +%z' command and get the output\n", + " if not tz_offset:\n", + " tz_offset = !date +%z\n", + " tz_offset = tz_offset[0]\n", + " \n", + " # Extract the sign and the hour/minute offset\n", + " sign = tz_offset[0]\n", + " hours = int(tz_offset[1:3])\n", + " minutes = int(tz_offset[3:])\n", + "\n", + " # Convert the offset to a GMT value\n", + " gmt_offset = hours + (minutes / 60)\n", + " if sign == '+':\n", + " gmt_offset = -gmt_offset\n", + " else:\n", + " gmt_offset = abs(gmt_offset)\n", + "\n", + " # Construct the Etc/GMT string\n", + " etc_gmt = f\"Etc/GMT{int(gmt_offset):+d}\"\n", + " return etc_gmt" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "# TestTPC" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "code_folding": [] + }, + "outputs": [], + "source": [ + "import os\n", + "import socket\n", + "from dataclasses import dataclass\n", + "from functools import wraps\n", + "from pathlib import Path\n", + "from typing import List \n", + "\n", + "class TestTPC:\n", + " @dataclass\n", + " class query_info:\n", + " tables: List[str]\n", + " sql: List[str]\n", + "\n", + " query_infos = {}\n", + " query_ids =[]\n", + "\n", + " tpctables=[]\n", + " tpc_query_path = ''\n", + " \n", + " def __init__(self, spark, table_dir, run_gluten, workload, server, base_dir, nb_name, data_source = 'parquet'):\n", + " self.spark = spark\n", + " self.sc = spark.sparkSession.sparkContext\n", + " self.appid = self.sc.applicationId\n", + " self.run_gluten = run_gluten\n", + " self.workload = workload\n", + " self.table_dir = table_dir\n", + " self.server = server\n", + " self.base_dir = base_dir\n", + " self.nb_name = nb_name\n", + " self.data_source = data_source\n", + " self.table_loaded = False\n", + " self.result = {}\n", + " self.stopped = False\n", + " self.perf_html = ''\n", + " self.finished_nb = ''\n", + " for l in os.listdir(self.tpc_query_path):\n", + " if (l[-3:] == 'sql'):\n", + " with open(self.tpc_query_path+l,\"r\") as f:\n", + " self.query_infos[l.split(\".\")[0]]=self.query_info(self.tpctables,[\"\\n\".join(f.readlines())])\n", + " self.query_ids = sorted(self.query_infos.keys(), key=lambda x: str(len(x))+x if x[-1] != 'a' and x[-1] != 'b' else str(len(x)-1) + x)\n", + " print(\"http://{}:18080/history/{}/jobs/\".format(local_ip, self.sc.applicationId))\n", + " \n", + " def start_monitor(self, clients, **kw):\n", + " startmonitor(clients, self.appid, **kw)\n", + " \n", + " def stop_monitor(self, clients, **kw):\n", + " if self.stopped:\n", + " return\n", + " stopmonitor(clients, self.sc, self.appid, **kw)\n", + " if self.server:\n", + " output_nb = f'{self.nb_name[:-6]}-{self.appid}.ipynb'\n", + " if output_nb.startswith(home):\n", + " output_nb_name = os.path.relpath(output_nb, home)\n", + " else:\n", + " output_nb_name = output_nb\n", + " output_nb_dir = os.path.dirname(output_nb_name)\n", + " server_nb_dir = os.path.join('PAUS', self.base_dir, output_nb_dir)\n", + " !ssh {self.server} \"mkdir -p {server_nb_dir}\"\n", + " !scp {output_nb} {self.server}:{server_nb_dir}\n", + " self.finished_nb = f\"http://{self.server}:8888/tree/{self.base_dir}/{output_nb_name}\"\n", + " self.stopped = True\n", + "\n", + " def run_perf_analysis(self, disk_dev, nic_dev):\n", + " if not self.server:\n", + " return\n", + "\n", + " upload_profile(self.server, self.base_dir, self.appid)\n", + "\n", + " ts=time.strftime(\"%Y_%m_%d_%H%M%S\")\n", + " name=f'{self.workload}_gluten' if self.run_gluten else f'{self.workload}_vanilla'\n", + " run_script=f'{gluten_home}/tools/workload/benchmark_velox/analysis/run_perf_analysis.sh'\n", + " \n", + " disk=','.join(disk_dev)\n", + " nic=','.join(nic_dev)\n", + "\n", + " command =' '.join(['bash', run_script, '--ts', ts, '--base-dir', self.base_dir, '--name', name, '--appid', self.appid, '--disk', disk, '--nic', nic, '--tz', convert_to_etc_gmt()])\n", + " print(command)\n", + "\n", + " # Block if running on local cluster.\n", + " if self.server == localhost:\n", + " !ssh {self.server} \"{command} > /dev/null 2>&1\"\n", + " else:\n", + " !ssh {self.server} \"{command} > /dev/null 2>&1 &\"\n", + "\n", + " self.perf_html=f'http://{self.server}:8888/view/{self.base_dir}/html/{ts}_{name}_{self.appid}.html'\n", + " display(HTML(f'{self.perf_html}'))\n", + " \n", + " def load_table(self, table):\n", + " if type(self.table_dir)==list:\n", + " return self.spark.read.format(self.data_source).load([os.path.join(t, table) for t in self.table_dir])\n", + " else:\n", + " return self.spark.read.format(self.data_source).load(os.path.join(self.table_dir, table))\n", + " \n", + " def load_tables_as_tempview(self, tables):\n", + " for table in tables:\n", + " df = self.load_table(table)\n", + " df.createOrReplaceTempView(table)\n", + " \n", + " def load_all_tables_as_tempview(self):\n", + " print(f\"Loading all tables: {self.tpctables}\")\n", + " self.load_tables_as_tempview(self.tpctables)\n", + " \n", + " def load_query(self, query):\n", + " info = self.query_infos[query]\n", + " return [self.spark.sql(q) for q in info.sql]\n", + " \n", + " def run_query(self, query, explain = False, print_result=False, load_table=True):\n", + " if load_table:\n", + " self.load_all_tables_as_tempview()\n", + " start_time = timeit.default_timer()\n", + " print(\"start query \" + query + \", application id \" + self.sc.applicationId)\n", + " print(\"{} : {}\".format(\"Start time\", start_time))\n", + " self.sc.setJobDescription(query)\n", + "\n", + " queries = self.load_query(query)\n", + " for q in queries:\n", + " if explain: q.explain()\n", + " collect=q.collect()\n", + " end_time = timeit.default_timer()\n", + " duration = end_time - start_time\n", + " display(HTML(('Completed Query. Time(sec): {:f}'.format(duration))))\n", + " \n", + " self.result[query] = duration\n", + " if print_result:\n", + " print(collect)\n", + "\n", + " def power_run(self, explain=False, print_result=False, load_table=True):\n", + " if load_table:\n", + " self.load_all_tables_as_tempview()\n", + " for l in self.query_ids:\n", + " self.run_query(l, explain=explain, print_result=print_result, load_table=False)\n", + "\n", + " def print_result(self):\n", + " print(self.result)\n", + " print()\n", + " durations = [float(i) for i in self.result.values()]\n", + " print(\"total duration:\")\n", + " print(sum(durations))\n", + " print()\n", + " if self.server:\n", + " print(self.finished_nb)\n", + " print(f\"http://{self.server}:1088/tracing_examples/trace_viewer.html#/tracing/test_data/{self.appid}.json\")\n", + " print(f\"http://{self.server}:18080/history/{self.appid}\")\n", + " print(self.perf_html)\n", + " print(self.appid)\n", + " for i in durations:\n", + " print(i)\n", + " \n", + "class TestTPCH(TestTPC):\n", + " tpctables = ['customer', 'lineitem', 'nation', 'orders', 'part', 'partsupp', 'region', 'supplier']\n", + " tpc_query_path = f'{gluten_home}/tools/gluten-it/common/src/main/resources/tpch-queries/'\n", + " \n", + " def __init__(self, spark, table_dir, run_gluten, server, base_dir, nb_name, data_source = 'parquet'):\n", + " TestTPC.__init__(self,spark, table_dir, run_gluten, 'tpch', server, base_dir, nb_name, data_source)\n", + " \n", + "class TestTPCDS(TestTPC):\n", + " tpctables = [ 'call_center',\n", + " 'catalog_page',\n", + " 'catalog_returns',\n", + " 'catalog_sales',\n", + " 'customer',\n", + " 'customer_address',\n", + " 'customer_demographics',\n", + " 'date_dim',\n", + " 'household_demographics',\n", + " 'income_band',\n", + " 'inventory',\n", + " 'item',\n", + " 'promotion',\n", + " 'reason',\n", + " 'ship_mode',\n", + " 'store',\n", + " 'store_returns',\n", + " 'store_sales',\n", + " 'time_dim',\n", + " 'warehouse',\n", + " 'web_page',\n", + " 'web_returns',\n", + " 'web_sales',\n", + " 'web_site']\n", + " tpc_query_path = f'{gluten_home}/tools/gluten-it/common/src/main/resources/tpcds-queries/'\n", + " \n", + " def __init__(self, spark, table_dir, run_gluten, server, base_dir, nb_name, data_source = 'parquet'):\n", + " TestTPC.__init__(self,spark, table_dir, run_gluten, 'tpcds', server, base_dir, nb_name, data_source)" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "# Create SparkContext" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## default config" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "code_folding": [] + }, + "outputs": [], + "source": [ + "import os\n", + "\n", + "def findjemalloc():\n", + " l = clients[0]\n", + " jemallocDir = !ssh $l \"whereis libjemalloc.so.2\"\n", + " libjemalloc = jemallocDir[0].split(' ')\n", + " return libjemalloc[1]\n", + "\n", + "def get_py4jzip():\n", + " spark_home=os.environ['SPARK_HOME']\n", + " py4jzip = !ls {spark_home}/python/lib/py4j*.zip\n", + " return py4jzip[0]\n", + "\n", + "def default_conf(executors_per_node, cores_per_executor, task_per_core, memory_per_node, extra_jars='', app_name='', master='yarn', run_gluten=False):\n", + " # Create a temp directory that gets cleaned up on exit\n", + " output_dir = os.path.abspath(tempfile.mkdtemp())\n", + " def cleanup():\n", + " shutil.rmtree(output_dir, True)\n", + " atexit.register(cleanup)\n", + " signal.signal(signal.SIGTERM, cleanup)\n", + "\n", + "##################################################\n", + " def convert_to_bytes(size):\n", + " units = {'k': 1, 'm': 2, 'g': 3}\n", + " size = size.lower()\n", + " if size[-1] in units:\n", + " return int(size[:-1]) * 1024 ** units[size[-1]]\n", + " else:\n", + " return int(size)\n", + "\n", + " def yarn_padding(size):\n", + " min_size = convert_to_bytes('1g')\n", + " step = min_size\n", + " while size > min_size:\n", + " min_size += step\n", + " return min_size - size\n", + " \n", + " num_nodes = len(clients)\n", + " num_executors = num_nodes*executors_per_node\n", + " parallelism = num_executors*cores_per_executor*task_per_core\n", + "\n", + " if run_gluten:\n", + " offheap_ratio = gluten_offheap_ratio\n", + " else:\n", + " offheap_ratio = vanilla_offheap_ratio\n", + " driver_memory = convert_to_bytes('20g')\n", + " executor_memory_overhead = convert_to_bytes('1g')\n", + " \n", + " # Minimun executor memory\n", + " min_memory = convert_to_bytes('1g')\n", + "\n", + " # Calculate executor onheap memory\n", + " num_driver = 1 if localhost in clients else 0\n", + " executor_memory = math.floor((convert_to_bytes(memory_per_node) - (executor_memory_overhead + min_memory)*executors_per_node - (driver_memory + min_memory)*num_driver)/(offheap_ratio*num_driver + (1+offheap_ratio)*executors_per_node))\n", + " executor_memory = max(executor_memory, min_memory)\n", + " # Calculate driver/executor offheap memory in MB\n", + " #offheap_memory_per_node = convert_to_bytes(memory_per_node) - (executor_memory + executor_memory_overhead) * executors_per_node\n", + " if offheap_ratio > 0:\n", + " enable_offheap = True\n", + " offheap_memory = math.floor(executor_memory*offheap_ratio)\n", + " else:\n", + " enable_offheap = False\n", + " offheap_memory = 0\n", + "\n", + " byte_to_mb = lambda x: int(x/(1024 ** 2))\n", + " driver_memory_mb = byte_to_mb(driver_memory)\n", + " executor_memory_overhead_mb = byte_to_mb(executor_memory_overhead)\n", + " executor_memory_mb = byte_to_mb(executor_memory)\n", + " offheap_memory_mb = byte_to_mb(offheap_memory)\n", + " \n", + " executor_totalmem_mb = executor_memory_overhead_mb + executor_memory_mb + offheap_memory_mb\n", + " executor_totalmem_mb = yarn_padding(executor_totalmem_mb)\n", + " if byte_to_mb(convert_to_bytes(memory_per_node)) - executor_totalmem_mb*executors_per_node > executor_totalmem_mb:\n", + " executor_memory_overhead_mb += 1024\n", + " \n", + " print('''\n", + " executors per node: {:d}\n", + " parallelism: {:d}\n", + " executor memory: {:d}m\n", + " offheap memory: {:d}m\n", + " '''.format(executors_per_node, parallelism, executor_memory_mb, offheap_memory_mb))\n", + "\n", + " conf = SparkConf() \\\n", + " .set('spark.app.name', app_name)\\\n", + " .set('spark.master',master)\\\n", + " .set('spark.executor.memory', '{:d}m'.format(executor_memory_mb))\\\n", + " .set('spark.memory.offHeap.enabled', enable_offheap)\\\n", + " .set('spark.memory.offHeap.size','{:d}m'.format(offheap_memory_mb))\\\n", + " .set('spark.sql.shuffle.partitions', parallelism)\\\n", + " .set('spark.executor.instances', '{:d}'.format(num_executors))\\\n", + " .set('spark.executor.cores','{:d}'.format(cores_per_executor))\\\n", + " .set('spark.task.cpus','{:d}'.format(1))\\\n", + " .set('spark.driver.memory', '{:d}m'.format(driver_memory_mb))\\\n", + " .set('spark.executor.memoryOverhead', '{:d}m'.format(executor_memory_overhead_mb))\\\n", + " .set('spark.driver.maxResultSize', '4g')\\\n", + " .set('spark.executor.extraJavaOptions',\\\n", + " f'-XX:+UseParallelOldGC -XX:ParallelGCThreads=2 -XX:NewRatio=1 -XX:SurvivorRatio=1 -XX:+UseCompressedOops -verbose:gc -XX:+PrintGCDetails -XX:+PrintGCTimeStamps -XX:ErrorFile={home}/logs/java/hs_err_pid%p.log')\\\n", + " .set('spark.driver.extraClassPath', extra_jars) \\\n", + " .set('spark.executor.extraClassPath', extra_jars) \\\n", + " .set('spark.executorEnv.PYTHONPATH',f\"{os.environ['SPARK_HOME']}python:{get_py4jzip()}\") \\\n", + " .set(\"spark.repl.class.outputDir\", output_dir) \\\n", + " .set(\"spark.sql.broadcastTimeout\", \"4800\") \\\n", + " .set('spark.serializer','org.apache.spark.serializer.KryoSerializer')\\\n", + " .set('spark.kryoserializer.buffer.max','512m')\\\n", + " .set('spark.kryo.unsafe',False)\\\n", + " .set('spark.sql.adaptive.enabled',True)\\\n", + " .set('spark.sql.autoBroadcastJoinThreshold',\"10m\")\\\n", + " .set('spark.sql.catalogImplementation','hive')\\\n", + " .set('spark.sql.optimizer.dynamicPartitionPruning.enabled',True)\\\n", + " .set('spark.cleaner.periodicGC.interval', '10s')\n", + "\n", + " return conf\n", + "\n", + "\n", + "def create_cntx_with_config(conf,conf_overwrite=None):\n", + "\n", + " importlib.reload(pyspark.java_gateway)\n", + "\n", + " def Popen(*args, **kwargs):\n", + " \"\"\"Wraps subprocess.Popen to force stdout and stderr from the child process\n", + " to pipe to this process without buffering.\n", + " \"\"\"\n", + " global spark_jvm_proc\n", + " # Override these in kwargs to avoid duplicate value errors\n", + " # Set streams to unbuffered so that we read whatever bytes are available\n", + " # when ready, https://docs.python.org/3.6/library/subprocess.html#popen-constructor\n", + " kwargs['bufsize'] = 0\n", + " # Capture everything from stdout for display in the notebook\n", + " kwargs['stdout'] = subprocess.PIPE\n", + " print(\"java proc gateway popen\")\n", + " spark_jvm_proc = subprocess.Popen(*args, **kwargs)\n", + " return spark_jvm_proc\n", + " pyspark.java_gateway.Popen = Popen\n", + "\n", + " spylon_kernel.scala_interpreter.scala_intp=None\n", + " \n", + " if conf_overwrite is not None:\n", + " conf=conf_overwrite(conf)\n", + " print(\"spark.serializer: \",conf.get(\"spark.serializer\"))\n", + " print(\"master: \",conf.get(\"spark.master\"))\n", + " \n", + " sc = SparkContext(conf = conf,master=conf.get(\"spark.master\"))\n", + " sc.setLogLevel('ERROR')\n", + " \n", + " sc.addPyFile(f\"{os.environ['SPARK_HOME']}/python/lib/pyspark.zip\")\n", + " sc.addPyFile(get_py4jzip())\n", + " \n", + " spark = SQLContext(sc)\n", + " \n", + " time.sleep(30)\n", + " \n", + " for client in clients:\n", + " pids=!ssh $client \"jps | grep CoarseGrainedExecutorBackend | cut -d' ' -f1\"\n", + " print(client,\":\",len(pids),\" \",\"\\t\".join(map(str,pids)))\n", + " \n", + " spark_session = SparkSession(sc)\n", + " spark_jvm_helpers = SparkJVMHelpers(spark_session._sc)\n", + " spylon_kernel.scala_interpreter.spark_state = spylon_kernel.scala_interpreter.SparkState(spark_session, spark_jvm_helpers, spark_jvm_proc)\n", + " \n", + " print(\"appid: \",sc.applicationId)\n", + " print(\"SparkConf:\")\n", + "\n", + " df = pd.DataFrame(sc.getConf().getAll(), columns=['key', 'value'])\n", + " display(df)\n", + "\n", + " return sc, spark" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Vanilla Spark" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "def vanilla_tpch_conf_overwrite(conf):\n", + " return conf\n", + "\n", + "def vanilla_tpcds_conf_overwrite(conf):\n", + " conf.set('spark.sql.optimizer.runtime.bloomFilter.applicationSideScanSizeThreshold', '0')\\\n", + " .set('spark.sql.optimizer.runtime.bloomFilter.enabled', 'true')\n", + " return conf" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "def create_cntx_vanilla(executors_per_node, cores_per_executor, task_per_core, memory_per_node, extra_jars, app_name='', master='yarn', conf_overwrite=None):\n", + " conf = default_conf(executors_per_node, cores_per_executor, task_per_core, memory_per_node, extra_jars, app_name, master, run_gluten=False)\n", + " conf.set(\"spark.sql.execution.arrow.maxRecordsPerBatch\",20480)\\\n", + " .set(\"spark.sql.parquet.columnarReaderBatchSize\",20480)\\\n", + " .set(\"spark.sql.inMemoryColumnarStorage.batchSize\",20480)\n", + " return create_cntx_with_config(conf,conf_overwrite)" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Gluten" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "def gluten_tpch_conf_overwrite(conf):\n", + " return conf\n", + "\n", + "def gluten_tpcds_conf_overwrite(conf):\n", + " conf.set('spark.sql.optimizer.runtime.bloomFilter.applicationSideScanSizeThreshold', '0')\\\n", + " .set('spark.sql.optimizer.runtime.bloomFilter.enabled', 'true')\\\n", + " .set('spark.gluten.sql.columnar.joinOptimizationLevel', '18')\\\n", + " .set('spark.gluten.sql.columnar.physicalJoinOptimizeEnable', 'true')\\\n", + " .set('spark.gluten.sql.columnar.physicalJoinOptimizationLevel', '18')\\\n", + " .set('spark.gluten.sql.columnar.logicalJoinOptimizeEnable', 'true')\\\n", + " .set('spark.gluten.sql.columnar.logicalJoinOptimizationLevel', '19')\n", + " return conf" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "def create_cntx_gluten(executors_per_node, cores_per_executor, task_per_core, memory_per_node, extra_jars, app_name='', master='yarn', conf_overwrite=None):\n", + " conf = default_conf(executors_per_node, cores_per_executor, task_per_core, memory_per_node, extra_jars, app_name, master, run_gluten=True)\n", + " conf.set('spark.sql.files.maxPartitionBytes', '4g')\\\n", + " .set('spark.plugins','org.apache.gluten.GlutenPlugin')\\\n", + " .set('spark.shuffle.manager','org.apache.spark.shuffle.sort.ColumnarShuffleManager')\\\n", + " .set('spark.gluten.sql.columnar.backend.lib','velox')\\\n", + " .set('spark.gluten.sql.columnar.maxBatchSize',4096)\\\n", + " .set('spark.gluten.sql.columnar.forceshuffledhashjoin',True)\\\n", + " .set('spark.executorEnv.LD_PRELOAD', findjemalloc())\\\n", + " .set('spark.gluten.sql.columnar.coalesce.batches', 'true')\n", + " \n", + " return create_cntx_with_config(conf,conf_overwrite)" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Create Context" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "def create_cntx(run_gluten=False, workload='tpch', app_conf_overwrite=None, server='', base_dir='', nb_name='tpc_workload.ipynb', app_name=''):\n", + " table_dir=''\n", + " extra_jars = ''\n", + " is_tpch_workload=False\n", + " is_tpcds_workload=False\n", + " workload_conf_overwrite=None\n", + " create_cntx_func=None\n", + " test_tpc=None\n", + "\n", + " if workload.lower() == 'tpch':\n", + " if not app_name:\n", + " app_name = 'tpch_power'\n", + " tabledir = tpch_tabledir\n", + " is_tpch_workload=True\n", + " elif workload.lower() == 'tpcds':\n", + " if not app_name:\n", + " app_name = 'tpcds_power'\n", + " tabledir = tpcds_tabledir\n", + " is_tpcds_workload=True\n", + " else:\n", + " raise ValueError(f\"Unknown workload: {workload}\")\n", + "\n", + " lastgit=!git --git-dir {gluten_home}/.git log --format=\"%H\" -n 1\n", + " lastgit = lastgit[0]\n", + " print(f'lastgit: {lastgit}')\n", + "\n", + " nodes=len(clients)\n", + "\n", + " if run_gluten:\n", + " jars_base=f\"{home}/jars/\"+lastgit\n", + " \n", + " for target_jar in gluten_target_jar.split(\",\"):\n", + " !ls -l {target_jar}\n", + " !mkdir -p {jars_base}\n", + " !rm -rf {jars_base}/*\n", + " !cp {target_jar} {jars_base}/\n", + " if target_jar[-4:] != '.jar':\n", + " !cp -f {target_jar} {jars_base}/gluten-{lastgit}.jar\n", + "\n", + " jars=!ls -d {jars_base}/*.jar\n", + " extra_jars=\":\".join([\"file://\"+j for j in jars])\n", + " print(f'extra_jars: {extra_jars}')\n", + "\n", + " for c in clients:\n", + " if c!=localhost:\n", + " !ssh {c} \"rm -rf {jars_base}\"\n", + " !ssh {c} \"mkdir -p {jars_base}\"\n", + " !scp {jars_base}/*.jar {c}:{jars_base} >/dev/null 2>&1\n", + "\n", + " app_name = ' '.join(['gluten', app_name, lastgit[:6]])\n", + " create_cntx_func=create_cntx_gluten\n", + " if is_tpch_workload:\n", + " task_per_core = gluten_tpch_task_per_core\n", + " workload_conf_overwrite = gluten_tpch_conf_overwrite\n", + " elif is_tpcds_workload:\n", + " task_per_core = gluten_tpcds_task_per_core\n", + " workload_conf_overwrite = gluten_tpcds_conf_overwrite\n", + " else:\n", + " app_name = ' '.join(['vanilla', app_name, lastgit[:6]])\n", + " create_cntx_func=create_cntx_vanilla\n", + " if is_tpch_workload:\n", + " task_per_core = vanilla_tpch_task_per_core\n", + " workload_conf_overwrite = vanilla_tpch_conf_overwrite\n", + " elif is_tpcds_workload:\n", + " task_per_core = vanilla_tpcds_task_per_core\n", + " workload_conf_overwrite = vanilla_tpcds_conf_overwrite\n", + " \n", + " conf_overwrite = lambda conf: app_conf_overwrite(workload_conf_overwrite(conf))\n", + " \n", + " sc, spark = create_cntx_func(executors_per_node, cores_per_executor, task_per_core, memory_per_node, extra_jars, app_name, master, conf_overwrite)\n", + " \n", + " # Pin executors to numa nodes for Gluten\n", + " if run_gluten:\n", + " pinexecutor_numa(clients)\n", + "\n", + " appid = sc.applicationId\n", + " print(\"start run: \", appid)\n", + " \n", + " if is_tpch_workload:\n", + " test_tpc = TestTPCH(spark, tabledir, run_gluten, server, base_dir, nb_name)\n", + " elif is_tpcds_workload:\n", + " test_tpc = TestTPCDS(spark, tabledir, run_gluten, server, base_dir, nb_name)\n", + " \n", + " return sc, spark, appid, test_tpc" + ] + } + ], + "metadata": { + "hide_input": false, + "kernelspec": { + "display_name": "Python 3 (ipykernel)", + "language": "python", + "name": "python3" + }, + "language_info": { + "codemirror_mode": { + "name": "ipython", + "version": 3 + }, + "file_extension": ".py", + "mimetype": "text/x-python", + "name": "python", + "nbconvert_exporter": "python", + "pygments_lexer": "ipython3", + "version": "3.10.12" + }, + "nbTranslate": { + "displayLangs": [ + "*" + ], + "hotkey": "alt-t", + "langInMainMenu": true, + "sourceLang": "en", + "targetLang": "fr", + "useGoogleTranslate": true + }, + "toc": { + "base_numbering": 1, + "nav_menu": {}, + "number_sections": true, + "sideBar": false, + "skip_h1_title": false, + "title_cell": "Table of Contents", + "title_sidebar": "Contents", + "toc_cell": false, + "toc_position": { + "height": "364.469px", + "left": "2086.8px", + "top": "150.516px", + "width": "375px" + }, + "toc_section_display": true, + "toc_window_display": true + }, + "toc-autonumbering": true, + "varInspector": { + "cols": { + "lenName": 16, + "lenType": 16, + "lenVar": 40 + }, + "kernels_config": { + "python": { + "delete_cmd_postfix": "", + "delete_cmd_prefix": "del ", + "library": "var_list.py", + "varRefreshCmd": "print(var_dic_list())" + }, + "r": { + "delete_cmd_postfix": ") ", + "delete_cmd_prefix": "rm(", + "library": "var_list.r", + "varRefreshCmd": "cat(var_dic_list()) " + } + }, + "types_to_exclude": [ + "module", + "function", + "builtin_function_or_method", + "instance", + "_Feature" + ], + "window_display": false + } + }, + "nbformat": 4, + "nbformat_minor": 4 +} diff --git a/tools/workload/benchmark_velox/params.yaml.template b/tools/workload/benchmark_velox/params.yaml.template new file mode 100644 index 000000000000..1c70e428bc99 --- /dev/null +++ b/tools/workload/benchmark_velox/params.yaml.template @@ -0,0 +1,71 @@ +# Local path to gluten project. +gluten_home: /home/sparkuser/gluten + +# Local path to gluten jar. +gluten_target_jar: /home/sparkuser/gluten-velox-bundle-spark3.3_2.12-centos_7_x86_64-1.3.0-SNAPSHOT.jar + +# Spark app master. +master: yarn + +# List of workers. +clients: + - localhost + +# List of block devices. +disk_dev: + - nvme1n1 + - nvme2n1 + +# List of network devices. +nic_dev: + - ens787f0 + +# Hostname or IP to server for perf analysis. Able to connect via ssh. +server: '' + +# Specify the directory on perf analysis server. Usually a codename for this run. +base_dir: emr + +# Proxy used to connect to server for perf analysis. +proxy: '' + +# Whether to upload profile to perf analysis server and run perf analysis scripts. Only takes effect if server is set. +analyze_perf: True + +# Select workload. Can be either 'tpch' or 'tpcds'. +workload: tpch + +# Run with gluten. If False, run vanilla Spark. +run_gluten: True + +# TPC tables +tpch_tabledir: /tpch_sf3000 +tpcds_tabledir: /tpcds_sf3000 + +# Parallelism +executors_per_node: 32 +cores_per_executor: 8 + +gluten_tpch_task_per_core: 2 +gluten_tpcds_task_per_core: 2 +vanilla_tpch_task_per_core: 4 +vanilla_tpcds_task_per_core: 4 + +# Physical memory on each worker node. +memory_per_node: 1000g + +# Offheap ratio. 0 to disable offheap for vanilla Spark. +# onheap:offheap = 1:2 +vanilla_offheap_ratio: 2.0 +# onheap:offheap = 1:7 +gluten_offheap_ratio: 7.0 + +# spark.io.compression.codec +vanilla_codec: lz4 +# spark.gluten.sql.columnar.shuffle.codec +gluten_codec: lz4 +# spark.gluten.sql.columnar.shuffle.codecBackend +gluten_codec_backend: '' +# spark.gluten.sql.columnar.maxBatchSize +max_batch_size: 4096 + diff --git a/tools/workload/benchmark_velox/run_tpc_workload.sh b/tools/workload/benchmark_velox/run_tpc_workload.sh new file mode 100755 index 000000000000..f6de6ff0f538 --- /dev/null +++ b/tools/workload/benchmark_velox/run_tpc_workload.sh @@ -0,0 +1,86 @@ +#! /bin/bash + +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +set -eu + +PAPERMILL_ARGS=() +OUTPUT_DIR=$PWD + +while [[ $# -gt 0 ]]; do + case $1 in + --notebook) + NOTEBOOK="$2" + shift # past argument + shift # past value + ;; + --output-dir) + OUTPUT_DIR="$2" + shift # past argument + shift # past value + ;; + --output-name) + OUTPUT_NAME="$2" + shift # past argument + shift # past value + ;; + *) + PAPERMILL_ARGS+=("$1") # save positional arg + shift # past argument + ;; + esac +done + +if [ -z ${NOTEBOOK+x} ]; then + echo "Usage: $0 --notebook NOTEBOOK [--output-dir OUTPUT_DIR] [--output-name OUTPUT_NAME] [PAPERMILL_ARGS]" + exit 0 +fi + + +BASEDIR=$(dirname $0) +echo "Script location: ${BASEDIR}" + +nbname=$(basename $NOTEBOOK .ipynb) + +if [ -z ${OUTPUT_NAME+x} ]; then output_name=$nbname; else output_name=$(basename $OUTPUT_NAME .ipynb); fi + +output_dir=$(realpath $OUTPUT_DIR) +mkdir -p $output_dir + +rename_append_appid() { + output_name=$1 + orig_nb=$2 + + output_appid=`grep "appid: " $orig_nb | awk -F' ' '{print $2}' | sed 's/....$//'` + if [ -n "$output_appid" ]; + then + rename_nb=${output_dir}/${output_name}-${output_appid}.ipynb + echo "Rename notebook $orig_nb to $rename_nb" + mv $orig_nb $rename_nb + fi +} + +run() { + output_name=${output_name}-$(date +"%H%M%S") + output_nb=${output_dir}/${output_name}.ipynb + papermill --inject-output-path $NOTEBOOK \ + ${PAPERMILL_ARGS[@]} \ + $output_nb + rename_append_appid $output_name $output_nb +} + +run + diff --git a/tools/workload/benchmark_velox/tpc_workload.ipynb b/tools/workload/benchmark_velox/tpc_workload.ipynb new file mode 100644 index 000000000000..5dcb50a8a066 --- /dev/null +++ b/tools/workload/benchmark_velox/tpc_workload.ipynb @@ -0,0 +1,381 @@ +{ + "cells": [ + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "# initialization" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "tags": [ + "parameters" + ] + }, + "outputs": [], + "source": [ + "# Local path to gluten project.\n", + "gluten_home='/home/sparkuser/gluten'\n", + "\n", + "# Local path to gluten jar.\n", + "gluten_target_jar='/home/sparkuser/gluten-velox-bundle-spark3.3_2.12-centos_7_x86_64-1.3.0-SNAPSHOT.jar'\n", + "\n", + "# Spark app master. e.g. 'yarn'\n", + "master='yarn'\n", + "\n", + "# List of workers.\n", + "clients=['localhost']\n", + "\n", + "# List of block devices. e.g. ['nvme1n1', 'nvme2n1']\n", + "disk_dev=[]\n", + "\n", + "# List of network devices. e.g. ['ens787f0']\n", + "nic_dev=[]\n", + "\n", + "# Hostname or IP to server for perf analysis. Able to connect via ssh.\n", + "server=''\n", + "\n", + "# Specify the directory on perf analysis server. Usually a codename for this run.\n", + "base_dir=''\n", + "\n", + "# Proxy used to connect to server for perf analysis.\n", + "proxy=''\n", + "\n", + "# Whether to upload profile to perf analysis server and run perf analysis scripts. Only takes effect if server is set.\n", + "analyze_perf=True\n", + "\n", + "# Select workload. Can be either 'tpch' or 'tpcds'.\n", + "workload='tpch'\n", + "\n", + "# Run with gluten. If False, run vanilla Spark.\n", + "run_gluten=True\n", + "\n", + "# TPC tables\n", + "tpch_tabledir=''\n", + "tpcds_tabledir=''\n", + "\n", + "# Parallelism\n", + "executors_per_node=32\n", + "cores_per_executor=8\n", + "\n", + "gluten_tpch_task_per_core=2\n", + "gluten_tpcds_task_per_core=4\n", + "vanilla_tpch_task_per_core=8\n", + "vanilla_tpcds_task_per_core=8\n", + "\n", + "# Physical memory on each worker node.\n", + "memory_per_node='1000g'\n", + "\n", + "# Offheap ratio. 0 to disable offheap for vanilla Spark.\n", + "# onheap:offheap = 1:2\n", + "vanilla_offheap_ratio=2.0\n", + "# onheap:offheap = 1:7\n", + "gluten_offheap_ratio=7.0\n", + "\n", + "# spark.io.compression.codec\n", + "vanilla_codec='lz4'\n", + "# spark.gluten.sql.columnar.shuffle.codec\n", + "gluten_codec='lz4'\n", + "# spark.gluten.sql.columnar.shuffle.codecBackend\n", + "gluten_codec_backend=''\n", + "# spark.gluten.sql.columnar.maxBatchSize\n", + "max_batch_size=4096" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "initialize_ipynb = !realpath native_sql_initialize.ipynb\n", + "print(f\"Running notebook: {initialize_ipynb[0]}\\n\")\n", + "%run {initialize_ipynb[0]}" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "newClients = []\n", + "for l in clients:\n", + " if l == 'localhost':\n", + " newClients.append(localhost)\n", + " else:\n", + " newClients.append(l)\n", + "clients = newClients\n", + "\n", + "if server == 'localhost':\n", + " server = localhost" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "%%javascript\n", + "IPython.notebook.kernel.execute('nb_name = \"' + IPython.notebook.notebook_name + '\"')" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "deletable": false, + "editable": false, + "run_control": { + "frozen": true + } + }, + "outputs": [], + "source": [ + "nb_name=PAPERMILL_OUTPUT_PATH" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "# Application Level Configuration" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "tpch_workload=False\n", + "tpcds_workload=False\n", + "\n", + "if workload.lower() == 'tpch':\n", + " tpch_workload=True\n", + "elif workload.lower() == 'tpcds':\n", + " tpcds_workload=True\n", + "else:\n", + " raise ValueError(f\"Unknown workload: {workload}\")\n", + "\n", + "def gluten_conf_overwrite(conf):\n", + " conf.set('spark.gluten.sql.columnar.shuffle.codec', gluten_codec)\\\n", + " .set('spark.gluten.sql.columnar.shuffle.codecBackend', gluten_codec_backend)\\\n", + " .set('spark.gluten.sql.columnar.maxBatchSize', max_batch_size)\\\n", + " .set('spark.executor.extraJavaOptions',\\\n", + " '-XX:+UseParallelOldGC -XX:ParallelGCThreads=2 -XX:NewRatio=1 -XX:SurvivorRatio=1 -XX:+UseCompressedOops -verbose:gc -XX:+PrintGCDetails -XX:+PrintGCTimeStamps -XX:ErrorFile=/home/sparkuser/logs/java/hs_err_pid%p.log')\\\n", + " .set('spark.gluten.memory.overAcquiredMemoryRatio','0')\\\n", + "\n", + " if tpch_workload:\n", + " pass\n", + " elif tpcds_workload:\n", + " pass\n", + " return conf\n", + "\n", + "def vanilla_conf_overwrite(conf):\n", + " conf.set('spark.io.compression.codec', vanilla_codec)\\\n", + " .set('spark.executorEnv.LD_LIBRARY_PATH',f\"{os.getenv('HADOOP_HOME')}/lib/native/\") \\\n", + " .set('spark.yarn.appMasterEnv.LD_LIBRARY_PATH',f\"{os.getenv('HADOOP_HOME')}/lib/native/\") \\\n", + "\n", + " if tpch_workload:\n", + " pass\n", + " elif tpcds_workload:\n", + " pass\n", + " return conf\n", + "\n", + "def app_conf_overwrite(conf):\n", + " if run_gluten:\n", + " return gluten_conf_overwrite(conf)\n", + " return vanilla_conf_overwrite(conf)" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "# Run Workload" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "# Config and clean pagecache before each run\n", + "config_pagecache(clients, run_gluten)\n", + "dropcache(clients)\n", + "print_kernel_params(clients)" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "# Create SparkSession\n", + "sc, spark, appid, test_tpc=create_cntx(run_gluten, workload, app_conf_overwrite, server, base_dir, nb_name)" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "if run_gluten:\n", + " config_mem_cgroup(clients)" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "test_tpc.start_monitor(clients)" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "test_tpc.power_run(explain=False, print_result=False, load_table=True)" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "test_tpc.stop_monitor(clients)" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "if analyze_perf:\n", + " test_tpc.run_perf_analysis(disk_dev, nic_dev)" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "# Show Performance" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "test_tpc.print_result()" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "code_folding": [] + }, + "outputs": [], + "source": [ + "for client in clients:\n", + " draw_sar(appid, qtime=test_tpc.result, disk_dev=disk_dev, nic_dev=nic_dev, client=client)" + ] + } + ], + "metadata": { + "celltoolbar": "Tags", + "hide_input": false, + "kernelspec": { + "display_name": "Python 3 (ipykernel)", + "language": "python", + "name": "python3" + }, + "language_info": { + "codemirror_mode": { + "name": "ipython", + "version": 3 + }, + "file_extension": ".py", + "mimetype": "text/x-python", + "name": "python", + "nbconvert_exporter": "python", + "pygments_lexer": "ipython3", + "version": "3.10.12" + }, + "nbTranslate": { + "displayLangs": [ + "*" + ], + "hotkey": "alt-t", + "langInMainMenu": true, + "sourceLang": "en", + "targetLang": "fr", + "useGoogleTranslate": true + }, + "toc": { + "base_numbering": 1, + "nav_menu": {}, + "number_sections": true, + "sideBar": false, + "skip_h1_title": false, + "title_cell": "Table of Contents", + "title_sidebar": "Contents", + "toc_cell": false, + "toc_position": { + "height": "428.672px", + "left": "1339.91px", + "top": "374.297px", + "width": "456.969px" + }, + "toc_section_display": true, + "toc_window_display": true + }, + "varInspector": { + "cols": { + "lenName": 16, + "lenType": 16, + "lenVar": 40 + }, + "kernels_config": { + "python": { + "delete_cmd_postfix": "", + "delete_cmd_prefix": "del ", + "library": "var_list.py", + "varRefreshCmd": "print(var_dic_list())" + }, + "r": { + "delete_cmd_postfix": ") ", + "delete_cmd_prefix": "rm(", + "library": "var_list.r", + "varRefreshCmd": "cat(var_dic_list()) " + } + }, + "types_to_exclude": [ + "module", + "function", + "builtin_function_or_method", + "instance", + "_Feature" + ], + "window_display": false + } + }, + "nbformat": 4, + "nbformat_minor": 4 +} diff --git a/tools/workload/tpcds/gen_data/parquet_dataset/tpcds_datagen_parquet.sh b/tools/workload/tpcds/gen_data/parquet_dataset/tpcds_datagen_parquet.sh index 8f62560dc168..60f10f094d5b 100644 --- a/tools/workload/tpcds/gen_data/parquet_dataset/tpcds_datagen_parquet.sh +++ b/tools/workload/tpcds/gen_data/parquet_dataset/tpcds_datagen_parquet.sh @@ -13,7 +13,6 @@ # See the License for the specific language governing permissions and # limitations under the License. -batchsize=10240 SPARK_HOME=/home/sparkuser/spark/ spark_sql_perf_jar=/PATH/TO/spark-sql-perf_2.12-0.5.1-SNAPSHOT.jar cat tpcds_datagen_parquet.scala | ${SPARK_HOME}/bin/spark-shell \ @@ -22,14 +21,13 @@ cat tpcds_datagen_parquet.scala | ${SPARK_HOME}/bin/spark-shell \ --executor-memory 25g \ --executor-cores 8 \ --master yarn \ - --driver-memory 50g \ + --driver-memory 10g \ --deploy-mode client \ --conf spark.executor.memoryOverhead=1g \ - --conf spark.sql.parquet.columnarReaderBatchSize=${batchsize} \ - --conf spark.sql.inMemoryColumnarStorage.batchSize=${batchsize} \ - --conf spark.sql.execution.arrow.maxRecordsPerBatch=${batchsize} \ --conf spark.sql.broadcastTimeout=4800 \ --conf spark.driver.maxResultSize=4g \ - --conf spark.sql.sources.useV1SourceList=avro \ --conf spark.sql.shuffle.partitions=224 \ + --conf spark.sql.parquet.compression.codec=snappy \ + --conf spark.network.timeout=800s \ + --conf spark.executor.heartbeatInterval=200s \ --jars ${spark_sql_perf_jar} diff --git a/tools/workload/tpch/gen_data/dwrf_dataset/tpch_convert_parquet_dwrf.scala b/tools/workload/tpch/gen_data/dwrf_dataset/tpch_convert_parquet_dwrf.scala deleted file mode 100644 index 958a98f57020..000000000000 --- a/tools/workload/tpch/gen_data/dwrf_dataset/tpch_convert_parquet_dwrf.scala +++ /dev/null @@ -1,62 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -import org.apache.spark.sql.execution.debug._ -import scala.io.Source -import java.io.File -import java.util.Arrays -import org.apache.spark.sql.types.{DoubleType, TimestampType, LongType, IntegerType} - -val parquet_file_path = "/PATH/TO/TPCH_PARQUET_PATH" -val dwrf_file_path = "/PATH/TO/TPCH_DWRF_PATH" - -val lineitem_parquet_path = "file://" + parquet_file_path + "/lineitem" -val customer_parquet_path = "file://" + parquet_file_path + "/customer" -val nation_parquet_path = "file://" + parquet_file_path + "/nation" -val orders_parquet_path = "file://" + parquet_file_path + "/orders" -val part_parquet_path = "file://" + parquet_file_path + "/part" -val partsupp_parquet_path = "file://" + parquet_file_path + "/partsupp" -val region_path_path = "file://" + parquet_file_path + "/region" -val supplier_parquet_path = "file://" + parquet_file_path + "/supplier" - -val lineitem = spark.read.format("parquet").load(lineitem_parquet_path) -val customer = spark.read.format("parquet").load(customer_parquet_path) -val nation = spark.read.format("parquet").load(nation_parquet_path) -val orders = spark.read.format("parquet").load(orders_parquet_path) -val part = spark.read.format("parquet").load(part_parquet_path) -val partsupp = spark.read.format("parquet").load(partsupp_parquet_path) -val region = spark.read.format("parquet").load(region_path_path) -val supplier = spark.read.format("parquet").load(supplier_parquet_path) - -val lineitem_dwrf_path = "file://" + dwrf_file_path + "/lineitem" -val customer_dwrf_path = "file://" + dwrf_file_path + "/customer" -val nation_dwrf_path = "file://" + dwrf_file_path + "/nation" -val orders_dwrf_path = "file://" + dwrf_file_path + "/orders" -val part_dwrf_path = "file://" + dwrf_file_path + "/part" -val partsupp_dwrf_path = "file://" + dwrf_file_path + "/partsupp" -val region_dwrf_path = "file://" + dwrf_file_path + "/region" -val supplier_dwrf_path = "file://" + dwrf_file_path + "/supplier" - -lineitem.write.mode("append").format("dwrf").save(lineitem_dwrf_path) -customer.write.mode("append").format("dwrf").save(customer_dwrf_path) -nation.write.mode("append").format("dwrf").save(nation_dwrf_path) -orders.write.mode("append").format("dwrf").save(orders_dwrf_path) -part.write.mode("append").format("dwrf").save(part_dwrf_path) -partsupp.write.mode("append").format("dwrf").save(partsupp_dwrf_path) -region.write.mode("append").format("dwrf").save(region_dwrf_path) -supplier.write.mode("append").format("dwrf").save(supplier_dwrf_path) - - diff --git a/tools/workload/tpch/gen_data/dwrf_dataset/tpch_convert_parquet_dwrf.sh b/tools/workload/tpch/gen_data/dwrf_dataset/tpch_convert_parquet_dwrf.sh deleted file mode 100644 index 2dc05c9aa7dd..000000000000 --- a/tools/workload/tpch/gen_data/dwrf_dataset/tpch_convert_parquet_dwrf.sh +++ /dev/null @@ -1,47 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -batchsize=20480 - -export GLUTEN_HOME=/PATH/TO/gluten/ -#please choose right os system jar -export GLUTEN_JVM_JAR=${GLUTEN_HOME}/package/target/ -SPARK_HOME=/home/sparkuser/spark/ - -cat tpch_convert_parquet_dwrf.scala | ${SPARK_HOME}/bin/spark-shell \ - --name convert_parquet_dwrf \ - --master yarn \ - --deploy-mode client \ - --driver-memory 20g \ - --executor-cores 8 \ - --num-executors 14 \ - --executor-memory 30g \ - --conf spark.plugins=org.apache.gluten.GlutenPlugin \ - --conf spark.driver.extraClassPath=${GLUTEN_JVM_JAR} \ - --conf spark.executor.extraClassPath=${GLUTEN_JVM_JAR} \ - --conf spark.memory.offHeap.size=30g \ - --conf spark.executor.memoryOverhead=5g \ - --conf spark.driver.maxResultSize=32g \ - --conf spark.sql.autoBroadcastJoinThreshold=-1 \ - --conf spark.sql.parquet.columnarReaderBatchSize=${batchsize} \ - --conf spark.sql.inMemoryColumnarStorage.batchSize=${batchsize} \ - --conf spark.sql.execution.arrow.maxRecordsPerBatch=${batchsize} \ - --conf spark.gluten.sql.columnar.forceShuffledHashJoin=true \ - --conf spark.sql.broadcastTimeout=4800 \ - --conf spark.driver.maxResultSize=4g \ - --conf spark.sql.adaptive.enabled=true \ - --conf spark.sql.shuffle.partitions=112 \ - --conf spark.sql.sources.useV1SourceList=avro \ - --conf spark.shuffle.manager=org.apache.spark.shuffle.sort.ColumnarShuffleManager \ - --conf spark.sql.files.maxPartitionBytes=1073741824 \ diff --git a/tools/workload/tpch/gen_data/parquet_dataset/tpch_datagen_parquet.sh b/tools/workload/tpch/gen_data/parquet_dataset/tpch_datagen_parquet.sh index 29512ed80b15..8db9d443331d 100644 --- a/tools/workload/tpch/gen_data/parquet_dataset/tpch_datagen_parquet.sh +++ b/tools/workload/tpch/gen_data/parquet_dataset/tpch_datagen_parquet.sh @@ -13,7 +13,6 @@ # See the License for the specific language governing permissions and # limitations under the License. -batchsize=10240 SPARK_HOME=/home/sparkuser/spark/ spark_sql_perf_jar=/PATH/TO/spark-sql-perf_2.12-0.5.1-SNAPSHOT.jar cat tpch_datagen_parquet.scala | ${SPARK_HOME}/bin/spark-shell \ @@ -25,11 +24,10 @@ cat tpch_datagen_parquet.scala | ${SPARK_HOME}/bin/spark-shell \ --driver-memory 50g \ --deploy-mode client \ --conf spark.executor.memoryOverhead=1g \ - --conf spark.sql.parquet.columnarReaderBatchSize=${batchsize} \ - --conf spark.sql.inMemoryColumnarStorage.batchSize=${batchsize} \ - --conf spark.sql.execution.arrow.maxRecordsPerBatch=${batchsize} \ --conf spark.sql.broadcastTimeout=4800 \ --conf spark.driver.maxResultSize=4g \ - --conf spark.sql.sources.useV1SourceList=avro \ --conf spark.sql.shuffle.partitions=224 \ + --conf spark.sql.parquet.compression.codec=snappy \ + --conf spark.network.timeout=800s \ + --conf spark.executor.heartbeatInterval=200s \ --jars ${spark_sql_perf_jar} From 59d52486b189336ee3fd31a9a8fe98e866d5805a Mon Sep 17 00:00:00 2001 From: lgbo Date: Thu, 14 Nov 2024 10:31:29 +0800 Subject: [PATCH 16/35] remove duplicated columns in agg results (#7937) --- .../org/apache/gluten/extension/LazyAggregateExpandRule.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/extension/LazyAggregateExpandRule.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/LazyAggregateExpandRule.scala index d21cbfa2505a..d64300a12fcb 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/extension/LazyAggregateExpandRule.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/LazyAggregateExpandRule.scala @@ -353,7 +353,7 @@ case class LazyAggregateExpandRule(session: SparkSession) extends Rule[SparkPlan s"resultExpressions:${partialAggregate.resultExpressions}") partialAggregate.copy( groupingExpressions = groupingExpressions, - resultExpressions = resultExpressions, + resultExpressions = resultExpressions.distinct, child = expand.child) } From 73c06100e454723546b396b02110504b6c76c61e Mon Sep 17 00:00:00 2001 From: Xiuli Wei Date: Thu, 14 Nov 2024 12:27:55 +0800 Subject: [PATCH 17/35] [GLUTEN-7856][CORE] Ensure correct enabling of GlutenCostEvaluator (#7857) Closes #7856 --- .../src/main/scala/org/apache/gluten/GlutenPlugin.scala | 9 ++++++--- .../src/main/scala/org/apache/gluten/GlutenConfig.scala | 9 ++++----- 2 files changed, 10 insertions(+), 8 deletions(-) diff --git a/gluten-core/src/main/scala/org/apache/gluten/GlutenPlugin.scala b/gluten-core/src/main/scala/org/apache/gluten/GlutenPlugin.scala index d5f5fd2e2150..e52c53d1d558 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/GlutenPlugin.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/GlutenPlugin.scala @@ -65,7 +65,7 @@ private[gluten] class GlutenDriverPlugin extends DriverPlugin with Logging { postBuildInfoEvent(sc) - setPredefinedConfigs(sc, conf) + setPredefinedConfigs(conf) // Initialize Backend. Backend.get().onDriverStart(sc, pluginContext) @@ -123,7 +123,7 @@ private[gluten] class GlutenDriverPlugin extends DriverPlugin with Logging { GlutenEventUtils.post(sc, event) } - private def setPredefinedConfigs(sc: SparkContext, conf: SparkConf): Unit = { + private def setPredefinedConfigs(conf: SparkConf): Unit = { // Spark SQL extensions val extensions = if (conf.contains(SPARK_SESSION_EXTENSIONS.key)) { s"${conf.get(SPARK_SESSION_EXTENSIONS.key)}," + @@ -134,7 +134,10 @@ private[gluten] class GlutenDriverPlugin extends DriverPlugin with Logging { conf.set(SPARK_SESSION_EXTENSIONS.key, extensions) // adaptive custom cost evaluator class - if (GlutenConfig.getConf.enableGluten && GlutenConfig.getConf.enableGlutenCostEvaluator) { + val enableGlutenCostEvaluator = conf.getBoolean( + GlutenConfig.GLUTEN_COST_EVALUATOR_ENABLED, + GLUTEN_COST_EVALUATOR_ENABLED_DEFAULT_VALUE) + if (enableGlutenCostEvaluator) { val costEvaluator = "org.apache.spark.sql.execution.adaptive.GlutenCostEvaluator" conf.set(SQLConf.ADAPTIVE_CUSTOM_COST_EVALUATOR_CLASS.key, costEvaluator) } diff --git a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala index 0bfe16270e5f..0e41fc65376c 100644 --- a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala +++ b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala @@ -466,8 +466,6 @@ class GlutenConfig(conf: SQLConf) extends Logging { def enableCastAvgAggregateFunction: Boolean = conf.getConf(COLUMNAR_NATIVE_CAST_AGGREGATE_ENABLED) - def enableGlutenCostEvaluator: Boolean = conf.getConf(COST_EVALUATOR_ENABLED) - def dynamicOffHeapSizingEnabled: Boolean = conf.getConf(DYNAMIC_OFFHEAP_SIZING_ENABLED) @@ -648,6 +646,7 @@ object GlutenConfig { "spark.gluten.memory.dynamic.offHeap.sizing.memory.fraction" val GLUTEN_COST_EVALUATOR_ENABLED = "spark.gluten.sql.adaptive.costEvaluator.enabled" + val GLUTEN_COST_EVALUATOR_ENABLED_DEFAULT_VALUE = true var ins: GlutenConfig = _ @@ -2116,15 +2115,15 @@ object GlutenConfig { .createWithDefault(true) val COST_EVALUATOR_ENABLED = - buildConf(GlutenConfig.GLUTEN_COST_EVALUATOR_ENABLED) + buildStaticConf(GlutenConfig.GLUTEN_COST_EVALUATOR_ENABLED) .internal() .doc( - "If true and gluten enabled, use " + + "If true, use " + "org.apache.spark.sql.execution.adaptive.GlutenCostEvaluator as custom cost " + "evaluator class, else follow the configuration " + "spark.sql.adaptive.customCostEvaluatorClass.") .booleanConf - .createWithDefault(true) + .createWithDefault(GLUTEN_COST_EVALUATOR_ENABLED_DEFAULT_VALUE) val DYNAMIC_OFFHEAP_SIZING_ENABLED = buildConf(GlutenConfig.GLUTEN_DYNAMIC_OFFHEAP_SIZING_ENABLED) From fc549eee511304e3dbe02a5295c4016b827c4f10 Mon Sep 17 00:00:00 2001 From: PHILO-HE Date: Thu, 14 Nov 2024 13:30:25 +0800 Subject: [PATCH 18/35] [VL] Fix wrong lib suffix for google_cloud_cpp_storage (#7933) --- cpp/velox/CMakeLists.txt | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/cpp/velox/CMakeLists.txt b/cpp/velox/CMakeLists.txt index 329da49497f3..3ed7574d1e4c 100644 --- a/cpp/velox/CMakeLists.txt +++ b/cpp/velox/CMakeLists.txt @@ -138,8 +138,8 @@ endmacro() macro(find_gcssdk) set(CMAKE_FIND_LIBRARY_SUFFIXES_BCK ${CMAKE_FIND_LIBRARY_SUFFIXES}) - set(CMAKE_FIND_LIBRARY_SUFFIXES ".so") - find_package(google_cloud_cpp_storage REQUIRED) + set(CMAKE_FIND_LIBRARY_SUFFIXES ".a") + find_package(google_cloud_cpp_storage CONFIG 2.22.0 REQUIRED) set(CMAKE_FIND_LIBRARY_SUFFIXES ${CMAKE_FIND_LIBRARY_SUFFIXES_BCK}) endmacro() From 20136c86acb3e63bf7834e1c1a636db26c92bd88 Mon Sep 17 00:00:00 2001 From: Rui Mo Date: Thu, 14 Nov 2024 15:21:22 +0800 Subject: [PATCH 19/35] [VL] Add test for scan operator with filter on decimal/timestamp/binary field (#7945) --- .../gluten/execution/VeloxScanSuite.scala | 36 +++++++++++++++++++ .../RandomParquetDataGenerator.scala | 6 ++-- 2 files changed, 39 insertions(+), 3 deletions(-) diff --git a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxScanSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxScanSuite.scala index a50bffa3edcd..a4f16ecc3c8f 100644 --- a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxScanSuite.scala +++ b/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxScanSuite.scala @@ -18,11 +18,13 @@ package org.apache.gluten.execution import org.apache.gluten.GlutenConfig import org.apache.gluten.backendsapi.velox.VeloxBackendSettings +import org.apache.gluten.benchmarks.RandomParquetDataGenerator import org.apache.gluten.utils.VeloxFileSystemValidationJniWrapper import org.apache.spark.SparkConf import org.apache.spark.sql.catalyst.expressions.GreaterThan import org.apache.spark.sql.execution.ScalarSubquery +import org.apache.spark.sql.types._ class VeloxScanSuite extends VeloxWholeStageTransformerSuite { protected val rootPath: String = getClass.getResource("/").getPath @@ -114,4 +116,38 @@ class VeloxScanSuite extends VeloxWholeStageTransformerSuite { !VeloxFileSystemValidationJniWrapper.allSupportedByRegisteredFileSystems( Array("file:/test_path/", "unsupported://test_path"))) } + + test("scan with filter on decimal/timestamp/binary field") { + withTempView("t") { + withTempDir { + dir => + val path = dir.getAbsolutePath + val schema = StructType( + Array( + StructField("short_decimal_field", DecimalType(5, 2), nullable = true), + StructField("long_decimal_field", DecimalType(32, 8), nullable = true), + StructField("binary_field", BinaryType, nullable = true), + StructField("timestamp_field", TimestampType, nullable = true) + )) + RandomParquetDataGenerator(0).generateRandomData(spark, schema, 10, Some(path)) + spark.catalog.createTable("t", path, "parquet") + + runQueryAndCompare( + """select * from t where long_decimal_field = 3.14""".stripMargin + )(checkGlutenOperatorMatch[FileSourceScanExecTransformer]) + + runQueryAndCompare( + """select * from t where short_decimal_field = 3.14""".stripMargin + )(checkGlutenOperatorMatch[FileSourceScanExecTransformer]) + + runQueryAndCompare( + """select * from t where binary_field = '3.14'""".stripMargin + )(checkGlutenOperatorMatch[FileSourceScanExecTransformer]) + + runQueryAndCompare( + """select * from t where timestamp_field = current_timestamp()""".stripMargin + )(checkGlutenOperatorMatch[FileSourceScanExecTransformer]) + } + } + } } diff --git a/gluten-substrait/src/test/scala/org/apache/gluten/benchmarks/RandomParquetDataGenerator.scala b/gluten-substrait/src/test/scala/org/apache/gluten/benchmarks/RandomParquetDataGenerator.scala index a27a4991cf4a..ce2f85af7781 100644 --- a/gluten-substrait/src/test/scala/org/apache/gluten/benchmarks/RandomParquetDataGenerator.scala +++ b/gluten-substrait/src/test/scala/org/apache/gluten/benchmarks/RandomParquetDataGenerator.scala @@ -22,7 +22,7 @@ import org.apache.spark.sql.types._ import com.github.javafaker.Faker -import java.sql.Date +import java.sql.{Date, Timestamp} import java.util.Random case class RandomParquetDataGenerator(initialSeed: Long = 0L) extends Logging { @@ -67,7 +67,7 @@ case class RandomParquetDataGenerator(initialSeed: Long = 0L) extends Logging { case DoubleType => faker.number().randomDouble(2, Double.MinValue.toLong, Double.MaxValue.toLong) case DateType => new Date(faker.date().birthday().getTime) -// case TimestampType => new Timestamp(faker.date().birthday().getTime) + case TimestampType => new Timestamp(faker.date().birthday().getTime) case t: DecimalType => BigDecimal( faker.number().randomDouble(t.scale, 0, Math.pow(10, t.precision - t.scale).toLong)) @@ -124,7 +124,7 @@ case class RandomParquetDataGenerator(initialSeed: Long = 0L) extends Logging { () => StructField(fieldName, FloatType, nullable = true), () => StructField(fieldName, DoubleType, nullable = true), () => StructField(fieldName, DateType, nullable = true), -// () => StructField(fieldName, TimestampType, nullable = true), + () => StructField(fieldName, TimestampType, nullable = true), () => StructField(fieldName, DecimalType(10, 2), nullable = true), () => StructField(fieldName, DecimalType(30, 10), nullable = true) ) From 21b4e6539b5494ee6e7506368dfeca4711b6971a Mon Sep 17 00:00:00 2001 From: Mingliang Zhu Date: Thu, 14 Nov 2024 16:54:59 +0800 Subject: [PATCH 20/35] [GLUTEN-7362][VL] Add test for 'IN' and 'OR' filter in Scan (#7363) --- .../org/apache/gluten/execution/MiscOperatorSuite.scala | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/backends-velox/src/test/scala/org/apache/gluten/execution/MiscOperatorSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/execution/MiscOperatorSuite.scala index 2b42d7fb8d64..58c748d37bfa 100644 --- a/backends-velox/src/test/scala/org/apache/gluten/execution/MiscOperatorSuite.scala +++ b/backends-velox/src/test/scala/org/apache/gluten/execution/MiscOperatorSuite.scala @@ -208,6 +208,13 @@ class MiscOperatorSuite extends VeloxWholeStageTransformerSuite with AdaptiveSpa "select l_orderkey from lineitem " + "where l_partkey in (1552, 674) or l_partkey in (1552) and l_orderkey > 1") { _ => } checkLengthAndPlan(df, 73) + + runQueryAndCompare( + "select count(1) from lineitem " + + "where (l_shipmode in ('TRUCK', 'MAIL') or l_shipmode in ('AIR', 'FOB')) " + + "and l_shipmode in ('RAIL','SHIP')") { + checkGlutenOperatorMatch[FileSourceScanExecTransformer] + } } test("in_not") { From f8a2dca340c2c47e2349b1b4304d54ce2a58e6df Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9D=8E=E6=89=AC?= <654010905@qq.com> Date: Thu, 14 Nov 2024 18:44:01 +0800 Subject: [PATCH 21/35] [GLUTEN-7387][CH] Allow parallel downloading in scan operator for hive text/json table when the whole compresse(not bzip2) file is a single file split (#7598) * enable parallel downloading for text/json * wip * wip * finish dev * update version * update initialization of thread pool * fix style --- cpp-ch/clickhouse.version | 2 +- cpp-ch/local-engine/Common/CHUtil.cpp | 17 +++++- .../SubstraitSource/ReadBufferBuilder.cpp | 55 +++++++++++++++++-- .../SubstraitSource/ReadBufferBuilder.h | 7 +++ 4 files changed, 73 insertions(+), 8 deletions(-) diff --git a/cpp-ch/clickhouse.version b/cpp-ch/clickhouse.version index e51caf2aa788..beda11ec0a99 100644 --- a/cpp-ch/clickhouse.version +++ b/cpp-ch/clickhouse.version @@ -1,3 +1,3 @@ CH_ORG=Kyligence CH_BRANCH=rebase_ch/20241111 -CH_COMMIT=06ffc32a255 +CH_COMMIT=3f7e46d4e9e diff --git a/cpp-ch/local-engine/Common/CHUtil.cpp b/cpp-ch/local-engine/Common/CHUtil.cpp index d253de82938f..778c4f257b19 100644 --- a/cpp-ch/local-engine/Common/CHUtil.cpp +++ b/cpp-ch/local-engine/Common/CHUtil.cpp @@ -32,6 +32,7 @@ #include #include #include +#include #include #include #include @@ -86,6 +87,14 @@ extern const int BAD_ARGUMENTS; extern const int UNKNOWN_TYPE; extern const int CANNOT_PARSE_PROTOBUF_SCHEMA; } + +namespace ServerSetting +{ +extern const ServerSettingsUInt64 max_thread_pool_size; +extern const ServerSettingsUInt64 thread_pool_queue_size; +extern const ServerSettingsUInt64 max_io_thread_pool_size; +extern const ServerSettingsUInt64 io_thread_pool_queue_size; +} } namespace local_engine @@ -757,6 +766,7 @@ void BackendInitializerUtil::initSettings(const SparkConfigs::ConfigMap & spark_ LOG_DEBUG(&Poco::Logger::get("CHUtil"), "Set settings key:{} value:{}", key, value); } } + /// Finally apply some fixed kvs to settings. settings.set("join_use_nulls", true); settings.set("input_format_orc_allow_missing_columns", true); @@ -970,7 +980,12 @@ void BackendInitializerUtil::initBackend(const SparkConfigs::ConfigMap & spark_c initCompiledExpressionCache(config); LOG_INFO(logger, "Init compiled expressions cache factory."); - GlobalThreadPool::initialize(); + ServerSettings server_settings; + server_settings.loadSettingsFromConfig(*config); + GlobalThreadPool::initialize( + server_settings[ServerSetting::max_thread_pool_size], 0, server_settings[ServerSetting::thread_pool_queue_size]); + getIOThreadPool().initialize( + server_settings[ServerSetting::max_io_thread_pool_size], 0, server_settings[ServerSetting::io_thread_pool_queue_size]); const size_t active_parts_loading_threads = config->getUInt("max_active_parts_loading_thread_pool_size", 64); DB::getActivePartsLoadingThreadPool().initialize( diff --git a/cpp-ch/local-engine/Storages/SubstraitSource/ReadBufferBuilder.cpp b/cpp-ch/local-engine/Storages/SubstraitSource/ReadBufferBuilder.cpp index f5f7f95e6202..c1796f3e3cda 100644 --- a/cpp-ch/local-engine/Storages/SubstraitSource/ReadBufferBuilder.cpp +++ b/cpp-ch/local-engine/Storages/SubstraitSource/ReadBufferBuilder.cpp @@ -31,6 +31,7 @@ #include #include #include +#include #include #include #include @@ -70,6 +71,9 @@ namespace Setting extern const SettingsUInt64 s3_max_redirects; extern const SettingsBool s3_disable_checksum; extern const SettingsUInt64 s3_retry_attempts; +extern const SettingsMaxThreads max_download_threads; +extern const SettingsUInt64 max_download_buffer_size; +extern const SettingsBool input_format_allow_seeks; } namespace ErrorCodes { @@ -183,7 +187,7 @@ adjustReadRangeIfNeeded(std::unique_ptr read_buffer, const s return std::move(read_buffer); /// Skip text/json files with compression. - /// TODO implement adjustFileReadPosition when compression method is bzip2 + /// When the file is compressed, its read range is adjusted in [[buildWithCompressionWrapper]] Poco::URI file_uri(file_info.uri_file()); DB::CompressionMethod compression = DB::chooseCompressionMethod(file_uri.getPath(), "auto"); if (compression != CompressionMethod::None) @@ -216,6 +220,8 @@ class LocalFileReadBufferBuilder : public ReadBufferBuilder explicit LocalFileReadBufferBuilder(DB::ContextPtr context_) : ReadBufferBuilder(context_) { } ~LocalFileReadBufferBuilder() override = default; + bool isRemote() const override { return false; } + std::unique_ptr build(const substrait::ReadRel::LocalFiles::FileOrFiles & file_info) override { @@ -660,10 +666,6 @@ void registerReadBufferBuilders() #endif } -ReadBufferBuilder::ReadBufferBuilder(DB::ContextPtr context_) : context(context_) -{ -} - DB::ReadSettings ReadBufferBuilder::getReadSettings() const { DB::ReadSettings read_settings = context->getReadSettings(); @@ -678,6 +680,10 @@ DB::ReadSettings ReadBufferBuilder::getReadSettings() const return read_settings; } +ReadBufferBuilder::ReadBufferBuilder(DB::ContextPtr context_) : context(context_) +{ +} + std::unique_ptr ReadBufferBuilder::wrapWithBzip2(std::unique_ptr in, const substrait::ReadRel::LocalFiles::FileOrFiles & file_info) { @@ -758,7 +764,11 @@ ReadBufferBuilder::buildWithCompressionWrapper(const substrait::ReadRel::LocalFi if (compression == CompressionMethod::Bzip2) return wrapWithBzip2(std::move(in), file_info); else - return wrapReadBufferWithCompressionMethod(std::move(in), compression); + { + /// In this case we are pretty sure that current split covers the whole file because only bzip2 compression is splittable + auto parallel = wrapWithParallelIfNeeded(std::move(in), file_info); + return wrapReadBufferWithCompressionMethod(std::move(parallel), compression); + } } ReadBufferBuilder::ReadBufferCreator ReadBufferBuilder::wrapWithCache( @@ -843,6 +853,39 @@ void ReadBufferBuilder::updateCaches(const String & key, size_t last_modified_ti } } +std::unique_ptr ReadBufferBuilder::wrapWithParallelIfNeeded( + std::unique_ptr in, const substrait::ReadRel::LocalFiles::FileOrFiles & file_info) +{ + /// Only use parallel downloading for text and json format because data are read serially in those formats. + if (!file_info.has_text() && !file_info.has_json()) + return std::move(in); + + const auto & settings = context->getSettingsRef(); + auto max_download_threads = settings[DB::Setting::max_download_threads]; + auto max_download_buffer_size = settings[DB::Setting::max_download_buffer_size]; + + bool parallel_read = isRemote() && max_download_threads > 1 && isBufferWithFileSize(*in); + if (!parallel_read) + return std::move(in); + + size_t file_size = getFileSizeFromReadBuffer(*in); + if (file_size < 4 * max_download_buffer_size) + return std::move(in); + + LOG_TRACE( + getLogger("ReadBufferBuilder"), + "Using ParallelReadBuffer with {} workers with chunks of {} bytes", + max_download_threads, + max_download_buffer_size); + + return wrapInParallelReadBufferIfSupported( + {std::move(in)}, + DB::threadPoolCallbackRunnerUnsafe(DB::getIOThreadPool().get(), "ParallelRead"), + max_download_threads, + max_download_buffer_size, + file_size); +} + ReadBufferBuilderFactory & ReadBufferBuilderFactory::instance() { static ReadBufferBuilderFactory instance; diff --git a/cpp-ch/local-engine/Storages/SubstraitSource/ReadBufferBuilder.h b/cpp-ch/local-engine/Storages/SubstraitSource/ReadBufferBuilder.h index f2c1c10a6f2c..e76e79a9b0ba 100644 --- a/cpp-ch/local-engine/Storages/SubstraitSource/ReadBufferBuilder.h +++ b/cpp-ch/local-engine/Storages/SubstraitSource/ReadBufferBuilder.h @@ -19,6 +19,7 @@ #include #include #include +#include #include #include #include @@ -35,6 +36,8 @@ class ReadBufferBuilder virtual ~ReadBufferBuilder() = default; + virtual bool isRemote() const { return true; } + /// build a new read buffer virtual std::unique_ptr build(const substrait::ReadRel::LocalFiles::FileOrFiles & file_info) = 0; @@ -55,7 +58,11 @@ class ReadBufferBuilder size_t last_modified_time, size_t file_size); + std::unique_ptr + wrapWithParallelIfNeeded(std::unique_ptr in, const substrait::ReadRel::LocalFiles::FileOrFiles & file_info); + DB::ReadSettings getReadSettings() const; + DB::ContextPtr context; private: From 9a7814d96f1d49d8eb0ec4fe15d1968bdcdcdb18 Mon Sep 17 00:00:00 2001 From: Gluten Performance Bot <137994563+GlutenPerfBot@users.noreply.github.com> Date: Thu, 14 Nov 2024 19:14:16 +0800 Subject: [PATCH 22/35] [GLUTEN-6887][VL] Daily Update Velox Version (2024_11_14) (#7942) Upstream Velox's New Commits: 99979c4e7 by Krishna Pai, Back out "Readd: [velox][PR] Add runner for local distributed execution" (11529) 24a36c059 by Pedro Pedreira, docs: Add CONTRIBUTING guidelines for broken CI jobs (11521) beada2b62 by Kevin Wilfong, Fix Presto's date_trunc function when dealing with daylight savings (11509) 08dba0f73 by Deepak Majeti, Add GCS Writer Sink and fix Gcs naming (11328) f34035b03 by zhli1142015, Support decimal type for Spark unaryminus function (11454) a993e053b by duanmeng, Extracts trace node name automatically in trace replayer (11519) 06ab00135 by NEUpanning, Disable aggregate pushdown for decimal type (11298) 6c25a9120 by Krishna Pai, Throw if Aggregation Filter clause has non boolean expression (11524) 8b1717546 by aditi-pandit, refactor: Combine loops in PrestoSerializerTest.validateLexer (11434) 5ab1a8f45 by Minhan Cao, refactor: Remove Parquet thrift code from Arrow writer (11489) 145f1650e by Orri Erling, Readd: [velox][PR] Add runner for local distributed execution (11507) 711494383 by Richard Barnes, Remove unused-variable in velox/dwio/parquet/reader/DeltaByteArrayDecoder.h (11528) a5a2cad2d by David Reveman, Initial import of breeze (10770) 39aa1caf7 by Jimmy Lu, feat(dwio): Delta update support in selective readers (11501) 1f4ccc3c1 by Joe Abraham, Rename variables in Base64.* for readability (11299) 65c49f4b6 by aditi-pandit, Code cleanup in TopNRowNumber (11422) 05dc84155 by Chengcheng Jin, Support long decimal in prefixsort (10385) 117b5df60 by Jialiang Tan, Add fs fault injection in memory arbitrator fuzzer spill (11432) d4bdc3b0e by zhli1142015, Ensure prefix length does not exceed configured maximum (11496) c42320042 by Jialiang Tan, Make arbitration timed lock TSAN compatible (11517) f1ff2dff0 by Pramod, Fix `l_quantity` data generated by Tpch connector (11511) f6276bb6a by Deepak Majeti, doc: Add S3 Bucket config, fix Generic config and Spark-specific config (11418) --- cpp/velox/compute/VeloxBackend.cc | 4 ++-- cpp/velox/operators/writer/VeloxParquetDataSource.h | 2 +- cpp/velox/operators/writer/VeloxParquetDataSourceGCS.h | 2 +- cpp/velox/utils/ConfigExtractor.cc | 8 ++++---- ep/build-velox/src/get_velox.sh | 2 +- 5 files changed, 9 insertions(+), 9 deletions(-) diff --git a/cpp/velox/compute/VeloxBackend.cc b/cpp/velox/compute/VeloxBackend.cc index 889f979b2869..0d5e1d41a54d 100644 --- a/cpp/velox/compute/VeloxBackend.cc +++ b/cpp/velox/compute/VeloxBackend.cc @@ -41,7 +41,7 @@ #include "velox/connectors/hive/HiveConnector.h" #include "velox/connectors/hive/HiveDataSource.h" #include "velox/connectors/hive/storage_adapters/abfs/RegisterAbfsFileSystem.h" // @manual -#include "velox/connectors/hive/storage_adapters/gcs/RegisterGCSFileSystem.h" // @manual +#include "velox/connectors/hive/storage_adapters/gcs/RegisterGcsFileSystem.h" // @manual #include "velox/connectors/hive/storage_adapters/hdfs/RegisterHdfsFileSystem.h" // @manual #include "velox/connectors/hive/storage_adapters/s3fs/RegisterS3FileSystem.h" // @manual #include "velox/dwio/orc/reader/OrcReader.h" @@ -142,7 +142,7 @@ void VeloxBackend::init(const std::unordered_map& conf velox::filesystems::registerS3FileSystem(); #endif #ifdef ENABLE_GCS - velox::filesystems::registerGCSFileSystem(); + velox::filesystems::registerGcsFileSystem(); #endif #ifdef ENABLE_ABFS velox::filesystems::abfs::registerAbfsFileSystem(); diff --git a/cpp/velox/operators/writer/VeloxParquetDataSource.h b/cpp/velox/operators/writer/VeloxParquetDataSource.h index e7428999f02e..a94ab00b9e2a 100644 --- a/cpp/velox/operators/writer/VeloxParquetDataSource.h +++ b/cpp/velox/operators/writer/VeloxParquetDataSource.h @@ -37,7 +37,7 @@ #include "velox/connectors/hive/storage_adapters/s3fs/S3Util.h" #endif #ifdef ENABLE_GCS -#include "velox/connectors/hive/storage_adapters/gcs/GCSFileSystem.h" +#include "velox/connectors/hive/storage_adapters/gcs/GcsFileSystem.h" #endif #ifdef ENABLE_HDFS #include "velox/connectors/hive/storage_adapters/hdfs/HdfsFileSystem.h" diff --git a/cpp/velox/operators/writer/VeloxParquetDataSourceGCS.h b/cpp/velox/operators/writer/VeloxParquetDataSourceGCS.h index 22d4e96efe7f..579a425d1833 100644 --- a/cpp/velox/operators/writer/VeloxParquetDataSourceGCS.h +++ b/cpp/velox/operators/writer/VeloxParquetDataSourceGCS.h @@ -43,7 +43,7 @@ class VeloxParquetDataSourceGCS final : public VeloxParquetDataSource { void initSink(const std::unordered_map& /* sparkConfs */) override { auto fileSystem = filesystems::getFileSystem(filePath_, nullptr); - auto* gcsFileSystem = dynamic_cast(fileSystem.get()); + auto* gcsFileSystem = dynamic_cast(fileSystem.get()); sink_ = std::make_unique( gcsFileSystem->openFileForWrite(filePath_, {{}, sinkPool_.get()}), filePath_); } diff --git a/cpp/velox/utils/ConfigExtractor.cc b/cpp/velox/utils/ConfigExtractor.cc index 3175c375fafc..78366b37568b 100644 --- a/cpp/velox/utils/ConfigExtractor.cc +++ b/cpp/velox/utils/ConfigExtractor.cc @@ -141,7 +141,7 @@ std::shared_ptr getHiveConfig( std::string gcsEndpoint = gsStorageRootUrl.value(); if (!gcsEndpoint.empty()) { - hiveConfMap[facebook::velox::connector::hive::HiveConfig::kGCSEndpoint] = gcsEndpoint; + hiveConfMap[facebook::velox::connector::hive::HiveConfig::kGcsEndpoint] = gcsEndpoint; } } @@ -149,13 +149,13 @@ std::shared_ptr getHiveConfig( // https://cloud.google.com/cpp/docs/reference/storage/latest/classgoogle_1_1cloud_1_1storage_1_1LimitedErrorCountRetryPolicy auto gsMaxRetryCount = conf->get("spark.hadoop.fs.gs.http.max.retry"); if (gsMaxRetryCount.hasValue()) { - hiveConfMap[facebook::velox::connector::hive::HiveConfig::kGCSMaxRetryCount] = gsMaxRetryCount.value(); + hiveConfMap[facebook::velox::connector::hive::HiveConfig::kGcsMaxRetryCount] = gsMaxRetryCount.value(); } // https://cloud.google.com/cpp/docs/reference/storage/latest/classgoogle_1_1cloud_1_1storage_1_1LimitedTimeRetryPolicy auto gsMaxRetryTime = conf->get("spark.hadoop.fs.gs.http.max.retry-time"); if (gsMaxRetryTime.hasValue()) { - hiveConfMap[facebook::velox::connector::hive::HiveConfig::kGCSMaxRetryTime] = gsMaxRetryTime.value(); + hiveConfMap[facebook::velox::connector::hive::HiveConfig::kGcsMaxRetryTime] = gsMaxRetryTime.value(); } // https://github.com/GoogleCloudDataproc/hadoop-connectors/blob/master/gcs/CONFIGURATION.md#authentication @@ -166,7 +166,7 @@ std::shared_ptr getHiveConfig( auto gsAuthServiceAccountJsonKeyfile = conf->get("spark.hadoop.fs.gs.auth.service.account.json.keyfile"); if (gsAuthServiceAccountJsonKeyfile.hasValue()) { - hiveConfMap[facebook::velox::connector::hive::HiveConfig::kGCSCredentialsPath] = + hiveConfMap[facebook::velox::connector::hive::HiveConfig::kGcsCredentialsPath] = gsAuthServiceAccountJsonKeyfile.value(); } else { LOG(WARNING) << "STARTUP: conf spark.hadoop.fs.gs.auth.type is set to SERVICE_ACCOUNT_JSON_KEYFILE, " diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh index 2be1481fc05d..06a7215d827a 100755 --- a/ep/build-velox/src/get_velox.sh +++ b/ep/build-velox/src/get_velox.sh @@ -17,7 +17,7 @@ set -exu VELOX_REPO=https://github.com/oap-project/velox.git -VELOX_BRANCH=2024_11_13_new +VELOX_BRANCH=2024_11_14 VELOX_HOME="" OS=`uname -s` From 7b4005055408b7cbdc4f5e9bec5b6ac442e52259 Mon Sep 17 00:00:00 2001 From: lgbo Date: Thu, 14 Nov 2024 20:01:16 +0800 Subject: [PATCH 23/35] [GLUTEN-7647][CH] Drop literals in aggregation results (#7951) --- .../gluten/extension/LazyAggregateExpandRule.scala | 13 +++++++++++-- .../GlutenClickHouseTPCHSaltNullParquetSuite.scala | 8 ++++++++ 2 files changed, 19 insertions(+), 2 deletions(-) diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/extension/LazyAggregateExpandRule.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/LazyAggregateExpandRule.scala index d64300a12fcb..0f7752339707 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/extension/LazyAggregateExpandRule.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/LazyAggregateExpandRule.scala @@ -345,8 +345,17 @@ case class LazyAggregateExpandRule(session: SparkSession) extends Rule[SparkPlan s"groupingExpressions: ${partialAggregate.groupingExpressions}") // Remove group id column from result expressions - val resultExpressions = partialAggregate.resultExpressions - .filter(!_.toAttribute.semanticEquals(groupIdAttribute)) + val groupingKeysCount = + partialAggregate.resultExpressions.length - partialAggregate.aggregateExpressions.length + var resultExpressions = partialAggregate.resultExpressions + .slice(0, groupingKeysCount) + .filter( + e => + !e.toAttribute.semanticEquals(groupIdAttribute) && + attributesToReplace.find(kv => kv._1.semanticEquals(e.toAttribute)).isDefined) + .map(e => getReplaceAttribute(e.toAttribute, attributesToReplace)) + resultExpressions = resultExpressions ++ partialAggregate.resultExpressions + .slice(groupingKeysCount, partialAggregate.resultExpressions.length) .map(e => getReplaceAttribute(e.toAttribute, attributesToReplace)) logDebug( s"xxx newResultExpressions: $resultExpressions\n" + diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHSaltNullParquetSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHSaltNullParquetSuite.scala index 9affdeb7f7fc..865d3fa40c0f 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHSaltNullParquetSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHSaltNullParquetSuite.scala @@ -3066,6 +3066,14 @@ class GlutenClickHouseTPCHSaltNullParquetSuite extends GlutenClickHouseTPCHAbstr |order by n_regionkey, n_nationkey |""".stripMargin compareResultsAgainstVanillaSpark(sql, true, checkLazyExpand) + + sql = """ + |select x, n_regionkey, n_nationkey, + |sum(n_regionkey), count(n_name), max(n_regionkey), min(n_regionkey) + |from (select '123' as x, * from nation) group by x, n_regionkey, n_nationkey with cube + |order by x, n_regionkey, n_nationkey + |""".stripMargin + compareResultsAgainstVanillaSpark(sql, true, checkLazyExpand) } test("GLUTEN-7647 lazy expand for avg and sum") { From f65087ece35fc13a56189c450c56075107aad446 Mon Sep 17 00:00:00 2001 From: Chang chen Date: Thu, 14 Nov 2024 23:51:01 +0800 Subject: [PATCH 24/35] [CH] Fix issues due to https://github.com/ClickHouse/ClickHouse/pull/71539. (#7952) * Fix issues due to https://github.com/ClickHouse/ClickHouse/pull/71539. Issue 1 BuildQueryPipelineSettings is created manually instead of calling BuildQueryPipelineSettings::fromContext(); so even https://github.com/ClickHouse/ClickHouse/pull/71890 disable 'query_plan_merge_filters', UTs are still failed. To fix this issue, we need set correct default parameters in CHUtil.cpp Issue 2 If we set query_plan_merge_filters to true, then https://github.com/ClickHouse/ClickHouse/pull/71539 will try to split the left most AND atom to a separate DAG and hence create FilterTransformer for each And atom, which cause collecting metrics failed. I am not sure the benefits of setting it to true, let's keep it to false. * Calling `QueryPlan::explainPlan` after building pipeline is not correct, due to `action_dag` is [moved](https://github.com/ClickHouse/ClickHouse/blob/22d2c856a70dfb8b6e4c506fcb22ac03d59df9be/src/Processors/QueryPlan/FilterStep.cpp#L161). --- cpp-ch/local-engine/Common/CHUtil.cpp | 11 +++++++ cpp-ch/local-engine/Common/DebugUtils.cpp | 16 +++++---- cpp-ch/local-engine/Common/DebugUtils.h | 2 +- .../Parser/SerializedPlanParser.cpp | 33 ++++++++----------- .../Parser/SerializedPlanParser.h | 4 +-- 5 files changed, 37 insertions(+), 29 deletions(-) diff --git a/cpp-ch/local-engine/Common/CHUtil.cpp b/cpp-ch/local-engine/Common/CHUtil.cpp index 778c4f257b19..2413fae9e350 100644 --- a/cpp-ch/local-engine/Common/CHUtil.cpp +++ b/cpp-ch/local-engine/Common/CHUtil.cpp @@ -80,6 +80,9 @@ namespace Setting { extern const SettingsUInt64 prefer_external_sort_block_bytes; extern const SettingsUInt64 max_bytes_before_external_sort; +extern const SettingsBool query_plan_merge_filters; +extern const SettingsBool compile_expressions; +extern const SettingsShortCircuitFunctionEvaluation short_circuit_function_evaluation; } namespace ErrorCodes { @@ -722,6 +725,14 @@ void BackendInitializerUtil::initSettings(const SparkConfigs::ConfigMap & spark_ settings.set("max_download_threads", 1); settings.set("input_format_parquet_enable_row_group_prefetch", false); + /// update per https://github.com/ClickHouse/ClickHouse/pull/71539 + /// if true, we can't get correct metrics for the query + settings[Setting::query_plan_merge_filters] = false; + /// We now set BuildQueryPipelineSettings according to config. + settings[Setting::compile_expressions] = true; + settings[Setting::short_circuit_function_evaluation] = ShortCircuitFunctionEvaluation::DISABLE; + /// + for (const auto & [key, value] : spark_conf_map) { // Firstly apply spark.gluten.sql.columnar.backend.ch.runtime_config.local_engine.settings.* to settings diff --git a/cpp-ch/local-engine/Common/DebugUtils.cpp b/cpp-ch/local-engine/Common/DebugUtils.cpp index 2fcab59bf856..8a4323cb1c13 100644 --- a/cpp-ch/local-engine/Common/DebugUtils.cpp +++ b/cpp-ch/local-engine/Common/DebugUtils.cpp @@ -38,7 +38,7 @@ namespace pb_util = google::protobuf::util; namespace debug { -void dumpPlan(DB::QueryPlan & plan, bool force, LoggerPtr logger) +void dumpPlan(DB::QueryPlan & plan, const char * type, bool force, LoggerPtr logger) { if (!logger) { @@ -51,10 +51,12 @@ void dumpPlan(DB::QueryPlan & plan, bool force, LoggerPtr logger) return; auto out = local_engine::PlanUtil::explainPlan(plan); + auto task_id = local_engine::QueryContext::instance().currentTaskIdOrEmpty(); + task_id = task_id.empty() ? "" : "(" + task_id + ")"; if (force) // force - LOG_ERROR(logger, "clickhouse plan({}):\n{}", local_engine::QueryContext::instance().currentTaskIdOrEmpty(), out); + LOG_ERROR(logger, "{}{} =>\n{}", type, task_id, out); else - LOG_DEBUG(logger, "clickhouse plan({}):\n{}", local_engine::QueryContext::instance().currentTaskIdOrEmpty(), out); + LOG_DEBUG(logger, "{}{} =>\n{}", type, task_id, out); } void dumpMessage(const google::protobuf::Message & message, const char * type, bool force, LoggerPtr logger) @@ -70,13 +72,15 @@ void dumpMessage(const google::protobuf::Message & message, const char * type, b return; pb_util::JsonOptions options; std::string json; - if (auto s = google::protobuf::json::MessageToJsonString(message, &json, options); !s.ok()) + if (auto s = MessageToJsonString(message, &json, options); !s.ok()) throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Can not convert {} to Json", type); + auto task_id = local_engine::QueryContext::instance().currentTaskIdOrEmpty(); + task_id = task_id.empty() ? "" : "(" + task_id + ")"; if (force) // force - LOG_ERROR(logger, "{}({}):\n{}", type, local_engine::QueryContext::instance().currentTaskIdOrEmpty(), json); + LOG_ERROR(logger, "{}{} =>\n{}", type, task_id, json); else - LOG_DEBUG(logger, "{}({}):\n{}", type, local_engine::QueryContext::instance().currentTaskIdOrEmpty(), json); + LOG_DEBUG(logger, "{}{} =>\n{}", type, task_id, json); } void headBlock(const DB::Block & block, size_t count) diff --git a/cpp-ch/local-engine/Common/DebugUtils.h b/cpp-ch/local-engine/Common/DebugUtils.h index 55a0be5140c5..338326b05e0e 100644 --- a/cpp-ch/local-engine/Common/DebugUtils.h +++ b/cpp-ch/local-engine/Common/DebugUtils.h @@ -29,7 +29,7 @@ class QueryPlan; namespace debug { -void dumpPlan(DB::QueryPlan & plan, bool force = false, LoggerPtr = nullptr); +void dumpPlan(DB::QueryPlan & plan, const char * type = "clickhouse plan", bool force = false, LoggerPtr = nullptr); void dumpMessage(const google::protobuf::Message & message, const char * type, bool force = false, LoggerPtr = nullptr); void headBlock(const DB::Block & block, size_t count = 10); diff --git a/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp b/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp index 748ff88acbd1..4e461a5c4954 100644 --- a/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp +++ b/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp @@ -101,8 +101,9 @@ std::string join(const ActionsDAG::NodeRawConstPtrs & v, char c) return res; } -void SerializedPlanParser::adjustOutput(const DB::QueryPlanPtr & query_plan, const substrait::PlanRel & root_rel) const +void SerializedPlanParser::adjustOutput(const DB::QueryPlanPtr & query_plan, const substrait::Plan & plan) { + const substrait::PlanRel & root_rel = plan.relations().at(0); if (root_rel.root().names_size()) { ActionsDAG actions_dag{blockToNameAndTypeList(query_plan->getCurrentHeader())}; @@ -110,8 +111,8 @@ void SerializedPlanParser::adjustOutput(const DB::QueryPlanPtr & query_plan, con const auto cols = query_plan->getCurrentHeader().getNamesAndTypesList(); if (cols.getNames().size() != static_cast(root_rel.root().names_size())) { - debug::dumpPlan(*query_plan, true); - debug::dumpMessage(root_rel, "substrait::PlanRel", true); + debug::dumpPlan(*query_plan, "clickhouse plan", true); + debug::dumpMessage(plan, "substrait::Plan", true); throw Exception( ErrorCodes::LOGICAL_ERROR, "Missmatch result columns size. plan column size {}, subtrait plan name size {}.", @@ -134,8 +135,8 @@ void SerializedPlanParser::adjustOutput(const DB::QueryPlanPtr & query_plan, con const auto & original_cols = original_header.getColumnsWithTypeAndName(); if (static_cast(output_schema.types_size()) != original_cols.size()) { - debug::dumpPlan(*query_plan, true); - debug::dumpMessage(root_rel, "substrait::PlanRel", true); + debug::dumpPlan(*query_plan, "clickhouse plan", true); + debug::dumpMessage(plan, "substrait::Plan", true); throw Exception( ErrorCodes::LOGICAL_ERROR, "Missmatch result columns size. plan column size {}, subtrait plan output schema size {}, subtrait plan name size {}.", @@ -198,7 +199,7 @@ QueryPlanPtr SerializedPlanParser::parse(const substrait::Plan & plan) std::list rel_stack; auto query_plan = parseOp(first_read_rel, rel_stack); if (!writePipeline) - adjustOutput(query_plan, root_rel); + adjustOutput(query_plan, plan); #ifndef NDEBUG PlanUtil::checkOuputType(*query_plan); @@ -297,12 +298,10 @@ DB::QueryPipelineBuilderPtr SerializedPlanParser::buildQueryPipeline(DB::QueryPl settings, 0); const QueryPlanOptimizationSettings optimization_settings{.optimize_plan = settings[Setting::query_plan_enable_optimizations]}; - return query_plan.buildQueryPipeline( - optimization_settings, - BuildQueryPipelineSettings{ - .actions_settings - = ExpressionActionsSettings{.can_compile_expressions = true, .min_count_to_compile_expression = 3, .compile_expressions = CompileExpressions::yes}, - .process_list_element = query_status}); + BuildQueryPipelineSettings build_settings = BuildQueryPipelineSettings::fromContext(context); + build_settings.process_list_element = query_status; + build_settings.progress_callback = nullptr; + return query_plan.buildQueryPipeline(optimization_settings,build_settings); } std::unique_ptr SerializedPlanParser::createExecutor(const std::string_view plan) @@ -311,11 +310,10 @@ std::unique_ptr SerializedPlanParser::createExecutor(const std::s return createExecutor(parse(s_plan), s_plan); } -std::unique_ptr SerializedPlanParser::createExecutor(DB::QueryPlanPtr query_plan, const substrait::Plan & s_plan) +std::unique_ptr SerializedPlanParser::createExecutor(DB::QueryPlanPtr query_plan, const substrait::Plan & s_plan) const { Stopwatch stopwatch; - const Settings & settings = parser_context->queryContext()->getSettingsRef(); DB::QueryPipelineBuilderPtr builder = nullptr; try { @@ -323,7 +321,7 @@ std::unique_ptr SerializedPlanParser::createExecutor(DB::QueryPla } catch (...) { - LOG_ERROR(getLogger("SerializedPlanParser"), "Invalid plan:\n{}", PlanUtil::explainPlan(*query_plan)); + debug::dumpPlan(*query_plan, "Invalid clickhouse plan", true); throw; } @@ -333,11 +331,6 @@ std::unique_ptr SerializedPlanParser::createExecutor(DB::QueryPla if (root_rel.root().input().has_write()) addSinkTransform(parser_context->queryContext(), root_rel.root().input().write(), builder); LOG_INFO(getLogger("SerializedPlanParser"), "build pipeline {} ms", stopwatch.elapsedMicroseconds() / 1000.0); - LOG_DEBUG( - getLogger("SerializedPlanParser"), - "clickhouse plan [optimization={}]:\n{}", - settings[Setting::query_plan_enable_optimizations], - PlanUtil::explainPlan(*query_plan)); auto config = ExecutorConfig::loadFromContext(parser_context->queryContext()); return std::make_unique(std::move(query_plan), std::move(builder), config.dump_pipeline); diff --git a/cpp-ch/local-engine/Parser/SerializedPlanParser.h b/cpp-ch/local-engine/Parser/SerializedPlanParser.h index f0ec608a330f..eadc7112c266 100644 --- a/cpp-ch/local-engine/Parser/SerializedPlanParser.h +++ b/cpp-ch/local-engine/Parser/SerializedPlanParser.h @@ -68,7 +68,7 @@ class NonNullableColumnsResolver class SerializedPlanParser { private: - std::unique_ptr createExecutor(DB::QueryPlanPtr query_plan, const substrait::Plan & s_plan); + std::unique_ptr createExecutor(DB::QueryPlanPtr query_plan, const substrait::Plan & s_plan) const; public: explicit SerializedPlanParser(std::shared_ptr parser_context_); @@ -118,7 +118,7 @@ class SerializedPlanParser private: DB::QueryPlanPtr parseOp(const substrait::Rel & rel, std::list & rel_stack); - void adjustOutput(const DB::QueryPlanPtr & query_plan, const substrait::PlanRel & root_rel) const; + static void adjustOutput(const DB::QueryPlanPtr & query_plan, const substrait::Plan & plan); std::vector input_iters; std::vector split_infos; From 1544307d0c91133fbc059ec3f149869c9701c0f9 Mon Sep 17 00:00:00 2001 From: Rong Ma Date: Fri, 15 Nov 2024 10:42:07 +0800 Subject: [PATCH 25/35] [GLUTEN-6896] Add buffered read for hash/sort shuffle reader (#7897) --- .../vectorized/ColumnarBatchSerializer.scala | 2 + cpp/core/jni/JniWrapper.cc | 3 +- cpp/core/shuffle/Options.h | 2 + cpp/core/shuffle/Payload.cc | 18 ++++++--- cpp/core/shuffle/Payload.h | 1 + cpp/velox/compute/VeloxRuntime.cc | 1 + cpp/velox/shuffle/VeloxShuffleReader.cc | 37 ++++++++++++------- cpp/velox/shuffle/VeloxShuffleReader.h | 7 +++- .../utils/tests/VeloxShuffleWriterTestBase.h | 1 + docs/Configuration.md | 1 + .../vectorized/ShuffleReaderJniWrapper.java | 1 + ...VeloxCelebornColumnarBatchSerializer.scala | 2 + .../org/apache/gluten/GlutenConfig.scala | 13 +++++++ 13 files changed, 68 insertions(+), 21 deletions(-) diff --git a/backends-velox/src/main/scala/org/apache/gluten/vectorized/ColumnarBatchSerializer.scala b/backends-velox/src/main/scala/org/apache/gluten/vectorized/ColumnarBatchSerializer.scala index fa4d92652ca3..88215e36bb77 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/vectorized/ColumnarBatchSerializer.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/vectorized/ColumnarBatchSerializer.scala @@ -98,6 +98,7 @@ private class ColumnarBatchSerializerInstance( val compressionCodecBackend = GlutenConfig.getConf.columnarShuffleCodecBackend.orNull val batchSize = GlutenConfig.getConf.maxBatchSize + val bufferSize = GlutenConfig.getConf.columnarShuffleReaderBufferSize val runtime = Runtimes.contextInstance("ShuffleReader") val jniWrapper = ShuffleReaderJniWrapper.create(runtime) val shuffleReaderHandle = jniWrapper.make( @@ -105,6 +106,7 @@ private class ColumnarBatchSerializerInstance( compressionCodec, compressionCodecBackend, batchSize, + bufferSize, shuffleWriterType) // Close shuffle reader instance as lately as the end of task processing, // since the native reader could hold a reference to memory pool that diff --git a/cpp/core/jni/JniWrapper.cc b/cpp/core/jni/JniWrapper.cc index 6a0a5b0057f7..9da55894864e 100644 --- a/cpp/core/jni/JniWrapper.cc +++ b/cpp/core/jni/JniWrapper.cc @@ -996,6 +996,7 @@ JNIEXPORT jlong JNICALL Java_org_apache_gluten_vectorized_ShuffleReaderJniWrappe jstring compressionType, jstring compressionBackend, jint batchSize, + jlong bufferSize, jstring shuffleWriterType) { JNI_METHOD_START auto ctx = getRuntime(env, wrapper); @@ -1007,7 +1008,7 @@ JNIEXPORT jlong JNICALL Java_org_apache_gluten_vectorized_ShuffleReaderJniWrappe options.codecBackend = getCodecBackend(env, compressionBackend); } options.batchSize = batchSize; - // TODO: Add coalesce option and maximum coalesced size. + options.bufferSize = bufferSize; options.shuffleWriterType = ShuffleWriter::stringToType(jStringToCString(env, shuffleWriterType)); std::shared_ptr schema = diff --git a/cpp/core/shuffle/Options.h b/cpp/core/shuffle/Options.h index a3dc9f6260b0..6a9e0ec4b340 100644 --- a/cpp/core/shuffle/Options.h +++ b/cpp/core/shuffle/Options.h @@ -38,6 +38,7 @@ static constexpr double kDefaultMergeBufferThreshold = 0.25; static constexpr bool kEnableBufferedWrite = true; static constexpr bool kDefaultUseRadixSort = true; static constexpr int32_t kDefaultSortBufferSize = 4096; +static constexpr int64_t kDefaultReadBufferSize = 1 << 20; enum ShuffleWriterType { kHashShuffle, kSortShuffle, kRssSortShuffle }; enum PartitionWriterType { kLocal, kRss }; @@ -49,6 +50,7 @@ struct ShuffleReaderOptions { ShuffleWriterType shuffleWriterType = kHashShuffle; CodecBackend codecBackend = CodecBackend::NONE; int32_t batchSize = kDefaultBatchSize; + int64_t bufferSize = kDefaultReadBufferSize; }; struct ShuffleWriterOptions { diff --git a/cpp/core/shuffle/Payload.cc b/cpp/core/shuffle/Payload.cc index d5cdb6d08eda..55f3a4339677 100644 --- a/cpp/core/shuffle/Payload.cc +++ b/cpp/core/shuffle/Payload.cc @@ -118,9 +118,9 @@ arrow::Status compressAndFlush( return arrow::Status::OK(); } -arrow::Result> readUncompressedBuffer( - arrow::io::InputStream* inputStream, - arrow::MemoryPool* pool) { +arrow::Result> +readUncompressedBuffer(arrow::io::InputStream* inputStream, arrow::MemoryPool* pool, int64_t& deserializedTime) { + ScopedTimer timer(&deserializedTime); int64_t bufferLength; RETURN_NOT_OK(inputStream->Read(sizeof(int64_t), &bufferLength)); if (bufferLength == kNullBuffer) { @@ -135,7 +135,9 @@ arrow::Result> readCompressedBuffer( arrow::io::InputStream* inputStream, const std::shared_ptr& codec, arrow::MemoryPool* pool, + int64_t& deserializeTime, int64_t& decompressTime) { + ScopedTimer timer(&deserializeTime); int64_t compressedLength; RETURN_NOT_OK(inputStream->Read(sizeof(int64_t), &compressedLength)); if (compressedLength == kNullBuffer) { @@ -155,7 +157,7 @@ arrow::Result> readCompressedBuffer( ARROW_ASSIGN_OR_RAISE(auto compressed, arrow::AllocateResizableBuffer(compressedLength, pool)); RETURN_NOT_OK(inputStream->Read(compressedLength, compressed->mutable_data())); - ScopedTimer timer(&decompressTime); + timer.switchTo(&decompressTime); ARROW_ASSIGN_OR_RAISE(auto output, arrow::AllocateResizableBuffer(uncompressedLength, pool)); RETURN_NOT_OK(codec->Decompress(compressedLength, compressed->data(), uncompressedLength, output->mutable_data())); return output; @@ -296,7 +298,9 @@ arrow::Result>> BlockPayload::deseria const std::shared_ptr& codec, arrow::MemoryPool* pool, uint32_t& numRows, + int64_t& deserializeTime, int64_t& decompressTime) { + auto timer = std::make_unique(&deserializeTime); static const std::vector> kEmptyBuffers{}; ARROW_ASSIGN_OR_RAISE(auto type, readType(inputStream)); if (type == 0) { @@ -306,6 +310,7 @@ arrow::Result>> BlockPayload::deseria RETURN_NOT_OK(inputStream->Read(sizeof(uint32_t), &numRows)); uint32_t numBuffers; RETURN_NOT_OK(inputStream->Read(sizeof(uint32_t), &numBuffers)); + timer.reset(); bool isCompressionEnabled = type == Type::kCompressed; std::vector> buffers; @@ -313,9 +318,10 @@ arrow::Result>> BlockPayload::deseria for (auto i = 0; i < numBuffers; ++i) { buffers.emplace_back(); if (isCompressionEnabled) { - ARROW_ASSIGN_OR_RAISE(buffers.back(), readCompressedBuffer(inputStream, codec, pool, decompressTime)); + ARROW_ASSIGN_OR_RAISE( + buffers.back(), readCompressedBuffer(inputStream, codec, pool, deserializeTime, decompressTime)); } else { - ARROW_ASSIGN_OR_RAISE(buffers.back(), readUncompressedBuffer(inputStream, pool)); + ARROW_ASSIGN_OR_RAISE(buffers.back(), readUncompressedBuffer(inputStream, pool, deserializeTime)); } } return buffers; diff --git a/cpp/core/shuffle/Payload.h b/cpp/core/shuffle/Payload.h index ea8c897e96d0..611b2310d547 100644 --- a/cpp/core/shuffle/Payload.h +++ b/cpp/core/shuffle/Payload.h @@ -92,6 +92,7 @@ class BlockPayload final : public Payload { const std::shared_ptr& codec, arrow::MemoryPool* pool, uint32_t& numRows, + int64_t& deserializeTime, int64_t& decompressTime); static int64_t maxCompressedLength( diff --git a/cpp/velox/compute/VeloxRuntime.cc b/cpp/velox/compute/VeloxRuntime.cc index 332c75dbd725..4c6b52e6fe04 100644 --- a/cpp/velox/compute/VeloxRuntime.cc +++ b/cpp/velox/compute/VeloxRuntime.cc @@ -259,6 +259,7 @@ std::shared_ptr VeloxRuntime::createShuffleReader( veloxCompressionType, rowType, options.batchSize, + options.bufferSize, memoryManager()->getArrowMemoryPool(), ctxVeloxPool, options.shuffleWriterType); diff --git a/cpp/velox/shuffle/VeloxShuffleReader.cc b/cpp/velox/shuffle/VeloxShuffleReader.cc index 92751d454aa9..0407be736a70 100644 --- a/cpp/velox/shuffle/VeloxShuffleReader.cc +++ b/cpp/velox/shuffle/VeloxShuffleReader.cc @@ -38,7 +38,6 @@ #include #include -// using namespace facebook; using namespace facebook::velox; namespace gluten { @@ -291,14 +290,14 @@ VeloxHashShuffleReaderDeserializer::VeloxHashShuffleReaderDeserializer( const std::shared_ptr& codec, const facebook::velox::RowTypePtr& rowType, int32_t batchSize, + int64_t bufferSize, arrow::MemoryPool* memoryPool, facebook::velox::memory::MemoryPool* veloxPool, std::vector* isValidityBuffer, bool hasComplexType, int64_t& deserializeTime, int64_t& decompressTime) - : in_(std::move(in)), - schema_(schema), + : schema_(schema), codec_(codec), rowType_(rowType), batchSize_(batchSize), @@ -307,13 +306,16 @@ VeloxHashShuffleReaderDeserializer::VeloxHashShuffleReaderDeserializer( isValidityBuffer_(isValidityBuffer), hasComplexType_(hasComplexType), deserializeTime_(deserializeTime), - decompressTime_(decompressTime) {} + decompressTime_(decompressTime) { + GLUTEN_ASSIGN_OR_THROW(in_, arrow::io::BufferedInputStream::Create(bufferSize, memoryPool, std::move(in))); +} std::shared_ptr VeloxHashShuffleReaderDeserializer::next() { if (hasComplexType_) { uint32_t numRows = 0; GLUTEN_ASSIGN_OR_THROW( - auto arrowBuffers, BlockPayload::deserialize(in_.get(), codec_, memoryPool_, numRows, decompressTime_)); + auto arrowBuffers, + BlockPayload::deserialize(in_.get(), codec_, memoryPool_, numRows, deserializeTime_, decompressTime_)); if (arrowBuffers.empty()) { // Reach EOS. return nullptr; @@ -332,7 +334,8 @@ std::shared_ptr VeloxHashShuffleReaderDeserializer::next() { uint32_t numRows = 0; while (!merged_ || merged_->numRows() < batchSize_) { GLUTEN_ASSIGN_OR_THROW( - arrowBuffers, BlockPayload::deserialize(in_.get(), codec_, memoryPool_, numRows, decompressTime_)); + arrowBuffers, + BlockPayload::deserialize(in_.get(), codec_, memoryPool_, numRows, deserializeTime_, decompressTime_)); if (arrowBuffers.empty()) { reachEos_ = true; break; @@ -372,22 +375,24 @@ VeloxSortShuffleReaderDeserializer::VeloxSortShuffleReaderDeserializer( const std::shared_ptr& codec, const RowTypePtr& rowType, int32_t batchSize, + int64_t bufferSize, arrow::MemoryPool* memoryPool, facebook::velox::memory::MemoryPool* veloxPool, int64_t& deserializeTime, int64_t& decompressTime) - : in_(std::move(in)), - schema_(schema), + : schema_(schema), codec_(codec), rowType_(rowType), batchSize_(batchSize), arrowPool_(memoryPool), veloxPool_(veloxPool), deserializeTime_(deserializeTime), - decompressTime_(decompressTime) {} + decompressTime_(decompressTime) { + GLUTEN_ASSIGN_OR_THROW(in_, arrow::io::BufferedInputStream::Create(bufferSize, memoryPool, std::move(in))); +} std::shared_ptr VeloxSortShuffleReaderDeserializer::next() { - if (reachEos_) { + if (reachedEos_) { if (cachedRows_ > 0) { return deserializeToBatch(); } @@ -401,10 +406,11 @@ std::shared_ptr VeloxSortShuffleReaderDeserializer::next() { while (cachedRows_ < batchSize_) { uint32_t numRows = 0; GLUTEN_ASSIGN_OR_THROW( - auto arrowBuffers, BlockPayload::deserialize(in_.get(), codec_, arrowPool_, numRows, decompressTime_)); + auto arrowBuffers, + BlockPayload::deserialize(in_.get(), codec_, arrowPool_, numRows, deserializeTime_, decompressTime_)); if (arrowBuffers.empty()) { - reachEos_ = true; + reachedEos_ = true; if (cachedRows_ > 0) { return deserializeToBatch(); } @@ -467,7 +473,8 @@ void VeloxSortShuffleReaderDeserializer::readLargeRow(std::vectorsize(); buffers.emplace_back(std::move(arrowBuffers[0])); @@ -575,6 +582,7 @@ VeloxColumnarBatchDeserializerFactory::VeloxColumnarBatchDeserializerFactory( const facebook::velox::common::CompressionKind veloxCompressionType, const RowTypePtr& rowType, int32_t batchSize, + int64_t bufferSize, arrow::MemoryPool* memoryPool, std::shared_ptr veloxPool, ShuffleWriterType shuffleWriterType) @@ -583,6 +591,7 @@ VeloxColumnarBatchDeserializerFactory::VeloxColumnarBatchDeserializerFactory( veloxCompressionType_(veloxCompressionType), rowType_(rowType), batchSize_(batchSize), + bufferSize_(bufferSize), memoryPool_(memoryPool), veloxPool_(veloxPool), shuffleWriterType_(shuffleWriterType) { @@ -599,6 +608,7 @@ std::unique_ptr VeloxColumnarBatchDeserializerFactory::cr codec_, rowType_, batchSize_, + bufferSize_, memoryPool_, veloxPool_.get(), &isValidityBuffer_, @@ -612,6 +622,7 @@ std::unique_ptr VeloxColumnarBatchDeserializerFactory::cr codec_, rowType_, batchSize_, + bufferSize_, memoryPool_, veloxPool_.get(), deserializeTime_, diff --git a/cpp/velox/shuffle/VeloxShuffleReader.h b/cpp/velox/shuffle/VeloxShuffleReader.h index d39e38936369..af35f977127f 100644 --- a/cpp/velox/shuffle/VeloxShuffleReader.h +++ b/cpp/velox/shuffle/VeloxShuffleReader.h @@ -21,6 +21,7 @@ #include "shuffle/Payload.h" #include "shuffle/ShuffleReader.h" #include "shuffle/VeloxSortShuffleWriter.h" +#include "utils/Timer.h" #include "velox/type/Type.h" #include "velox/vector/ComplexVector.h" @@ -36,6 +37,7 @@ class VeloxHashShuffleReaderDeserializer final : public ColumnarBatchIterator { const std::shared_ptr& codec, const facebook::velox::RowTypePtr& rowType, int32_t batchSize, + int64_t bufferSize, arrow::MemoryPool* memoryPool, facebook::velox::memory::MemoryPool* veloxPool, std::vector* isValidityBuffer, @@ -73,6 +75,7 @@ class VeloxSortShuffleReaderDeserializer final : public ColumnarBatchIterator { const std::shared_ptr& codec, const facebook::velox::RowTypePtr& rowType, int32_t batchSize, + int64_t bufferSize, arrow::MemoryPool* memoryPool, facebook::velox::memory::MemoryPool* veloxPool, int64_t& deserializeTime, @@ -97,7 +100,7 @@ class VeloxSortShuffleReaderDeserializer final : public ColumnarBatchIterator { std::list> cachedInputs_; uint32_t cachedRows_{0}; - bool reachEos_{false}; + bool reachedEos_{false}; uint32_t rowOffset_{0}; size_t byteOffset_{0}; @@ -139,6 +142,7 @@ class VeloxColumnarBatchDeserializerFactory : public DeserializerFactory { const facebook::velox::common::CompressionKind veloxCompressionType, const facebook::velox::RowTypePtr& rowType, int32_t batchSize, + int64_t bufferSize, arrow::MemoryPool* memoryPool, std::shared_ptr veloxPool, ShuffleWriterType shuffleWriterType); @@ -161,6 +165,7 @@ class VeloxColumnarBatchDeserializerFactory : public DeserializerFactory { facebook::velox::common::CompressionKind veloxCompressionType_; facebook::velox::RowTypePtr rowType_; int32_t batchSize_; + int64_t bufferSize_; arrow::MemoryPool* memoryPool_; std::shared_ptr veloxPool_; diff --git a/cpp/velox/utils/tests/VeloxShuffleWriterTestBase.h b/cpp/velox/utils/tests/VeloxShuffleWriterTestBase.h index 4b4abfb525a5..d2995e251c68 100644 --- a/cpp/velox/utils/tests/VeloxShuffleWriterTestBase.h +++ b/cpp/velox/utils/tests/VeloxShuffleWriterTestBase.h @@ -366,6 +366,7 @@ class VeloxShuffleWriterTest : public ::testing::TestWithParam::max(), + kDefaultReadBufferSize, defaultArrowMemoryPool().get(), pool_, GetParam().shuffleWriterType); diff --git a/docs/Configuration.md b/docs/Configuration.md index 76549dd4fe7c..d4ca9c10fb76 100644 --- a/docs/Configuration.md +++ b/docs/Configuration.md @@ -51,6 +51,7 @@ You can add these configurations into spark-defaults.conf to enable or disable t | spark.gluten.sql.columnar.shuffle.compression.threshold | If number of rows in a batch falls below this threshold, will copy all buffers into one buffer to compress. | 100 | | spark.gluten.sql.columnar.shuffle.realloc.threshold | Set the threshold to dynamically adjust the size of shuffle split buffers. The size of each split buffer is recalculated for each incoming batch of data. If the new size deviates from the current partition buffer size by a factor outside the range of [1 - threshold, 1 + threshold], the split buffer will be re-allocated using the newly calculated size | 0.25 | | spark.gluten.sql.columnar.shuffle.merge.threshold | Set the threshold control the minimum merged size. When a partition buffer is full, and the number of rows is below (`threshold * spark.gluten.sql.columnar.maxBatchSize`), it will be saved for merging. | 0.25 | +| spark.gluten.sql.columnar.shuffle.readerBufferSize | Buffer size in bytes for shuffle reader reading input stream from local or remote. | 1MB | | spark.gluten.sql.columnar.numaBinding | Set up NUMABinding, default is false | true | | spark.gluten.sql.columnar.coreRange | Set up the core range for NUMABinding, only works when numaBinding set to true.
The setting is based on the number of cores in your system. Use 72 cores as an example. | 0-17,36-53 |18-35,54-71 | | spark.gluten.sql.columnar.wholeStage.fallback.threshold | Configure the threshold for whether whole stage will fall back in AQE supported case by counting the number of ColumnarToRow & vanilla leaf node | \>= 1 | diff --git a/gluten-arrow/src/main/java/org/apache/gluten/vectorized/ShuffleReaderJniWrapper.java b/gluten-arrow/src/main/java/org/apache/gluten/vectorized/ShuffleReaderJniWrapper.java index 09f97cbd06b9..8d031cf0792c 100644 --- a/gluten-arrow/src/main/java/org/apache/gluten/vectorized/ShuffleReaderJniWrapper.java +++ b/gluten-arrow/src/main/java/org/apache/gluten/vectorized/ShuffleReaderJniWrapper.java @@ -40,6 +40,7 @@ public native long make( String compressionType, String compressionCodecBackend, int batchSize, + long bufferSize, String shuffleWriterType); public native long readStream(long shuffleReaderHandle, JniByteInputStream jniIn); diff --git a/gluten-celeborn/velox/src/main/scala/org/apache/spark/shuffle/VeloxCelebornColumnarBatchSerializer.scala b/gluten-celeborn/velox/src/main/scala/org/apache/spark/shuffle/VeloxCelebornColumnarBatchSerializer.scala index a1fb190e4593..d5f20c8dea00 100644 --- a/gluten-celeborn/velox/src/main/scala/org/apache/spark/shuffle/VeloxCelebornColumnarBatchSerializer.scala +++ b/gluten-celeborn/velox/src/main/scala/org/apache/spark/shuffle/VeloxCelebornColumnarBatchSerializer.scala @@ -88,12 +88,14 @@ private class CelebornColumnarBatchSerializerInstance( .replace(GLUTEN_SORT_SHUFFLE_WRITER, GLUTEN_RSS_SORT_SHUFFLE_WRITER) val jniWrapper = ShuffleReaderJniWrapper.create(runtime) val batchSize = GlutenConfig.getConf.maxBatchSize + val bufferSize = GlutenConfig.getConf.columnarShuffleReaderBufferSize val handle = jniWrapper .make( cSchema.memoryAddress(), compressionCodec, compressionCodecBackend, batchSize, + bufferSize, shuffleWriterType ) // Close shuffle reader instance as lately as the end of task processing, diff --git a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala index 0e41fc65376c..f049e045c7b0 100644 --- a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala +++ b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala @@ -193,6 +193,9 @@ class GlutenConfig(conf: SQLConf) extends Logging { def columnarShuffleCompressionThreshold: Int = conf.getConf(COLUMNAR_SHUFFLE_COMPRESSION_THRESHOLD) + def columnarShuffleReaderBufferSize: Long = + conf.getConf(COLUMNAR_SHUFFLE_READER_BUFFER_SIZE) + def maxBatchSize: Int = conf.getConf(COLUMNAR_MAX_BATCH_SIZE) def columnarToRowMemThreshold: Long = @@ -607,6 +610,9 @@ object GlutenConfig { val GLUTEN_SHUFFLE_WRITER_MERGE_THRESHOLD = "spark.gluten.sql.columnar.shuffle.merge.threshold" val GLUTEN_SHUFFLE_DEFUALT_COMPRESSION_BUFFER_SIZE = 32 * 1024 + // Shuffle reader buffer size. + val GLUTEN_SHUFFLE_READER_BUFFER_SIZE = "spark.gluten.sql.columnar.shuffle.readerBufferSize" + // Controls whether to load DLL from jars. User can get dependent native libs packed into a jar // by executing dev/package.sh. Then, with that jar configured, Gluten can load the native libs // at runtime. This config is just for velox backend. And it is NOT applicable to the situation @@ -1155,6 +1161,13 @@ object GlutenConfig { .checkValue(v => v >= 0 && v <= 1, "Shuffle writer merge threshold must between [0, 1]") .createWithDefault(0.25) + val COLUMNAR_SHUFFLE_READER_BUFFER_SIZE = + buildConf(GLUTEN_SHUFFLE_READER_BUFFER_SIZE) + .internal() + .doc("Buffer size in bytes for shuffle reader reading input stream from local or remote.") + .bytesConf(ByteUnit.BYTE) + .createWithDefaultString("1MB") + val COLUMNAR_MAX_BATCH_SIZE = buildConf(GLUTEN_MAX_BATCH_SIZE_KEY) .internal() From fd716bf1420566608f093e8354bcf577a2511c99 Mon Sep 17 00:00:00 2001 From: Xuedong Luan Date: Fri, 15 Nov 2024 13:01:12 +0800 Subject: [PATCH 26/35] [GLUTEN-7837][VL] Spark driver should not initialize cache if not in local mode Closes #7837 --- .../gluten/backendsapi/velox/VeloxListenerApi.scala | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxListenerApi.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxListenerApi.scala index 850509db3e91..e5c9afc096f4 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxListenerApi.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxListenerApi.scala @@ -89,7 +89,7 @@ class VeloxListenerApi extends ListenerApi with Logging { SparkDirectoryUtil.init(conf) UDFResolver.resolveUdfConf(conf, isDriver = true) - initialize(conf) + initialize(conf, isDriver = true) UdfJniWrapper.registerFunctionSignatures() } @@ -116,12 +116,12 @@ class VeloxListenerApi extends ListenerApi with Logging { SparkDirectoryUtil.init(conf) UDFResolver.resolveUdfConf(conf, isDriver = false) - initialize(conf) + initialize(conf, isDriver = false) } override def onExecutorShutdown(): Unit = shutdown() - private def initialize(conf: SparkConf): Unit = { + private def initialize(conf: SparkConf, isDriver: Boolean): Unit = { // Force batch type initializations. VeloxBatch.getClass ArrowJavaBatch.getClass @@ -157,7 +157,11 @@ class VeloxListenerApi extends ListenerApi with Logging { } // Initial native backend with configurations. - val parsed = GlutenConfigUtil.parseConfig(conf.getAll.toMap) + var parsed = GlutenConfigUtil.parseConfig(conf.getAll.toMap) + // Workaround for https://github.com/apache/incubator-gluten/issues/7837 + if (isDriver && !inLocalMode(conf)) { + parsed += (GlutenConfig.COLUMNAR_VELOX_CACHE_ENABLED.key -> "false") + } NativeBackendInitializer.initializeBackend(parsed) // Inject backend-specific implementations to override spark classes. From 596858adde56caa4167e73ffdf53dedf8700874d Mon Sep 17 00:00:00 2001 From: kevinyhzou <37431499+KevinyhZou@users.noreply.github.com> Date: Fri, 15 Nov 2024 14:19:18 +0800 Subject: [PATCH 27/35] support nested column pruning (#7268) Co-authored-by: zouyunhe <811-zouyunhe@users.noreply.git.sysop.bigo.sg> --- .../backendsapi/clickhouse/CHBackend.scala | 4 + .../hive/GlutenClickHouseHiveTableSuite.scala | 53 ++++ .../backendsapi/BackendSettingsApi.scala | 2 + .../heuristic/OffloadSingleNode.scala | 8 +- .../hive/HiveTableScanExecTransformer.scala | 11 +- .../HiveTableScanNestedColumnPruning.scala | 252 ++++++++++++++++++ .../org/apache/gluten/GlutenConfig.scala | 10 + .../execution/AbstractHiveTableScanExec.scala | 7 +- .../execution/AbstractHiveTableScanExec.scala | 7 +- .../execution/AbstractHiveTableScanExec.scala | 7 +- .../execution/AbstractHiveTableScanExec.scala | 7 +- 11 files changed, 360 insertions(+), 8 deletions(-) create mode 100644 gluten-substrait/src/main/scala/org/apache/spark/sql/hive/HiveTableScanNestedColumnPruning.scala diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala index 54ab38569bb8..6e73ff6b29d1 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala @@ -396,4 +396,8 @@ object CHBackendSettings extends BackendSettingsApi with Logging { } override def supportWindowGroupLimitExec(rankLikeFunction: Expression): Boolean = true + + override def supportHiveTableScanNestedColumnPruning: Boolean = + GlutenConfig.getConf.enableColumnarHiveTableScanNestedColumnPruning + } diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseHiveTableSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseHiveTableSuite.scala index ff2d13996dc6..7e31e73040d4 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseHiveTableSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseHiveTableSuite.scala @@ -1499,4 +1499,57 @@ class GlutenClickHouseHiveTableSuite spark.sql("drop table if exists aj") } + test("test hive table scan nested column pruning") { + val json_table_name = "test_tbl_7267_json" + val pq_table_name = "test_tbl_7267_pq" + val create_table_sql = + s""" + | create table if not exists %s( + | id bigint, + | d1 STRUCT>>, + | d2 STRUCT>>, + | day string, + | hour string + | ) partitioned by(day, hour) + |""".stripMargin + val create_table_json = create_table_sql.format(json_table_name) + + s""" + | ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe' + | STORED AS INPUTFORMAT 'org.apache.hadoop.mapred.TextInputFormat' + | OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat' + |""".stripMargin + val create_table_pq = create_table_sql.format(pq_table_name) + " Stored as PARQUET" + val insert_sql = + """ + | insert into %s values(1, + | named_struct('c', 'c123', 'd', array(named_struct('x', 'x123', 'y', 'y123'))), + | named_struct('c', 'c124', 'd', map('m124', named_struct('x', 'x124', 'y', 'y124'))), + | '2024-09-26', '12' + | ) + |""".stripMargin + val select_sql = + "select id, d1.c, d1.d[0].x, d2.d['m124'].y from %s where day = '2024-09-26' and hour = '12'" + val table_names = Array.apply(json_table_name, pq_table_name) + val create_table_sqls = Array.apply(create_table_json, create_table_pq) + for (i <- table_names.indices) { + val table_name = table_names(i) + val create_table = create_table_sqls(i) + spark.sql(create_table) + spark.sql(insert_sql.format(table_name)) + withSQLConf(("spark.sql.hive.convertMetastoreParquet" -> "false")) { + compareResultsAgainstVanillaSpark( + select_sql.format(table_name), + compareResult = true, + df => { + val scan = collect(df.queryExecution.executedPlan) { + case l: HiveTableScanExecTransformer => l + } + assert(scan.size == 1) + } + ) + } + spark.sql("drop table if exists %s".format(table_name)) + } + } + } diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/BackendSettingsApi.scala b/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/BackendSettingsApi.scala index 177d19c0c709..700571fd2815 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/BackendSettingsApi.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/BackendSettingsApi.scala @@ -131,4 +131,6 @@ trait BackendSettingsApi { def supportColumnarArrowUdf(): Boolean = false def needPreComputeRangeFrameBoundary(): Boolean = false + + def supportHiveTableScanNestedColumnPruning(): Boolean = false } diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/OffloadSingleNode.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/OffloadSingleNode.scala index a8c200e9be44..bae98bec2ec6 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/OffloadSingleNode.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/OffloadSingleNode.scala @@ -37,7 +37,7 @@ import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ShuffleEx import org.apache.spark.sql.execution.joins._ import org.apache.spark.sql.execution.python.{ArrowEvalPythonExec, BatchEvalPythonExec} import org.apache.spark.sql.execution.window.{WindowExec, WindowGroupLimitExecShim} -import org.apache.spark.sql.hive.HiveTableScanExecTransformer +import org.apache.spark.sql.hive.{HiveTableScanExecTransformer, HiveTableScanNestedColumnPruning} /** * Converts a vanilla Spark plan node into Gluten plan node. Gluten plan is supposed to be executed @@ -226,7 +226,11 @@ object OffloadOthers { case plan: ProjectExec => val columnarChild = plan.child logDebug(s"Columnar Processing for ${plan.getClass} is currently supported.") - ProjectExecTransformer(plan.projectList, columnarChild) + if (HiveTableScanNestedColumnPruning.supportNestedColumnPruning(plan)) { + HiveTableScanNestedColumnPruning.apply(plan) + } else { + ProjectExecTransformer(plan.projectList, columnarChild) + } case plan: HashAggregateExec => logDebug(s"Columnar Processing for ${plan.getClass} is currently supported.") HashAggregateExecBaseTransformer.from(plan) diff --git a/gluten-substrait/src/main/scala/org/apache/spark/sql/hive/HiveTableScanExecTransformer.scala b/gluten-substrait/src/main/scala/org/apache/spark/sql/hive/HiveTableScanExecTransformer.scala index 85432350d4a2..f701c76b1813 100644 --- a/gluten-substrait/src/main/scala/org/apache/spark/sql/hive/HiveTableScanExecTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/spark/sql/hive/HiveTableScanExecTransformer.scala @@ -45,7 +45,8 @@ import java.net.URI case class HiveTableScanExecTransformer( requestedAttributes: Seq[Attribute], relation: HiveTableRelation, - partitionPruningPred: Seq[Expression])(@transient session: SparkSession) + partitionPruningPred: Seq[Expression], + prunedOutput: Seq[Attribute] = Seq.empty[Attribute])(@transient session: SparkSession) extends AbstractHiveTableScanExec(requestedAttributes, relation, partitionPruningPred)(session) with BasicScanExecTransformer { @@ -63,7 +64,13 @@ case class HiveTableScanExecTransformer( override def getMetadataColumns(): Seq[AttributeReference] = Seq.empty - override def outputAttributes(): Seq[Attribute] = output + override def outputAttributes(): Seq[Attribute] = { + if (prunedOutput.nonEmpty) { + prunedOutput + } else { + output + } + } override def getPartitions: Seq[InputPartition] = partitions diff --git a/gluten-substrait/src/main/scala/org/apache/spark/sql/hive/HiveTableScanNestedColumnPruning.scala b/gluten-substrait/src/main/scala/org/apache/spark/sql/hive/HiveTableScanNestedColumnPruning.scala new file mode 100644 index 000000000000..7a20e5c37da5 --- /dev/null +++ b/gluten-substrait/src/main/scala/org/apache/spark/sql/hive/HiveTableScanNestedColumnPruning.scala @@ -0,0 +1,252 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.hive + +import org.apache.gluten.backendsapi.BackendsApiManager +import org.apache.gluten.execution.ProjectExecTransformer + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.catalyst.catalog.HiveTableRelation +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.execution.{FilterExec, LeafExecNode, ProjectExec, SparkPlan} +import org.apache.spark.sql.hive.HiveTableScanExecTransformer.{ORC_INPUT_FORMAT_CLASS, PARQUET_INPUT_FORMAT_CLASS, TEXT_INPUT_FORMAT_CLASS} +import org.apache.spark.sql.types.{ArrayType, DataType, MapType, StructType} +import org.apache.spark.sql.util.SchemaUtils._ +import org.apache.spark.util.Utils + +object HiveTableScanNestedColumnPruning extends Logging { + import org.apache.spark.sql.catalyst.expressions.SchemaPruning._ + + def supportNestedColumnPruning(projectExec: ProjectExec): Boolean = { + if (BackendsApiManager.getSettings.supportHiveTableScanNestedColumnPruning()) { + projectExec.child match { + case HiveTableScanExecTransformer(_, relation, _, _) => + relation.tableMeta.storage.inputFormat match { + case Some(inputFormat) + if TEXT_INPUT_FORMAT_CLASS.isAssignableFrom(Utils.classForName(inputFormat)) => + relation.tableMeta.storage.serde match { + case Some("org.openx.data.jsonserde.JsonSerDe") | Some( + "org.apache.hive.hcatalog.data.JsonSerDe") => + return true + case _ => + } + case Some(inputFormat) + if ORC_INPUT_FORMAT_CLASS.isAssignableFrom(Utils.classForName(inputFormat)) => + return true + case Some(inputFormat) + if PARQUET_INPUT_FORMAT_CLASS.isAssignableFrom(Utils.classForName(inputFormat)) => + return true + case _ => + } + case _ => + } + } + false + } + + def apply(plan: SparkPlan): SparkPlan = { + plan match { + case ProjectExec(projectList, child) => + child match { + case h: HiveTableScanExecTransformer => + val newPlan = prunePhysicalColumns( + h.relation, + projectList, + Seq.empty[Expression], + (prunedDataSchema, prunedMetadataSchema) => { + buildNewHiveTableScan(h, prunedDataSchema, prunedMetadataSchema) + }, + (schema, requestFields) => { + h.pruneSchema(schema, requestFields) + } + ) + if (newPlan.nonEmpty) { + return newPlan.get + } else { + return ProjectExecTransformer(projectList, child) + } + case _ => + return ProjectExecTransformer(projectList, child) + } + case _ => + } + plan + } + + private def prunePhysicalColumns( + relation: HiveTableRelation, + projects: Seq[NamedExpression], + filters: Seq[Expression], + leafNodeBuilder: (StructType, StructType) => LeafExecNode, + pruneSchemaFunc: (StructType, Seq[SchemaPruning.RootField]) => StructType) + : Option[SparkPlan] = { + val (normalizedProjects, normalizedFilters) = + normalizeAttributeRefNames(relation.output, projects, filters) + val requestedRootFields = identifyRootFields(normalizedProjects, normalizedFilters) + // If requestedRootFields includes a nested field, continue. Otherwise, + // return op + if (requestedRootFields.exists { root: RootField => !root.derivedFromAtt }) { + val prunedDataSchema = pruneSchemaFunc(relation.tableMeta.dataSchema, requestedRootFields) + val metaFieldNames = relation.tableMeta.schema.fieldNames + val metadataSchema = relation.output.collect { + case attr: AttributeReference if metaFieldNames.contains(attr.name) => attr + }.toStructType + val prunedMetadataSchema = if (metadataSchema.nonEmpty) { + pruneSchemaFunc(metadataSchema, requestedRootFields) + } else { + metadataSchema + } + // If the data schema is different from the pruned data schema + // OR + // the metadata schema is different from the pruned metadata schema, continue. + // Otherwise, return None. + if ( + countLeaves(relation.tableMeta.dataSchema) > countLeaves(prunedDataSchema) || + countLeaves(metadataSchema) > countLeaves(prunedMetadataSchema) + ) { + val leafNode = leafNodeBuilder(prunedDataSchema, prunedMetadataSchema) + val projectionOverSchema = ProjectionOverSchema( + prunedDataSchema.merge(prunedMetadataSchema), + AttributeSet(relation.output)) + Some( + buildNewProjection( + projects, + normalizedProjects, + normalizedFilters, + leafNode, + projectionOverSchema)) + } else { + None + } + } else { + None + } + } + + /** + * Normalizes the names of the attribute references in the given projects and filters to reflect + * the names in the given logical relation. This makes it possible to compare attributes and + * fields by name. Returns a tuple with the normalized projects and filters, respectively. + */ + private def normalizeAttributeRefNames( + output: Seq[AttributeReference], + projects: Seq[NamedExpression], + filters: Seq[Expression]): (Seq[NamedExpression], Seq[Expression]) = { + val normalizedAttNameMap = output.map(att => (att.exprId, att.name)).toMap + val normalizedProjects = projects + .map(_.transform { + case att: AttributeReference if normalizedAttNameMap.contains(att.exprId) => + att.withName(normalizedAttNameMap(att.exprId)) + }) + .map { case expr: NamedExpression => expr } + val normalizedFilters = filters.map(_.transform { + case att: AttributeReference if normalizedAttNameMap.contains(att.exprId) => + att.withName(normalizedAttNameMap(att.exprId)) + }) + (normalizedProjects, normalizedFilters) + } + + /** Builds the new output [[Project]] Spark SQL operator that has the `leafNode`. */ + private def buildNewProjection( + projects: Seq[NamedExpression], + normalizedProjects: Seq[NamedExpression], + filters: Seq[Expression], + leafNode: LeafExecNode, + projectionOverSchema: ProjectionOverSchema): ProjectExecTransformer = { + // Construct a new target for our projection by rewriting and + // including the original filters where available + val projectionChild = + if (filters.nonEmpty) { + val projectedFilters = filters.map(_.transformDown { + case projectionOverSchema(expr) => expr + }) + val newFilterCondition = projectedFilters.reduce(And) + FilterExec(newFilterCondition, leafNode) + } else { + leafNode + } + + // Construct the new projections of our Project by + // rewriting the original projections + val newProjects = + normalizedProjects.map(_.transformDown { case projectionOverSchema(expr) => expr }).map { + case expr: NamedExpression => expr + } + + ProjectExecTransformer( + restoreOriginalOutputNames(newProjects, projects.map(_.name)), + projectionChild) + } + + private def buildNewHiveTableScan( + hiveTableScan: HiveTableScanExecTransformer, + prunedDataSchema: StructType, + prunedMetadataSchema: StructType): HiveTableScanExecTransformer = { + val relation = hiveTableScan.relation + val partitionSchema = relation.tableMeta.partitionSchema + val prunedBaseSchema = StructType( + prunedDataSchema.fields.filterNot( + f => partitionSchema.fieldNames.contains(f.name)) ++ partitionSchema.fields) + val finalSchema = prunedBaseSchema.merge(prunedMetadataSchema) + val prunedOutput = getPrunedOutput(relation.output, finalSchema) + var finalOutput = Seq.empty[Attribute] + for (p <- hiveTableScan.output) { + var flag = false + for (q <- prunedOutput if !flag) { + if (p.name.equals(q.name)) { + finalOutput :+= q + flag = true + } + } + } + HiveTableScanExecTransformer( + hiveTableScan.requestedAttributes, + relation, + hiveTableScan.partitionPruningPred, + finalOutput)(hiveTableScan.session) + } + + // Prune the given output to make it consistent with `requiredSchema`. + private def getPrunedOutput( + output: Seq[AttributeReference], + requiredSchema: StructType): Seq[Attribute] = { + // We need to update the data type of the output attributes to use the pruned ones. + // so that references to the original relation's output are not broken + val nameAttributeMap = output.map(att => (att.name, att)).toMap + val requiredAttributes = + requiredSchema.map(f => AttributeReference(f.name, f.dataType, f.nullable, f.metadata)()) + requiredAttributes.map { + case att if nameAttributeMap.contains(att.name) => + nameAttributeMap(att.name).withDataType(att.dataType) + case att => att + } + } + + /** + * Counts the "leaf" fields of the given dataType. Informally, this is the number of fields of + * non-complex data type in the tree representation of [[DataType]]. + */ + private def countLeaves(dataType: DataType): Int = { + dataType match { + case array: ArrayType => countLeaves(array.elementType) + case map: MapType => countLeaves(map.keyType) + countLeaves(map.valueType) + case struct: StructType => + struct.map(field => countLeaves(field.dataType)).sum + case _ => 1 + } + } +} diff --git a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala index f049e045c7b0..5b15faf6466a 100644 --- a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala +++ b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala @@ -50,6 +50,9 @@ class GlutenConfig(conf: SQLConf) extends Logging { def enableColumnarHiveTableScan: Boolean = conf.getConf(COLUMNAR_HIVETABLESCAN_ENABLED) + def enableColumnarHiveTableScanNestedColumnPruning: Boolean = + conf.getConf(COLUMNAR_HIVETABLESCAN_NESTED_COLUMN_PRUNING_ENABLED) + def enableVanillaVectorizedReaders: Boolean = conf.getConf(VANILLA_VECTORIZED_READERS_ENABLED) def enableColumnarHashAgg: Boolean = conf.getConf(COLUMNAR_HASHAGG_ENABLED) @@ -865,6 +868,13 @@ object GlutenConfig { .booleanConf .createWithDefault(true) + val COLUMNAR_HIVETABLESCAN_NESTED_COLUMN_PRUNING_ENABLED = + buildConf("spark.gluten.sql.columnar.enableNestedColumnPruningInHiveTableScan") + .internal() + .doc("Enable or disable nested column pruning in hivetablescan.") + .booleanConf + .createWithDefault(true) + val VANILLA_VECTORIZED_READERS_ENABLED = buildStaticConf("spark.gluten.sql.columnar.enableVanillaVectorizedReaders") .internal() diff --git a/shims/spark32/src/main/scala/org/apache/spark/sql/hive/execution/AbstractHiveTableScanExec.scala b/shims/spark32/src/main/scala/org/apache/spark/sql/hive/execution/AbstractHiveTableScanExec.scala index 46b59ac306c2..f38c85a49dde 100644 --- a/shims/spark32/src/main/scala/org/apache/spark/sql/hive/execution/AbstractHiveTableScanExec.scala +++ b/shims/spark32/src/main/scala/org/apache/spark/sql/hive/execution/AbstractHiveTableScanExec.scala @@ -22,12 +22,13 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.CastSupport import org.apache.spark.sql.catalyst.catalog.HiveTableRelation import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.SchemaPruning.RootField import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.hive._ import org.apache.spark.sql.hive.client.HiveClientImpl import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.{BooleanType, DataType} +import org.apache.spark.sql.types.{BooleanType, DataType, StructType} import org.apache.spark.util.Utils import org.apache.hadoop.conf.Configuration @@ -232,4 +233,8 @@ abstract private[hive] class AbstractHiveTableScanExec( } override def otherCopyArgs: Seq[AnyRef] = Seq(sparkSession) + + def pruneSchema(schema: StructType, requestedFields: Seq[RootField]): StructType = { + SchemaPruning.pruneDataSchema(schema, requestedFields) + } } diff --git a/shims/spark33/src/main/scala/org/apache/spark/sql/hive/execution/AbstractHiveTableScanExec.scala b/shims/spark33/src/main/scala/org/apache/spark/sql/hive/execution/AbstractHiveTableScanExec.scala index dd095f0ff247..d9b6bb936f67 100644 --- a/shims/spark33/src/main/scala/org/apache/spark/sql/hive/execution/AbstractHiveTableScanExec.scala +++ b/shims/spark33/src/main/scala/org/apache/spark/sql/hive/execution/AbstractHiveTableScanExec.scala @@ -22,12 +22,13 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.CastSupport import org.apache.spark.sql.catalyst.catalog.HiveTableRelation import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.SchemaPruning.RootField import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.hive._ import org.apache.spark.sql.hive.client.HiveClientImpl import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.{BooleanType, DataType} +import org.apache.spark.sql.types.{BooleanType, DataType, StructType} import org.apache.spark.util.Utils import org.apache.hadoop.conf.Configuration @@ -239,4 +240,8 @@ abstract private[hive] class AbstractHiveTableScanExec( } override def otherCopyArgs: Seq[AnyRef] = Seq(sparkSession) + + def pruneSchema(schema: StructType, requestedFields: Seq[RootField]): StructType = { + SchemaPruning.pruneSchema(schema, requestedFields) + } } diff --git a/shims/spark34/src/main/scala/org/apache/spark/sql/hive/execution/AbstractHiveTableScanExec.scala b/shims/spark34/src/main/scala/org/apache/spark/sql/hive/execution/AbstractHiveTableScanExec.scala index 87aba00b0f59..3521d496546d 100644 --- a/shims/spark34/src/main/scala/org/apache/spark/sql/hive/execution/AbstractHiveTableScanExec.scala +++ b/shims/spark34/src/main/scala/org/apache/spark/sql/hive/execution/AbstractHiveTableScanExec.scala @@ -22,12 +22,13 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.CastSupport import org.apache.spark.sql.catalyst.catalog.HiveTableRelation import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.SchemaPruning.RootField import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.hive._ import org.apache.spark.sql.hive.client.HiveClientImpl import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.{BooleanType, DataType} +import org.apache.spark.sql.types.{BooleanType, DataType, StructType} import org.apache.spark.util.Utils import org.apache.hadoop.conf.Configuration @@ -257,4 +258,8 @@ abstract private[hive] class AbstractHiveTableScanExec( } override def otherCopyArgs: Seq[AnyRef] = Seq(sparkSession) + + def pruneSchema(schema: StructType, requestedFields: Seq[RootField]): StructType = { + SchemaPruning.pruneSchema(schema, requestedFields) + } } diff --git a/shims/spark35/src/main/scala/org/apache/spark/sql/hive/execution/AbstractHiveTableScanExec.scala b/shims/spark35/src/main/scala/org/apache/spark/sql/hive/execution/AbstractHiveTableScanExec.scala index 87aba00b0f59..3521d496546d 100644 --- a/shims/spark35/src/main/scala/org/apache/spark/sql/hive/execution/AbstractHiveTableScanExec.scala +++ b/shims/spark35/src/main/scala/org/apache/spark/sql/hive/execution/AbstractHiveTableScanExec.scala @@ -22,12 +22,13 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.CastSupport import org.apache.spark.sql.catalyst.catalog.HiveTableRelation import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.SchemaPruning.RootField import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.hive._ import org.apache.spark.sql.hive.client.HiveClientImpl import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.{BooleanType, DataType} +import org.apache.spark.sql.types.{BooleanType, DataType, StructType} import org.apache.spark.util.Utils import org.apache.hadoop.conf.Configuration @@ -257,4 +258,8 @@ abstract private[hive] class AbstractHiveTableScanExec( } override def otherCopyArgs: Seq[AnyRef] = Seq(sparkSession) + + def pruneSchema(schema: StructType, requestedFields: Seq[RootField]): StructType = { + SchemaPruning.pruneSchema(schema, requestedFields) + } } From 7e4a13abd956e243ad47d7f44ef66754b1c63a11 Mon Sep 17 00:00:00 2001 From: PHILO-HE Date: Fri, 15 Nov 2024 14:58:44 +0800 Subject: [PATCH 28/35] [GLUTEN-7499][VL][CI] Print ccache statistics for tracking its efficacy (#7957) Just a minor change in case of ccache loses its efficacy for some reason. P.S. For the current pr, we got the below statistics printed. cache directory /__w/incubator-gluten/incubator-gluten/.ccache primary config /__w/incubator-gluten/incubator-gluten/.ccache/ccache.conf secondary config (readonly) /etc/ccache.conf stats updated Fri Nov 15 03:03:00 2024 cache hit (direct) 112337 cache hit (preprocessed) 899 cache miss 10042 cache hit rate 91.85 % called for link 368 cleanups performed 59 files in cache 11473 cache size 4.3 GB max cache size 5.0 GB --- .github/workflows/velox_backend.yml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.github/workflows/velox_backend.yml b/.github/workflows/velox_backend.yml index fafceb1aa06c..42e1f2527816 100644 --- a/.github/workflows/velox_backend.yml +++ b/.github/workflows/velox_backend.yml @@ -71,6 +71,7 @@ jobs: df -a cd $GITHUB_WORKSPACE/ bash dev/ci-velox-buildstatic-centos-7.sh + ccache -s - name: "Save ccache" uses: actions/cache/save@v3 id: ccache @@ -954,6 +955,7 @@ jobs: run: | df -a bash dev/ci-velox-buildshared-centos-8.sh + ccache -s - name: "Save ccache" uses: actions/cache/save@v3 id: ccache From c7548aa005a0350b30fa987b36b9d7a5b25747f7 Mon Sep 17 00:00:00 2001 From: "shuai.xu" Date: Fri, 15 Nov 2024 15:07:27 +0800 Subject: [PATCH 29/35] [GLUTEN-7594] [CH] support cast const map to string (#7599) * [GLUTEN-7594] [CH] support cast const map to string * fix test * fix test --- .../GlutenClickhouseFunctionSuite.scala | 18 +++++++++++++++++- .../Functions/SparkFunctionMapToString.h | 8 ++++++-- 2 files changed, 23 insertions(+), 3 deletions(-) diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/compatibility/GlutenClickhouseFunctionSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/compatibility/GlutenClickhouseFunctionSuite.scala index 2cfe935ef7e0..02af10657c73 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/compatibility/GlutenClickhouseFunctionSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/compatibility/GlutenClickhouseFunctionSuite.scala @@ -17,7 +17,7 @@ package org.apache.gluten.execution.compatibility import org.apache.gluten.GlutenConfig -import org.apache.gluten.execution.GlutenClickHouseTPCHAbstractSuite +import org.apache.gluten.execution.{GlutenClickHouseTPCHAbstractSuite, ProjectExecTransformer} import org.apache.gluten.utils.UTSystemParameters import org.apache.spark.SparkConf @@ -269,6 +269,22 @@ class GlutenClickhouseFunctionSuite extends GlutenClickHouseTPCHAbstractSuite { } } + test("GLUTEN-7594: cast const map to string") { + withSQLConf( + ( + "spark.sql.optimizer.excludedRules", + "org.apache.spark.sql.catalyst.optimizer.ConstantFolding," + + "org.apache.spark.sql.catalyst.optimizer.NullPropagation")) { + runQueryAndCompare( + """ + |select cast(map(1,'2') as string) + |""".stripMargin, + true, + false + )(checkGlutenOperatorMatch[ProjectExecTransformer]) + } + } + test("GLUTEN-7550 get_json_object in IN") { withTable("test_7550") { sql("create table test_7550(a string) using parquet") diff --git a/cpp-ch/local-engine/Functions/SparkFunctionMapToString.h b/cpp-ch/local-engine/Functions/SparkFunctionMapToString.h index 444e4542c807..3f8a0c97dc07 100644 --- a/cpp-ch/local-engine/Functions/SparkFunctionMapToString.h +++ b/cpp-ch/local-engine/Functions/SparkFunctionMapToString.h @@ -76,7 +76,10 @@ class SparkFunctionMapToString : public DB::IFunction arguments[1].type->getName(), arguments[2].type->getName()); } - return makeNullable(std::make_shared()); + if (arguments[0].type->isNullable()) + return makeNullable(std::make_shared()); + else + return std::make_shared(); } ColumnPtr executeImpl(const DB::ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t /*input_rows*/) const override @@ -89,7 +92,8 @@ class SparkFunctionMapToString : public DB::IFunction } const auto & col_with_type_and_name = columnGetNested(arguments[0]); - const IColumn & col_from = *col_with_type_and_name.column; + const IColumn & column = *col_with_type_and_name.column; + const IColumn & col_from = column.isConst() ? reinterpret_cast(column).getDataColumn() : column; size_t size = col_from.size(); auto col_to = removeNullable(result_type)->createColumn(); From 802dfa690dc0eff18cfc6fe0f66e36e968f11436 Mon Sep 17 00:00:00 2001 From: Mingliang Zhu Date: Fri, 15 Nov 2024 17:19:45 +0800 Subject: [PATCH 30/35] [CORE] Add buildSide info for BroadcastNestedLoopJoinExecTransformer simpleStringWithNodeId (#7948) --- .../execution/BroadcastNestedLoopJoinExecTransformer.scala | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/BroadcastNestedLoopJoinExecTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/BroadcastNestedLoopJoinExecTransformer.scala index ae407b3b3efa..1e0c3d5a7e70 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/BroadcastNestedLoopJoinExecTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/BroadcastNestedLoopJoinExecTransformer.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide} import org.apache.spark.sql.catalyst.plans.{FullOuter, InnerLike, JoinType, LeftExistence, LeftOuter, RightOuter} import org.apache.spark.sql.catalyst.plans.physical.Partitioning -import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.{ExplainUtils, SparkPlan} import org.apache.spark.sql.execution.joins.BaseJoinExec import org.apache.spark.sql.execution.metric.SQLMetric @@ -45,6 +45,11 @@ abstract class BroadcastNestedLoopJoinExecTransformer( def joinBuildSide: BuildSide = buildSide + override def simpleStringWithNodeId(): String = { + val opId = ExplainUtils.getOpId(this) + s"$nodeName $joinType $joinBuildSide ($opId)".trim + } + override def leftKeys: Seq[Expression] = Nil override def rightKeys: Seq[Expression] = Nil From 19e1caea853570e180ae16ccea5a4178bcfe1f8f Mon Sep 17 00:00:00 2001 From: Gluten Performance Bot <137994563+GlutenPerfBot@users.noreply.github.com> Date: Fri, 15 Nov 2024 19:29:20 +0800 Subject: [PATCH 31/35] [GLUTEN-6887][VL] Daily Update Velox Version (2024_11_15) (#7954) Upstream Velox's New Commits: 9ebe36609 by zuyu, Fix OutputBuffer comments regarding bufferedBytes_ (11423) 935d30ee1 by zhli1142015, Combine low selectivity vectors generated by the hash join filter (10987) 5ac572b47 by rui-mo, Fix Spark expression fuzzer test on deny_precision_loss functions (11526) fe87f0b75 by hengjiang.ly, Add a faster string-search method with simd instructions than std::find (10858) 5b274e3cf by NEUpanning, Allow partial date parsing when simple datetime formatter is used (11386) c069192e9 by Jimmy Lu, Buckify breeze (11512) f37dc004e by Xiaoxuan Meng, Allow single thread execution mode for local partition to support spark union for gluten (11531) 837480242 by Christian Zentgraf, Add Clang Linux build to CI pipeline (10767) 427ac31e6 by Yenda Li, Add support for TTL in directories (11498) --- cpp/velox/jni/JniFileSystem.cc | 4 ++-- ep/build-velox/src/get_velox.sh | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/cpp/velox/jni/JniFileSystem.cc b/cpp/velox/jni/JniFileSystem.cc index 21cdf2b23a02..fa849664e3ae 100644 --- a/cpp/velox/jni/JniFileSystem.cc +++ b/cpp/velox/jni/JniFileSystem.cc @@ -241,7 +241,7 @@ class FileSystemWrapper : public facebook::velox::filesystems::FileSystem { return fs_->list(rewrite(path)); } - void mkdir(std::string_view path) override { + void mkdir(std::string_view path, const facebook::velox::filesystems::DirectoryOptions& options = {}) override { fs_->mkdir(rewrite(path)); } @@ -349,7 +349,7 @@ class JniFileSystem : public facebook::velox::filesystems::FileSystem { return out; } - void mkdir(std::string_view path) override { + void mkdir(std::string_view path, const facebook::velox::filesystems::DirectoryOptions& options = {}) override { JNIEnv* env = nullptr; attachCurrentThreadAsDaemonOrThrow(vm, &env); env->CallVoidMethod(obj_, jniFileSystemMkdir, createJString(env, path)); diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh index 06a7215d827a..9e0f3c0160ff 100755 --- a/ep/build-velox/src/get_velox.sh +++ b/ep/build-velox/src/get_velox.sh @@ -17,7 +17,7 @@ set -exu VELOX_REPO=https://github.com/oap-project/velox.git -VELOX_BRANCH=2024_11_14 +VELOX_BRANCH=2024_11_15 VELOX_HOME="" OS=`uname -s` From a7f43625cbe613cab8f70d9c99399d515552a37f Mon Sep 17 00:00:00 2001 From: Gluten Performance Bot <137994563+GlutenPerfBot@users.noreply.github.com> Date: Sat, 16 Nov 2024 09:46:21 +0800 Subject: [PATCH 32/35] [GLUTEN-6887][VL] Daily Update Velox Version (2024_11_16) (#7958) Upstream Velox's New Commits: b40cec3f3 by Joe Giardino, - Fix FlatVector copy memory issue (11483) 035750010 by yingsu00, Fix PlanNodeStats overflow (11532) 8aeb51cf4 by Jimmy Lu, fix: SSE only build (11542) ab02900db by Orri Erling, feat: NVRTC work with Wave headers (11500) 213f0a053 by Yenda Li, Add comparison support for RowView (11499) cce529e08 by Wei He, Fix Window operator with NaN as the frame bound (11293) ebd597a02 by Jialiang Tan, Change memory-reclaim-max-wait-time to max-memory-arbitration-time (11405) 88e3e6118 by Jacob Wujciak-Jens, build(ci): Add conventional commit title check (11520) 9494cbca8 by Chengcheng Jin, Clean up prefix sort function defaultPrefixSortConfig (11533) --- ep/build-velox/src/get_velox.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh index 9e0f3c0160ff..02656d45e3cb 100755 --- a/ep/build-velox/src/get_velox.sh +++ b/ep/build-velox/src/get_velox.sh @@ -17,7 +17,7 @@ set -exu VELOX_REPO=https://github.com/oap-project/velox.git -VELOX_BRANCH=2024_11_15 +VELOX_BRANCH=2024_11_16 VELOX_HOME="" OS=`uname -s` From 3941a2d1d02864c6d3992c2c6e2b0563912cb058 Mon Sep 17 00:00:00 2001 From: Gluten Performance Bot <137994563+GlutenPerfBot@users.noreply.github.com> Date: Mon, 18 Nov 2024 07:25:05 +0800 Subject: [PATCH 33/35] [GLUTEN-6887][VL] Daily Update Velox Version (2024_11_17) (#7961) Upstream Velox's New Commits: d1ce7195b by duanmeng, Init filesystem after registered in query replyaer (11559) 90bbdaae6 by Yenda Li, fix(replace_first): Fix up registration of replace_first (11556) 00e814941 by Pramod, Add function serializeSingleColumn to PrestoVectorSerde (10657) 3265e791c by Jialiang Tan, refactor(arbitrator): Simplify arbitration participant lock (11525) 1029d8b35 by Heidi Han, Add support for array join on json (11446) 0a422f738 by Jimmy Lu, fix: Remove same page hack for simdStrstr (11553) 31ae379d4 by David Reveman, Breeze fixes (11537) c21020c5a by yingsu00, fix: ExchangeBenchmark "Named vector serde 'Presto' is not registered" (11548) --- ep/build-velox/src/get_velox.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh index 02656d45e3cb..0a1f168db7b5 100755 --- a/ep/build-velox/src/get_velox.sh +++ b/ep/build-velox/src/get_velox.sh @@ -17,7 +17,7 @@ set -exu VELOX_REPO=https://github.com/oap-project/velox.git -VELOX_BRANCH=2024_11_16 +VELOX_BRANCH=2024_11_17 VELOX_HOME="" OS=`uname -s` From c144443443580baf0f97dbfb721d66bc9cb21faa Mon Sep 17 00:00:00 2001 From: JiaKe Date: Mon, 18 Nov 2024 09:01:19 +0800 Subject: [PATCH 34/35] Add config to support viewfs in Gluten. (#7892) --- .../clickhouse/CHIteratorApi.scala | 4 +- ...tenClickHouseMergeTreeWriteOnS3Suite.scala | 2 +- .../GlutenClickHouseMergeTreeWriteSuite.scala | 6 +-- .../backendsapi/velox/VeloxIteratorApi.scala | 20 +++------- .../execution/IcebergScanTransformer.scala | 5 +-- .../gluten/execution/VeloxIcebergSuite.scala | 6 +-- .../gluten/substrait/rel/LocalFilesNode.java | 5 +++ .../gluten/backendsapi/IteratorApi.scala | 4 +- .../execution/BasicScanExecTransformer.scala | 12 ++---- .../execution/WholeStageTransformer.scala | 37 +++++++++++++++---- .../org/apache/gluten/GlutenConfig.scala | 9 +++++ 11 files changed, 62 insertions(+), 48 deletions(-) diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHIteratorApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHIteratorApi.scala index dd5a736e7571..ff268b95d8de 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHIteratorApi.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHIteratorApi.scala @@ -43,7 +43,6 @@ import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.types._ import org.apache.spark.sql.utils.SparkInputMetricsUtil.InputMetricsWrapper import org.apache.spark.sql.vectorized.ColumnarBatch -import org.apache.spark.util.SerializableConfiguration import java.lang.{Long => JLong} import java.net.URI @@ -133,8 +132,7 @@ class CHIteratorApi extends IteratorApi with Logging with LogLevelUtil { partitionSchema: StructType, fileFormat: ReadFileFormat, metadataColumnNames: Seq[String], - properties: Map[String, String], - serializableHadoopConf: SerializableConfiguration): SplitInfo = { + properties: Map[String, String]): SplitInfo = { partition match { case p: GlutenMergeTreePartition => ExtensionTableBuilder diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteOnS3Suite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteOnS3Suite.scala index 571dc4ba9258..c0f509c68cda 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteOnS3Suite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteOnS3Suite.scala @@ -771,7 +771,7 @@ class GlutenClickHouseMergeTreeWriteOnS3Suite case scanExec: BasicScanExecTransformer => scanExec } assertResult(1)(plans.size) - assertResult(1)(plans.head.getSplitInfos(null).size) + assertResult(1)(plans.head.getSplitInfos.size) } } } diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteSuite.scala index 85c8c2d92a52..72adee309dd7 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteSuite.scala @@ -1807,7 +1807,7 @@ class GlutenClickHouseMergeTreeWriteSuite case scanExec: BasicScanExecTransformer => scanExec } assertResult(1)(plans.size) - assertResult(conf._2)(plans.head.getSplitInfos(null).size) + assertResult(conf._2)(plans.head.getSplitInfos.size) } } }) @@ -1831,7 +1831,7 @@ class GlutenClickHouseMergeTreeWriteSuite case scanExec: BasicScanExecTransformer => scanExec } assertResult(1)(plans.size) - assertResult(1)(plans.head.getSplitInfos(null).size) + assertResult(1)(plans.head.getSplitInfos.size) } } } @@ -1939,7 +1939,7 @@ class GlutenClickHouseMergeTreeWriteSuite case f: BasicScanExecTransformer => f } assertResult(2)(scanExec.size) - assertResult(conf._2)(scanExec(1).getSplitInfos(null).size) + assertResult(conf._2)(scanExec(1).getSplitInfos.size) } } }) diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxIteratorApi.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxIteratorApi.scala index 320d1f366c23..061daaac0fad 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxIteratorApi.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxIteratorApi.scala @@ -39,9 +39,7 @@ import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.types._ import org.apache.spark.sql.utils.SparkInputMetricsUtil.InputMetricsWrapper import org.apache.spark.sql.vectorized.ColumnarBatch -import org.apache.spark.util.{ExecutorManager, SerializableConfiguration, SparkDirectoryUtil} - -import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.spark.util.{ExecutorManager, SparkDirectoryUtil} import java.lang.{Long => JLong} import java.nio.charset.StandardCharsets @@ -57,8 +55,7 @@ class VeloxIteratorApi extends IteratorApi with Logging { partitionSchema: StructType, fileFormat: ReadFileFormat, metadataColumnNames: Seq[String], - properties: Map[String, String], - serializableHadoopConf: SerializableConfiguration): SplitInfo = { + properties: Map[String, String]): SplitInfo = { partition match { case f: FilePartition => val ( @@ -69,7 +66,7 @@ class VeloxIteratorApi extends IteratorApi with Logging { modificationTimes, partitionColumns, metadataColumns) = - constructSplitInfo(partitionSchema, f.files, metadataColumnNames, serializableHadoopConf) + constructSplitInfo(partitionSchema, f.files, metadataColumnNames) val preferredLocations = SoftAffinity.getFilePartitionLocations(f) LocalFilesBuilder.makeLocalFiles( @@ -112,8 +109,7 @@ class VeloxIteratorApi extends IteratorApi with Logging { private def constructSplitInfo( schema: StructType, files: Array[PartitionedFile], - metadataColumnNames: Seq[String], - serializableHadoopConf: SerializableConfiguration) = { + metadataColumnNames: Seq[String]) = { val paths = new JArrayList[String]() val starts = new JArrayList[JLong] val lengths = new JArrayList[JLong]() @@ -125,15 +121,9 @@ class VeloxIteratorApi extends IteratorApi with Logging { file => // The "file.filePath" in PartitionedFile is not the original encoded path, so the decoded // path is incorrect in some cases and here fix the case of ' ' by using GlutenURLDecoder - var filePath = file.filePath.toString - if (filePath.startsWith("viewfs")) { - val viewPath = new Path(filePath) - val viewFileSystem = FileSystem.get(viewPath.toUri, serializableHadoopConf.value) - filePath = viewFileSystem.resolvePath(viewPath).toString - } paths.add( GlutenURLDecoder - .decode(filePath, StandardCharsets.UTF_8.name())) + .decode(file.filePath.toString, StandardCharsets.UTF_8.name())) starts.add(JLong.valueOf(file.start)) lengths.add(JLong.valueOf(file.length)) val (fileSize, modificationTime) = diff --git a/gluten-iceberg/src/main/scala/org/apache/gluten/execution/IcebergScanTransformer.scala b/gluten-iceberg/src/main/scala/org/apache/gluten/execution/IcebergScanTransformer.scala index 10d24c317cc1..1cbeb52a9213 100644 --- a/gluten-iceberg/src/main/scala/org/apache/gluten/execution/IcebergScanTransformer.scala +++ b/gluten-iceberg/src/main/scala/org/apache/gluten/execution/IcebergScanTransformer.scala @@ -27,7 +27,6 @@ import org.apache.spark.sql.connector.catalog.Table import org.apache.spark.sql.connector.read.{InputPartition, Scan} import org.apache.spark.sql.execution.datasources.v2.BatchScanExec import org.apache.spark.sql.types.StructType -import org.apache.spark.util.SerializableConfiguration import org.apache.iceberg.spark.source.GlutenIcebergSourceUtil @@ -59,9 +58,7 @@ case class IcebergScanTransformer( override lazy val fileFormat: ReadFileFormat = GlutenIcebergSourceUtil.getFileFormat(scan) - override def getSplitInfosFromPartitions( - partitions: Seq[InputPartition], - serializableHadoopConf: SerializableConfiguration): Seq[SplitInfo] = { + override def getSplitInfosFromPartitions(partitions: Seq[InputPartition]): Seq[SplitInfo] = { val groupedPartitions = SparkShimLoader.getSparkShims.orderPartitions( scan, keyGroupedPartitioning, diff --git a/gluten-iceberg/src/test/scala/org/apache/gluten/execution/VeloxIcebergSuite.scala b/gluten-iceberg/src/test/scala/org/apache/gluten/execution/VeloxIcebergSuite.scala index 7f399ce629cf..de71d341db69 100644 --- a/gluten-iceberg/src/test/scala/org/apache/gluten/execution/VeloxIcebergSuite.scala +++ b/gluten-iceberg/src/test/scala/org/apache/gluten/execution/VeloxIcebergSuite.scala @@ -128,7 +128,7 @@ class VeloxIcebergSuite extends WholeStageTransformerSuite { case plan if plan.isInstanceOf[IcebergScanTransformer] => assert( plan.asInstanceOf[IcebergScanTransformer].getKeyGroupPartitioning.isDefined) - assert(plan.asInstanceOf[IcebergScanTransformer].getSplitInfos(null).length == 3) + assert(plan.asInstanceOf[IcebergScanTransformer].getSplitInfos.length == 3) case _ => // do nothing } checkLengthAndPlan(df, 7) @@ -208,7 +208,7 @@ class VeloxIcebergSuite extends WholeStageTransformerSuite { case plan if plan.isInstanceOf[IcebergScanTransformer] => assert( plan.asInstanceOf[IcebergScanTransformer].getKeyGroupPartitioning.isDefined) - assert(plan.asInstanceOf[IcebergScanTransformer].getSplitInfos(null).length == 3) + assert(plan.asInstanceOf[IcebergScanTransformer].getSplitInfos.length == 3) case _ => // do nothing } checkLengthAndPlan(df, 7) @@ -289,7 +289,7 @@ class VeloxIcebergSuite extends WholeStageTransformerSuite { case plan if plan.isInstanceOf[IcebergScanTransformer] => assert( plan.asInstanceOf[IcebergScanTransformer].getKeyGroupPartitioning.isDefined) - assert(plan.asInstanceOf[IcebergScanTransformer].getSplitInfos(null).length == 1) + assert(plan.asInstanceOf[IcebergScanTransformer].getSplitInfos.length == 1) case _ => // do nothing } checkLengthAndPlan(df, 5) diff --git a/gluten-substrait/src/main/java/org/apache/gluten/substrait/rel/LocalFilesNode.java b/gluten-substrait/src/main/java/org/apache/gluten/substrait/rel/LocalFilesNode.java index 04bb9d8cf400..9513f497602a 100644 --- a/gluten-substrait/src/main/java/org/apache/gluten/substrait/rel/LocalFilesNode.java +++ b/gluten-substrait/src/main/java/org/apache/gluten/substrait/rel/LocalFilesNode.java @@ -92,6 +92,11 @@ public List getPaths() { return paths; } + public void setPaths(List newPaths) { + paths.clear(); + paths.addAll(newPaths); + } + public void setFileSchema(StructType schema) { this.fileSchema = schema; } diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/IteratorApi.scala b/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/IteratorApi.scala index 69c9d37334de..11211bd0da91 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/IteratorApi.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/IteratorApi.scala @@ -29,7 +29,6 @@ import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.types.StructType import org.apache.spark.sql.utils.SparkInputMetricsUtil.InputMetricsWrapper import org.apache.spark.sql.vectorized.ColumnarBatch -import org.apache.spark.util.SerializableConfiguration trait IteratorApi { @@ -38,8 +37,7 @@ trait IteratorApi { partitionSchema: StructType, fileFormat: ReadFileFormat, metadataColumnNames: Seq[String], - properties: Map[String, String], - serializableHadoopConf: SerializableConfiguration): SplitInfo + properties: Map[String, String]): SplitInfo /** Generate native row partition. */ def genPartitions( diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/BasicScanExecTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/BasicScanExecTransformer.scala index d768ac2c5936..73ed35e7190b 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/BasicScanExecTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/BasicScanExecTransformer.scala @@ -31,7 +31,6 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.connector.read.InputPartition import org.apache.spark.sql.hive.HiveTableScanExecTransformer import org.apache.spark.sql.types.{BooleanType, StringType, StructField, StructType} -import org.apache.spark.util.SerializableConfiguration import com.google.protobuf.StringValue import io.substrait.proto.NamedStruct @@ -63,13 +62,11 @@ trait BasicScanExecTransformer extends LeafTransformSupport with BaseDataSource def getProperties: Map[String, String] = Map.empty /** Returns the split infos that will be processed by the underlying native engine. */ - def getSplitInfos(serializableHadoopConf: SerializableConfiguration): Seq[SplitInfo] = { - getSplitInfosFromPartitions(getPartitions, serializableHadoopConf) + def getSplitInfos(): Seq[SplitInfo] = { + getSplitInfosFromPartitions(getPartitions) } - def getSplitInfosFromPartitions( - partitions: Seq[InputPartition], - serializableHadoopConf: SerializableConfiguration): Seq[SplitInfo] = { + def getSplitInfosFromPartitions(partitions: Seq[InputPartition]): Seq[SplitInfo] = { partitions.map( BackendsApiManager.getIteratorApiInstance .genSplitInfo( @@ -77,8 +74,7 @@ trait BasicScanExecTransformer extends LeafTransformSupport with BaseDataSource getPartitionSchema, fileFormat, getMetadataColumns.map(_.name), - getProperties, - serializableHadoopConf)) + getProperties)) } override protected def doValidateInternal(): ValidationResult = { diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/WholeStageTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/WholeStageTransformer.scala index 70839ffc2eba..beb7fe5f99d2 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/WholeStageTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/WholeStageTransformer.scala @@ -25,7 +25,7 @@ import org.apache.gluten.metrics.{GlutenTimeMetric, MetricsUpdater} import org.apache.gluten.substrait.`type`.{TypeBuilder, TypeNode} import org.apache.gluten.substrait.SubstraitContext import org.apache.gluten.substrait.plan.{PlanBuilder, PlanNode} -import org.apache.gluten.substrait.rel.{RelNode, SplitInfo} +import org.apache.gluten.substrait.rel.{LocalFilesNode, RelNode, SplitInfo} import org.apache.gluten.utils.SubstraitPlanPrinterUtil import org.apache.spark._ @@ -43,7 +43,9 @@ import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.SerializableConfiguration import com.google.common.collect.Lists +import org.apache.hadoop.fs.{FileSystem, Path} +import scala.collection.JavaConverters._ import scala.collection.mutable import scala.collection.mutable.ArrayBuffer @@ -127,8 +129,10 @@ case class WholeStageTransformer(child: SparkPlan, materializeInput: Boolean = f BackendsApiManager.getMetricsApiInstance.genWholeStageTransformerMetrics(sparkContext) val sparkConf: SparkConf = sparkContext.getConf + val serializableHadoopConf: SerializableConfiguration = new SerializableConfiguration( sparkContext.hadoopConfiguration) + val numaBindingInfo: GlutenNumaBindingInfo = GlutenConfig.getConf.numaBindingInfo @transient @@ -289,10 +293,28 @@ case class WholeStageTransformer(child: SparkPlan, materializeInput: Boolean = f */ val allScanPartitions = basicScanExecTransformers.map(_.getPartitions) val allScanSplitInfos = - getSplitInfosFromPartitions( - basicScanExecTransformers, - allScanPartitions, - serializableHadoopConf) + getSplitInfosFromPartitions(basicScanExecTransformers, allScanPartitions) + if (GlutenConfig.getConf.enableHdfsViewfs) { + allScanSplitInfos.foreach { + splitInfos => + splitInfos.foreach { + case splitInfo: LocalFilesNode => + val paths = splitInfo.getPaths.asScala + if (paths.nonEmpty && paths.head.startsWith("viewfs")) { + // Convert the viewfs path into hdfs + val newPaths = paths.map { + viewfsPath => + val viewPath = new Path(viewfsPath) + val viewFileSystem = + FileSystem.get(viewPath.toUri, serializableHadoopConf.value) + viewFileSystem.resolvePath(viewPath).toString + } + splitInfo.setPaths(newPaths.asJava) + } + } + } + } + val inputPartitions = BackendsApiManager.getIteratorApiInstance.genPartitions( wsCtx, @@ -384,8 +406,7 @@ case class WholeStageTransformer(child: SparkPlan, materializeInput: Boolean = f private def getSplitInfosFromPartitions( basicScanExecTransformers: Seq[BasicScanExecTransformer], - allScanPartitions: Seq[Seq[InputPartition]], - serializableHadoopConf: SerializableConfiguration): Seq[Seq[SplitInfo]] = { + allScanPartitions: Seq[Seq[InputPartition]]): Seq[Seq[SplitInfo]] = { // If these are two scan transformers, they must have same partitions, // otherwise, exchange will be inserted. We should combine the two scan // transformers' partitions with same index, and set them together in @@ -404,7 +425,7 @@ case class WholeStageTransformer(child: SparkPlan, materializeInput: Boolean = f val allScanSplitInfos = allScanPartitions.zip(basicScanExecTransformers).map { case (partition, transformer) => - transformer.getSplitInfosFromPartitions(partition, serializableHadoopConf) + transformer.getSplitInfosFromPartitions(partition) } val partitionLength = allScanSplitInfos.head.size if (allScanSplitInfos.exists(_.size != partitionLength)) { diff --git a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala index 5b15faf6466a..107c33a241fd 100644 --- a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala +++ b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala @@ -478,6 +478,8 @@ class GlutenConfig(conf: SQLConf) extends Logging { def enableHiveFileFormatWriter: Boolean = conf.getConf(NATIVE_HIVEFILEFORMAT_WRITER_ENABLED) def enableCelebornFallback: Boolean = conf.getConf(CELEBORN_FALLBACK_ENABLED) + + def enableHdfsViewfs: Boolean = conf.getConf(HDFS_VIEWFS_ENABLED) } object GlutenConfig { @@ -2193,4 +2195,11 @@ object GlutenConfig { "Otherwise, do nothing.") .booleanConf .createWithDefault(false) + + val HDFS_VIEWFS_ENABLED = + buildStaticConf("spark.gluten.storage.hdfsViewfs.enabled") + .internal() + .doc("If enabled, gluten will convert the viewfs path to hdfs path in scala side") + .booleanConf + .createWithDefault(false) } From 17d9cd8dfbfa0afc551d71be954553f959ce7f26 Mon Sep 17 00:00:00 2001 From: lgbo Date: Mon, 18 Nov 2024 10:18:43 +0800 Subject: [PATCH 35/35] [GLUTEN-7959][CH] `AdvancedExpandStep` generates less row than expected (#7960) * detect cardinality * fix aggregate params --- .../Operator/AdvancedExpandStep.cpp | 19 ++++++------------- 1 file changed, 6 insertions(+), 13 deletions(-) diff --git a/cpp-ch/local-engine/Operator/AdvancedExpandStep.cpp b/cpp-ch/local-engine/Operator/AdvancedExpandStep.cpp index 6ac5f5fc8f8b..b777731a9103 100644 --- a/cpp-ch/local-engine/Operator/AdvancedExpandStep.cpp +++ b/cpp-ch/local-engine/Operator/AdvancedExpandStep.cpp @@ -115,6 +115,7 @@ void AdvancedExpandStep::transformPipeline(DB::QueryPipelineBuilder & pipeline, break; aggregate_grouping_keys.push_back(col.name); } + // partial to partial aggregate DB::Aggregator::Params params( aggregate_grouping_keys, aggregate_descriptions, @@ -122,10 +123,10 @@ void AdvancedExpandStep::transformPipeline(DB::QueryPipelineBuilder & pipeline, settings[DB::Setting::max_rows_to_group_by], settings[DB::Setting::group_by_overflow_mode], settings[DB::Setting::group_by_two_level_threshold], - settings[DB::Setting::group_by_two_level_threshold_bytes], - settings[DB::Setting::max_bytes_before_external_group_by], + 0, + 0, settings[DB::Setting::empty_result_for_aggregation_by_empty_set], - context->getTempDataOnDisk(), + nullptr, settings[DB::Setting::max_threads], settings[DB::Setting::min_free_disk_space_for_temporary_data], true, @@ -149,7 +150,7 @@ void AdvancedExpandStep::transformPipeline(DB::QueryPipelineBuilder & pipeline, new_processors.push_back(expand_processor); auto expand_output_header = expand_processor->getOutputs().front().getHeader(); - + auto transform_params = std::make_shared(expand_output_header, params, false); auto aggregate_processor = std::make_shared(expand_output_header, transform_params, context, false, false); @@ -188,14 +189,10 @@ AdvancedExpandTransform::AdvancedExpandTransform( , input_header(input_header_) { for (size_t i = 0; i < project_set_exprs.getKinds().size(); ++i) - { is_low_cardinality_expand.push_back(true); - } for (auto & port : outputs) - { output_ports.push_back(&port); - } } DB::IProcessor::Status AdvancedExpandTransform::prepare() @@ -245,9 +242,7 @@ DB::IProcessor::Status AdvancedExpandTransform::prepare() input.setNeeded(); if (!input.hasData()) - { return Status::NeedData; - } input_chunk = input.pull(true); has_input = true; expand_expr_iterator = 0; @@ -265,9 +260,7 @@ void AdvancedExpandTransform::work() has_input = false; } if ((input_finished || cardinality_detect_rows >= rows_for_detect_cardinality) && !cardinality_detect_blocks.empty()) - { detectCardinality(); - } else if (!input_finished && cardinality_detect_rows < rows_for_detect_cardinality) return; @@ -281,7 +274,7 @@ void AdvancedExpandTransform::detectCardinality() std::vector is_col_low_cardinality; for (size_t i = 0; i < grouping_keys; ++i) { - DB::WeakHash32 hash(cardinality_detect_rows); + DB::WeakHash32 hash = block.getByPosition(i).column->getWeakHash32(); std::unordered_set distinct_ids; const auto & data = hash.getData(); for (size_t j = 0; j < cardinality_detect_rows; ++j)