From 35de38f4ad1daa25a990659ef1243924e9964e27 Mon Sep 17 00:00:00 2001 From: Suraj Naik Date: Mon, 25 Nov 2024 08:37:53 +0530 Subject: [PATCH 01/13] [GLUTEN-7950][VL] Keep Core module's build flag consistent with Velox (#8027) --- cpp/core/CMakeLists.txt | 30 ++++++++++++++++++++++-------- ep/build-velox/src/get_velox.sh | 1 - 2 files changed, 22 insertions(+), 9 deletions(-) diff --git a/cpp/core/CMakeLists.txt b/cpp/core/CMakeLists.txt index 94484eab826cf..88b0fd883e5c8 100644 --- a/cpp/core/CMakeLists.txt +++ b/cpp/core/CMakeLists.txt @@ -22,10 +22,29 @@ include(FindPkgConfig) include(GNUInstallDirs) include(CheckCXXCompilerFlag) -# Only set arch=native for non-AppleClang compilers. -if(NOT CMAKE_CXX_COMPILER_ID MATCHES "AppleClang") - set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -march=native") +set(CMAKE_CXX_FLAGS + "${CMAKE_CXX_FLAGS} -Wno-deprecated-declarations -Wno-attributes") +if(NOT ${CMAKE_SYSTEM_NAME} STREQUAL "Darwin") + set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wno-class-memaccess") +endif() + +if(NOT DEFINED VELOX_HOME) + set(VELOX_HOME ${GLUTEN_HOME}/ep/build-velox/build/velox_ep) + message(STATUS "Set VELOX_HOME to ${VELOX_HOME}") +endif() + +# Keep same compile option with Velox. +execute_process( + COMMAND + bash -c + "( source ${VELOX_HOME}/scripts/setup-helper-functions.sh && echo -n $(get_cxx_flags $ENV{CPU_TARGET}))" + OUTPUT_VARIABLE SCRIPT_CXX_FLAGS + RESULT_VARIABLE COMMAND_STATUS) +if(COMMAND_STATUS EQUAL "1") + message(FATAL_ERROR "Unable to determine compiler flags!") endif() +set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${SCRIPT_CXX_FLAGS}") +message("Core module final CMAKE_CXX_FLAGS=${CMAKE_CXX_FLAGS}") set(BOOST_MIN_VERSION "1.42.0") find_package(Boost REQUIRED) @@ -99,11 +118,6 @@ set_source_files_properties(${GLUTEN_PROTO_OUTPUT_FILES} PROPERTIES GENERATED TRUE) get_filename_component(GLUTEN_PROTO_DIR ${GLUTEN_PROTO_SRC_DIR}/ DIRECTORY) -set(CMAKE_CXX_FLAGS - "${CMAKE_CXX_FLAGS} -Wno-deprecated-declarations -Wno-attributes") - -message("Core module final CMAKE_CXX_FLAGS=${CMAKE_CXX_FLAGS}") - set(SPARK_COLUMNAR_PLUGIN_SRCS ${SUBSTRAIT_PROTO_SRCS} ${GLUTEN_PROTO_SRCS} diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh index f50e244dffa64..d95f742a55153 100755 --- a/ep/build-velox/src/get_velox.sh +++ b/ep/build-velox/src/get_velox.sh @@ -189,7 +189,6 @@ function setup_linux { local LINUX_VERSION_ID=$(. /etc/os-release && echo ${VERSION_ID}) # apply patches - sed -i 's/-mavx2 -mfma -mavx -mf16c -mlzcnt -std=c++17/-march=native -std=c++17 -mno-avx512f/g' scripts/setup-helper-functions.sh sed -i 's/SUDO="${SUDO:-""}"/SUDO="${SUDO:-"sudo --preserve-env"}"/g' scripts/setup-helper-functions.sh if [[ "$LINUX_DISTRIBUTION" == "ubuntu" || "$LINUX_DISTRIBUTION" == "debian" || "$LINUX_DISTRIBUTION" == "pop" ]]; then process_setup_ubuntu From 0bd6584eab916e79028c6275ae72e4cb3a0364a8 Mon Sep 17 00:00:00 2001 From: Hongze Zhang Date: Mon, 25 Nov 2024 14:04:45 +0800 Subject: [PATCH 02/13] [VL] RAS: Remove alternative constraint sets passing to RAS planner (#8033) --- .../enumerated/EnumeratedTransform.scala | 11 +----- .../scala/org/apache/gluten/ras/Ras.scala | 17 +++------ .../org/apache/gluten/ras/RasPlanner.scala | 10 ++---- .../org/apache/gluten/ras/dp/DpPlanner.scala | 15 ++------ .../ras/exaustive/ExhaustivePlanner.scala | 10 ++---- .../org/apache/gluten/ras/PropertySuite.scala | 35 ------------------- 6 files changed, 13 insertions(+), 85 deletions(-) diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedTransform.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedTransform.scala index 43c5f10bbde36..fad0ae386c040 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedTransform.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedTransform.scala @@ -21,7 +21,6 @@ import org.apache.gluten.exception.GlutenException import org.apache.gluten.extension.columnar.ColumnarRuleApplier.ColumnarRuleCall import org.apache.gluten.extension.columnar.enumerated.planner.GlutenOptimization import org.apache.gluten.extension.columnar.enumerated.planner.property.Conv -import org.apache.gluten.extension.columnar.transition.ConventionReq import org.apache.gluten.extension.injector.Injector import org.apache.gluten.extension.util.AdaptiveContext import org.apache.gluten.logging.LogLevelUtil @@ -59,17 +58,9 @@ case class EnumeratedTransform(costModel: CostModel[SparkPlan], rules: Seq[RasRu private val reqConvention = Conv.any - private val altConventions = { - val rowBased: Conv = Conv.req(ConventionReq.row) - val backendBatchBased: Conv = Conv.req(ConventionReq.backendBatch) - Seq(rowBased, backendBatchBased) - } - override def apply(plan: SparkPlan): SparkPlan = { val constraintSet = PropertySet(List(reqConvention)) - val altConstraintSets = - altConventions.map(altConv => PropertySet(List(altConv))) - val planner = optimization.newPlanner(plan, constraintSet, altConstraintSets) + val planner = optimization.newPlanner(plan, constraintSet) val out = planner.plan() out } diff --git a/gluten-ras/common/src/main/scala/org/apache/gluten/ras/Ras.scala b/gluten-ras/common/src/main/scala/org/apache/gluten/ras/Ras.scala index f705a2901be92..6cf15b0e8d6ce 100644 --- a/gluten-ras/common/src/main/scala/org/apache/gluten/ras/Ras.scala +++ b/gluten-ras/common/src/main/scala/org/apache/gluten/ras/Ras.scala @@ -26,10 +26,7 @@ import scala.collection.mutable * https://github.com/apache/incubator-gluten/issues/5057. */ trait Optimization[T <: AnyRef] { - def newPlanner( - plan: T, - constraintSet: PropertySet[T], - altConstraintSets: Seq[PropertySet[T]]): RasPlanner[T] + def newPlanner(plan: T, constraintSet: PropertySet[T]): RasPlanner[T] def anyPropSet(): PropertySet[T] def withNewConfig(confFunc: RasConfig => RasConfig): Optimization[T] } @@ -47,10 +44,7 @@ object Optimization { implicit class OptimizationImplicits[T <: AnyRef](opt: Optimization[T]) { def newPlanner(plan: T): RasPlanner[T] = { - opt.newPlanner(plan, opt.anyPropSet(), List.empty) - } - def newPlanner(plan: T, constraintSet: PropertySet[T]): RasPlanner[T] = { - opt.newPlanner(plan, constraintSet, List.empty) + opt.newPlanner(plan, opt.anyPropSet()) } } } @@ -113,11 +107,8 @@ class Ras[T <: AnyRef] private ( } } - override def newPlanner( - plan: T, - constraintSet: PropertySet[T], - altConstraintSets: Seq[PropertySet[T]]): RasPlanner[T] = { - RasPlanner(this, altConstraintSets, constraintSet, plan) + override def newPlanner(plan: T, constraintSet: PropertySet[T]): RasPlanner[T] = { + RasPlanner(this, constraintSet, plan) } override def anyPropSet(): PropertySet[T] = propertySetFactory().any() diff --git a/gluten-ras/common/src/main/scala/org/apache/gluten/ras/RasPlanner.scala b/gluten-ras/common/src/main/scala/org/apache/gluten/ras/RasPlanner.scala index 327b980f38ecb..626d2b1caf9c9 100644 --- a/gluten-ras/common/src/main/scala/org/apache/gluten/ras/RasPlanner.scala +++ b/gluten-ras/common/src/main/scala/org/apache/gluten/ras/RasPlanner.scala @@ -32,16 +32,12 @@ trait RasPlanner[T <: AnyRef] { } object RasPlanner { - def apply[T <: AnyRef]( - ras: Ras[T], - altConstraintSets: Seq[PropertySet[T]], - constraintSet: PropertySet[T], - plan: T): RasPlanner[T] = { + def apply[T <: AnyRef](ras: Ras[T], constraintSet: PropertySet[T], plan: T): RasPlanner[T] = { ras.config.plannerType match { case PlannerType.Exhaustive => - ExhaustivePlanner(ras, altConstraintSets, constraintSet, plan) + ExhaustivePlanner(ras, constraintSet, plan) case PlannerType.Dp => - DpPlanner(ras, altConstraintSets, constraintSet, plan) + DpPlanner(ras, constraintSet, plan) } } } diff --git a/gluten-ras/common/src/main/scala/org/apache/gluten/ras/dp/DpPlanner.scala b/gluten-ras/common/src/main/scala/org/apache/gluten/ras/dp/DpPlanner.scala index 3f2590dff8a4d..2b601720bfd55 100644 --- a/gluten-ras/common/src/main/scala/org/apache/gluten/ras/dp/DpPlanner.scala +++ b/gluten-ras/common/src/main/scala/org/apache/gluten/ras/dp/DpPlanner.scala @@ -26,11 +26,7 @@ import org.apache.gluten.ras.property.PropertySet import org.apache.gluten.ras.rule.{EnforcerRuleSet, RuleApplier, Shape} // TODO: Branch and bound pruning. -private class DpPlanner[T <: AnyRef] private ( - ras: Ras[T], - altConstraintSets: Seq[PropertySet[T]], - constraintSet: PropertySet[T], - plan: T) +private class DpPlanner[T <: AnyRef] private (ras: Ras[T], constraintSet: PropertySet[T], plan: T) extends RasPlanner[T] { import DpPlanner._ @@ -43,7 +39,6 @@ private class DpPlanner[T <: AnyRef] private ( } private lazy val best: (Best[T], KnownCostPath[T]) = { - altConstraintSets.foreach(propSet => memo.memorize(plan, propSet)) val groupId = rootGroupId val memoTable = memo.table() val best = findBest(memoTable, groupId) @@ -70,12 +65,8 @@ private class DpPlanner[T <: AnyRef] private ( } object DpPlanner { - def apply[T <: AnyRef]( - ras: Ras[T], - altConstraintSets: Seq[PropertySet[T]], - constraintSet: PropertySet[T], - plan: T): RasPlanner[T] = { - new DpPlanner(ras, altConstraintSets: Seq[PropertySet[T]], constraintSet, plan) + def apply[T <: AnyRef](ras: Ras[T], constraintSet: PropertySet[T], plan: T): RasPlanner[T] = { + new DpPlanner(ras, constraintSet, plan) } // Visited flag. diff --git a/gluten-ras/common/src/main/scala/org/apache/gluten/ras/exaustive/ExhaustivePlanner.scala b/gluten-ras/common/src/main/scala/org/apache/gluten/ras/exaustive/ExhaustivePlanner.scala index c4d3e4881c43e..f0cb42cf662a5 100644 --- a/gluten-ras/common/src/main/scala/org/apache/gluten/ras/exaustive/ExhaustivePlanner.scala +++ b/gluten-ras/common/src/main/scala/org/apache/gluten/ras/exaustive/ExhaustivePlanner.scala @@ -27,7 +27,6 @@ import org.apache.gluten.ras.rule.{EnforcerRuleSet, RuleApplier, Shape} private class ExhaustivePlanner[T <: AnyRef] private ( ras: Ras[T], - altConstraintSets: Seq[PropertySet[T]], constraintSet: PropertySet[T], plan: T) extends RasPlanner[T] { @@ -40,7 +39,6 @@ private class ExhaustivePlanner[T <: AnyRef] private ( } private lazy val best: (Best[T], KnownCostPath[T]) = { - altConstraintSets.foreach(propSet => memo.memorize(plan, propSet)) val groupId = rootGroupId explore() val memoState = memo.newState() @@ -72,12 +70,8 @@ private class ExhaustivePlanner[T <: AnyRef] private ( } object ExhaustivePlanner { - def apply[T <: AnyRef]( - ras: Ras[T], - altConstraintSets: Seq[PropertySet[T]], - constraintSet: PropertySet[T], - plan: T): RasPlanner[T] = { - new ExhaustivePlanner(ras, altConstraintSets, constraintSet, plan) + def apply[T <: AnyRef](ras: Ras[T], constraintSet: PropertySet[T], plan: T): RasPlanner[T] = { + new ExhaustivePlanner(ras, constraintSet, plan) } private class ExhaustiveExplorer[T <: AnyRef]( diff --git a/gluten-ras/common/src/test/scala/org/apache/gluten/ras/PropertySuite.scala b/gluten-ras/common/src/test/scala/org/apache/gluten/ras/PropertySuite.scala index eb4babe069e5f..06bb806f7d6bf 100644 --- a/gluten-ras/common/src/test/scala/org/apache/gluten/ras/PropertySuite.scala +++ b/gluten-ras/common/src/test/scala/org/apache/gluten/ras/PropertySuite.scala @@ -250,41 +250,6 @@ abstract class PropertySuite extends AnyFunSuite { assert(out == TypedUnary(TypeA, 8, PassNodeType(5, TypedLeaf(TypeA, 10)))) } - test(s"Property convert - (A, B), alternative conventions") { - object ConvertEnforcerAndTypeAToTypeB extends RasRule[TestNode] { - override def shift(node: TestNode): Iterable[TestNode] = node match { - case TypeEnforcer(TypeB, _, TypedBinary(TypeA, 5, left, right)) => - List(TypedBinary(TypeB, 0, left, right)) - case _ => List.empty - } - override def shape(): Shape[TestNode] = Shapes.fixedHeight(2) - } - - val ras = - Ras[TestNode]( - PlanModelImpl, - CostModelImpl, - MetadataModelImpl, - propertyModel(zeroDepth), - ExplainImpl, - RasRule.Factory.reuse(List(ConvertEnforcerAndTypeAToTypeB))) - .withNewConfig(_ => conf) - val plan = - TypedBinary(TypeA, 5, TypedUnary(TypeA, 10, TypedLeaf(TypeA, 10)), TypedLeaf(TypeA, 10)) - val planner = ras.newPlanner( - plan, - PropertySet(Seq(TypeAny)), - List(PropertySet(Seq(TypeB)), PropertySet(Seq(TypeC)))) - val out = planner.plan() - assert( - out == TypedBinary( - TypeB, - 0, - TypeEnforcer(TypeB, 1, TypedUnary(TypeA, 10, TypedLeaf(TypeA, 10))), - TypeEnforcer(TypeB, 1, TypedLeaf(TypeA, 10)))) - assert(planner.newState().memoState().allGroups().size == 9) - } - test(s"Property convert - (A, B), Unary only has TypeA") { object ReplaceNonUnaryByTypeBRule extends RasRule[TestNode] { override def shift(node: TestNode): Iterable[TestNode] = { From e5b4b4ce081f99090ac532b3d16ea31b2de12aac Mon Sep 17 00:00:00 2001 From: Hongze Zhang Date: Mon, 25 Nov 2024 14:29:54 +0800 Subject: [PATCH 03/13] [GLUTEN-6920][CORE] Move API `Backend#defaultBatchType` down to `BackendSettingsApi` in module gluten-substrait (#8016) --- .../backendsapi/clickhouse/CHBackend.scala | 17 +++++- .../backendsapi/clickhouse/CHRuleApi.scala | 3 +- .../backendsapi/velox/VeloxBackend.scala | 10 ++-- .../backendsapi/velox/VeloxRuleApi.scala | 5 +- .../ColumnarPartialProjectExec.scala | 2 - .../execution/VeloxResizeBatchesExec.scala | 1 - .../python/ColumnarArrowEvalPythonExec.scala | 3 +- .../execution/ArrowFileSourceScanExec.scala | 2 - .../sql/execution/BaseArrowScanExec.scala | 3 +- .../transition/VeloxTransitionSuite.scala | 46 ++++++++-------- .../org/apache/gluten/backend/Backend.scala | 5 +- .../execution/ColumnarToColumnarExec.scala | 21 ++++++-- .../gluten/extension/GlutenColumnarRule.scala | 9 ++-- .../enumerated/EnumeratedTransform.scala | 4 +- .../planner/cost/LongCostModel.scala | 9 ++-- .../planner/plan/GlutenPlanModel.scala | 19 ++++--- .../enumerated/planner/property/Conv.scala | 12 +++++ .../columnar/transition/Convention.scala | 52 ++++++++++++++++++- .../columnar/transition/ConventionFunc.scala | 52 ++++++++++++------- .../columnar/transition/ConventionReq.scala | 13 ++--- .../columnar/transition/Transition.scala | 19 ++++--- .../columnar/transition/Transitions.scala | 40 +++++++------- .../extension/injector/GlutenInjector.scala | 4 +- .../apache/gluten/iterator/Iterators.scala | 2 +- .../apache/spark/util/SparkVersionUtil.scala | 30 +++++++++++ .../backendsapi/BackendSettingsApi.scala | 5 ++ .../BasicPhysicalOperatorTransformer.scala | 2 - .../CartesianProductExecTransformer.scala | 1 - .../execution/ColumnarCoalesceExec.scala | 2 - .../execution/ColumnarToRowExecBase.scala | 11 ++-- .../execution/RowToColumnarExecBase.scala | 5 +- ...TakeOrderedAndProjectExecTransformer.scala | 1 - .../execution/WholeStageTransformer.scala | 2 - .../apache/gluten/extension/GlutenPlan.scala | 30 ++++++----- .../columnar/MiscColumnarRules.scala | 4 +- .../heuristic/ExpandFallbackPolicy.scala | 7 ++- .../transition/BackendTransitions.scala | 36 +++++++++++++ .../ColumnarBroadcastExchangeExec.scala | 2 - .../ColumnarCollapseTransformStages.scala | 5 +- .../ColumnarShuffleExchangeExec.scala | 1 - .../execution/ColumnarWriteFilesExec.scala | 14 +++-- .../GlutenWriterColumnarRules.scala | 6 ++- .../columnar/transition/TransitionSuite.scala | 15 +++--- .../transition/TransitionSuiteBase.scala | 11 ++-- .../execution/FallbackStrategiesSuite.scala | 20 +++---- .../execution/FallbackStrategiesSuite.scala | 20 +++---- .../execution/FallbackStrategiesSuite.scala | 20 +++---- .../benchmarks/ParquetReadBenchmark.scala | 3 +- .../execution/FallbackStrategiesSuite.scala | 20 +++---- .../AbstractFileSourceScanExec.scala | 7 ++- .../AbstractFileSourceScanExec.scala | 7 ++- .../AbstractFileSourceScanExec.scala | 12 ++--- .../AbstractFileSourceScanExec.scala | 12 ++--- 53 files changed, 401 insertions(+), 263 deletions(-) create mode 100644 gluten-core/src/main/scala/org/apache/spark/util/SparkVersionUtil.scala create mode 100644 gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/transition/BackendTransitions.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 061ec9856e809..f6cacba42b9f8 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 @@ -24,7 +24,7 @@ import org.apache.gluten.columnarbatch.CHBatch import org.apache.gluten.execution.WriteFilesExecTransformer import org.apache.gluten.expression.WindowFunctionsBuilder import org.apache.gluten.extension.ValidationResult -import org.apache.gluten.extension.columnar.transition.Convention +import org.apache.gluten.extension.columnar.transition.{Convention, ConventionFunc} import org.apache.gluten.substrait.rel.LocalFilesNode.ReadFileFormat import org.apache.gluten.substrait.rel.LocalFilesNode.ReadFileFormat._ @@ -34,6 +34,8 @@ import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression import org.apache.spark.sql.catalyst.plans._ +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec import org.apache.spark.sql.execution.datasources.FileFormat import org.apache.spark.sql.execution.datasources.orc.OrcFileFormat import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat @@ -45,10 +47,11 @@ import java.util.Locale import scala.util.control.Breaks.{break, breakable} class CHBackend extends SubstraitBackend { + import CHBackend._ override def name(): String = CHConf.BACKEND_NAME - override def defaultBatchType: Convention.BatchType = CHBatch override def buildInfo(): Backend.BuildInfo = Backend.BuildInfo("ClickHouse", CH_BRANCH, CH_COMMIT, "UNKNOWN") + override def convFuncOverride(): ConventionFunc.Override = new ConvFunc() override def iteratorApi(): IteratorApi = new CHIteratorApi override def sparkPlanExecApi(): SparkPlanExecApi = new CHSparkPlanExecApi override def transformerApi(): TransformerApi = new CHTransformerApi @@ -59,7 +62,17 @@ class CHBackend extends SubstraitBackend { override def settings(): BackendSettingsApi = CHBackendSettings } +object CHBackend { + private class ConvFunc() extends ConventionFunc.Override { + override def batchTypeOf: PartialFunction[SparkPlan, Convention.BatchType] = { + case a: AdaptiveSparkPlanExec if a.supportsColumnar => + CHBatch + } + } +} + object CHBackendSettings extends BackendSettingsApi with Logging { + override def primaryBatchType: Convention.BatchType = CHBatch private val GLUTEN_CLICKHOUSE_SEP_SCAN_RDD = "spark.gluten.sql.columnar.separate.scan.rdd.for.ch" private val GLUTEN_CLICKHOUSE_SEP_SCAN_RDD_DEFAULT = "false" 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 081e815588faf..f6d2b85d9d19c 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 @@ -18,6 +18,7 @@ package org.apache.gluten.backendsapi.clickhouse import org.apache.gluten.GlutenConfig import org.apache.gluten.backendsapi.RuleApi +import org.apache.gluten.columnarbatch.CHBatch import org.apache.gluten.extension._ import org.apache.gluten.extension.columnar._ import org.apache.gluten.extension.columnar.MiscColumnarRules.{RemoveGlutenTableCacheColumnarToRow, RemoveTopmostColumnarToRow, RewriteSubqueryBroadcast} @@ -114,7 +115,7 @@ object CHRuleApi { intercept( SparkPlanRules.extendedColumnarRule(c.glutenConf.extendedColumnarTransformRules)( c.session))) - injector.injectPostTransform(c => InsertTransitions(c.outputsColumnar)) + injector.injectPostTransform(c => InsertTransitions.create(c.outputsColumnar, CHBatch)) // Gluten columnar: Fallback policies. injector.injectFallbackPolicy( diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxBackend.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxBackend.scala index 03d5aa2549e56..e05fd92e32018 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxBackend.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxBackend.scala @@ -37,6 +37,7 @@ import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, import org.apache.spark.sql.catalyst.plans.{JoinType, LeftOuter, RightOuter} import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, CharVarcharUtils} import org.apache.spark.sql.execution.{ColumnarCachedBatchSerializer, SparkPlan} +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec import org.apache.spark.sql.execution.command.CreateDataSourceTableAsSelectCommand import org.apache.spark.sql.execution.datasources.{FileFormat, InsertIntoHadoopFsRelationCommand} @@ -52,10 +53,9 @@ import scala.util.control.Breaks.breakable class VeloxBackend extends SubstraitBackend { import VeloxBackend._ override def name(): String = VeloxBackend.BACKEND_NAME - override def defaultBatchType: Convention.BatchType = VeloxBatch - override def convFuncOverride(): ConventionFunc.Override = new ConvFunc() override def buildInfo(): Backend.BuildInfo = Backend.BuildInfo("Velox", VELOX_BRANCH, VELOX_REVISION, VELOX_REVISION_TIME) + override def convFuncOverride(): ConventionFunc.Override = new ConvFunc() override def iteratorApi(): IteratorApi = new VeloxIteratorApi override def sparkPlanExecApi(): SparkPlanExecApi = new VeloxSparkPlanExecApi override def transformerApi(): TransformerApi = new VeloxTransformerApi @@ -72,6 +72,8 @@ object VeloxBackend { private class ConvFunc() extends ConventionFunc.Override { override def batchTypeOf: PartialFunction[SparkPlan, Convention.BatchType] = { + case a: AdaptiveSparkPlanExec if a.supportsColumnar => + VeloxBatch case i: InMemoryTableScanExec if i.supportsColumnar && i.relation.cacheBuilder.serializer .isInstanceOf[ColumnarCachedBatchSerializer] => @@ -81,13 +83,15 @@ object VeloxBackend { } object VeloxBackendSettings extends BackendSettingsApi { - val SHUFFLE_SUPPORTED_CODEC = Set("lz4", "zstd") val GLUTEN_VELOX_UDF_LIB_PATHS = VeloxBackend.CONF_PREFIX + ".udfLibraryPaths" val GLUTEN_VELOX_DRIVER_UDF_LIB_PATHS = VeloxBackend.CONF_PREFIX + ".driver.udfLibraryPaths" val GLUTEN_VELOX_INTERNAL_UDF_LIB_PATHS = VeloxBackend.CONF_PREFIX + ".internal.udfLibraryPaths" val GLUTEN_VELOX_UDF_ALLOW_TYPE_CONVERSION = VeloxBackend.CONF_PREFIX + ".udfAllowTypeConversion" + /** The columnar-batch type this backend is by default using. */ + override def primaryBatchType: Convention.BatchType = VeloxBatch + override def validateScanExec( format: ReadFileFormat, fields: Array[StructField], 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 c5f46dae67dd6..0a62a41baaf0d 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 @@ -18,6 +18,7 @@ package org.apache.gluten.backendsapi.velox import org.apache.gluten.GlutenConfig import org.apache.gluten.backendsapi.RuleApi +import org.apache.gluten.columnarbatch.VeloxBatch import org.apache.gluten.datasource.ArrowConvertorRule import org.apache.gluten.extension._ import org.apache.gluten.extension.columnar._ @@ -101,7 +102,7 @@ object VeloxRuleApi { injector.injectPostTransform(_ => EliminateLocalSort) injector.injectPostTransform(_ => CollapseProjectExecTransformer) injector.injectPostTransform(c => FlushableHashAggregateRule.apply(c.session)) - injector.injectPostTransform(c => InsertTransitions(c.outputsColumnar)) + injector.injectPostTransform(c => InsertTransitions.create(c.outputsColumnar, VeloxBatch)) // Gluten columnar: Fallback policies. injector.injectFallbackPolicy( @@ -187,7 +188,7 @@ object VeloxRuleApi { injector.injectPostTransform(_ => EliminateLocalSort) injector.injectPostTransform(_ => CollapseProjectExecTransformer) injector.injectPostTransform(c => FlushableHashAggregateRule.apply(c.session)) - injector.injectPostTransform(c => InsertTransitions(c.outputsColumnar)) + injector.injectPostTransform(c => InsertTransitions.create(c.outputsColumnar, VeloxBatch)) injector.injectPostTransform( c => RemoveTopmostColumnarToRow(c.session, c.ac.isAdaptiveContext())) SparkShimLoader.getSparkShims diff --git a/backends-velox/src/main/scala/org/apache/gluten/execution/ColumnarPartialProjectExec.scala b/backends-velox/src/main/scala/org/apache/gluten/execution/ColumnarPartialProjectExec.scala index d993e399dbf4a..84de41daa015b 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/execution/ColumnarPartialProjectExec.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/execution/ColumnarPartialProjectExec.scala @@ -82,8 +82,6 @@ case class ColumnarPartialProjectExec(original: ProjectExec, child: SparkPlan)( replacedAliasUdf :: Nil } - final override val supportsColumnar: Boolean = true - private def validateExpression(expr: Expression): Boolean = { expr.deterministic && !expr.isInstanceOf[LambdaFunction] && expr.children .forall(validateExpression) diff --git a/backends-velox/src/main/scala/org/apache/gluten/execution/VeloxResizeBatchesExec.scala b/backends-velox/src/main/scala/org/apache/gluten/execution/VeloxResizeBatchesExec.scala index 995582024ba5c..5283ab61e3b97 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/execution/VeloxResizeBatchesExec.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/execution/VeloxResizeBatchesExec.scala @@ -53,7 +53,6 @@ case class VeloxResizeBatchesExec( "selfTime" -> SQLMetrics.createTimingMetric(sparkContext, "time to append / split batches") ) - override def supportsColumnar: Boolean = true override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() override protected def doExecuteColumnar(): RDD[ColumnarBatch] = { diff --git a/backends-velox/src/main/scala/org/apache/spark/api/python/ColumnarArrowEvalPythonExec.scala b/backends-velox/src/main/scala/org/apache/spark/api/python/ColumnarArrowEvalPythonExec.scala index f1f5eb906238b..a2eee53660cee 100644 --- a/backends-velox/src/main/scala/org/apache/spark/api/python/ColumnarArrowEvalPythonExec.scala +++ b/backends-velox/src/main/scala/org/apache/spark/api/python/ColumnarArrowEvalPythonExec.scala @@ -214,9 +214,8 @@ case class ColumnarArrowEvalPythonExec( extends EvalPythonExec with GlutenPlan with KnownChildrenConventions { - override def supportsColumnar: Boolean = true - override protected def batchType0(): Convention.BatchType = ArrowJavaBatch + override def batchType(): Convention.BatchType = ArrowJavaBatch override def requiredChildrenConventions(): Seq[ConventionReq] = List( ConventionReq.of(ConventionReq.RowType.Any, ConventionReq.BatchType.Is(ArrowJavaBatch))) diff --git a/backends-velox/src/main/scala/org/apache/spark/sql/execution/ArrowFileSourceScanExec.scala b/backends-velox/src/main/scala/org/apache/spark/sql/execution/ArrowFileSourceScanExec.scala index 85bc68223487f..16b8fb0e9f6f2 100644 --- a/backends-velox/src/main/scala/org/apache/spark/sql/execution/ArrowFileSourceScanExec.scala +++ b/backends-velox/src/main/scala/org/apache/spark/sql/execution/ArrowFileSourceScanExec.scala @@ -33,8 +33,6 @@ case class ArrowFileSourceScanExec(original: FileSourceScanExec) override def output: Seq[Attribute] = original.output - override def supportsColumnar: Boolean = original.supportsColumnar - override def doCanonicalize(): FileSourceScanExec = original.doCanonicalize() override protected def doExecuteColumnar(): RDD[ColumnarBatch] = { diff --git a/backends-velox/src/main/scala/org/apache/spark/sql/execution/BaseArrowScanExec.scala b/backends-velox/src/main/scala/org/apache/spark/sql/execution/BaseArrowScanExec.scala index 38a6d1803d0be..6617e8b138ba7 100644 --- a/backends-velox/src/main/scala/org/apache/spark/sql/execution/BaseArrowScanExec.scala +++ b/backends-velox/src/main/scala/org/apache/spark/sql/execution/BaseArrowScanExec.scala @@ -21,8 +21,7 @@ import org.apache.gluten.extension.GlutenPlan import org.apache.gluten.extension.columnar.transition.Convention trait BaseArrowScanExec extends GlutenPlan { - - final override protected def batchType0(): Convention.BatchType = { + final override def batchType(): Convention.BatchType = { ArrowBatches.ArrowJavaBatch } } diff --git a/backends-velox/src/test/scala/org/apache/gluten/extension/columnar/transition/VeloxTransitionSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/extension/columnar/transition/VeloxTransitionSuite.scala index d12faae0f73d9..e14ffd43d82d0 100644 --- a/backends-velox/src/test/scala/org/apache/gluten/extension/columnar/transition/VeloxTransitionSuite.scala +++ b/backends-velox/src/test/scala/org/apache/gluten/extension/columnar/transition/VeloxTransitionSuite.scala @@ -33,37 +33,37 @@ class VeloxTransitionSuite extends SharedSparkSession { test("Vanilla C2R - outputs row") { val in = BatchLeaf(VanillaBatch) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) assert(out == ColumnarToRowExec(BatchLeaf(VanillaBatch))) } test("Vanilla C2R - requires row input") { val in = RowUnary(BatchLeaf(VanillaBatch)) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) assert(out == RowUnary(ColumnarToRowExec(BatchLeaf(VanillaBatch)))) } test("Vanilla R2C - requires vanilla input") { val in = BatchUnary(VanillaBatch, RowLeaf()) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) assert(out == ColumnarToRowExec(BatchUnary(VanillaBatch, RowToColumnarExec(RowLeaf())))) } test("ArrowNative C2R - outputs row") { val in = BatchLeaf(ArrowNativeBatch) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) assert(out == ColumnarToRowExec(LoadArrowDataExec(BatchLeaf(ArrowNativeBatch)))) } test("ArrowNative C2R - requires row input") { val in = RowUnary(BatchLeaf(ArrowNativeBatch)) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) assert(out == RowUnary(ColumnarToRowExec(LoadArrowDataExec(BatchLeaf(ArrowNativeBatch))))) } test("ArrowNative R2C - requires Arrow input") { val in = BatchUnary(ArrowNativeBatch, RowLeaf()) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) assert( out == ColumnarToRowExec( LoadArrowDataExec(BatchUnary(ArrowNativeBatch, RowToVeloxColumnarExec(RowLeaf()))))) @@ -71,7 +71,7 @@ class VeloxTransitionSuite extends SharedSparkSession { test("ArrowNative-to-Velox C2C") { val in = BatchUnary(VeloxBatch, BatchLeaf(ArrowNativeBatch)) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) // No explicit transition needed for ArrowNative-to-Velox. // FIXME: Add explicit transitions. // See https://github.com/apache/incubator-gluten/issues/7313. @@ -82,7 +82,7 @@ class VeloxTransitionSuite extends SharedSparkSession { test("Velox-to-ArrowNative C2C") { val in = BatchUnary(ArrowNativeBatch, BatchLeaf(VeloxBatch)) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) assert( out == ColumnarToRowExec( LoadArrowDataExec(BatchUnary(ArrowNativeBatch, BatchLeaf(VeloxBatch))))) @@ -90,7 +90,7 @@ class VeloxTransitionSuite extends SharedSparkSession { test("Vanilla-to-ArrowNative C2C") { val in = BatchUnary(ArrowNativeBatch, BatchLeaf(VanillaBatch)) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) assert( out == ColumnarToRowExec( LoadArrowDataExec(BatchUnary( @@ -100,7 +100,7 @@ class VeloxTransitionSuite extends SharedSparkSession { test("ArrowNative-to-Vanilla C2C") { val in = BatchUnary(VanillaBatch, BatchLeaf(ArrowNativeBatch)) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) assert( out == ColumnarToRowExec( BatchUnary(VanillaBatch, LoadArrowDataExec(BatchLeaf(ArrowNativeBatch))))) @@ -108,19 +108,19 @@ class VeloxTransitionSuite extends SharedSparkSession { test("ArrowJava C2R - outputs row") { val in = BatchLeaf(ArrowJavaBatch) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) assert(out == ColumnarToRowExec(BatchLeaf(ArrowJavaBatch))) } test("ArrowJava C2R - requires row input") { val in = RowUnary(BatchLeaf(ArrowJavaBatch)) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) assert(out == RowUnary(ColumnarToRowExec(BatchLeaf(ArrowJavaBatch)))) } test("ArrowJava R2C - requires Arrow input") { val in = BatchUnary(ArrowJavaBatch, RowLeaf()) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) assert( out == ColumnarToRowExec( BatchUnary(ArrowJavaBatch, LoadArrowDataExec(RowToVeloxColumnarExec(RowLeaf()))))) @@ -128,7 +128,7 @@ class VeloxTransitionSuite extends SharedSparkSession { test("ArrowJava-to-Velox C2C") { val in = BatchUnary(VeloxBatch, BatchLeaf(ArrowJavaBatch)) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) assert( out == VeloxColumnarToRowExec( BatchUnary( @@ -138,7 +138,7 @@ class VeloxTransitionSuite extends SharedSparkSession { test("Velox-to-ArrowJava C2C") { val in = BatchUnary(ArrowJavaBatch, BatchLeaf(VeloxBatch)) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) assert( out == ColumnarToRowExec( BatchUnary(ArrowJavaBatch, LoadArrowDataExec(BatchLeaf(VeloxBatch))))) @@ -146,7 +146,7 @@ class VeloxTransitionSuite extends SharedSparkSession { test("Vanilla-to-ArrowJava C2C") { val in = BatchUnary(ArrowJavaBatch, BatchLeaf(VanillaBatch)) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) assert( out == ColumnarToRowExec( BatchUnary( @@ -156,37 +156,37 @@ class VeloxTransitionSuite extends SharedSparkSession { test("ArrowJava-to-Vanilla C2C") { val in = BatchUnary(VanillaBatch, BatchLeaf(ArrowJavaBatch)) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) assert(out == ColumnarToRowExec(BatchUnary(VanillaBatch, BatchLeaf(ArrowJavaBatch)))) } test("Velox C2R - outputs row") { val in = BatchLeaf(VeloxBatch) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) assert(out == VeloxColumnarToRowExec(BatchLeaf(VeloxBatch))) } test("Velox C2R - requires row input") { val in = RowUnary(BatchLeaf(VeloxBatch)) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) assert(out == RowUnary(VeloxColumnarToRowExec(BatchLeaf(VeloxBatch)))) } test("Velox R2C - outputs Velox") { val in = RowLeaf() - val out = Transitions.insertTransitions(in, outputsColumnar = true) + val out = BackendTransitions.insert(in, outputsColumnar = true) assert(out == RowToVeloxColumnarExec(RowLeaf())) } test("Velox R2C - requires Velox input") { val in = BatchUnary(VeloxBatch, RowLeaf()) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) assert(out == VeloxColumnarToRowExec(BatchUnary(VeloxBatch, RowToVeloxColumnarExec(RowLeaf())))) } test("Vanilla-to-Velox C2C") { val in = BatchUnary(VeloxBatch, BatchLeaf(VanillaBatch)) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) assert( out == VeloxColumnarToRowExec( BatchUnary(VeloxBatch, RowToVeloxColumnarExec(ColumnarToRowExec(BatchLeaf(VanillaBatch)))))) @@ -194,7 +194,7 @@ class VeloxTransitionSuite extends SharedSparkSession { test("Velox-to-Vanilla C2C") { val in = BatchUnary(VanillaBatch, BatchLeaf(VeloxBatch)) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) assert( out == ColumnarToRowExec(BatchUnary(VanillaBatch, LoadArrowDataExec(BatchLeaf(VeloxBatch))))) } 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 346181e140ac8..f406a6ac4d7d1 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 @@ -16,7 +16,7 @@ */ package org.apache.gluten.backend -import org.apache.gluten.extension.columnar.transition.{Convention, ConventionFunc} +import org.apache.gluten.extension.columnar.transition.ConventionFunc import org.apache.gluten.extension.injector.Injector import org.apache.spark.SparkContext @@ -39,9 +39,6 @@ trait Backend { def onExecutorStart(pc: PluginContext): Unit = {} def onExecutorShutdown(): Unit = {} - /** The columnar-batch type this backend is by default using. */ - def defaultBatchType: Convention.BatchType - /** * Overrides [[org.apache.gluten.extension.columnar.transition.ConventionFunc]] Gluten is using to * determine the convention (its row-based processing / columnar-batch processing support) of a diff --git a/gluten-core/src/main/scala/org/apache/gluten/execution/ColumnarToColumnarExec.scala b/gluten-core/src/main/scala/org/apache/gluten/execution/ColumnarToColumnarExec.scala index d0eacc1e4dacf..7ca4b36b06d2a 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/execution/ColumnarToColumnarExec.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/execution/ColumnarToColumnarExec.scala @@ -17,8 +17,6 @@ package org.apache.gluten.execution import org.apache.gluten.extension.columnar.transition.{Convention, ConventionReq} -import org.apache.gluten.extension.columnar.transition.Convention.KnownBatchType -import org.apache.gluten.extension.columnar.transition.ConventionReq.KnownChildrenConventions import org.apache.gluten.iterator.Iterators import org.apache.spark.rdd.RDD @@ -32,8 +30,9 @@ import java.util.concurrent.atomic.AtomicLong abstract class ColumnarToColumnarExec(from: Convention.BatchType, to: Convention.BatchType) extends ColumnarToColumnarTransition - with KnownBatchType - with KnownChildrenConventions { + with Convention.KnownBatchType + with Convention.KnownRowTypeForSpark33AndLater + with ConventionReq.KnownChildrenConventions { def child: SparkPlan protected def mapIterator(in: Iterator[ColumnarBatch]): Iterator[ColumnarBatch] @@ -47,8 +46,20 @@ abstract class ColumnarToColumnarExec(from: Convention.BatchType, to: Convention "selfTime" -> SQLMetrics.createTimingMetric(sparkContext, "time to convert batches") ) - override def supportsColumnar: Boolean = true + final override val supportsColumnar: Boolean = { + batchType() != Convention.BatchType.None + } + override def batchType(): Convention.BatchType = to + + final override val supportsRowBased: Boolean = { + rowType() != Convention.RowType.None + } + + override def rowType0(): Convention.RowType = { + Convention.RowType.None + } + override def requiredChildrenConventions(): Seq[ConventionReq] = List( ConventionReq.of(ConventionReq.RowType.Any, ConventionReq.BatchType.Is(from))) diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/GlutenColumnarRule.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/GlutenColumnarRule.scala index 3378344253dd1..5b440302a09b4 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/GlutenColumnarRule.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/GlutenColumnarRule.scala @@ -31,11 +31,9 @@ import org.apache.spark.sql.execution._ import org.apache.spark.sql.vectorized.ColumnarBatch object GlutenColumnarRule { - // Utilities to infer columnar rule's caller's property: // ApplyColumnarRulesAndInsertTransitions#outputsColumnar. - - case class DummyRowOutputExec(override val child: SparkPlan) extends UnaryExecNode { + private case class DummyRowOutputExec(override val child: SparkPlan) extends UnaryExecNode { override def supportsColumnar: Boolean = false override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() override protected def doExecuteColumnar(): RDD[ColumnarBatch] = @@ -47,7 +45,7 @@ object GlutenColumnarRule { copy(child = newChild) } - case class DummyColumnarOutputExec(override val child: SparkPlan) extends UnaryExecNode { + private case class DummyColumnarOutputExec(override val child: SparkPlan) extends UnaryExecNode { override def supportsColumnar: Boolean = true override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() override protected def doExecuteColumnar(): RDD[ColumnarBatch] = @@ -99,9 +97,8 @@ case class GlutenColumnarRule( "This should not happen. Please leave an issue at" + " https://github.com/apache/incubator-gluten.") } - val vanillaPlan = Transitions.insertTransitions(originalPlan, outputsColumnar) + val vanillaPlan = Transitions.insert(originalPlan, outputsColumnar) val applier = applierBuilder.apply(session) applier.apply(vanillaPlan, outputsColumnar) } - } diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedTransform.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedTransform.scala index fad0ae386c040..67399e25d4125 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedTransform.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedTransform.scala @@ -56,10 +56,10 @@ case class EnumeratedTransform(costModel: CostModel[SparkPlan], rules: Seq[RasRu .create() } - private val reqConvention = Conv.any + private val convReq = Conv.any override def apply(plan: SparkPlan): SparkPlan = { - val constraintSet = PropertySet(List(reqConvention)) + val constraintSet = PropertySet(Seq(convReq)) val planner = optimization.newPlanner(plan, constraintSet) val out = planner.plan() out diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/LongCostModel.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/LongCostModel.scala index 1cfe132d84ed2..393ac35de42f0 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/LongCostModel.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/LongCostModel.scala @@ -69,19 +69,20 @@ object LongCostModel extends Logging { */ sealed trait Kind { import Kind._ - values.synchronized { + all.synchronized { val n = name() - if (values.contains(n)) { + if (all.contains(n)) { throw new GlutenException(s"Cost mode kind $n already registered") } - values += n -> this + all += n -> this } def name(): String } object Kind { - val values: mutable.Map[String, Kind] = mutable.Map() + private val all: mutable.Map[String, Kind] = mutable.Map() + def values(): Map[String, Kind] = all.toMap } /** diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/plan/GlutenPlanModel.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/plan/GlutenPlanModel.scala index fa92eacd4d47b..568ea50396bd2 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/plan/GlutenPlanModel.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/plan/GlutenPlanModel.scala @@ -19,7 +19,6 @@ package org.apache.gluten.extension.columnar.enumerated.planner.plan import org.apache.gluten.extension.columnar.enumerated.planner.metadata.GlutenMetadata import org.apache.gluten.extension.columnar.enumerated.planner.property.{Conv, ConvDef} import org.apache.gluten.extension.columnar.transition.{Convention, ConventionReq} -import org.apache.gluten.extension.columnar.transition.Convention.{KnownBatchType, KnownRowType} import org.apache.gluten.ras.{Metadata, PlanModel} import org.apache.gluten.ras.property.PropertySet import org.apache.gluten.sql.shims.SparkShimLoader @@ -43,17 +42,13 @@ object GlutenPlanModel { metadata: GlutenMetadata, constraintSet: PropertySet[SparkPlan]) extends LeafExecNode - with KnownBatchType - with KnownRowType { + with Convention.KnownBatchType + with Convention.KnownRowTypeForSpark33AndLater { private val req: Conv.Req = constraintSet.get(ConvDef).asInstanceOf[Conv.Req] override protected def doExecute(): RDD[InternalRow] = throw new IllegalStateException() override def output: Seq[Attribute] = metadata.schema().output - override def supportsColumnar: Boolean = { - batchType != Convention.BatchType.None - } - override val batchType: Convention.BatchType = { val out = req.req.requiredBatchType match { case ConventionReq.BatchType.Any => Convention.BatchType.None @@ -62,13 +57,21 @@ object GlutenPlanModel { out } - override val rowType: Convention.RowType = { + final override val supportsColumnar: Boolean = { + batchType != Convention.BatchType.None + } + + override val rowType0: Convention.RowType = { val out = req.req.requiredRowType match { case ConventionReq.RowType.Any => Convention.RowType.None case ConventionReq.RowType.Is(r) => r } out } + + final override val supportsRowBased: Boolean = { + rowType() != Convention.RowType.None + } } private object PlanModelImpl extends PlanModel[SparkPlan] { diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/property/Conv.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/property/Conv.scala index 831b212e1f96e..7b2b801ac91e8 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/property/Conv.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/property/Conv.scala @@ -29,6 +29,18 @@ sealed trait Conv extends Property[SparkPlan] { } override def satisfies(other: Property[SparkPlan]): Boolean = { + // The following enforces strict type checking against `this` and `other` + // to make sure: + // + // 1. `this`, which came from user implementation of PropertyDef.getProperty, must be a `Prop` + // 2. `other` which came from user implementation of PropertyDef.getChildrenConstraints, + // must be a `Req` + // + // If the user implementation doesn't follow the criteria, cast error will be thrown. + // + // This can be a common practice to implement a safe Property for RAS. + // + // TODO: Add a similar case to RAS UTs. val req = other.asInstanceOf[Req] if (req.isAny) { return true diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/Convention.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/Convention.scala index 840b62fb672da..b57f3e0c0a29c 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/Convention.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/Convention.scala @@ -16,10 +16,15 @@ */ package org.apache.gluten.extension.columnar.transition +import org.apache.gluten.exception.GlutenException + import org.apache.spark.sql.execution.{ColumnarToRowExec, RowToColumnarExec, SparkPlan} +import org.apache.spark.util.SparkVersionUtil import java.util.concurrent.atomic.AtomicBoolean +import scala.collection.mutable + /** * Convention of a query plan consists of the row data type and columnar data type it supports to * output. @@ -74,6 +79,7 @@ object Convention { } trait BatchType extends TransitionGraph.Vertex with Serializable { + import BatchType._ private val initialized: AtomicBoolean = new AtomicBoolean(false) final def ensureRegistered(): Unit = { @@ -84,7 +90,8 @@ object Convention { register() } - final private def register(): Unit = { + final private def register(): Unit = BatchType.synchronized { + assert(all.add(this)) Transition.graph.addVertex(this) registerTransitions() } @@ -117,6 +124,8 @@ object Convention { } object BatchType { + private val all: mutable.Set[BatchType] = mutable.Set() + def values(): Set[BatchType] = all.toSet // None indicates that the plan doesn't support batch-based processing. final case object None extends BatchType { override protected[this] def registerTransitions(): Unit = {} @@ -133,7 +142,46 @@ object Convention { def batchType(): BatchType } - trait KnownRowType { + sealed trait KnownRowType extends KnownRowType.SupportsRowBasedCompatible { def rowType(): RowType } + + object KnownRowType { + // To be compatible with Spark (version < 3.3) + sealed trait SupportsRowBasedCompatible { + def supportsRowBased(): Boolean = { + throw new GlutenException("Illegal state: The method is not expected to be called") + } + } + } + + trait KnownRowTypeForSpark33AndLater extends KnownRowType { + this: SparkPlan => + import KnownRowTypeForSpark33AndLater._ + + final override def rowType(): RowType = { + if (lteSpark32) { + // It's known that in Spark 3.2, one Spark plan node is considered either only having + // row-based support or only having columnar support at a time. + // Hence, if the plan supports columnar output, we'd disable its row-based support. + // The same for the opposite. + if (supportsColumnar) { + Convention.RowType.None + } else { + Convention.RowType.VanillaRow + } + } else { + rowType0() + } + } + + def rowType0(): RowType + } + + object KnownRowTypeForSpark33AndLater { + private val lteSpark32: Boolean = { + val v = SparkVersionUtil.majorMinorVersion() + SparkVersionUtil.compareMajorMinorVersion(v, (3, 2)) <= 0 + } + } } diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/ConventionFunc.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/ConventionFunc.scala index c3feefe9435ab..5cb3d44a15626 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/ConventionFunc.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/ConventionFunc.scala @@ -21,7 +21,7 @@ import org.apache.gluten.extension.columnar.transition.ConventionReq.KnownChildr import org.apache.gluten.sql.shims.SparkShimLoader import org.apache.spark.sql.execution.{ColumnarToRowExec, SparkPlan, UnionExec} -import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, QueryStageExec} +import org.apache.spark.sql.execution.adaptive.QueryStageExec import org.apache.spark.sql.execution.command.DataWritingCommandExec import org.apache.spark.sql.execution.exchange.ReusedExchangeExec @@ -89,17 +89,6 @@ object ConventionFunc { } case q: QueryStageExec => conventionOf0(q.plan) case r: ReusedExchangeExec => conventionOf0(r.child) - case a: AdaptiveSparkPlanExec => - val rowType = rowTypeOf(a) - val batchType = if (a.supportsColumnar) { - // By default, we execute columnar AQE with backend batch output. - // See org.apache.gluten.extension.columnar.transition.InsertTransitions.apply - Backend.get().defaultBatchType - } else { - Convention.BatchType.None - } - val conv = Convention.of(rowType, batchType) - conv case other => val conv = Convention.of(rowTypeOf(other), batchTypeOf(other)) conv @@ -119,12 +108,24 @@ object ConventionFunc { case _ => Convention.RowType.None } - assert( - out == Convention.RowType.None || plan.isInstanceOf[Convention.KnownRowType] || - SparkShimLoader.getSparkShims.supportsRowBased(plan)) + checkRowType(plan, out) out } + private def checkRowType(plan: SparkPlan, rowType: Convention.RowType): Unit = { + if (SparkShimLoader.getSparkShims.supportsRowBased(plan)) { + assert( + rowType != Convention.RowType.None, + s"Plan ${plan.nodeName} supports row-based execution, " + + s"however #rowTypeOf returns None") + } else { + assert( + rowType == Convention.RowType.None, + s"Plan ${plan.nodeName} doesn't support row-based " + + s"execution, however #rowTypeOf returns $rowType") + } + } + private def batchTypeOf(plan: SparkPlan): Convention.BatchType = { val out = o.batchTypeOf.applyOrElse(plan, batchTypeOf0) out @@ -139,10 +140,24 @@ object ConventionFunc { case _ => Convention.BatchType.None } - assert(out == Convention.BatchType.None || plan.supportsColumnar) + checkBatchType(plan, out) out } + private def checkBatchType(plan: SparkPlan, batchType: Convention.BatchType): Unit = { + if (plan.supportsColumnar) { + assert( + batchType != Convention.BatchType.None, + s"Plan ${plan.nodeName} supports columnar " + + s"execution, however #batchTypeOf returns None") + } else { + assert( + batchType == Convention.BatchType.None, + s"Plan ${plan.nodeName} doesn't support " + + s"columnar execution, however #batchTypeOf returns $batchType") + } + } + override def conventionReqOf(plan: SparkPlan): ConventionReq = { val req = o.conventionReqOf.applyOrElse(plan, conventionReqOf0) req @@ -169,14 +184,15 @@ object ConventionFunc { // To align with ApplyColumnarRulesAndInsertTransitions#insertTransitions ConventionReq.any case u: UnionExec => - // We force vanilla union to output row data to get best compatibility with vanilla Spark. + // We force vanilla union to output row data to get the best compatibility with vanilla + // Spark. // As a result it's a common practice to rewrite it with GlutenPlan for offloading. ConventionReq.of( ConventionReq.RowType.Is(Convention.RowType.VanillaRow), ConventionReq.BatchType.Any) case other => // In the normal case, children's convention should follow parent node's convention. - // Note, we don't have consider C2R / R2C here since they are already removed by + // Note, we don't have to consider C2R / R2C here since they are already removed by // RemoveTransitions. val thisConv = conventionOf0(other) thisConv.asReq() diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/ConventionReq.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/ConventionReq.scala index ce613bf7dbba4..a081f214347bc 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/ConventionReq.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/ConventionReq.scala @@ -16,8 +16,6 @@ */ package org.apache.gluten.extension.columnar.transition -import org.apache.gluten.backend.Backend - import org.apache.spark.sql.execution.SparkPlan /** @@ -53,15 +51,14 @@ object ConventionReq { override val requiredBatchType: BatchType ) extends ConventionReq - val any: ConventionReq = Impl(RowType.Any, BatchType.Any) - val row: ConventionReq = Impl(RowType.Is(Convention.RowType.VanillaRow), BatchType.Any) - val vanillaBatch: ConventionReq = - Impl(RowType.Any, BatchType.Is(Convention.BatchType.VanillaBatch)) - lazy val backendBatch: ConventionReq = - Impl(RowType.Any, BatchType.Is(Backend.get().defaultBatchType)) + val any: ConventionReq = of(RowType.Any, BatchType.Any) + val row: ConventionReq = ofRow(RowType.Is(Convention.RowType.VanillaRow)) + val vanillaBatch: ConventionReq = ofBatch(BatchType.Is(Convention.BatchType.VanillaBatch)) def get(plan: SparkPlan): ConventionReq = ConventionFunc.create().conventionReqOf(plan) def of(rowType: RowType, batchType: BatchType): ConventionReq = Impl(rowType, batchType) + def ofRow(rowType: RowType): ConventionReq = Impl(rowType, BatchType.Any) + def ofBatch(batchType: BatchType): ConventionReq = Impl(RowType.Any, batchType) trait KnownChildrenConventions { def requiredChildrenConventions(): Seq[ConventionReq] diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/Transition.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/Transition.scala index ced9378ad69e9..0a7f635b8bb02 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/Transition.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/Transition.scala @@ -101,22 +101,27 @@ object Transition { case (ConventionReq.RowType.Is(toRowType), ConventionReq.BatchType.Any) => from.rowType match { case Convention.RowType.None => + // Input query plan doesn't have recognizable row-based output, + // find columnar-to-row transition. graph.transitionOfOption(from.batchType, toRowType).getOrElse(orElse) - case fromRowType => + case fromRowType if toRowType == fromRowType => // We have only one single built-in row type. - assert(toRowType == fromRowType) Transition.empty + case _ => + throw new UnsupportedOperationException( + "Row-to-row transition is not yet supported") } case (ConventionReq.RowType.Any, ConventionReq.BatchType.Is(toBatchType)) => from.batchType match { case Convention.BatchType.None => + // Input query plan doesn't have recognizable columnar output, + // find row-to-columnar transition. graph.transitionOfOption(from.rowType, toBatchType).getOrElse(orElse) + case fromBatchType if toBatchType == fromBatchType => + Transition.empty case fromBatchType => - if (toBatchType == fromBatchType) { - Transition.empty - } else { - graph.transitionOfOption(fromBatchType, toBatchType).getOrElse(orElse) - } + // Find columnar-to-columnar transition. + graph.transitionOfOption(fromBatchType, toBatchType).getOrElse(orElse) } case (ConventionReq.RowType.Any, ConventionReq.BatchType.Any) => Transition.empty diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/Transitions.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/Transitions.scala index 2f2840b52b7fd..10d50f453d6d2 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/Transitions.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/Transitions.scala @@ -16,24 +16,22 @@ */ package org.apache.gluten.extension.columnar.transition -import org.apache.gluten.backend.Backend +import org.apache.gluten.extension.columnar.transition.Convention.BatchType import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.SparkPlan import scala.annotation.tailrec -case class InsertTransitions(outputsColumnar: Boolean) extends Rule[SparkPlan] { +case class InsertTransitions(convReq: ConventionReq) extends Rule[SparkPlan] { private val convFunc = ConventionFunc.create() override def apply(plan: SparkPlan): SparkPlan = { // Remove all transitions at first. val removed = RemoveTransitions.apply(plan) val filled = fillWithTransitions(removed) - if (!outputsColumnar) { - return Transitions.toRowPlan(filled) - } - Transitions.toBackendBatchPlan(filled) + val out = Transitions.enforceReq(filled, convReq) + out } private def fillWithTransitions(plan: SparkPlan): SparkPlan = plan.transformUp { @@ -63,6 +61,17 @@ case class InsertTransitions(outputsColumnar: Boolean) extends Rule[SparkPlan] { } } +object InsertTransitions { + def create(outputsColumnar: Boolean, batchType: BatchType): InsertTransitions = { + val conventionReq = if (outputsColumnar) { + ConventionReq.ofBatch(ConventionReq.BatchType.Is(batchType)) + } else { + ConventionReq.row + } + InsertTransitions(conventionReq) + } +} + object RemoveTransitions extends Rule[SparkPlan] { override def apply(plan: SparkPlan): SparkPlan = plan.transformDown { case p => removeForNode(p) } @@ -76,8 +85,8 @@ object RemoveTransitions extends Rule[SparkPlan] { } object Transitions { - def insertTransitions(plan: SparkPlan, outputsColumnar: Boolean): SparkPlan = { - InsertTransitions(outputsColumnar).apply(plan) + def insert(plan: SparkPlan, outputsColumnar: Boolean): SparkPlan = { + InsertTransitions.create(outputsColumnar, BatchType.VanillaBatch).apply(plan) } def toRowPlan(plan: SparkPlan): SparkPlan = { @@ -88,24 +97,13 @@ object Transitions { ConventionReq.BatchType.Any)) } - def toBackendBatchPlan(plan: SparkPlan): SparkPlan = { - val backendBatchType = Backend.get().defaultBatchType - val out = toBatchPlan(plan, backendBatchType) - out - } - - def toVanillaBatchPlan(plan: SparkPlan): SparkPlan = { - val out = toBatchPlan(plan, Convention.BatchType.VanillaBatch) - out - } - - private def toBatchPlan(plan: SparkPlan, toBatchType: Convention.BatchType): SparkPlan = { + def toBatchPlan(plan: SparkPlan, toBatchType: Convention.BatchType): SparkPlan = { enforceReq( plan, ConventionReq.of(ConventionReq.RowType.Any, ConventionReq.BatchType.Is(toBatchType))) } - private def enforceReq(plan: SparkPlan, req: ConventionReq): SparkPlan = { + def enforceReq(plan: SparkPlan, req: ConventionReq): SparkPlan = { val convFunc = ConventionFunc.create() val removed = RemoveTransitions.removeForNode(plan) val transition = Transition diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/injector/GlutenInjector.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/injector/GlutenInjector.scala index 89314dfcf37fb..efe584d44126c 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/injector/GlutenInjector.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/injector/GlutenInjector.scala @@ -150,8 +150,8 @@ object GlutenInjector { private def findCostModel( registry: LongCostModel.Registry, aliasOrClass: String): CostModel[SparkPlan] = { - if (LongCostModel.Kind.values.contains(aliasOrClass)) { - val kind = LongCostModel.Kind.values(aliasOrClass) + if (LongCostModel.Kind.values().contains(aliasOrClass)) { + val kind = LongCostModel.Kind.values()(aliasOrClass) val model = registry.get(kind) return model } diff --git a/gluten-core/src/main/scala/org/apache/gluten/iterator/Iterators.scala b/gluten-core/src/main/scala/org/apache/gluten/iterator/Iterators.scala index 2de1c7b4ed809..ef8ca2c974cbd 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/iterator/Iterators.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/iterator/Iterators.scala @@ -41,7 +41,7 @@ object Iterators { } def wrap[A](in: Iterator[A]): WrapperBuilder[A] = { - wrap(V1, in) + wrap(DEFAULT_VERSION, in) } def wrap[A](version: Version, in: Iterator[A]): WrapperBuilder[A] = { diff --git a/gluten-core/src/main/scala/org/apache/spark/util/SparkVersionUtil.scala b/gluten-core/src/main/scala/org/apache/spark/util/SparkVersionUtil.scala new file mode 100644 index 0000000000000..6864d3caa0319 --- /dev/null +++ b/gluten-core/src/main/scala/org/apache/spark/util/SparkVersionUtil.scala @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.util + +object SparkVersionUtil { + def majorMinorVersion(): (Int, Int) = { + VersionUtils.majorMinorVersion(org.apache.spark.SPARK_VERSION) + } + + // Returns X. X < 0 if one < other, x == 0 if one == other, x > 0 if one > other. + def compareMajorMinorVersion(one: (Int, Int), other: (Int, Int)): Int = { + val base = 1000 + assert(one._2 < base && other._2 < base) + one._1 * base + one._2 - (other._1 * base + other._2) + } +} diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/BackendSettingsApi.scala b/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/BackendSettingsApi.scala index 506424b79cc3f..1eb69da6e5f04 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 @@ -18,6 +18,7 @@ package org.apache.gluten.backendsapi import org.apache.gluten.GlutenConfig import org.apache.gluten.extension.ValidationResult +import org.apache.gluten.extension.columnar.transition.Convention import org.apache.gluten.substrait.rel.LocalFilesNode.ReadFileFormat import org.apache.spark.sql.catalyst.catalog.BucketSpec @@ -28,6 +29,10 @@ import org.apache.spark.sql.execution.datasources.{FileFormat, InsertIntoHadoopF import org.apache.spark.sql.types.StructField trait BackendSettingsApi { + + /** The columnar-batch type this backend is by default using. */ + def primaryBatchType: Convention.BatchType + def validateScanExec( format: ReadFileFormat, fields: Array[StructField], diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/BasicPhysicalOperatorTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/BasicPhysicalOperatorTransformer.scala index 76505da3e0ccd..dbe667ebb2aaf 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/BasicPhysicalOperatorTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/BasicPhysicalOperatorTransformer.scala @@ -263,8 +263,6 @@ case class ColumnarUnionExec(children: Seq[SparkPlan]) extends GlutenPlan { case _ => } - override def supportsColumnar: Boolean = true - override def output: Seq[Attribute] = { children.map(_.output).transpose.map { attrs => diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/CartesianProductExecTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/CartesianProductExecTransformer.scala index 7f7e54e9c72c1..28fb691896fff 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/CartesianProductExecTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/CartesianProductExecTransformer.scala @@ -45,7 +45,6 @@ import java.io.{IOException, ObjectOutputStream} */ case class ColumnarCartesianProductBridge(child: SparkPlan) extends UnaryExecNode with GlutenPlan { override def output: Seq[Attribute] = child.output - override def supportsColumnar: Boolean = true override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() override protected def doExecuteColumnar(): RDD[ColumnarBatch] = child.executeColumnar() diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/ColumnarCoalesceExec.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/ColumnarCoalesceExec.scala index f40a7f8f07f60..3b13207c93506 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/ColumnarCoalesceExec.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/ColumnarCoalesceExec.scala @@ -30,8 +30,6 @@ case class ColumnarCoalesceExec(numPartitions: Int, child: SparkPlan) extends UnaryExecNode with GlutenPlan { - override def supportsColumnar: Boolean = true - override def output: Seq[Attribute] = child.output override def outputPartitioning: Partitioning = { diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/ColumnarToRowExecBase.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/ColumnarToRowExecBase.scala index fd86106bf3677..fae3115981ebc 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/ColumnarToRowExecBase.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/ColumnarToRowExecBase.scala @@ -18,7 +18,7 @@ package org.apache.gluten.execution import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.extension.GlutenPlan -import org.apache.gluten.extension.columnar.transition.ConventionReq +import org.apache.gluten.extension.columnar.transition.{Convention, ConventionReq} import org.apache.gluten.extension.columnar.transition.ConventionReq.KnownChildrenConventions import org.apache.spark.broadcast.Broadcast @@ -43,6 +43,10 @@ abstract class ColumnarToRowExecBase(child: SparkPlan) final override def outputOrdering: Seq[SortOrder] = child.outputOrdering + override def batchType(): Convention.BatchType = Convention.BatchType.None + + override def rowType0(): Convention.RowType = Convention.RowType.VanillaRow + override def doExecuteBroadcast[T](): Broadcast[T] = { // Require for explicit implementation, otherwise throw error. super.doExecuteBroadcast[T]() @@ -55,7 +59,8 @@ abstract class ColumnarToRowExecBase(child: SparkPlan) } override def requiredChildrenConventions(): Seq[ConventionReq] = { - List(ConventionReq.backendBatch) + List( + ConventionReq.ofBatch( + ConventionReq.BatchType.Is(BackendsApiManager.getSettings.primaryBatchType))) } - } diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/RowToColumnarExecBase.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/RowToColumnarExecBase.scala index f4dd160b58b4c..2a52616361a1b 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/RowToColumnarExecBase.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/RowToColumnarExecBase.scala @@ -18,6 +18,7 @@ package org.apache.gluten.execution import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.extension.GlutenPlan +import org.apache.gluten.extension.columnar.transition.Convention import org.apache.spark.broadcast import org.apache.spark.rdd.RDD @@ -45,6 +46,8 @@ abstract class RowToColumnarExecBase(child: SparkPlan) final override def outputOrdering: Seq[SortOrder] = child.outputOrdering + override def rowType0(): Convention.RowType = Convention.RowType.None + final override def doExecute(): RDD[InternalRow] = { child.execute() } @@ -54,8 +57,6 @@ abstract class RowToColumnarExecBase(child: SparkPlan) super.doExecuteBroadcast[T]() } - final override def supportsColumnar: Boolean = true - def doExecuteColumnarInternal(): RDD[ColumnarBatch] override def doExecuteColumnar(): RDD[ColumnarBatch] = { diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/TakeOrderedAndProjectExecTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/TakeOrderedAndProjectExecTransformer.scala index c4e192d1837b5..c960bda2490ee 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/TakeOrderedAndProjectExecTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/TakeOrderedAndProjectExecTransformer.scala @@ -43,7 +43,6 @@ case class TakeOrderedAndProjectExecTransformer( with GlutenPlan { override def outputPartitioning: Partitioning = SinglePartition override def outputOrdering: Seq[SortOrder] = sortOrder - override def supportsColumnar: Boolean = true override def output: Seq[Attribute] = { projectList.map(_.toAttribute) 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 beb7fe5f99d2d..e8a42883a54f7 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 @@ -60,8 +60,6 @@ trait TransformSupport extends GlutenPlan { s"${this.getClass.getSimpleName} doesn't support doExecute") } - final override lazy val supportsColumnar: Boolean = true - /** * Returns all the RDDs of ColumnarBatch which generates the input rows. * 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 a6ca9b1dcc5ca..3639ac522f37e 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 @@ -17,11 +17,11 @@ package org.apache.gluten.extension import org.apache.gluten.GlutenConfig -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.FallbackTag +import org.apache.gluten.extension.columnar.FallbackTag.{Appendable, Converter} import org.apache.gluten.extension.columnar.FallbackTags.add import org.apache.gluten.extension.columnar.transition.Convention import org.apache.gluten.extension.columnar.validator.Validator @@ -34,7 +34,6 @@ import org.apache.gluten.test.TestStats import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.execution.SparkPlan -import FallbackTag.{Appendable, Converter} import com.google.common.collect.Lists sealed trait ValidationResult { @@ -83,7 +82,11 @@ object ValidationResult { } /** Every Gluten Operator should extend this trait. */ -trait GlutenPlan extends SparkPlan with Convention.KnownBatchType with LogLevelUtil { +trait GlutenPlan + extends SparkPlan + with Convention.KnownBatchType + with Convention.KnownRowTypeForSpark33AndLater + with LogLevelUtil { protected lazy val enableNativeValidation = glutenConf.enableNativeValidation protected def glutenConf: GlutenConfig = GlutenConfig.getConf @@ -126,17 +129,20 @@ trait GlutenPlan extends SparkPlan with Convention.KnownBatchType with LogLevelU } } - final override def batchType(): Convention.BatchType = { - if (!supportsColumnar) { - return Convention.BatchType.None - } - val batchType = batchType0() - assert(batchType != Convention.BatchType.None) - batchType + final override val supportsColumnar: Boolean = { + batchType() != Convention.BatchType.None + } + + override def batchType(): Convention.BatchType = { + BackendsApiManager.getSettings.primaryBatchType + } + + final override val supportsRowBased: Boolean = { + rowType() != Convention.RowType.None } - protected def batchType0(): Convention.BatchType = { - Backend.get().defaultBatchType + override def rowType0(): Convention.RowType = { + Convention.RowType.None } protected def doValidateInternal(): ValidationResult = ValidationResult.succeeded 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 a199b5920ca42..e11c6139547a8 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 @@ -16,6 +16,7 @@ */ package org.apache.gluten.extension.columnar +import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.extension.columnar.transition.{ColumnarToRowLike, Transitions} import org.apache.gluten.utils.PlanUtil @@ -106,7 +107,8 @@ object MiscColumnarRules { private def toColumnarBroadcastExchange( exchange: BroadcastExchangeExec): ColumnarBroadcastExchangeExec = { - val newChild = Transitions.toBackendBatchPlan(exchange.child) + val newChild = + Transitions.toBatchPlan(exchange.child, BackendsApiManager.getSettings.primaryBatchType) ColumnarBroadcastExchangeExec(exchange.mode, newChild) } diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/ExpandFallbackPolicy.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/ExpandFallbackPolicy.scala index e1c8b6f41f448..44ed81f565816 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/ExpandFallbackPolicy.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/ExpandFallbackPolicy.scala @@ -20,7 +20,7 @@ import org.apache.gluten.GlutenConfig import org.apache.gluten.extension.GlutenPlan import org.apache.gluten.extension.columnar.{FallbackTag, FallbackTags} import org.apache.gluten.extension.columnar.FallbackTags.add -import org.apache.gluten.extension.columnar.transition.{ColumnarToRowLike, RowToColumnarLike, Transitions} +import org.apache.gluten.extension.columnar.transition.{BackendTransitions, ColumnarToRowLike, RowToColumnarLike} import org.apache.gluten.utils.PlanUtil import org.apache.spark.sql.catalyst.rules.Rule @@ -32,6 +32,9 @@ import org.apache.spark.sql.execution.command.ExecutedCommandExec import org.apache.spark.sql.execution.exchange.Exchange + + + // format: off /** * Note, this rule should only fallback to row-based plan if there is no harm. @@ -226,7 +229,7 @@ case class ExpandFallbackPolicy(isAdaptiveContext: Boolean, originalPlan: SparkP case _ => } - val planWithTransitions = Transitions.insertTransitions(originalPlan, outputsColumnar) + val planWithTransitions = BackendTransitions.insert(originalPlan, outputsColumnar) planWithTransitions } diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/transition/BackendTransitions.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/transition/BackendTransitions.scala new file mode 100644 index 0000000000000..86d4b40d55d87 --- /dev/null +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/transition/BackendTransitions.scala @@ -0,0 +1,36 @@ +/* + * 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.transition + +import org.apache.gluten.backendsapi.BackendsApiManager + +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.SparkPlan + +case class InsertBackendTransitions(outputsColumnar: Boolean) extends Rule[SparkPlan] { + def apply(plan: SparkPlan): SparkPlan = { + InsertTransitions + .create(outputsColumnar, BackendsApiManager.getSettings.primaryBatchType) + .apply(plan) + } +} + +object BackendTransitions { + def insert(plan: SparkPlan, outputsColumnar: Boolean): SparkPlan = { + InsertBackendTransitions(outputsColumnar)(plan) + } +} diff --git a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarBroadcastExchangeExec.scala b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarBroadcastExchangeExec.scala index d55733fe4e971..01a4380a148f5 100644 --- a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarBroadcastExchangeExec.scala +++ b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarBroadcastExchangeExec.scala @@ -121,8 +121,6 @@ case class ColumnarBroadcastExchangeExec(mode: BroadcastMode, child: SparkPlan) @transient private val timeout: Long = SQLConf.get.broadcastTimeout - override def supportsColumnar: Boolean = true - override def output: Seq[Attribute] = child.output override def outputPartitioning: Partitioning = BroadcastPartitioning(mode) diff --git a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarCollapseTransformStages.scala b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarCollapseTransformStages.scala index 5ea5e4159ddac..9ec078e003da2 100644 --- a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarCollapseTransformStages.scala +++ b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarCollapseTransformStages.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.execution import org.apache.gluten.GlutenConfig -import org.apache.gluten.backend.Backend import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.execution._ import org.apache.gluten.extension.columnar.transition.Convention @@ -178,9 +177,9 @@ case class ColumnarInputAdapter(child: SparkPlan) extends InputAdapterGenerateTreeStringShim with Convention.KnownBatchType { override def output: Seq[Attribute] = child.output - override def supportsColumnar: Boolean = true + override val supportsColumnar: Boolean = true override def batchType(): Convention.BatchType = - Backend.get().defaultBatchType + BackendsApiManager.getSettings.primaryBatchType override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() override protected def doExecuteColumnar(): RDD[ColumnarBatch] = child.executeColumnar() override def outputPartitioning: Partitioning = child.outputPartitioning diff --git a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarShuffleExchangeExec.scala b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarShuffleExchangeExec.scala index 4f62377b09e36..d4b33be292dfc 100644 --- a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarShuffleExchangeExec.scala +++ b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarShuffleExchangeExec.scala @@ -126,7 +126,6 @@ case class ColumnarShuffleExchangeExec( override def nodeName: String = "ColumnarExchange" - override def supportsColumnar: Boolean = true override def numMappers: Int = shuffleDependency.rdd.getNumPartitions override def numPartitions: Int = shuffleDependency.partitioner.numPartitions diff --git a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarWriteFilesExec.scala b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarWriteFilesExec.scala index fcd82d8c19870..25d6c4ed61e7c 100644 --- a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarWriteFilesExec.scala +++ b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarWriteFilesExec.scala @@ -19,9 +19,8 @@ package org.apache.spark.sql.execution import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.exception.GlutenException import org.apache.gluten.extension.GlutenPlan -import org.apache.gluten.extension.columnar.transition.Convention.{KnownRowType, RowType} +import org.apache.gluten.extension.columnar.transition.Convention.RowType import org.apache.gluten.extension.columnar.transition.ConventionReq -import org.apache.gluten.extension.columnar.transition.ConventionReq.KnownChildrenConventions import org.apache.gluten.sql.shims.SparkShimLoader import org.apache.spark.TaskContext @@ -45,18 +44,17 @@ abstract class ColumnarWriteFilesExec protected ( override val right: SparkPlan) extends BinaryExecNode with GlutenPlan - with KnownChildrenConventions - with KnownRowType + with ConventionReq.KnownChildrenConventions with ColumnarWriteFilesExec.ExecuteWriteCompatible { val child: SparkPlan = left override lazy val references: AttributeSet = AttributeSet.empty - override def supportsColumnar: Boolean = true - override def requiredChildrenConventions(): Seq[ConventionReq] = { - List(ConventionReq.backendBatch) + List( + ConventionReq.ofBatch( + ConventionReq.BatchType.Is(BackendsApiManager.getSettings.primaryBatchType))) } /** @@ -69,7 +67,7 @@ abstract class ColumnarWriteFilesExec protected ( * * Since https://github.com/apache/incubator-gluten/pull/6745. */ - override def rowType(): RowType = { + override def rowType0(): RowType = { RowType.VanillaRow } diff --git a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/datasources/GlutenWriterColumnarRules.scala b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/datasources/GlutenWriterColumnarRules.scala index 917f6c03dfe04..fb42c55ba0a70 100644 --- a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/datasources/GlutenWriterColumnarRules.scala +++ b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/datasources/GlutenWriterColumnarRules.scala @@ -20,7 +20,7 @@ import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.execution.ColumnarToRowExecBase import org.apache.gluten.execution.datasource.GlutenFormatFactory import org.apache.gluten.extension.GlutenPlan -import org.apache.gluten.extension.columnar.transition.Transitions +import org.apache.gluten.extension.columnar.transition.{Convention, Transitions} import org.apache.spark.rdd.RDD import org.apache.spark.sql.SparkSession @@ -61,6 +61,8 @@ case class FakeRowAdaptor(child: SparkPlan) override def output: Seq[Attribute] = child.output + override def rowType0(): Convention.RowType = Convention.RowType.VanillaRow + override protected def doExecute(): RDD[InternalRow] = { doExecuteColumnar().map(cb => new FakeRow(cb)) } @@ -74,7 +76,7 @@ case class FakeRowAdaptor(child: SparkPlan) if (child.supportsColumnar) { child.executeColumnar() } else { - val r2c = Transitions.toBackendBatchPlan(child) + val r2c = Transitions.toBatchPlan(child, BackendsApiManager.getSettings.primaryBatchType) r2c.executeColumnar() } } diff --git a/gluten-substrait/src/test/scala/org/apache/gluten/extension/columnar/transition/TransitionSuite.scala b/gluten-substrait/src/test/scala/org/apache/gluten/extension/columnar/transition/TransitionSuite.scala index 9712bd2c219f3..5daca9bede9e4 100644 --- a/gluten-substrait/src/test/scala/org/apache/gluten/extension/columnar/transition/TransitionSuite.scala +++ b/gluten-substrait/src/test/scala/org/apache/gluten/extension/columnar/transition/TransitionSuite.scala @@ -32,7 +32,7 @@ class TransitionSuite extends SharedSparkSession { test("Trivial C2R") { val in = BatchLeaf(TypeA) val out = ConventionFunc.ignoreBackend { - Transitions.insertTransitions(in, outputsColumnar = false) + Transitions.insert(in, outputsColumnar = false) } assert(out == BatchToRow(TypeA, BatchLeaf(TypeA))) } @@ -40,7 +40,7 @@ class TransitionSuite extends SharedSparkSession { test("Insert C2R") { val in = RowUnary(BatchLeaf(TypeA)) val out = ConventionFunc.ignoreBackend { - Transitions.insertTransitions(in, outputsColumnar = false) + Transitions.insert(in, outputsColumnar = false) } assert(out == RowUnary(BatchToRow(TypeA, BatchLeaf(TypeA)))) } @@ -48,7 +48,7 @@ class TransitionSuite extends SharedSparkSession { test("Insert R2C") { val in = BatchUnary(TypeA, RowLeaf()) val out = ConventionFunc.ignoreBackend { - Transitions.insertTransitions(in, outputsColumnar = false) + Transitions.insert(in, outputsColumnar = false) } assert(out == BatchToRow(TypeA, BatchUnary(TypeA, RowToBatch(TypeA, RowLeaf())))) } @@ -56,7 +56,7 @@ class TransitionSuite extends SharedSparkSession { test("Insert C2R2C") { val in = BatchUnary(TypeA, BatchLeaf(TypeB)) val out = ConventionFunc.ignoreBackend { - Transitions.insertTransitions(in, outputsColumnar = false) + Transitions.insert(in, outputsColumnar = false) } assert( out == BatchToRow( @@ -67,7 +67,7 @@ class TransitionSuite extends SharedSparkSession { test("Insert C2C") { val in = BatchUnary(TypeA, BatchLeaf(TypeC)) val out = ConventionFunc.ignoreBackend { - Transitions.insertTransitions(in, outputsColumnar = false) + Transitions.insert(in, outputsColumnar = false) } assert( out == BatchToRow( @@ -79,7 +79,7 @@ class TransitionSuite extends SharedSparkSession { val in = BatchUnary(TypeA, BatchLeaf(TypeD)) assertThrows[GlutenException] { ConventionFunc.ignoreBackend { - Transitions.insertTransitions(in, outputsColumnar = false) + Transitions.insert(in, outputsColumnar = false) } } } @@ -116,8 +116,7 @@ object TransitionSuite extends TransitionSuiteBase { case class RowToBatch(toBatchType: Convention.BatchType, override val child: SparkPlan) extends RowToColumnarTransition with GlutenPlan { - override def supportsColumnar: Boolean = true - override protected def batchType0(): Convention.BatchType = toBatchType + override def batchType(): Convention.BatchType = toBatchType override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan = copy(child = newChild) override protected def doExecute(): RDD[InternalRow] = diff --git a/gluten-substrait/src/test/scala/org/apache/gluten/extension/columnar/transition/TransitionSuiteBase.scala b/gluten-substrait/src/test/scala/org/apache/gluten/extension/columnar/transition/TransitionSuiteBase.scala index d82cc3aac9f12..43805b3d659b2 100644 --- a/gluten-substrait/src/test/scala/org/apache/gluten/extension/columnar/transition/TransitionSuiteBase.scala +++ b/gluten-substrait/src/test/scala/org/apache/gluten/extension/columnar/transition/TransitionSuiteBase.scala @@ -24,22 +24,18 @@ import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.execution.{BinaryExecNode, LeafExecNode, SparkPlan, UnaryExecNode} trait TransitionSuiteBase { - case class BatchLeaf(override val batchType0: Convention.BatchType) + case class BatchLeaf(override val batchType: Convention.BatchType) extends LeafExecNode with GlutenPlan { - override def supportsColumnar: Boolean = true override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() override def output: Seq[Attribute] = List.empty } - case class BatchUnary( - override val batchType0: Convention.BatchType, - override val child: SparkPlan) + case class BatchUnary(override val batchType: Convention.BatchType, override val child: SparkPlan) extends UnaryExecNode with GlutenPlan { - override def supportsColumnar: Boolean = true override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan = copy(child = newChild) @@ -50,12 +46,11 @@ trait TransitionSuiteBase { } case class BatchBinary( - override val batchType0: Convention.BatchType, + override val batchType: Convention.BatchType, override val left: SparkPlan, override val right: SparkPlan) extends BinaryExecNode with GlutenPlan { - override def supportsColumnar: Boolean = true override protected def withNewChildrenInternal( newLeft: SparkPlan, diff --git a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala index 7abe8228facf9..7d4315e8d707d 100644 --- a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala +++ b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala @@ -23,7 +23,7 @@ import org.apache.gluten.extension.columnar.ColumnarRuleApplier.ColumnarRuleCall import org.apache.gluten.extension.columnar.MiscColumnarRules.RemoveTopmostColumnarToRow import org.apache.gluten.extension.columnar.RemoveFallbackTagRule import org.apache.gluten.extension.columnar.heuristic.{ExpandFallbackPolicy, HeuristicApplier} -import org.apache.gluten.extension.columnar.transition.InsertTransitions +import org.apache.gluten.extension.columnar.transition.InsertBackendTransitions import org.apache.spark.rdd.RDD import org.apache.spark.sql.{GlutenSQLTestsTrait, SparkSession} @@ -44,7 +44,7 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait { _ => { UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOp())))) }, - c => InsertTransitions(c.outputsColumnar))) + c => InsertBackendTransitions(c.outputsColumnar))) val outputPlan = rule.apply(originalPlan, false) // Expect to fall back the entire plan. assert(outputPlan == originalPlan) @@ -61,7 +61,7 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait { _ => { UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOp())))) }, - c => InsertTransitions(c.outputsColumnar))) + c => InsertBackendTransitions(c.outputsColumnar))) .enableAdaptiveContext() val outputPlan = rule.apply(originalPlan, false) // Expect to fall back the entire plan. @@ -79,7 +79,7 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait { _ => { UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOp())))) }, - c => InsertTransitions(c.outputsColumnar))) + c => InsertBackendTransitions(c.outputsColumnar))) .enableAdaptiveContext() val outputPlan = rule.apply(originalPlan, false) // Expect to get the plan with columnar rule applied. @@ -99,7 +99,7 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait { _ => { UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOpTransformer())))) }, - c => InsertTransitions(c.outputsColumnar))) + c => InsertBackendTransitions(c.outputsColumnar))) .enableAdaptiveContext() val outputPlan = rule.apply(originalPlan, false) // Expect to fall back the entire plan. @@ -119,7 +119,7 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait { _ => { UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOpTransformer())))) }, - c => InsertTransitions(c.outputsColumnar))) + c => InsertBackendTransitions(c.outputsColumnar))) .enableAdaptiveContext() val outputPlan = rule.apply(originalPlan, false) // Expect to get the plan with columnar rule applied. @@ -199,17 +199,13 @@ private object FallbackStrategiesSuite { } // For replacing LeafOp. - case class LeafOpTransformer(override val supportsColumnar: Boolean = true) - extends LeafExecNode - with GlutenPlan { + case class LeafOpTransformer() extends LeafExecNode with GlutenPlan { override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() override def output: Seq[Attribute] = Seq.empty } // For replacing UnaryOp1. - case class UnaryOp1Transformer( - override val child: SparkPlan, - override val supportsColumnar: Boolean = true) + case class UnaryOp1Transformer(override val child: SparkPlan) extends UnaryExecNode with GlutenPlan { override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala index d03619a2e07aa..e8cc7898c2d47 100644 --- a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala @@ -23,7 +23,7 @@ import org.apache.gluten.extension.columnar.{FallbackTags, RemoveFallbackTagRule import org.apache.gluten.extension.columnar.ColumnarRuleApplier.ColumnarRuleCall import org.apache.gluten.extension.columnar.MiscColumnarRules.RemoveTopmostColumnarToRow import org.apache.gluten.extension.columnar.heuristic.{ExpandFallbackPolicy, HeuristicApplier} -import org.apache.gluten.extension.columnar.transition.InsertTransitions +import org.apache.gluten.extension.columnar.transition.InsertBackendTransitions import org.apache.spark.rdd.RDD import org.apache.spark.sql.{GlutenSQLTestsTrait, SparkSession} @@ -43,7 +43,7 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait { _ => { UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOp())))) }, - c => InsertTransitions(c.outputsColumnar))) + c => InsertBackendTransitions(c.outputsColumnar))) val outputPlan = rule.apply(originalPlan, false) // Expect to fall back the entire plan. assert(outputPlan == originalPlan) @@ -60,7 +60,7 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait { _ => { UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOp())))) }, - c => InsertTransitions(c.outputsColumnar))) + c => InsertBackendTransitions(c.outputsColumnar))) .enableAdaptiveContext() val outputPlan = rule.apply(originalPlan, false) // Expect to fall back the entire plan. @@ -78,7 +78,7 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait { _ => { UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOp())))) }, - c => InsertTransitions(c.outputsColumnar))) + c => InsertBackendTransitions(c.outputsColumnar))) .enableAdaptiveContext() val outputPlan = rule.apply(originalPlan, false) // Expect to get the plan with columnar rule applied. @@ -98,7 +98,7 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait { _ => { UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOpTransformer())))) }, - c => InsertTransitions(c.outputsColumnar))) + c => InsertBackendTransitions(c.outputsColumnar))) .enableAdaptiveContext() val outputPlan = rule.apply(originalPlan, false) // Expect to fall back the entire plan. @@ -118,7 +118,7 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait { _ => { UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOpTransformer())))) }, - c => InsertTransitions(c.outputsColumnar))) + c => InsertBackendTransitions(c.outputsColumnar))) .enableAdaptiveContext() val outputPlan = rule.apply(originalPlan, false) // Expect to get the plan with columnar rule applied. @@ -228,17 +228,13 @@ private object FallbackStrategiesSuite { } // For replacing LeafOp. - case class LeafOpTransformer(override val supportsColumnar: Boolean = true) - extends LeafExecNode - with GlutenPlan { + case class LeafOpTransformer() extends LeafExecNode with GlutenPlan { override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() override def output: Seq[Attribute] = Seq.empty } // For replacing UnaryOp1. - case class UnaryOp1Transformer( - override val child: SparkPlan, - override val supportsColumnar: Boolean = true) + case class UnaryOp1Transformer(override val child: SparkPlan) extends UnaryExecNode with GlutenPlan { override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala index d03619a2e07aa..e8cc7898c2d47 100644 --- a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala @@ -23,7 +23,7 @@ import org.apache.gluten.extension.columnar.{FallbackTags, RemoveFallbackTagRule import org.apache.gluten.extension.columnar.ColumnarRuleApplier.ColumnarRuleCall import org.apache.gluten.extension.columnar.MiscColumnarRules.RemoveTopmostColumnarToRow import org.apache.gluten.extension.columnar.heuristic.{ExpandFallbackPolicy, HeuristicApplier} -import org.apache.gluten.extension.columnar.transition.InsertTransitions +import org.apache.gluten.extension.columnar.transition.InsertBackendTransitions import org.apache.spark.rdd.RDD import org.apache.spark.sql.{GlutenSQLTestsTrait, SparkSession} @@ -43,7 +43,7 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait { _ => { UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOp())))) }, - c => InsertTransitions(c.outputsColumnar))) + c => InsertBackendTransitions(c.outputsColumnar))) val outputPlan = rule.apply(originalPlan, false) // Expect to fall back the entire plan. assert(outputPlan == originalPlan) @@ -60,7 +60,7 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait { _ => { UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOp())))) }, - c => InsertTransitions(c.outputsColumnar))) + c => InsertBackendTransitions(c.outputsColumnar))) .enableAdaptiveContext() val outputPlan = rule.apply(originalPlan, false) // Expect to fall back the entire plan. @@ -78,7 +78,7 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait { _ => { UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOp())))) }, - c => InsertTransitions(c.outputsColumnar))) + c => InsertBackendTransitions(c.outputsColumnar))) .enableAdaptiveContext() val outputPlan = rule.apply(originalPlan, false) // Expect to get the plan with columnar rule applied. @@ -98,7 +98,7 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait { _ => { UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOpTransformer())))) }, - c => InsertTransitions(c.outputsColumnar))) + c => InsertBackendTransitions(c.outputsColumnar))) .enableAdaptiveContext() val outputPlan = rule.apply(originalPlan, false) // Expect to fall back the entire plan. @@ -118,7 +118,7 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait { _ => { UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOpTransformer())))) }, - c => InsertTransitions(c.outputsColumnar))) + c => InsertBackendTransitions(c.outputsColumnar))) .enableAdaptiveContext() val outputPlan = rule.apply(originalPlan, false) // Expect to get the plan with columnar rule applied. @@ -228,17 +228,13 @@ private object FallbackStrategiesSuite { } // For replacing LeafOp. - case class LeafOpTransformer(override val supportsColumnar: Boolean = true) - extends LeafExecNode - with GlutenPlan { + case class LeafOpTransformer() extends LeafExecNode with GlutenPlan { override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() override def output: Seq[Attribute] = Seq.empty } // For replacing UnaryOp1. - case class UnaryOp1Transformer( - override val child: SparkPlan, - override val supportsColumnar: Boolean = true) + case class UnaryOp1Transformer(override val child: SparkPlan) extends UnaryExecNode with GlutenPlan { override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/benchmarks/ParquetReadBenchmark.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/benchmarks/ParquetReadBenchmark.scala index 640b8cdc6f8f5..cdfb63aa20744 100644 --- a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/benchmarks/ParquetReadBenchmark.scala +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/benchmarks/ParquetReadBenchmark.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.execution.benchmarks import org.apache.gluten.GlutenConfig +import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.execution.{FileSourceScanExecTransformer, WholeStageTransformer} import org.apache.gluten.extension.columnar.transition.Transitions import org.apache.gluten.jni.JniLibLoader @@ -125,7 +126,7 @@ object ParquetReadBenchmark extends SqlBasedBenchmark { // generate ColumnarToRow val columnarToRowPlan = - Transitions.toBackendBatchPlan(newWholeStage) + Transitions.toBatchPlan(newWholeStage, BackendsApiManager.getSettings.primaryBatchType) val newWholeStageRDD = newWholeStage.executeColumnar() val newColumnarToRowRDD = columnarToRowPlan.execute() diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala index f43652a7d4ee4..1d45e8a672e3a 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala @@ -23,7 +23,7 @@ import org.apache.gluten.extension.columnar.{FallbackTags, RemoveFallbackTagRule import org.apache.gluten.extension.columnar.ColumnarRuleApplier.ColumnarRuleCall import org.apache.gluten.extension.columnar.MiscColumnarRules.RemoveTopmostColumnarToRow import org.apache.gluten.extension.columnar.heuristic.{ExpandFallbackPolicy, HeuristicApplier} -import org.apache.gluten.extension.columnar.transition.InsertTransitions +import org.apache.gluten.extension.columnar.transition.InsertBackendTransitions import org.apache.spark.rdd.RDD import org.apache.spark.sql.{GlutenSQLTestsTrait, SparkSession} @@ -44,7 +44,7 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait { _ => { UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOp())))) }, - c => InsertTransitions(c.outputsColumnar))) + c => InsertBackendTransitions(c.outputsColumnar))) val outputPlan = rule.apply(originalPlan, false) // Expect to fall back the entire plan. assert(outputPlan == originalPlan) @@ -61,7 +61,7 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait { _ => { UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOp())))) }, - c => InsertTransitions(c.outputsColumnar))) + c => InsertBackendTransitions(c.outputsColumnar))) .enableAdaptiveContext() val outputPlan = rule.apply(originalPlan, false) // Expect to fall back the entire plan. @@ -79,7 +79,7 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait { _ => { UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOp())))) }, - c => InsertTransitions(c.outputsColumnar))) + c => InsertBackendTransitions(c.outputsColumnar))) .enableAdaptiveContext() val outputPlan = rule.apply(originalPlan, false) // Expect to get the plan with columnar rule applied. @@ -99,7 +99,7 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait { _ => { UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOpTransformer())))) }, - c => InsertTransitions(c.outputsColumnar))) + c => InsertBackendTransitions(c.outputsColumnar))) .enableAdaptiveContext() val outputPlan = rule.apply(originalPlan, false) // Expect to fall back the entire plan. @@ -119,7 +119,7 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait { _ => { UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOpTransformer())))) }, - c => InsertTransitions(c.outputsColumnar))) + c => InsertBackendTransitions(c.outputsColumnar))) .enableAdaptiveContext() val outputPlan = rule.apply(originalPlan, false) // Expect to get the plan with columnar rule applied. @@ -229,17 +229,13 @@ private object FallbackStrategiesSuite { } // For replacing LeafOp. - case class LeafOpTransformer(override val supportsColumnar: Boolean = true) - extends LeafExecNode - with GlutenPlan { + case class LeafOpTransformer() extends LeafExecNode with GlutenPlan { override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() override def output: Seq[Attribute] = Seq.empty } // For replacing UnaryOp1. - case class UnaryOp1Transformer( - override val child: SparkPlan, - override val supportsColumnar: Boolean = true) + case class UnaryOp1Transformer(override val child: SparkPlan) extends UnaryExecNode with GlutenPlan { override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() diff --git a/shims/spark32/src/main/scala/org/apache/spark/sql/execution/AbstractFileSourceScanExec.scala b/shims/spark32/src/main/scala/org/apache/spark/sql/execution/AbstractFileSourceScanExec.scala index 2bdde3b4aaa01..a3bd5079b016a 100644 --- a/shims/spark32/src/main/scala/org/apache/spark/sql/execution/AbstractFileSourceScanExec.scala +++ b/shims/spark32/src/main/scala/org/apache/spark/sql/execution/AbstractFileSourceScanExec.scala @@ -71,10 +71,9 @@ abstract class AbstractFileSourceScanExec( disableBucketedScan: Boolean = false) extends DataSourceScanExec { - // Note that some vals referring the file-based relation are lazy intentionally - // so that this plan can be canonicalized on executor side too. See SPARK-23731. - override lazy val supportsColumnar: Boolean = { - relation.fileFormat.supportBatch(relation.sparkSession, schema) + override def supportsColumnar: Boolean = { + // The value should be defined in GlutenPlan. + throw new UnsupportedOperationException("Unreachable code") } private lazy val needsUnsafeRowConversion: Boolean = { diff --git a/shims/spark33/src/main/scala/org/apache/spark/sql/execution/AbstractFileSourceScanExec.scala b/shims/spark33/src/main/scala/org/apache/spark/sql/execution/AbstractFileSourceScanExec.scala index 6b3d679fcdd48..c885f0cf44b3e 100644 --- a/shims/spark33/src/main/scala/org/apache/spark/sql/execution/AbstractFileSourceScanExec.scala +++ b/shims/spark33/src/main/scala/org/apache/spark/sql/execution/AbstractFileSourceScanExec.scala @@ -75,10 +75,9 @@ abstract class AbstractFileSourceScanExec( lazy val metadataColumns: Seq[AttributeReference] = output.collect { case FileSourceMetadataAttribute(attr) => attr } - // Note that some vals referring the file-based relation are lazy intentionally - // so that this plan can be canonicalized on executor side too. See SPARK-23731. - override lazy val supportsColumnar: Boolean = { - relation.fileFormat.supportBatch(relation.sparkSession, schema) + override def supportsColumnar: Boolean = { + // The value should be defined in GlutenPlan. + throw new UnsupportedOperationException("Unreachable code") } private lazy val needsUnsafeRowConversion: Boolean = { diff --git a/shims/spark34/src/main/scala/org/apache/spark/sql/execution/AbstractFileSourceScanExec.scala b/shims/spark34/src/main/scala/org/apache/spark/sql/execution/AbstractFileSourceScanExec.scala index 5e751861861aa..53ea6f543a95e 100644 --- a/shims/spark34/src/main/scala/org/apache/spark/sql/execution/AbstractFileSourceScanExec.scala +++ b/shims/spark34/src/main/scala/org/apache/spark/sql/execution/AbstractFileSourceScanExec.scala @@ -67,15 +67,9 @@ abstract class AbstractFileSourceScanExec( override val disableBucketedScan: Boolean = false) extends FileSourceScanLike { - // Note that some vals referring the file-based relation are lazy intentionally - // so that this plan can be canonicalized on executor side too. See SPARK-23731. - override lazy val supportsColumnar: Boolean = { - val conf = relation.sparkSession.sessionState.conf - // Only output columnar if there is WSCG to read it. - val requiredWholeStageCodegenSettings = - conf.wholeStageEnabled && !WholeStageCodegenExec.isTooManyFields(conf, schema) - requiredWholeStageCodegenSettings && - relation.fileFormat.supportBatch(relation.sparkSession, schema) + override def supportsColumnar: Boolean = { + // The value should be defined in GlutenPlan. + throw new UnsupportedOperationException("Unreachable code") } private lazy val needsUnsafeRowConversion: Boolean = { diff --git a/shims/spark35/src/main/scala/org/apache/spark/sql/execution/AbstractFileSourceScanExec.scala b/shims/spark35/src/main/scala/org/apache/spark/sql/execution/AbstractFileSourceScanExec.scala index 32cbd435b013a..c8dbcc2fed4f4 100644 --- a/shims/spark35/src/main/scala/org/apache/spark/sql/execution/AbstractFileSourceScanExec.scala +++ b/shims/spark35/src/main/scala/org/apache/spark/sql/execution/AbstractFileSourceScanExec.scala @@ -67,15 +67,9 @@ abstract class AbstractFileSourceScanExec( override val disableBucketedScan: Boolean = false) extends FileSourceScanLike { - // Note that some vals referring the file-based relation are lazy intentionally - // so that this plan can be canonicalized on executor side too. See SPARK-23731. - override lazy val supportsColumnar: Boolean = { - val conf = relation.sparkSession.sessionState.conf - // Only output columnar if there is WSCG to read it. - val requiredWholeStageCodegenSettings = - conf.wholeStageEnabled && !WholeStageCodegenExec.isTooManyFields(conf, schema) - requiredWholeStageCodegenSettings && - relation.fileFormat.supportBatch(relation.sparkSession, schema) + override def supportsColumnar: Boolean = { + // The value should be defined in GlutenPlan. + throw new UnsupportedOperationException("Unreachable code") } private lazy val needsUnsafeRowConversion: Boolean = { From e3682fdafb505af87ad1c1605908c5ec30463a9e Mon Sep 17 00:00:00 2001 From: Mingliang Zhu Date: Mon, 25 Nov 2024 16:22:40 +0800 Subject: [PATCH 04/13] [GLUTEN-8010][CORE] Don't generate native metrics if transformer don't generate relNode (#8011) --- .../org/apache/gluten/metrics/MetricsUtil.scala | 3 +++ .../BasicPhysicalOperatorTransformer.scala | 15 ++++++++++----- .../gluten/execution/ExpandExecTransformer.scala | 12 ++++++++---- .../gluten/execution/SortExecTransformer.scala | 12 ++++++++---- .../gluten/execution/WholeStageTransformer.scala | 3 +++ .../gluten/execution/WindowExecTransformer.scala | 12 ++++++++---- .../gluten/substrait/SubstraitContext.scala | 13 ------------- 7 files changed, 40 insertions(+), 30 deletions(-) diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/metrics/MetricsUtil.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/metrics/MetricsUtil.scala index e1e0f7c11a09a..7d81467e978f3 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/metrics/MetricsUtil.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/metrics/MetricsUtil.scala @@ -38,6 +38,9 @@ object MetricsUtil extends Logging { j.metricsUpdater(), // must put the buildPlan first Seq(treeifyMetricsUpdaters(j.buildPlan), treeifyMetricsUpdaters(j.streamedPlan))) + case t: TransformSupport if t.metricsUpdater() == MetricsUpdater.None => + assert(t.children.size == 1, "MetricsUpdater.None can only be used on unary operator") + treeifyMetricsUpdaters(t.children.head) case t: TransformSupport => MetricsUpdaterTree(t.metricsUpdater(), t.children.map(treeifyMetricsUpdaters)) case _ => diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/BasicPhysicalOperatorTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/BasicPhysicalOperatorTransformer.scala index dbe667ebb2aaf..2830ef404c0e7 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/BasicPhysicalOperatorTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/BasicPhysicalOperatorTransformer.scala @@ -61,8 +61,13 @@ abstract class FilterExecTransformerBase(val cond: Expression, val input: SparkP case _ => false } - override def metricsUpdater(): MetricsUpdater = + override def isNoop: Boolean = getRemainingCondition == null + + override def metricsUpdater(): MetricsUpdater = if (isNoop) { + MetricsUpdater.None + } else { BackendsApiManager.getMetricsApiInstance.genFilterTransformerMetricsUpdater(metrics) + } def getRelNode( context: SubstraitContext, @@ -149,15 +154,15 @@ abstract class FilterExecTransformerBase(val cond: Expression, val input: SparkP override protected def doTransform(context: SubstraitContext): TransformContext = { val childCtx = child.asInstanceOf[TransformSupport].transform(context) - val remainingCondition = getRemainingCondition - val operatorId = context.nextOperatorId(this.nodeName) - if (remainingCondition == null) { + if (isNoop) { // The computing for this filter is not needed. - context.registerEmptyRelToOperator(operatorId) // Since some columns' nullability will be removed after this filter, we need to update the // outputAttributes of child context. return TransformContext(output, childCtx.root) } + + val operatorId = context.nextOperatorId(this.nodeName) + val remainingCondition = getRemainingCondition val currRel = getRelNode( context, remainingCondition, diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/ExpandExecTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/ExpandExecTransformer.scala index b600175b28269..c6936daaffe5c 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/ExpandExecTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/ExpandExecTransformer.scala @@ -48,8 +48,13 @@ case class ExpandExecTransformer( AttributeSet.fromAttributeSets(projections.flatten.map(_.references)) } - override def metricsUpdater(): MetricsUpdater = + override def isNoop: Boolean = projections == null || projections.isEmpty + + override def metricsUpdater(): MetricsUpdater = if (isNoop) { + MetricsUpdater.None + } else { BackendsApiManager.getMetricsApiInstance.genExpandTransformerMetricsUpdater(metrics) + } // The GroupExpressions can output data with arbitrary partitioning, so set it // as UNKNOWN partitioning @@ -112,13 +117,12 @@ case class ExpandExecTransformer( override protected def doTransform(context: SubstraitContext): TransformContext = { val childCtx = child.asInstanceOf[TransformSupport].transform(context) - val operatorId = context.nextOperatorId(this.nodeName) - if (projections == null || projections.isEmpty) { + if (isNoop) { // The computing for this Expand is not needed. - context.registerEmptyRelToOperator(operatorId) return childCtx } + val operatorId = context.nextOperatorId(this.nodeName) val currRel = getRelNode(context, projections, child.output, operatorId, childCtx.root, validation = false) assert(currRel != null, "Expand Rel should be valid") diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/SortExecTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/SortExecTransformer.scala index c62a30b846323..6f9564e6d54fc 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/SortExecTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/SortExecTransformer.scala @@ -44,8 +44,13 @@ case class SortExecTransformer( @transient override lazy val metrics = BackendsApiManager.getMetricsApiInstance.genSortTransformerMetrics(sparkContext) - override def metricsUpdater(): MetricsUpdater = + override def isNoop: Boolean = sortOrder == null || sortOrder.isEmpty + + override def metricsUpdater(): MetricsUpdater = if (isNoop) { + MetricsUpdater.None + } else { BackendsApiManager.getMetricsApiInstance.genSortTransformerMetricsUpdater(metrics) + } override def output: Seq[Attribute] = child.output @@ -103,13 +108,12 @@ case class SortExecTransformer( override protected def doTransform(context: SubstraitContext): TransformContext = { val childCtx = child.asInstanceOf[TransformSupport].transform(context) - val operatorId = context.nextOperatorId(this.nodeName) - if (sortOrder == null || sortOrder.isEmpty) { + if (isNoop) { // The computing for this project is not needed. - context.registerEmptyRelToOperator(operatorId) return childCtx } + val operatorId = context.nextOperatorId(this.nodeName) val currRel = getRelNode(context, sortOrder, child.output, operatorId, childCtx.root, validation = false) assert(currRel != null, "Sort Rel should be valid") 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 e8a42883a54f7..6414b67a80921 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 @@ -98,6 +98,9 @@ trait TransformSupport extends GlutenPlan { Seq(plan.executeColumnar()) } } + + // When true, it will not generate relNode, nor will it generate native metrics. + def isNoop: Boolean = false } trait LeafTransformSupport extends TransformSupport with LeafExecNode { diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/WindowExecTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/WindowExecTransformer.scala index 7b9e2865f8830..28d7809924922 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/WindowExecTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/WindowExecTransformer.scala @@ -51,8 +51,13 @@ case class WindowExecTransformer( @transient override lazy val metrics = BackendsApiManager.getMetricsApiInstance.genWindowTransformerMetrics(sparkContext) - override def metricsUpdater(): MetricsUpdater = + override def isNoop: Boolean = windowExpression == null || windowExpression.isEmpty + + override def metricsUpdater(): MetricsUpdater = if (isNoop) { + MetricsUpdater.None + } else { BackendsApiManager.getMetricsApiInstance.genWindowTransformerMetricsUpdater(metrics) + } override def output: Seq[Attribute] = child.output ++ windowExpression.map(_.toAttribute) @@ -177,13 +182,12 @@ case class WindowExecTransformer( override protected def doTransform(context: SubstraitContext): TransformContext = { val childCtx = child.asInstanceOf[TransformSupport].transform(context) - val operatorId = context.nextOperatorId(this.nodeName) - if (windowExpression == null || windowExpression.isEmpty) { + if (isNoop) { // The computing for this operator is not needed. - context.registerEmptyRelToOperator(operatorId) return childCtx } + val operatorId = context.nextOperatorId(this.nodeName) val currRel = getWindowRel(context, child.output, operatorId, childCtx.root, validation = false) assert(currRel != null, "Window Rel should be valid") diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/substrait/SubstraitContext.scala b/gluten-substrait/src/main/scala/org/apache/gluten/substrait/SubstraitContext.scala index 79148d9f3093b..1ceb2d4155aba 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/substrait/SubstraitContext.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/substrait/SubstraitContext.scala @@ -112,19 +112,6 @@ class SubstraitContext extends Serializable { id } - /** - * Register empty rel list to certain operator id. Used when the computing of a Spark transformer - * is omitted. - * @param operatorId - * operator id - */ - def registerEmptyRelToOperator(operatorId: JLong): Unit = { - if (!operatorToRelsMap.containsKey(operatorId)) { - val rels = new JArrayList[JLong]() - operatorToRelsMap.put(operatorId, rels) - } - } - /** * Return the registered map. * @return From cc25aa26a455b6708d3b9a0e9b416663d3e32531 Mon Sep 17 00:00:00 2001 From: PHILO-HE Date: Mon, 25 Nov 2024 17:54:05 +0800 Subject: [PATCH 05/13] [VL] Bump jemalloc version and update relevant documents (#8035) --- cpp/CMake/Buildjemalloc.cmake | 4 +- docs/developers/HowTo.md | 7 ++++ .../ProfileMemoryOfGlutenWithVelox.md | 37 ++++++++++--------- docs/get-started/build-guide.md | 2 +- 4 files changed, 31 insertions(+), 19 deletions(-) diff --git a/cpp/CMake/Buildjemalloc.cmake b/cpp/CMake/Buildjemalloc.cmake index 5491bd829533b..67a0cc5f11677 100644 --- a/cpp/CMake/Buildjemalloc.cmake +++ b/cpp/CMake/Buildjemalloc.cmake @@ -22,7 +22,7 @@ macro(build_jemalloc) if(DEFINED ENV{GLUTEN_JEMALLOC_URL}) set(JEMALLOC_SOURCE_URL "$ENV{GLUTEN_JEMALLOC_URL}") else() - set(JEMALLOC_BUILD_VERSION "5.2.1") + set(JEMALLOC_BUILD_VERSION "5.3.0") set(JEMALLOC_SOURCE_URL "https://github.com/jemalloc/jemalloc/releases/download/${JEMALLOC_BUILD_VERSION}/jemalloc-${JEMALLOC_BUILD_VERSION}.tar.bz2" "https://github.com/ursa-labs/thirdparty/releases/download/latest/jemalloc-${JEMALLOC_BUILD_VERSION}.tar.bz2" @@ -47,6 +47,8 @@ macro(build_jemalloc) # for dynamically linking. "--without-export" "--disable-cxx" "--disable-libdl" + # Enable heap profiling and leak detection functionality. + "--enable-prof" # For fixing an issue when loading native lib: cannot allocate memory in # static TLS block. "--disable-initial-exec-tls" diff --git a/docs/developers/HowTo.md b/docs/developers/HowTo.md index c4b1a03a36d8a..22ad3e30efc78 100644 --- a/docs/developers/HowTo.md +++ b/docs/developers/HowTo.md @@ -122,6 +122,13 @@ gdb ${GLUTEN_HOME}/cpp/build/releases/libgluten.so 'core-Executor task l-2000883 ``` - the `core-Executor task l-2000883-1671542526` represents the core file name. +# How to use jemalloc for Gluten native engine + +Currently, we have no dedicated memory allocator implemented by jemalloc. User can set environment variable `LD_PRELOAD` for lib jemalloc +to let it override the corresponding C standard functions entirely. It may help alleviate OOM issues. + +`spark.executorEnv.LD_PREALOD=/path/to/libjemalloc.so` + # How to run TPC-H on Velox backend Now, both Parquet and DWRF format files are supported, related scripts and files are under the directory of `${GLUTEN_HOME}/backends-velox/workload/tpch`. diff --git a/docs/developers/ProfileMemoryOfGlutenWithVelox.md b/docs/developers/ProfileMemoryOfGlutenWithVelox.md index 480755cb99c58..1f57f809218c1 100644 --- a/docs/developers/ProfileMemoryOfGlutenWithVelox.md +++ b/docs/developers/ProfileMemoryOfGlutenWithVelox.md @@ -5,13 +5,14 @@ nav_order: 8 has_children: true parent: /developer-overview/ --- -Gluten offloads most of computation to native engine. We can use [gperftools](https://github.com/gperftools/gperftools) or [jemalloc](https://github.com/jemalloc/jemalloc) to analyze the offheap memory and cpu profile. +Gluten offloads most of Spark SQL execution to native engine. We can use [gperftools](https://github.com/gperftools/gperftools) or [jemalloc](https://github.com/jemalloc/jemalloc) +to analyze the offheap memory and cpu profile. -# Profiling using gperftools +# Profile with gperftools `gperftools` is a collection of a high-performance multi-threaded malloc() implementation, plus some pretty nifty performance analysis -tools, see more: https://github.com/gperftools/gperftools/wiki +tools, see more: https://github.com/gperftools/gperftools/wiki. ## Build and install gperftools @@ -29,10 +30,10 @@ Then we can find the tcmalloc libraries in `$GPERFTOOLS_HOME/.lib`. ## Run Gluten with gperftools -Use `--file` or `spark.files` to upload tcmalloc library. +Configure `--files` or `spark.files` for Spark. ``` ---file /path/to/gperftools/libtcmalloc_and_profiler.so +--files /path/to/gperftools/libtcmalloc_and_profiler.so or spark.files /path/to/gperftools/libtcmalloc_and_profiler.so ``` @@ -48,14 +49,14 @@ spark.executorEnv.LD_PRELOAD ./libtcmalloc_and_profiler.so spark.executorEnv.HEAPPROFILE /tmp/gluten_heap_perf_${CONTAINER_ID} ``` -Finally, profiling files starting with `/tmp/gluten_heap_perf_${CONTAINER_ID}` will be generated in each spark executor. +Finally, profiling files prefixed with `/tmp/gluten_heap_perf_${CONTAINER_ID}` will be generated for each spark executor. -## Analyze output profiles +## Analyze profiling output -Prepare the required native libraries. We can extract the gluten and velox libraries from gluten bundle jar. (Maybe also need dependency libraries for non-static compilation) +Prepare the required native libraries. Assume static build is used for Gluten, there is no other shared dependency libs. ```bash -jar xf gluten-velox-bundle-spark3.5_2.12-centos_7_x86_64-1.2.0.jar libvelox.so libgluten.so +jar xf gluten-velox-bundle-spark3.5_2.12-centos_7_x86_64-1.2.0.jar relative/path/to/libvelox.so ralative/path/to/libgluten.so mv libvelox.so libgluten.so /path/to/gluten_lib_prefix ``` @@ -82,9 +83,11 @@ Result like: **\*\*** Get more help from https://github.com/gperftools/gperftools/wiki#documentation. -# Profiling using jemalloc +# Profile with jemalloc -`jemalloc` is a general purpose malloc(3) implementation that emphasizes fragmentation avoidance and scalable concurrency support. We can also use it to analyze Gluten performance. Getting Started with `jemalloc`: https://github.com/jemalloc/jemalloc/wiki/Getting-Started. +`jemalloc` is a general purpose malloc(3) implementation that emphasizes fragmentation +avoidance and scalable concurrency support. We can also use it to analyze Gluten performance. +Getting Started with `jemalloc`: https://github.com/jemalloc/jemalloc/wiki/Getting-Started. ## Build and install jemalloc @@ -99,10 +102,10 @@ Then we can find the jemalloc library in `$JEMALLOC_HOME/.lib`. ## Run Gluten with jemalloc -Use `--file` or `spark.files` to upload jemalloc library. +Configure `--files` or `spark.files` for Spark. ``` ---file /path/to/jemalloc/libjemalloc.so +--files /path/to/jemalloc/libjemalloc.so or spark.files /path/to/jemalloc/libjemalloc.so ``` @@ -114,14 +117,14 @@ spark.executorEnv.LD_PRELOAD ./libjemalloc.so spark.executorEnv.MALLOC_CONF prof:true,lg_prof_interval:30,prof_prefix:/tmp/gluten_heap_perf ``` -Finally, profiling files starting with `/tmp/gluten_heap_perf.${PID}` will be generated in each spark executor. +Finally, profiling files prefixed with `/tmp/gluten_heap_perf.${PID}` will be generated for each spark executor. -## Analyze output profiles +## Analyze profiling output -Prepare the required native libraries. We can extract the gluten and velox libraries from gluten bundle jar. (Maybe also need dependency libraries for non-static compilation) +Prepare the required native libraries. Assume static build is used for Gluten, so there is no other shared dependency libs. ```bash -jar xf gluten-velox-bundle-spark3.5_2.12-centos_7_x86_64-1.2.0.jar libvelox.so libgluten.so +jar xf gluten-velox-bundle-spark3.5_2.12-centos_7_x86_64-1.2.0.jar relative/path/to/libvelox.so relative/path/to/libgluten.so mv libvelox.so libgluten.so /path/to/gluten_lib_prefix ``` diff --git a/docs/get-started/build-guide.md b/docs/get-started/build-guide.md index 3c0c521e2181a..cbaad979fb909 100644 --- a/docs/get-started/build-guide.md +++ b/docs/get-started/build-guide.md @@ -14,7 +14,7 @@ Please set them via `--`, e.g. `--build_type=Release`. | build_tests | Build gluten cpp tests. | OFF | | build_examples | Build udf example. | OFF | | build_benchmarks | Build gluten cpp benchmarks. | OFF | -| enable_jemalloc_stats | Print jemalloc stats for debugging. | OFF | +| enable_jemalloc_stats | Print jemalloc stats for debugging. | OFF | | build_protobuf | Build protobuf lib. | OFF | | enable_qat | Enable QAT for shuffle data de/compression. | OFF | | enable_iaa | Enable IAA for shuffle data de/compression. | OFF | From 9f69d217e6cd9342cc31fd3a9d20a8065e30890c Mon Sep 17 00:00:00 2001 From: Yuan Date: Tue, 26 Nov 2024 07:34:39 +0800 Subject: [PATCH 06/13] [MISC] Velox maintainers as triage member(collaborators) (#8037) This patch adds Velox maintainer Deepak and Pedro as triage member, so they could help to manage the issues related with Velox easier. Signed-off-by: Yuan Zhou --- .asf.yaml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/.asf.yaml b/.asf.yaml index 76429ad286de8..dafb788883893 100644 --- a/.asf.yaml +++ b/.asf.yaml @@ -27,6 +27,9 @@ github: - spark-sql - vectorization - velox + collaborators: + - majetideepak + - pedroerp enabled_merge_buttons: squash: true merge: false From bcb768bde27688f0083861b566b787f07a2ffd40 Mon Sep 17 00:00:00 2001 From: Suraj Naik Date: Tue, 26 Nov 2024 08:54:51 +0530 Subject: [PATCH 07/13] [VL] Clean up duplicate CMake code for setting CMAKE_CXX_FLAGS (#8034) --- cpp/CMakeLists.txt | 21 ++++++++++++++++++++- cpp/core/CMakeLists.txt | 22 ---------------------- cpp/velox/CMakeLists.txt | 18 ------------------ 3 files changed, 20 insertions(+), 41 deletions(-) diff --git a/cpp/CMakeLists.txt b/cpp/CMakeLists.txt index 67fb9ec721ace..e0ab141a1f964 100644 --- a/cpp/CMakeLists.txt +++ b/cpp/CMakeLists.txt @@ -98,7 +98,9 @@ set(KNOWN_WARNINGS -Wno-error=unused-function \ -Wno-error=unused-variable \ -Wno-strict-aliasing \ - -Wno-ignored-qualifiers") + -Wno-ignored-qualifiers \ + -Wno-deprecated-declarations \ + -Wno-attributes") if("${CMAKE_CXX_COMPILER_ID}" STREQUAL "GNU") set(KNOWN_WARNINGS "-Wno-error=unused-but-set-variable \ @@ -141,8 +143,25 @@ if(${CMAKE_SYSTEM_NAME} MATCHES "Darwin") add_compile_definitions(_GNU_SOURCE) endif() +if(NOT ${CMAKE_SYSTEM_NAME} STREQUAL "Darwin") + set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wno-class-memaccess") +endif() + set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${KNOWN_WARNINGS}") +# Keep same compile option with Velox. +execute_process( + COMMAND + bash -c + "( source ${VELOX_HOME}/scripts/setup-helper-functions.sh && echo -n $(get_cxx_flags $ENV{CPU_TARGET}))" + OUTPUT_VARIABLE SCRIPT_CXX_FLAGS + RESULT_VARIABLE COMMAND_STATUS) +if(COMMAND_STATUS EQUAL "1") + message(FATAL_ERROR "Unable to determine compiler flags!") +endif() + +set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${SCRIPT_CXX_FLAGS}") + # # Dependencies # diff --git a/cpp/core/CMakeLists.txt b/cpp/core/CMakeLists.txt index 88b0fd883e5c8..5a5eeac354805 100644 --- a/cpp/core/CMakeLists.txt +++ b/cpp/core/CMakeLists.txt @@ -22,28 +22,6 @@ include(FindPkgConfig) include(GNUInstallDirs) include(CheckCXXCompilerFlag) -set(CMAKE_CXX_FLAGS - "${CMAKE_CXX_FLAGS} -Wno-deprecated-declarations -Wno-attributes") -if(NOT ${CMAKE_SYSTEM_NAME} STREQUAL "Darwin") - set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wno-class-memaccess") -endif() - -if(NOT DEFINED VELOX_HOME) - set(VELOX_HOME ${GLUTEN_HOME}/ep/build-velox/build/velox_ep) - message(STATUS "Set VELOX_HOME to ${VELOX_HOME}") -endif() - -# Keep same compile option with Velox. -execute_process( - COMMAND - bash -c - "( source ${VELOX_HOME}/scripts/setup-helper-functions.sh && echo -n $(get_cxx_flags $ENV{CPU_TARGET}))" - OUTPUT_VARIABLE SCRIPT_CXX_FLAGS - RESULT_VARIABLE COMMAND_STATUS) -if(COMMAND_STATUS EQUAL "1") - message(FATAL_ERROR "Unable to determine compiler flags!") -endif() -set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${SCRIPT_CXX_FLAGS}") message("Core module final CMAKE_CXX_FLAGS=${CMAKE_CXX_FLAGS}") set(BOOST_MIN_VERSION "1.42.0") diff --git a/cpp/velox/CMakeLists.txt b/cpp/velox/CMakeLists.txt index 9e110853eb6ee..4503d5947d78a 100644 --- a/cpp/velox/CMakeLists.txt +++ b/cpp/velox/CMakeLists.txt @@ -23,12 +23,6 @@ include(GNUInstallDirs) include(CheckCXXCompilerFlag) include(FindPackageHandleStandardArgs) -set(CMAKE_CXX_FLAGS - "${CMAKE_CXX_FLAGS} -Wno-deprecated-declarations -Wno-attributes") -if(NOT ${CMAKE_SYSTEM_NAME} STREQUAL "Darwin") - set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wno-class-memaccess") -endif() - set(SYSTEM_LIB_PATH "/usr/lib" CACHE PATH "System Lib dir") @@ -62,18 +56,6 @@ if(NOT DEFINED VELOX_HOME) message(STATUS "Set VELOX_HOME to ${VELOX_HOME}") endif() -# Keep same compile option with Velox. -execute_process( - COMMAND - bash -c - "( source ${VELOX_HOME}/scripts/setup-helper-functions.sh && echo -n $(get_cxx_flags $ENV{CPU_TARGET}))" - OUTPUT_VARIABLE SCRIPT_CXX_FLAGS - RESULT_VARIABLE COMMAND_STATUS) -if(COMMAND_STATUS EQUAL "1") - message(FATAL_ERROR "Unable to determine compiler flags!") -endif() -set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${SCRIPT_CXX_FLAGS}") - message("Velox module final CMAKE_CXX_FLAGS=${CMAKE_CXX_FLAGS}") # User can specify VELOX_BUILD_PATH, if Velox are built elsewhere. From 4be7bf17a6a031306759c1dfba95e4ef7ca9db1c Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Tue, 26 Nov 2024 11:37:26 +0800 Subject: [PATCH 08/13] [GLUTEN-7741][VL] Fix deprecated actions/upload-artifact version issue when building bundle package (#8017) V2 is deprecated: This request has been automatically failed because it uses a deprecated version of `actions/upload-artifact: v2`. Learn more: https://github.blog/changelog/2024-02-13-deprecation-notice-v1-and-v2-of-the-artifact-actions/ --- .github/workflows/build_bundle_package.yml | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/.github/workflows/build_bundle_package.yml b/.github/workflows/build_bundle_package.yml index a28720656651d..9b94d302f9629 100644 --- a/.github/workflows/build_bundle_package.yml +++ b/.github/workflows/build_bundle_package.yml @@ -54,13 +54,13 @@ jobs: cd $GITHUB_WORKSPACE/ bash dev/ci-velox-buildstatic-centos-7.sh - name: Upload native libs - uses: actions/upload-artifact@v2 + uses: actions/upload-artifact@v3 with: path: ./cpp/build/releases/ name: velox-native-lib-${{github.sha}} retention-days: 1 - name: Upload Artifact Arrow Jar - uses: actions/upload-artifact@v2 + uses: actions/upload-artifact@v3 with: path: /root/.m2/repository/org/apache/arrow/ name: velox-arrow-jar-centos-7-${{github.sha}} @@ -72,17 +72,19 @@ jobs: steps: - uses: actions/checkout@v2 - name: Download All Artifacts - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: velox-native-lib-${{github.sha}} path: ./cpp/build/releases - name: Download All Arrow Jar Artifacts - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: velox-arrow-jar-centos-7-${{github.sha}} path: /root/.m2/repository/org/apache/arrow/ - name: Setup java and maven run: | + sed -i 's/mirrorlist/#mirrorlist/g' /etc/yum.repos.d/CentOS-* && \ + sed -i 's|#baseurl=http://mirror.centos.org|baseurl=http://vault.centos.org|g' /etc/yum.repos.d/CentOS-* && \ yum update -y && yum install -y java-1.8.0-openjdk-devel wget && \ wget https://downloads.apache.org/maven/maven-3/3.8.8/binaries/apache-maven-3.8.8-bin.tar.gz && \ tar -xvf apache-maven-3.8.8-bin.tar.gz && \ @@ -94,7 +96,7 @@ jobs: export PATH=${PATH}:${MAVEN_HOME}/bin && \ mvn clean install -P${{ github.event.inputs.spark }} -Dhadoop.version=${{ github.event.inputs.hadoop }} -Pbackends-velox -Pceleborn -Puniffle -DskipTests -Dmaven.source.skip - name: Upload bundle package - uses: actions/upload-artifact@v2 + uses: actions/upload-artifact@v3 with: name: gluten-velox-bundle-package path: package/target/gluten-velox-bundle-*.jar From 1d1366a4229e0f0c9ead8ee24d7039ae8f99db65 Mon Sep 17 00:00:00 2001 From: Hongze Zhang Date: Tue, 26 Nov 2024 13:26:57 +0800 Subject: [PATCH 09/13] [VL] vcpkg: Broken libelf mirror (#8047) --- dev/vcpkg/ports/libelf/portfile.cmake | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/vcpkg/ports/libelf/portfile.cmake b/dev/vcpkg/ports/libelf/portfile.cmake index 8e9ea0c71fe1b..d1b347d693527 100644 --- a/dev/vcpkg/ports/libelf/portfile.cmake +++ b/dev/vcpkg/ports/libelf/portfile.cmake @@ -1,5 +1,5 @@ vcpkg_download_distfile(ARCHIVE - URLS "https://ftp.osuosl.org/pub/blfs/conglomeration/libelf/libelf-0.8.13.tar.gz" + URLS "https://fossies.org/linux/misc/old/libelf-0.8.13.tar.gz" FILENAME "libelf-0.8.13.tar.gz" SHA512 d2a4ea8ccc0bbfecac38fa20fbd96aefa8e86f8af38691fb6991cd9c5a03f587475ecc2365fc89a4954c11a679d93460ee9a5890693112f6133719af3e6582fe ) From 4dfdfd77b52f2f98fa0cf32eca143b47e4bd11b5 Mon Sep 17 00:00:00 2001 From: Gluten Performance Bot <137994563+GlutenPerfBot@users.noreply.github.com> Date: Tue, 26 Nov 2024 16:25:00 +0800 Subject: [PATCH 10/13] [GLUTEN-6887][VL] Daily Update Velox Version (2024_11_26) (#8042) Upstream Velox's New Commits: f33b40da0 by zhli1142015, feat(ABFS): Add dependency for ABFS Auth support (11633) 22392b175 by duanmeng, feat: Specifiy target tracing driver IDs (11560) 2b5e9f1a5 by Yenda Li, fix:Support hashing ipaddress (11640) 97bd72825 by Xiaoxuan Meng, feat: Add skewed partition balancer (11635) 97e47daf5 by Jacob Wujciak-Jens, build(ci): Fix scheduled.yml syntax (11652) e27867ca2 by Jia Ke, fix: Move the thread local parameter from method to member parameter in HdfsReadFile (11580) --- .github/workflows/velox_backend.yml | 22 ++++++++++++++-------- ep/build-velox/src/get_velox.sh | 2 +- 2 files changed, 15 insertions(+), 9 deletions(-) diff --git a/.github/workflows/velox_backend.yml b/.github/workflows/velox_backend.yml index d7445d1a2752d..56b43975acee1 100644 --- a/.github/workflows/velox_backend.yml +++ b/.github/workflows/velox_backend.yml @@ -56,9 +56,8 @@ concurrency: jobs: build-native-lib-centos-7: runs-on: ubuntu-20.04 - container: apache/gluten:vcpkg-centos-7 steps: - - uses: actions/checkout@v2 + - uses: actions/checkout@v4 - name: Get Ccache uses: actions/cache/restore@v3 with: @@ -68,10 +67,17 @@ jobs: ccache-centos7-release-default - name: Build Gluten native libraries run: | - df -a - cd $GITHUB_WORKSPACE/ - bash dev/ci-velox-buildstatic-centos-7.sh - ccache -s + docker pull apache/gluten:vcpkg-centos-7 + docker run -v $GITHUB_WORKSPACE:/work -w /work apache/gluten:vcpkg-centos-7 bash -c " + df -a + cd /work + export CCACHE_DIR=/work/.ccache + bash dev/ci-velox-buildstatic-centos-7.sh + ccache -s + mkdir -p /work/.m2/repository/org/apache/arrow/ + cp -r /root/.m2/repository/org/apache/arrow/* /work/.m2/repository/org/apache/arrow/ + " + - name: "Save ccache" uses: actions/cache/save@v3 id: ccache @@ -85,7 +91,7 @@ jobs: - uses: actions/upload-artifact@v3 with: name: arrow-jars-centos-7-${{github.sha}} - path: /root/.m2/repository/org/apache/arrow/ + path: .m2/repository/org/apache/arrow/ run-tpc-test-ubuntu: needs: build-native-lib-centos-7 @@ -163,7 +169,7 @@ jobs: strategy: fail-fast: false matrix: - os: [ "centos:7", "centos:8" ] + os: [ "centos:8" ] #TODO: add centos7 tests back with docker run spark: [ "spark-3.2", "spark-3.3", "spark-3.4", "spark-3.5" ] java: [ "java-8", "java-11", "java-17" ] # Spark supports JDK17 since 3.3 and later, see https://issues.apache.org/jira/browse/SPARK-33772 diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh index d95f742a55153..bcab2e31dbef4 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_24 +VELOX_BRANCH=2024_11_26 VELOX_HOME="" OS=`uname -s` From efd2cbde3064631179dede8ad2b7a0e96d426d06 Mon Sep 17 00:00:00 2001 From: kevinyhzou <37431499+KevinyhZou@users.noreply.github.com> Date: Tue, 26 Nov 2024 19:53:13 +0800 Subject: [PATCH 11/13] [GLUTEN-7896][CH]Fix to_date diff for time parser policy config (#7923) * fix pre-projection not take effect * Fix time_parser_plicy set legacy * fix * fix 11 * add test * fix ci test * Fix code bug * fix review * modify test --- ...enClickHouseTPCHSaltNullParquetSuite.scala | 18 ++- cpp-ch/local-engine/Common/CHUtil.cpp | 5 + cpp-ch/local-engine/Common/CHUtil.h | 1 + .../CommonScalarFunctionParser.cpp | 1 - .../scalar_function_parser/getTimestamp.cpp | 23 ++++ .../scalar_function_parser/getTimestamp.h | 106 ++++++++++++++++++ .../scalar_function_parser/unixTimestamp.cpp | 14 +-- .../org/apache/gluten/GlutenConfig.scala | 3 +- 8 files changed, 160 insertions(+), 11 deletions(-) create mode 100644 cpp-ch/local-engine/Parser/scalar_function_parser/getTimestamp.cpp create mode 100644 cpp-ch/local-engine/Parser/scalar_function_parser/getTimestamp.h 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 5d7bcf324ae86..4a2b7040fa5ea 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 @@ -2192,7 +2192,7 @@ class GlutenClickHouseTPCHSaltNullParquetSuite extends GlutenClickHouseTPCHAbstr } } - test("GLUTEN-3135: Bug fix to_date") { + test("GLUTEN-3135/GLUTEN-7896: Bug fix to_date") { val create_table_sql = """ | create table test_tbl_3135(id bigint, data string) using parquet @@ -2209,13 +2209,27 @@ class GlutenClickHouseTPCHSaltNullParquetSuite extends GlutenClickHouseTPCHAbstr |(7, '1970-01-01 00:00:00'), |(8, '2024-3-2'), |(9, '2024-03-2'), - |(10, '2024-03') + |(10, '2024-03'), + |(11, '2024-03-02 11:22:33') |""".stripMargin spark.sql(create_table_sql) spark.sql(insert_data_sql) val select_sql = "select id, to_date(data) from test_tbl_3135" compareResultsAgainstVanillaSpark(select_sql, true, { _ => }) + + withSQLConf(("spark.sql.legacy.timeParserPolicy" -> "corrected")) { + compareResultsAgainstVanillaSpark( + "select id, to_date('2024-03-2 11:22:33', 'yyyy-MM-dd') from test_tbl_3135 where id = 11", + true, + { _ => }) + } + withSQLConf(("spark.sql.legacy.timeParserPolicy" -> "legacy")) { + compareResultsAgainstVanillaSpark( + "select id, to_date(data, 'yyyy-MM-dd') from test_tbl_3135 where id = 11", + true, + { _ => }) + } spark.sql("drop table test_tbl_3135") } diff --git a/cpp-ch/local-engine/Common/CHUtil.cpp b/cpp-ch/local-engine/Common/CHUtil.cpp index 03df93c851e3e..8fef52e50a682 100644 --- a/cpp-ch/local-engine/Common/CHUtil.cpp +++ b/cpp-ch/local-engine/Common/CHUtil.cpp @@ -762,6 +762,11 @@ void BackendInitializerUtil::initSettings(const SparkConfigs::ConfigMap & spark_ settings.set(key, toField(key, value)); LOG_DEBUG(&Poco::Logger::get("CHUtil"), "Set settings key:{} value:{}", key, value); } + else if (key == TIMER_PARSER_POLICY) + { + settings.set(key, value); + LOG_DEBUG(&Poco::Logger::get("CHUtil"), "Set settings key:{} value:{}", key, value); + } } /// Finally apply some fixed kvs to settings. diff --git a/cpp-ch/local-engine/Common/CHUtil.h b/cpp-ch/local-engine/Common/CHUtil.h index cff69090ee313..a5fb24f6afeec 100644 --- a/cpp-ch/local-engine/Common/CHUtil.h +++ b/cpp-ch/local-engine/Common/CHUtil.h @@ -40,6 +40,7 @@ namespace local_engine static const String MERGETREE_INSERT_WITHOUT_LOCAL_STORAGE = "mergetree.insert_without_local_storage"; static const String MERGETREE_MERGE_AFTER_INSERT = "mergetree.merge_after_insert"; static const std::string DECIMAL_OPERATIONS_ALLOW_PREC_LOSS = "spark.sql.decimalOperations.allowPrecisionLoss"; +static const std::string TIMER_PARSER_POLICY = "spark.sql.legacy.timeParserPolicy"; static const std::unordered_set BOOL_VALUE_SETTINGS{ MERGETREE_MERGE_AFTER_INSERT, MERGETREE_INSERT_WITHOUT_LOCAL_STORAGE, DECIMAL_OPERATIONS_ALLOW_PREC_LOSS}; diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/CommonScalarFunctionParser.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/CommonScalarFunctionParser.cpp index d6584267455fb..ec8b4e0d12bf3 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/CommonScalarFunctionParser.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/CommonScalarFunctionParser.cpp @@ -57,7 +57,6 @@ REGISTER_COMMON_SCALAR_FUNCTION_PARSER(Not, not, not ); REGISTER_COMMON_SCALAR_FUNCTION_PARSER(Xor, xor, xor); REGISTER_COMMON_SCALAR_FUNCTION_PARSER(Cast, cast, CAST); -REGISTER_COMMON_SCALAR_FUNCTION_PARSER(GetTimestamp, get_timestamp, parseDateTime64InJodaSyntaxOrNull); REGISTER_COMMON_SCALAR_FUNCTION_PARSER(Quarter, quarter, toQuarter); // math functions diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/getTimestamp.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/getTimestamp.cpp new file mode 100644 index 0000000000000..4724f820099a7 --- /dev/null +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/getTimestamp.cpp @@ -0,0 +1,23 @@ +/* + * 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 + +namespace local_engine +{ + static FunctionParserRegister register_get_timestamp; +} diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/getTimestamp.h b/cpp-ch/local-engine/Parser/scalar_function_parser/getTimestamp.h new file mode 100644 index 0000000000000..5e32e00569f11 --- /dev/null +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/getTimestamp.h @@ -0,0 +1,106 @@ +/* + * 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 +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} +} + + +namespace local_engine +{ +class FunctionParserGetTimestamp : public FunctionParser +{ +public: + explicit FunctionParserGetTimestamp(ParserContextPtr parser_context_) : FunctionParser(parser_context_) {} + ~FunctionParserGetTimestamp() override = default; + + static constexpr auto name = "get_timestamp"; + String getName() const override { return name; } + + const ActionsDAG::Node * parse( + const substrait::Expression_ScalarFunction & substrait_func, + ActionsDAG & actions_dag) const override + { + /* + spark function: get_timestamp(expr, fmt) + 1. If timeParserPolicy is LEGACY + 1) fmt has 0 'S', ch function = parseDateTime64InJodaSyntaxOrNull(substr(expr,1,length(fmt)), fmt); + 2) fmt has 'S' more than 0, make the fmt has 3 'S', ch function = parseDateTime64InJodaSyntaxOrNull(expr, fmt) + 2. Else ch function = parseDateTime64InJodaSyntaxOrNull(expr, fmt) + */ + auto parsed_args = parseFunctionArguments(substrait_func, actions_dag); + if (parsed_args.size() != 2) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} requires exactly two arguments", getName()); + const auto * expr_arg = parsed_args[0]; + const auto * fmt_arg = parsed_args[1]; + + const auto & args = substrait_func.arguments(); + bool fmt_string_literal = args[1].value().has_literal(); + String fmt; + if (fmt_string_literal) + { + const auto & literal_fmt_expr = args[1].value().literal(); + fmt_string_literal = literal_fmt_expr.has_string(); + fmt = fmt_string_literal ? literal_fmt_expr.string() : ""; + } + if (!fmt_string_literal) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "The second of function {} must be const String.", name); + + UInt32 s_count = std::count(fmt.begin(), fmt.end(), 'S'); + String time_parser_policy = getContext()->getSettingsRef().has(TIMER_PARSER_POLICY) ? toString(getContext()->getSettingsRef().get(TIMER_PARSER_POLICY)) : ""; + boost::to_lower(time_parser_policy); + if (time_parser_policy == "legacy") + { + if (s_count == 0) + { + const auto * index_begin_node = addColumnToActionsDAG(actions_dag, std::make_shared(), 1); + const auto * index_end_node = addColumnToActionsDAG(actions_dag, std::make_shared(), fmt.size()); + const auto * substr_node = toFunctionNode(actions_dag, "substringUTF8", {expr_arg, index_begin_node, index_end_node}); + const auto * fmt_node = addColumnToActionsDAG(actions_dag, std::make_shared(), fmt); + const auto * result_node = toFunctionNode(actions_dag, "parseDateTime64InJodaSyntaxOrNull", {substr_node, fmt_node}); + return convertNodeTypeIfNeeded(substrait_func, result_node, actions_dag); + } + else if (s_count < 3) + fmt += String(3 - s_count, 'S'); + else + fmt = fmt.substr(0, fmt.size() - (s_count - 3)); + + const auto * fmt_node = addColumnToActionsDAG(actions_dag, std::make_shared(), fmt); + const auto * result_node = toFunctionNode(actions_dag, "parseDateTime64InJodaSyntaxOrNull", {expr_arg, fmt_node}); + return convertNodeTypeIfNeeded(substrait_func, result_node, actions_dag); + } + else + { + const auto * result_node = toFunctionNode(actions_dag, "parseDateTime64InJodaSyntaxOrNull", {expr_arg, fmt_arg}); + return convertNodeTypeIfNeeded(substrait_func, result_node, actions_dag); + } + } +}; +} diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/unixTimestamp.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/unixTimestamp.cpp index 622237da9707e..33997734c5e41 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/unixTimestamp.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/unixTimestamp.cpp @@ -17,7 +17,7 @@ #include #include - +#include namespace DB { @@ -34,10 +34,10 @@ namespace local_engine { template -class FunctionParserUnixTimestamp : public FunctionParser +class FunctionParserUnixTimestamp : public FunctionParserGetTimestamp { public: - explicit FunctionParserUnixTimestamp(ParserContextPtr parser_context_) : FunctionParser(parser_context_) {} + explicit FunctionParserUnixTimestamp(ParserContextPtr parser_context_) : FunctionParserGetTimestamp(parser_context_) {} ~FunctionParserUnixTimestamp() override = default; static constexpr auto name = Name::name; @@ -60,13 +60,13 @@ class FunctionParserUnixTimestamp : public FunctionParser const auto * expr_arg = parsed_args[0]; const auto * fmt_arg = parsed_args[1]; auto expr_type = removeNullable(expr_arg->result_type); + if (isString(expr_type)) + return FunctionParserGetTimestamp::parse(substrait_func, actions_dag); + const DateLUTImpl * date_lut = &DateLUT::instance(); const auto * time_zone_node = addColumnToActionsDAG(actions_dag, std::make_shared(), date_lut->getTimeZone()); - const DB::ActionsDAG::Node * result_node = nullptr; - if (isString(expr_type)) - result_node = toFunctionNode(actions_dag, "parseDateTime64InJodaSyntaxOrNull", {expr_arg, fmt_arg, time_zone_node}); - else if (isDateOrDate32(expr_type)) + if (isDateOrDate32(expr_type)) result_node = toFunctionNode(actions_dag, "sparkDateToUnixTimestamp", {expr_arg, time_zone_node}); else if (isDateTime(expr_type) || isDateTime64(expr_type)) result_node = toFunctionNode(actions_dag, "toUnixTimestamp", {expr_arg, time_zone_node}); 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 e0d06ce6fc0ff..2ccdcae99b5c8 100644 --- a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala +++ b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala @@ -805,7 +805,8 @@ object GlutenConfig { SPARK_OFFHEAP_ENABLED, SESSION_LOCAL_TIMEZONE.key, DECIMAL_OPERATIONS_ALLOW_PREC_LOSS.key, - SPARK_REDACTION_REGEX + SPARK_REDACTION_REGEX, + LEGACY_TIME_PARSER_POLICY.key ) nativeConfMap.putAll(conf.filter(e => keys.contains(e._1)).asJava) From 8d8ee3d54af359c747cea6310553f2e66d5f2122 Mon Sep 17 00:00:00 2001 From: LiuNeng <1398775315@qq.com> Date: Tue, 26 Nov 2024 22:33:32 +0800 Subject: [PATCH 12/13] [CH]Daily Update Clickhouse Version (20241118) (#7968) What changes were proposed in this pull request? Manual rebase Clickhouse repo How was this patch tested? unit tests (If this patch involves UI changes, please attach a screenshot; otherwise, remove this) --- cpp-ch/clickhouse.version | 4 +- .../CompactObjectStorageDiskTransaction.cpp | 39 +++++++++++++++++-- .../CompactObjectStorageDiskTransaction.h | 34 ++++++++++++++-- .../Disks/ObjectStorages/GlutenDiskHDFS.cpp | 2 +- .../Disks/ObjectStorages/GlutenDiskS3.cpp | 2 +- .../Functions/SparkFunctionRoundHalfUp.h | 29 +++++++++++++- .../Operator/GraceAggregatingTransform.cpp | 25 +++++++----- .../Operator/GraceAggregatingTransform.h | 6 +-- .../Parser/RelParsers/CrossRelParser.cpp | 9 ++++- .../Parser/RelParsers/JoinRelParser.cpp | 13 ++++++- .../MergeTree/SparkStorageMergeTree.cpp | 9 ++++- 11 files changed, 141 insertions(+), 31 deletions(-) diff --git a/cpp-ch/clickhouse.version b/cpp-ch/clickhouse.version index beda11ec0a998..62a70f06c1e13 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=3f7e46d4e9e +CH_BRANCH=rebase_ch/20241118 +CH_COMMIT=7f22fe487c88d3b988ea82a5c34882da23ea6289 diff --git a/cpp-ch/local-engine/Disks/ObjectStorages/CompactObjectStorageDiskTransaction.cpp b/cpp-ch/local-engine/Disks/ObjectStorages/CompactObjectStorageDiskTransaction.cpp index 5b1fe63a09c52..148a43580b96e 100644 --- a/cpp-ch/local-engine/Disks/ObjectStorages/CompactObjectStorageDiskTransaction.cpp +++ b/cpp-ch/local-engine/Disks/ObjectStorages/CompactObjectStorageDiskTransaction.cpp @@ -28,6 +28,36 @@ bool isMetaDataFile(const std::string & path) return !path.ends_with("bin"); } +TemporaryWriteBufferWrapper::TemporaryWriteBufferWrapper( + const String & file_name_, const std::shared_ptr & data_buffer_) + : WriteBufferFromFileBase(data_buffer_->buffer().size(), data_buffer_->buffer().begin(), 0) + , file_name(file_name_) + , data_buffer(data_buffer_) +{ +} +void TemporaryWriteBufferWrapper::preFinalize() +{ + next(); +} + +void TemporaryWriteBufferWrapper::finalizeImpl() +{ + next(); + data_buffer->finalizeImpl(); +} + +void TemporaryWriteBufferWrapper::cancelImpl() noexcept +{ + data_buffer->cancelImpl(); +} + +void TemporaryWriteBufferWrapper::nextImpl() +{ + data_buffer->position() = position(); + data_buffer->next(); + BufferBase::set(data_buffer->buffer().begin(), data_buffer->buffer().size(), data_buffer->offset()); +} + void CompactObjectStorageDiskTransaction::commit() { auto metadata_tx = disk.getMetadataStorage()->createTransaction(); @@ -52,9 +82,9 @@ void CompactObjectStorageDiskTransaction::commit() [&](auto & item) { DB::DiskObjectStorageMetadata metadata(object_storage->getCommonKeyPrefix(), item.first); - DB::ReadBufferFromFilePRead read(item.second->getAbsolutePath()); + auto read = item.second->read(); int file_size = 0; - while (int count = read.readBig(buffer.data(), buffer.size())) + while (int count = read->readBig(buffer.data(), buffer.size())) { file_size += count; out.write(buffer.data(), count); @@ -98,12 +128,13 @@ std::unique_ptr CompactObjectStorageDiskTransaction "Don't support write file in different dirs, path {}, prefix path: {}", path, prefix_path); - auto tmp = std::make_shared(tmp_data); + auto tmp = std::make_shared(tmp_data.get()); files.emplace_back(path, tmp); auto tx = disk.getMetadataStorage()->createTransaction(); tx->createDirectoryRecursive(std::filesystem::path(path).parent_path()); tx->createEmptyMetadataFile(path); tx->commit(); - return std::make_unique(tmp->getAbsolutePath(), buf_size); + + return std::make_unique(path, tmp); } } \ No newline at end of file diff --git a/cpp-ch/local-engine/Disks/ObjectStorages/CompactObjectStorageDiskTransaction.h b/cpp-ch/local-engine/Disks/ObjectStorages/CompactObjectStorageDiskTransaction.h index becb5371aad22..0f95ae01ec2be 100644 --- a/cpp-ch/local-engine/Disks/ObjectStorages/CompactObjectStorageDiskTransaction.h +++ b/cpp-ch/local-engine/Disks/ObjectStorages/CompactObjectStorageDiskTransaction.h @@ -32,15 +32,41 @@ extern const int NOT_IMPLEMENTED; namespace local_engine { +class TemporaryWriteBufferWrapper : public DB::WriteBufferFromFileBase +{ +public: + TemporaryWriteBufferWrapper(const String & file_name_, const std::shared_ptr & data_buffer_); + + void sync() override { data_buffer->nextImpl(); } + + void preFinalize() override; + +protected: + void finalizeImpl() override; + void cancelImpl() noexcept override; + +private: + void nextImpl() override; + +public: + std::string getFileName() const override + { + return file_name; + } + +private: + String file_name; + std::shared_ptr data_buffer; +}; + class CompactObjectStorageDiskTransaction: public DB::IDiskTransaction { public: static inline const String PART_DATA_FILE_NAME = "part_data.gluten"; static inline const String PART_META_FILE_NAME = "part_meta.gluten"; - explicit CompactObjectStorageDiskTransaction(DB::IDisk & disk_, const DB::DiskPtr tmp_) + explicit CompactObjectStorageDiskTransaction(DB::IDisk & disk_, const DB::TemporaryDataOnDiskScopePtr tmp_) : disk(disk_), tmp_data(tmp_) { - chassert(!tmp_->isRemote()); } void commit() override; @@ -170,8 +196,8 @@ class CompactObjectStorageDiskTransaction: public DB::IDiskTransaction { private: DB::IDisk & disk; - DB::DiskPtr tmp_data; - std::vector>> files; + DB::TemporaryDataOnDiskScopePtr tmp_data; + std::vector>> files; String prefix_path = ""; }; } diff --git a/cpp-ch/local-engine/Disks/ObjectStorages/GlutenDiskHDFS.cpp b/cpp-ch/local-engine/Disks/ObjectStorages/GlutenDiskHDFS.cpp index bd005132b9b93..fed23d7eef612 100644 --- a/cpp-ch/local-engine/Disks/ObjectStorages/GlutenDiskHDFS.cpp +++ b/cpp-ch/local-engine/Disks/ObjectStorages/GlutenDiskHDFS.cpp @@ -29,7 +29,7 @@ using namespace DB; DiskTransactionPtr GlutenDiskHDFS::createTransaction() { - return std::make_shared(*this, QueryContext::globalContext()->getTempDataOnDisk()->getVolume()->getDisk()); + return std::make_shared(*this, QueryContext::globalContext()->getTempDataOnDisk()); } void GlutenDiskHDFS::createDirectory(const String & path) diff --git a/cpp-ch/local-engine/Disks/ObjectStorages/GlutenDiskS3.cpp b/cpp-ch/local-engine/Disks/ObjectStorages/GlutenDiskS3.cpp index b2a6bb523d3a8..a180ebd7eaf09 100644 --- a/cpp-ch/local-engine/Disks/ObjectStorages/GlutenDiskS3.cpp +++ b/cpp-ch/local-engine/Disks/ObjectStorages/GlutenDiskS3.cpp @@ -31,7 +31,7 @@ namespace local_engine DB::DiskTransactionPtr GlutenDiskS3::createTransaction() { - return std::make_shared(*this, QueryContext::globalContext()->getTempDataOnDisk()->getVolume()->getDisk()); + return std::make_shared(*this, QueryContext::globalContext()->getSharedTempDataOnDisk()); } std::unique_ptr GlutenDiskS3::readFile( diff --git a/cpp-ch/local-engine/Functions/SparkFunctionRoundHalfUp.h b/cpp-ch/local-engine/Functions/SparkFunctionRoundHalfUp.h index 3e219b51a6955..2dfd25772ee74 100644 --- a/cpp-ch/local-engine/Functions/SparkFunctionRoundHalfUp.h +++ b/cpp-ch/local-engine/Functions/SparkFunctionRoundHalfUp.h @@ -127,6 +127,31 @@ class BaseFloatRoundingHalfUpComputation } }; +template <> +class BaseFloatRoundingHalfUpComputation +{ +public: + using ScalarType = BFloat16; + using VectorType = BFloat16; + static const size_t data_count = 1; + + static VectorType load(const ScalarType * in) { return *in; } + static VectorType load1(const ScalarType in) { return in; } + static VectorType store(ScalarType * out, ScalarType val) { return *out = val;} + static VectorType multiply(VectorType val, VectorType scale) { return val * scale; } + static VectorType divide(VectorType val, VectorType scale) { return val / scale; } + template + static VectorType apply(VectorType val) + { + return BFloat16(std::roundf(static_cast(val))); + } + + static VectorType prepare(size_t scale) + { + return load1(BFloat16(static_cast(scale))); + } +}; + /** Implementation of low-level round-off functions for floating-point values. */ @@ -167,7 +192,7 @@ struct FloatRoundingHalfUpImpl template ? Vectorize::No : Vectorize::Yes #else Vectorize::No #endif @@ -219,7 +244,7 @@ struct DispatcherRoundingHalfUp { template using FunctionRoundingImpl = std::conditional_t< - std::is_floating_point_v, + std::is_floating_point_v || std::is_same_v, FloatRoundingHalfUpImpl, IntegerRoundingImpl>; diff --git a/cpp-ch/local-engine/Operator/GraceAggregatingTransform.cpp b/cpp-ch/local-engine/Operator/GraceAggregatingTransform.cpp index adf25d13f29d7..63dc3c3457f6f 100644 --- a/cpp-ch/local-engine/Operator/GraceAggregatingTransform.cpp +++ b/cpp-ch/local-engine/Operator/GraceAggregatingTransform.cpp @@ -44,7 +44,7 @@ GraceAggregatingTransform::GraceAggregatingTransform( , aggregate_columns(params_->params.aggregates_size) , no_pre_aggregated(no_pre_aggregated_) , final_output(final_output_) - , tmp_data_disk(std::make_unique(context_->getTempDataOnDisk())) + , tmp_data_disk(context_->getTempDataOnDisk()) { output_header = params->getHeader(); auto config = GraceMergingAggregateConfig::loadFromContext(context); @@ -302,10 +302,13 @@ void GraceAggregatingTransform::flushBuckets() flushBucket(i); } -static size_t flushBlocksInfoDisk(DB::TemporaryFileStream * file_stream, std::list & blocks) +static size_t flushBlocksInfoDisk(std::optional& file_stream, std::list & blocks) { size_t flush_bytes = 0; DB::Blocks tmp_blocks; + if (!file_stream) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "file_stream is empty"); + auto & tmp_stream = file_stream.value(); while (!blocks.empty()) { while (!blocks.empty()) @@ -322,11 +325,11 @@ static size_t flushBlocksInfoDisk(DB::TemporaryFileStream * file_stream, std::li flush_bytes += merged_block.bytes(); if (merged_block.rows()) { - file_stream->write(merged_block); + tmp_stream->write(merged_block); } } if (flush_bytes) - file_stream->flush(); + tmp_stream->flush(); return flush_bytes; } @@ -338,7 +341,7 @@ size_t GraceAggregatingTransform::flushBucket(size_t bucket_index) if (!file_stream.original_blocks.empty()) { if (!file_stream.original_file_stream) - file_stream.original_file_stream = &tmp_data_disk->createStream(header); + file_stream.original_file_stream = DB::TemporaryBlockStreamHolder(header, tmp_data_disk.get()); flush_bytes += flushBlocksInfoDisk(file_stream.original_file_stream, file_stream.original_blocks); } if (!file_stream.intermediate_blocks.empty()) @@ -346,7 +349,7 @@ size_t GraceAggregatingTransform::flushBucket(size_t bucket_index) if (!file_stream.intermediate_file_stream) { auto intermediate_header = params->aggregator.getHeader(false); - file_stream.intermediate_file_stream = &tmp_data_disk->createStream(intermediate_header); + file_stream.intermediate_file_stream = DB::TemporaryBlockStreamHolder(header, tmp_data_disk.get()); } flush_bytes += flushBlocksInfoDisk(file_stream.intermediate_file_stream, file_stream.intermediate_blocks); } @@ -373,9 +376,10 @@ std::unique_ptr GraceAggregatingTransform::prepareB if (buffer_file_stream.intermediate_file_stream) { buffer_file_stream.intermediate_file_stream->finishWriting(); + auto reader = buffer_file_stream.intermediate_file_stream->getReadStream(); while (true) { - auto block = buffer_file_stream.intermediate_file_stream->read(); + auto block = reader->read(); if (!block.rows()) break; read_bytes += block.bytes(); @@ -383,7 +387,7 @@ std::unique_ptr GraceAggregatingTransform::prepareB mergeOneBlock(block, false); block = {}; } - buffer_file_stream.intermediate_file_stream = nullptr; + buffer_file_stream.intermediate_file_stream.reset(); total_read_disk_time += watch.elapsedMilliseconds(); } if (!buffer_file_stream.intermediate_blocks.empty()) @@ -398,9 +402,10 @@ std::unique_ptr GraceAggregatingTransform::prepareB if (buffer_file_stream.original_file_stream) { buffer_file_stream.original_file_stream->finishWriting(); + auto reader = buffer_file_stream.original_file_stream->getReadStream(); while (true) { - auto block = buffer_file_stream.original_file_stream->read(); + auto block = reader->read(); if (!block.rows()) break; read_bytes += block.bytes(); @@ -408,7 +413,7 @@ std::unique_ptr GraceAggregatingTransform::prepareB mergeOneBlock(block, true); block = {}; } - buffer_file_stream.original_file_stream = nullptr; + buffer_file_stream.original_file_stream.reset(); total_read_disk_time += watch.elapsedMilliseconds(); } if (!buffer_file_stream.original_blocks.empty()) diff --git a/cpp-ch/local-engine/Operator/GraceAggregatingTransform.h b/cpp-ch/local-engine/Operator/GraceAggregatingTransform.h index c2b787393a1be..612a58b3c9439 100644 --- a/cpp-ch/local-engine/Operator/GraceAggregatingTransform.h +++ b/cpp-ch/local-engine/Operator/GraceAggregatingTransform.h @@ -59,7 +59,7 @@ class GraceAggregatingTransform : public DB::IProcessor DB::Aggregator::AggregateColumns aggregate_columns; DB::AggregatingTransformParamsPtr params; DB::ContextPtr context; - DB::TemporaryDataOnDiskPtr tmp_data_disk; + DB::TemporaryDataOnDiskScopePtr tmp_data_disk; DB::AggregatedDataVariantsPtr current_data_variants = nullptr; size_t current_bucket_index = 0; @@ -83,9 +83,9 @@ class GraceAggregatingTransform : public DB::IProcessor /// Only be used when there is no pre-aggregated step, store the original input blocks. std::list original_blocks; /// store the intermediate result blocks. - DB::TemporaryFileStream * intermediate_file_stream = nullptr; + std::optional intermediate_file_stream; /// Only be used when there is no pre-aggregated step - DB::TemporaryFileStream * original_file_stream = nullptr; + std::optional original_file_stream; size_t pending_bytes = 0; }; std::unordered_map buckets; diff --git a/cpp-ch/local-engine/Parser/RelParsers/CrossRelParser.cpp b/cpp-ch/local-engine/Parser/RelParsers/CrossRelParser.cpp index 59d4c39f87377..5a6f229744fc1 100644 --- a/cpp-ch/local-engine/Parser/RelParsers/CrossRelParser.cpp +++ b/cpp-ch/local-engine/Parser/RelParsers/CrossRelParser.cpp @@ -41,6 +41,7 @@ namespace DB namespace Setting { extern const SettingsUInt64 max_block_size; +extern const SettingsUInt64 min_joined_block_size_bytes; } namespace ErrorCodes { @@ -200,7 +201,13 @@ DB::QueryPlanPtr CrossRelParser::parseJoin(const substrait::CrossRel & join, DB: { JoinPtr hash_join = std::make_shared(table_join, right->getCurrentHeader().cloneEmpty()); QueryPlanStepPtr join_step = std::make_unique( - left->getCurrentHeader(), right->getCurrentHeader(), hash_join, context->getSettingsRef()[Setting::max_block_size], 1, false); + left->getCurrentHeader(), + right->getCurrentHeader(), + hash_join, + context->getSettingsRef()[Setting::max_block_size], + context->getSettingsRef()[Setting::min_joined_block_size_bytes], + 1, + false); join_step->setStepDescription("CROSS_JOIN"); steps.emplace_back(join_step.get()); std::vector plans; diff --git a/cpp-ch/local-engine/Parser/RelParsers/JoinRelParser.cpp b/cpp-ch/local-engine/Parser/RelParsers/JoinRelParser.cpp index 46f7926cf7291..7493471697f1c 100644 --- a/cpp-ch/local-engine/Parser/RelParsers/JoinRelParser.cpp +++ b/cpp-ch/local-engine/Parser/RelParsers/JoinRelParser.cpp @@ -46,6 +46,7 @@ namespace Setting { extern const SettingsJoinAlgorithm join_algorithm; extern const SettingsUInt64 max_block_size; +extern const SettingsUInt64 min_joined_block_size_bytes; } namespace ErrorCodes { @@ -315,7 +316,13 @@ DB::QueryPlanPtr JoinRelParser::parseJoin(const substrait::JoinRel & join, DB::Q JoinPtr smj_join = std::make_shared(table_join, right->getCurrentHeader().cloneEmpty(), -1); MultiEnum join_algorithm = context->getSettingsRef()[Setting::join_algorithm]; QueryPlanStepPtr join_step = std::make_unique( - left->getCurrentHeader(), right->getCurrentHeader(), smj_join, context->getSettingsRef()[Setting::max_block_size], 1, false); + left->getCurrentHeader(), + right->getCurrentHeader(), + smj_join, + context->getSettingsRef()[Setting::max_block_size], + context->getSettingsRef()[Setting::min_joined_block_size_bytes], + 1, + false); join_step->setStepDescription("SORT_MERGE_JOIN"); steps.emplace_back(join_step.get()); @@ -448,7 +455,7 @@ void JoinRelParser::collectJoinKeys( table_join.addDisjunct(); const auto & expr = join_rel.expression(); auto & join_clause = table_join.getClauses().back(); - std::list expressions_stack; + std::list expressions_stack; expressions_stack.push_back(&expr); while (!expressions_stack.empty()) { @@ -778,6 +785,7 @@ DB::QueryPlanPtr JoinRelParser::buildMultiOnClauseHashJoin( right_plan->getCurrentHeader(), hash_join, context->getSettingsRef()[Setting::max_block_size], + context->getSettingsRef()[Setting::min_joined_block_size_bytes], 1, false); join_step->setStepDescription("Multi join on clause hash join"); @@ -817,6 +825,7 @@ DB::QueryPlanPtr JoinRelParser::buildSingleOnClauseHashJoin( right_plan->getCurrentHeader(), hash_join, context->getSettingsRef()[Setting::max_block_size], + context->getSettingsRef()[Setting::min_joined_block_size_bytes], 1, false); diff --git a/cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.cpp b/cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.cpp index 0be7e0d892b75..17587e5200ef0 100644 --- a/cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.cpp +++ b/cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.cpp @@ -74,7 +74,7 @@ void SparkStorageMergeTree::analysisPartsByRanges(DB::ReadFromMergeTree & source sum_ranges += part.ranges.size(); sum_marks += part.getMarksCount(); sum_rows += part.getRowsCount(); - total_marks_pk += part.data_part->index_granularity.getMarksCountWithoutFinal(); + total_marks_pk += part.data_part->index_granularity->getMarksCountWithoutFinal(); for (auto range : part.ranges) sum_marks_pk += range.getNumberOfMarks(); @@ -487,6 +487,12 @@ MergeTreeDataWriter::TemporaryPart SparkMergeTreeDataWriter::writeTempPart( /// either default lz4 or compression method with zero thresholds on absolute and relative part size. auto compression_codec = data.getContext()->chooseCompressionCodec(0, 0); auto txn = context->getCurrentTransaction(); + auto index_granularity_ptr = createMergeTreeIndexGranularity( + block.rows(), + block.bytes(), + *data.getSettings(), + new_data_part->index_granularity_info, + /*blocks_are_granules=*/false); auto out = std::make_unique( new_data_part, metadata_snapshot, @@ -494,6 +500,7 @@ MergeTreeDataWriter::TemporaryPart SparkMergeTreeDataWriter::writeTempPart( indices, MergeTreeStatisticsFactory::instance().getMany(metadata_snapshot->getColumns()), compression_codec, + index_granularity_ptr, txn ? txn->tid : Tx::PrehistoricTID, false, false, From 2649fa7b34edf99268bff0d95d9d9621683f839c Mon Sep 17 00:00:00 2001 From: Yuan Date: Wed, 27 Nov 2024 09:06:50 +0800 Subject: [PATCH 13/13] [GLUTEN-8046][VL] CI: fix velox cache/bundle package script (#8051) This patch fixes the cache/build script due to nodejs version update in GHA This patch also adds centos7 tests back fixes #8046 Signed-off-by: Yuan Zhou --- .github/workflows/build_bundle_package.yml | 19 ++++-- .github/workflows/velox_backend.yml | 78 +++++++++++++++++++--- .github/workflows/velox_backend_cache.yml | 8 +-- 3 files changed, 86 insertions(+), 19 deletions(-) diff --git a/.github/workflows/build_bundle_package.yml b/.github/workflows/build_bundle_package.yml index 9b94d302f9629..c7ddd8748b25b 100644 --- a/.github/workflows/build_bundle_package.yml +++ b/.github/workflows/build_bundle_package.yml @@ -38,9 +38,8 @@ on: jobs: build-native-lib: runs-on: ubuntu-20.04 - container: apache/gluten:vcpkg-centos-7 steps: - - uses: actions/checkout@v2 + - uses: actions/checkout@v4 - name: Get Ccache uses: actions/cache/restore@v3 with: @@ -50,9 +49,15 @@ jobs: ccache-centos7-release-default - name: Build Gluten velox third party run: | - df -a - cd $GITHUB_WORKSPACE/ - bash dev/ci-velox-buildstatic-centos-7.sh + docker run -v $GITHUB_WORKSPACE:/work -w /work apache/gluten:vcpkg-centos-7 bash -c " + df -a + cd /work + export CCACHE_DIR=/work/.ccache + bash dev/ci-velox-buildstatic-centos-7.sh + ccache -s + mkdir -p /work/.m2/repository/org/apache/arrow/ + cp -r /root/.m2/repository/org/apache/arrow/* /work/.m2/repository/org/apache/arrow/ + " - name: Upload native libs uses: actions/upload-artifact@v3 with: @@ -65,10 +70,10 @@ jobs: path: /root/.m2/repository/org/apache/arrow/ name: velox-arrow-jar-centos-7-${{github.sha}} - build-bundle-package-centos7: + build-bundle-package-centos8: needs: build-native-lib runs-on: ubuntu-20.04 - container: centos:7 + container: centos:8 steps: - uses: actions/checkout@v2 - name: Download All Artifacts diff --git a/.github/workflows/velox_backend.yml b/.github/workflows/velox_backend.yml index 56b43975acee1..3c4bb3e2e97df 100644 --- a/.github/workflows/velox_backend.yml +++ b/.github/workflows/velox_backend.yml @@ -164,12 +164,12 @@ jobs: && GLUTEN_IT_JVM_ARGS=-Xmx5G sbin/gluten-it.sh queries-compare \ --local --preset=velox --benchmark-type=ds --error-on-memleak --off-heap-size=10g -s=1.0 --threads=16 --iterations=1 - run-tpc-test-centos: + run-tpc-test-centos8: needs: build-native-lib-centos-7 strategy: fail-fast: false matrix: - os: [ "centos:8" ] #TODO: add centos7 tests back with docker run + os: [ "centos:8" ] spark: [ "spark-3.2", "spark-3.3", "spark-3.4", "spark-3.5" ] java: [ "java-8", "java-11", "java-17" ] # Spark supports JDK17 since 3.3 and later, see https://issues.apache.org/jira/browse/SPARK-33772 @@ -255,6 +255,68 @@ jobs: --local --preset=velox --benchmark-type=ds --error-on-memleak --off-heap-size=10g -s=1.0 --threads=16 --iterations=1 \ --extra-conf=spark.gluten.ras.enabled=true + run-tpc-test-centos7: + needs: build-native-lib-centos-7 + strategy: + fail-fast: false + matrix: + spark: [ "spark-3.2", "spark-3.3", "spark-3.4", "spark-3.5" ] + java: [ "java-8" ] + # Spark supports JDK17 since 3.3 and later, see https://issues.apache.org/jira/browse/SPARK-33772 + runs-on: ubuntu-20.04 + steps: + - uses: actions/checkout@v4 + - name: Download All Native Artifacts + uses: actions/download-artifact@v3 + with: + name: velox-native-lib-centos-7-${{github.sha}} + path: ./cpp/build/releases/ + - name: Download All Arrow Jar Artifacts + uses: actions/download-artifact@v3 + with: + name: arrow-jars-centos-7-${{github.sha}} + path: .m2/repository/org/apache/arrow/ + - name: Build and run TPCH/DS tests + run: | + docker pull centos:7 + docker run -v $GITHUB_WORKSPACE:/work -v /$GITHUB_WORKSPACE/.m2:/root/.m2/ -w /work \ + -e matrix.java=${{ matrix.java }} -e matrix.spark=${{ matrix.spark }} \ + centos:7 \ + bash -c " + sed -i -e 's|mirrorlist=|#mirrorlist=|g' /etc/yum.repos.d/CentOS-* || true + sed -i -e 's|#baseurl=http://mirror.centos.org|baseurl=http://vault.centos.org|g' /etc/yum.repos.d/CentOS-* || true + + # Setup java and maven + yum update -y && yum install -y java-1.8.0-openjdk-devel wget + + wget -nv https://downloads.apache.org/maven/maven-3/3.8.8/binaries/apache-maven-3.8.8-bin.tar.gz + tar -xvf apache-maven-3.8.8-bin.tar.gz && mv apache-maven-3.8.8 /usr/lib/maven + export PATH=${PATH}:/usr/lib/maven/bin + + # Set environment variables + export JAVA_HOME=/usr/lib/jvm/java-1.8.0-openjdk + + # Build gluten-it + mvn -ntp clean install -P${{ matrix.spark }} -P${{ matrix.java }} -Pbackends-velox -DskipTests + cd /work/tools/gluten-it + mvn -ntp clean install -P${{ matrix.spark }} -P${{ matrix.java }} + + # Run TPC-H / TPC-DS + GLUTEN_IT_JVM_ARGS=-Xmx5G sbin/gluten-it.sh queries-compare \ + --local --preset=velox --benchmark-type=h --error-on-memleak --off-heap-size=10g -s=1.0 --threads=16 --iterations=1 + GLUTEN_IT_JVM_ARGS=-Xmx5G sbin/gluten-it.sh queries-compare \ + --local --preset=velox --benchmark-type=ds --error-on-memleak --off-heap-size=10g -s=1.0 --threads=16 --iterations=1 + + # Run TPC-H / TPC-DS with RAS + cd /work/tools/gluten-it + GLUTEN_IT_JVM_ARGS=-Xmx5G sbin/gluten-it.sh queries-compare \ + --local --preset=velox --benchmark-type=h --error-on-memleak --off-heap-size=10g -s=1.0 --threads=16 --iterations=1 \ + --extra-conf=spark.gluten.ras.enabled=true \ + && GLUTEN_IT_JVM_ARGS=-Xmx5G sbin/gluten-it.sh queries-compare \ + --local --preset=velox --benchmark-type=ds --error-on-memleak --off-heap-size=10g -s=1.0 --threads=16 --iterations=1 \ + --extra-conf=spark.gluten.ras.enabled=true + " + run-tpc-test-ubuntu-oom: needs: build-native-lib-centos-7 strategy: @@ -962,12 +1024,12 @@ jobs: df -a bash dev/ci-velox-buildshared-centos-8.sh ccache -s - - name: "Save ccache" - uses: actions/cache/save@v3 - id: ccache - with: - path: '${{ env.CCACHE_DIR }}' - key: ccache-centos8-release-default-${{github.sha}} + # - name: "Save ccache" + # uses: actions/cache/save@v3 + # id: ccache + # with: + # path: '${{ env.CCACHE_DIR }}' + # key: ccache-centos8-release-default-${{github.sha}} - name: Run CPP unit test run: | cd ./cpp/build && ctest -V diff --git a/.github/workflows/velox_backend_cache.yml b/.github/workflows/velox_backend_cache.yml index e969ffb7415e4..d0b295b5f48e8 100644 --- a/.github/workflows/velox_backend_cache.yml +++ b/.github/workflows/velox_backend_cache.yml @@ -31,9 +31,8 @@ concurrency: jobs: cache-native-lib-centos-7: runs-on: ubuntu-20.04 - container: apache/gluten:vcpkg-centos-7 steps: - - uses: actions/checkout@v2 + - uses: actions/checkout@v4 - name: Get Ccache uses: actions/cache/restore@v3 with: @@ -43,8 +42,9 @@ jobs: ccache-centos7-release-default - name: Build Gluten native libraries run: | - df -a - bash dev/ci-velox-buildstatic-centos-7.sh + docker run -v $GITHUB_WORKSPACE:/work -w /work apache/gluten:vcpkg-centos-7 bash -c " + bash dev/ci-velox-buildstatic-centos-7.sh + " - name: Save Ccache uses: actions/cache/save@v3 id: ccache