diff --git a/backends-velox/src/main/scala/org/apache/gluten/config/VeloxConfig.scala b/backends-velox/src/main/scala/org/apache/gluten/config/VeloxConfig.scala index e87b18e07884..cd26117c28dc 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/config/VeloxConfig.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/config/VeloxConfig.scala @@ -80,6 +80,8 @@ class VeloxConfig(conf: SQLConf) extends GlutenConfig(conf) { getConf(ENABLE_ENHANCED_FEATURES) def veloxPreferredBatchBytes: Long = getConf(COLUMNAR_VELOX_PREFERRED_BATCH_BYTES) + + def cudfEnableTableScan: Boolean = getConf(CUDF_ENABLE_TABLE_SCAN) } object VeloxConfig { @@ -609,6 +611,12 @@ object VeloxConfig { .intConf .createWithDefault(50) + val CUDF_ENABLE_TABLE_SCAN = + buildStaticConf("spark.gluten.sql.columnar.backend.velox.cudf.enableTableScan") + .doc("Enable cudf table scan") + .booleanConf + .createWithDefault(false) + val MEMORY_DUMP_ON_EXIT = buildConf("spark.gluten.monitor.memoryDumpOnExit") .internal() diff --git a/backends-velox/src/main/scala/org/apache/gluten/extension/CudfNodeValidationRule.scala b/backends-velox/src/main/scala/org/apache/gluten/extension/CudfNodeValidationRule.scala index 9b63fb2c5b7e..20e819e21582 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/extension/CudfNodeValidationRule.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/extension/CudfNodeValidationRule.scala @@ -16,7 +16,7 @@ */ package org.apache.gluten.extension -import org.apache.gluten.config.GlutenConfig +import org.apache.gluten.config.{GlutenConfig, VeloxConfig} import org.apache.gluten.execution.{CudfTag, LeafTransformSupport, WholeStageTransformer} import org.apache.spark.sql.catalyst.rules.Rule @@ -31,12 +31,16 @@ case class CudfNodeValidationRule(glutenConf: GlutenConfig) extends Rule[SparkPl } plan.transformUp { case transformer: WholeStageTransformer => - // Spark3.2 does not have exists - val hasLeaf = transformer.find { - case _: LeafTransformSupport => true - case _ => false - }.isDefined - transformer.setTagValue(CudfTag.CudfTag, !hasLeaf) + if (!VeloxConfig.get.cudfEnableTableScan) { + // Spark3.2 does not have exists + val hasLeaf = transformer.find { + case _: LeafTransformSupport => true + case _ => false + }.isDefined + transformer.setTagValue(CudfTag.CudfTag, !hasLeaf) + } else { + transformer.setTagValue(CudfTag.CudfTag, true) + } transformer } } diff --git a/cpp/core/config/GlutenConfig.h b/cpp/core/config/GlutenConfig.h index 0384622e61af..2f5992b9c0df 100644 --- a/cpp/core/config/GlutenConfig.h +++ b/cpp/core/config/GlutenConfig.h @@ -94,12 +94,10 @@ const std::string kSparkLegacyStatisticalAggregate = "spark.sql.legacy.statistic const std::string kSparkJsonIgnoreNullFields = "spark.sql.jsonGenerator.ignoreNullFields"; // cudf -#ifdef GLUTEN_ENABLE_GPU const std::string kCudfEnabled = "spark.gluten.sql.columnar.cudf"; const bool kCudfEnabledDefault = "true"; const std::string kDebugCudf = "spark.gluten.sql.debug.cudf"; const bool kDebugCudfDefault = "false"; -#endif std::unordered_map parseConfMap(JNIEnv* env, const uint8_t* planData, const int32_t planDataLength); diff --git a/cpp/velox/CMakeLists.txt b/cpp/velox/CMakeLists.txt index dd3cd60d9cb3..741fa509b2e0 100644 --- a/cpp/velox/CMakeLists.txt +++ b/cpp/velox/CMakeLists.txt @@ -414,9 +414,27 @@ if(ENABLE_GPU) import_library( facebook::velox::velox_cudf_vector ${VELOX_BUILD_PATH}/velox/experimental/cudf/vector/libvelox_cudf_vector.a) + import_library( + facebook::velox::velox_cudf_hive_connector + ${VELOX_BUILD_PATH}/velox/experimental/cudf/connectors/hive/libvelox_cudf_hive_connector.a + ) + target_include_directories( + velox + PRIVATE ${VELOX_BUILD_PATH}/_deps/cudf-src/cpp/include + ${VELOX_BUILD_PATH}/_deps/rmm-src/cpp/include + ${VELOX_BUILD_PATH}/_deps/kvikio-src/cpp/include + ${VELOX_BUILD_PATH}/_deps/nvtx3-src/c/include + ${VELOX_BUILD_PATH}/_deps/nvcomp_proprietary_binary-src/include + ${VELOX_BUILD_PATH}/_deps/rapids_logger-src/include + /usr/local/cuda/include) + + target_compile_definitions( + velox PRIVATE LIBCUDACXX_ENABLE_EXPERIMENTAL_MEMORY_RESOURCE) - target_link_libraries(velox PUBLIC facebook::velox::velox_cudf_exec - facebook::velox::velox_cudf_vector) + target_link_libraries( + velox + PUBLIC facebook::velox::velox_cudf_exec facebook::velox::velox_cudf_vector + facebook::velox::velox_cudf_hive_connector) target_link_libraries(velox PRIVATE ${VELOX_BUILD_PATH}/_deps/cudf-build/libcudf.so) endif() diff --git a/cpp/velox/compute/VeloxBackend.cc b/cpp/velox/compute/VeloxBackend.cc index e26cd852b61a..54cb08bf572a 100644 --- a/cpp/velox/compute/VeloxBackend.cc +++ b/cpp/velox/compute/VeloxBackend.cc @@ -28,6 +28,7 @@ #include "utils/qat/QatCodec.h" #endif #ifdef GLUTEN_ENABLE_GPU +#include "velox/experimental/cudf/connectors/hive/CudfHiveConnector.h" #include "velox/experimental/cudf/exec/ToCudf.h" #endif @@ -306,6 +307,14 @@ void VeloxBackend::initConnector(const std::shared_ptr(kHiveConnectorId, hiveConf, ioExecutor_.get())); +#ifdef GLUTEN_ENABLE_GPU + if (backendConf_->get(kCudfEnableTableScan, kCudfEnableTableScanDefault) && + backendConf_->get(kCudfEnabled, kCudfEnabledDefault)) { + facebook::velox::cudf_velox::connector::hive::CudfHiveConnectorFactory factory; + auto hiveConnector = factory.newConnector(kCudfHiveConnectorId, hiveConf, ioExecutor_.get()); + facebook::velox::connector::registerConnector(hiveConnector); + } +#endif } void VeloxBackend::initUdf() { diff --git a/cpp/velox/compute/WholeStageResultIterator.cc b/cpp/velox/compute/WholeStageResultIterator.cc index d2b130f62982..45fe4895e005 100644 --- a/cpp/velox/compute/WholeStageResultIterator.cc +++ b/cpp/velox/compute/WholeStageResultIterator.cc @@ -23,7 +23,9 @@ #include "velox/exec/PlanNodeStats.h" #ifdef GLUTEN_ENABLE_GPU #include +#include #include "velox/experimental/cudf/exec/ToCudf.h" +#include "velox/experimental/cudf/connectors/hive/CudfHiveConnectorSplit.h" #endif using namespace facebook; @@ -131,14 +133,19 @@ WholeStageResultIterator::WholeStageResultIterator( const auto& format = scanInfo->format; const auto& partitionColumns = scanInfo->partitionColumns; const auto& metadataColumns = scanInfo->metadataColumns; + // Under the pre-condition that all the split infos has same partition column and format. + const auto canUseCudfConnector = scanInfo->canUseCudfConnector(); std::vector> connectorSplits; connectorSplits.reserve(paths.size()); for (int idx = 0; idx < paths.size(); idx++) { - auto partitionColumn = partitionColumns[idx]; auto metadataColumn = metadataColumns[idx]; std::unordered_map> partitionKeys; - constructPartitionColumns(partitionKeys, partitionColumn); + if (!partitionColumns.empty()) { + auto partitionColumn = partitionColumns[idx]; + constructPartitionColumns(partitionKeys, partitionColumn); + } + std::shared_ptr split; if (auto icebergSplitInfo = std::dynamic_pointer_cast(scanInfo)) { // Set Iceberg split. @@ -159,8 +166,16 @@ WholeStageResultIterator::WholeStageResultIterator( std::unordered_map(), properties[idx]); } else { + auto connectorId = kHiveConnectorId; +#ifdef GLUTEN_ENABLE_GPU + if (canUseCudfConnector) { + connectorId = kCudfHiveConnectorId; + VELOX_CHECK_EQ(starts[idx], 0, "Not support split file"); + VELOX_CHECK_EQ(lengths[idx], scanInfo->properties[idx]->fileSize, "Not support split file"); + } +#endif split = std::make_shared( - kHiveConnectorId, + connectorId, paths[idx], format, starts[idx], diff --git a/cpp/velox/config/VeloxConfig.h b/cpp/velox/config/VeloxConfig.h index e37c99987e1c..e74cc838862d 100644 --- a/cpp/velox/config/VeloxConfig.h +++ b/cpp/velox/config/VeloxConfig.h @@ -180,4 +180,9 @@ const int32_t kCudfMemoryPercentDefault = 50; /// Preferred size of batches in bytes to be returned by operators. const std::string kVeloxPreferredBatchBytes = "spark.gluten.sql.columnar.backend.velox.preferredBatchBytes"; +/// cudf +const std::string kCudfEnableTableScan = "spark.gluten.sql.columnar.backend.velox.cudf.enableTableScan"; +const bool kCudfEnableTableScanDefault = false; +const std::string kCudfHiveConnectorId = "cudf-hive"; + } // namespace gluten diff --git a/cpp/velox/substrait/SubstraitToVeloxPlan.cc b/cpp/velox/substrait/SubstraitToVeloxPlan.cc index ceaabb5ad34f..6f32c5237ec9 100644 --- a/cpp/velox/substrait/SubstraitToVeloxPlan.cc +++ b/cpp/velox/substrait/SubstraitToVeloxPlan.cc @@ -31,9 +31,29 @@ #include "config/GlutenConfig.h" #include "config/VeloxConfig.h" +#ifdef GLUTEN_ENABLE_GPU +#include "velox/experimental/cudf/connectors/hive/CudfHiveDataSink.h" +#include "velox/experimental/cudf/connectors/hive/CudfHiveTableHandle.h" +#include "velox/experimental/cudf/exec/ToCudf.h" +#include "velox/experimental/cudf/exec/VeloxCudfInterop.h" + +using namespace cudf_velox::connector::hive; +#endif + namespace gluten { namespace { +bool useCudfTableHandle(const std::vector>& splitInfos) { +#ifdef GLUTEN_ENABLE_GPU + if (splitInfos.empty()) { + return false; + } + return splitInfos[0]->canUseCudfConnector(); +#else + return false; +#endif +} + core::SortOrder toSortOrder(const ::substrait::SortField& sortField) { switch (sortField.direction()) { case ::substrait::SortField_SortDirection_SORT_DIRECTION_ASC_NULLS_FIRST: @@ -82,7 +102,7 @@ EmitInfo getEmitInfo(const ::substrait::RelCommon& relCommon, const core::PlanNo RowTypePtr getJoinInputType(const core::PlanNodePtr& leftNode, const core::PlanNodePtr& rightNode) { auto outputSize = leftNode->outputType()->size() + rightNode->outputType()->size(); std::vector outputNames; - std::vector> outputTypes; + std::vector outputTypes; outputNames.reserve(outputSize); outputTypes.reserve(outputSize); for (const auto& node : {leftNode, rightNode}) { @@ -119,7 +139,7 @@ RowTypePtr getJoinOutputType( if (outputMayIncludeLeftColumns) { if (core::isLeftSemiProjectJoin(joinType)) { std::vector outputNames = leftNode->outputType()->names(); - std::vector> outputTypes = leftNode->outputType()->children(); + std::vector outputTypes = leftNode->outputType()->children(); outputNames.emplace_back("exists"); outputTypes.emplace_back(BOOLEAN()); return std::make_shared(std::move(outputNames), std::move(outputTypes)); @@ -131,7 +151,7 @@ RowTypePtr getJoinOutputType( if (outputMayIncludeRightColumns) { if (core::isRightSemiProjectJoin(joinType)) { std::vector outputNames = rightNode->outputType()->names(); - std::vector> outputTypes = rightNode->outputType()->children(); + std::vector outputTypes = rightNode->outputType()->children(); outputNames.emplace_back("exists"); outputTypes.emplace_back(BOOLEAN()); return std::make_shared(std::move(outputNames), std::move(outputTypes)); @@ -144,6 +164,23 @@ RowTypePtr getJoinOutputType( } // namespace +bool SplitInfo::canUseCudfConnector() { + bool isEmpty = partitionColumns.empty(); + + if (!isEmpty) { + // Check if all maps are empty + bool allMapsEmpty = true; + for (const auto& m : partitionColumns) { + if (!m.empty()) { + allMapsEmpty = false; + break; + } + } + isEmpty = allMapsEmpty; + } + return isEmpty && format == dwio::common::FileFormat::PARQUET; +} + core::PlanNodePtr SubstraitToVeloxPlanConverter::processEmit( const ::substrait::RelCommon& relCommon, const core::PlanNodePtr& noEmitNode) { @@ -559,17 +596,19 @@ std::shared_ptr makeHiveInsertTableHandl } if (std::find(partitionedBy.cbegin(), partitionedBy.cend(), tableColumnNames.at(i)) != partitionedBy.cend()) { ++numPartitionColumns; - columnHandles.emplace_back(std::make_shared( - tableColumnNames.at(i), - connector::hive::HiveColumnHandle::ColumnType::kPartitionKey, - tableColumnTypes.at(i), - tableColumnTypes.at(i))); + columnHandles.emplace_back( + std::make_shared( + tableColumnNames.at(i), + connector::hive::HiveColumnHandle::ColumnType::kPartitionKey, + tableColumnTypes.at(i), + tableColumnTypes.at(i))); } else { - columnHandles.emplace_back(std::make_shared( - tableColumnNames.at(i), - connector::hive::HiveColumnHandle::ColumnType::kRegular, - tableColumnTypes.at(i), - tableColumnTypes.at(i))); + columnHandles.emplace_back( + std::make_shared( + tableColumnNames.at(i), + connector::hive::HiveColumnHandle::ColumnType::kRegular, + tableColumnTypes.at(i), + tableColumnTypes.at(i))); } } VELOX_CHECK_EQ(numPartitionColumns, partitionedBy.size()); @@ -585,6 +624,29 @@ std::shared_ptr makeHiveInsertTableHandl writerOptions); } +#ifdef GLUTEN_ENABLE_GPU +std::shared_ptr makeCudfHiveInsertTableHandle( + const std::vector& tableColumnNames, + const std::vector& tableColumnTypes, + std::shared_ptr locationHandle, + const std::optional compressionKind, + const std::unordered_map& serdeParameters, + const std::shared_ptr& writerOptions) { + std::vector> columnHandles; + + for (int i = 0; i < tableColumnNames.size(); ++i) { + columnHandles.push_back( + std::make_shared( + tableColumnNames.at(i), + tableColumnTypes.at(i), + cudf::data_type{cudf_velox::veloxToCudfTypeId(tableColumnTypes.at(i))})); + } + + return std::make_shared( + columnHandles, locationHandle, compressionKind, serdeParameters, writerOptions); +} +#endif + core::PlanNodePtr SubstraitToVeloxPlanConverter::toVeloxPlan(const ::substrait::WriteRel& writeRel) { core::PlanNodePtr childNode; if (writeRel.has_input()) { @@ -678,23 +740,40 @@ core::PlanNodePtr SubstraitToVeloxPlanConverter::toVeloxPlan(const ::substrait:: // Spark's default compression code is snappy. const auto& compressionKind = writerOptions->compressionKind.value_or(common::CompressionKind::CompressionKind_SNAPPY); - + std::shared_ptr tableHandle; + if (useCudfTableHandle(splitInfos_) && veloxCfg_->get(kCudfEnableTableScan, kCudfEnableTableScanDefault) && + veloxCfg_->get(kCudfEnabled, kCudfEnabledDefault)) { + #ifdef GLUTEN_ENABLE_GPU + tableHandle = std::make_shared( + kCudfHiveConnectorId, + makeCudfHiveInsertTableHandle( + tableColumnNames, /*inputType->names() clolumn name is different*/ + inputType->children(), + std::make_shared( + writePath, cudf_velox::connector::hive::LocationHandle::TableType::kNew, fileName), + compressionKind, + {}, + writerOptions)); +#endif + } else { + tableHandle = std::make_shared( + kHiveConnectorId, + makeHiveInsertTableHandle( + tableColumnNames, /*inputType->names() clolumn name is different*/ + inputType->children(), + partitionedKey, + bucketProperty, + makeLocationHandle(writePath, fileName, fileFormat, compressionKind, bucketProperty != nullptr), + writerOptions, + fileFormat, + compressionKind)); + } return std::make_shared( nextPlanNodeId(), inputType, tableColumnNames, std::nullopt, /*columnStatsSpec*/ - std::make_shared( - kHiveConnectorId, - makeHiveInsertTableHandle( - tableColumnNames, /*inputType->names() clolumn name is different*/ - inputType->children(), - partitionedKey, - bucketProperty, - makeLocationHandle(writePath, fileName, fileFormat, compressionKind, bucketProperty != nullptr), - writerOptions, - fileFormat, - compressionKind)), + tableHandle, (!partitionedKey.empty()), exec::TableWriteTraits::outputType(std::nullopt), connector::CommitStrategy::kNoCommit, @@ -1277,14 +1356,15 @@ core::PlanNodePtr SubstraitToVeloxPlanConverter::toVeloxPlan(const ::substrait:: auto names = colNameList; auto types = veloxTypeList; auto dataColumns = ROW(std::move(names), std::move(types)); - std::shared_ptr tableHandle; - if (!readRel.has_filter()) { - tableHandle = std::make_shared( - kHiveConnectorId, "hive_table", filterPushdownEnabled, common::SubfieldFilters{}, nullptr, dataColumns); + connector::ConnectorTableHandlePtr tableHandle; + auto remainingFilter = readRel.has_filter() ? exprConverter_->toVeloxExpr(readRel.filter(), dataColumns) : nullptr; + if (useCudfTableHandle(splitInfos_)) { +#ifdef GLUTEN_ENABLE_GPU + tableHandle = std::make_shared( + kCudfHiveConnectorId, "cudf_hive_table", filterPushdownEnabled, nullptr, remainingFilter, dataColumns); +#endif } else { common::SubfieldFilters subfieldFilters; - auto remainingFilter = exprConverter_->toVeloxExpr(readRel.filter(), dataColumns); - tableHandle = std::make_shared( kHiveConnectorId, "hive_table", diff --git a/cpp/velox/substrait/SubstraitToVeloxPlan.h b/cpp/velox/substrait/SubstraitToVeloxPlan.h index 1d816da612f6..b00c3447f712 100644 --- a/cpp/velox/substrait/SubstraitToVeloxPlan.h +++ b/cpp/velox/substrait/SubstraitToVeloxPlan.h @@ -57,6 +57,8 @@ struct SplitInfo { /// Make SplitInfo polymorphic virtual ~SplitInfo() = default; + + bool canUseCudfConnector(); }; /// This class is used to convert the Substrait plan into Velox plan. diff --git a/docs/Configuration.md b/docs/Configuration.md index b7e725278a35..9aacdb377743 100644 --- a/docs/Configuration.md +++ b/docs/Configuration.md @@ -75,21 +75,21 @@ nav_order: 15 | spark.gluten.sql.columnar.maxBatchSize | 4096 | | spark.gluten.sql.columnar.overwriteByExpression | true | Enable or disable columnar v2 command overwrite by expression. | | spark.gluten.sql.columnar.parquet.write.blockSize | 128MB | -| spark.gluten.sql.columnar.partial.project | true | Break up one project node into 2 phases when some of the expressions are non offload-able. Phase one is a regular offloaded project transformer that evaluates the offload-able expressions in native, phase two preserves the output from phase one and evaluates the remaining non-offload-able expressions using vanilla Spark projections | -| spark.gluten.sql.columnar.partial.generate | true | evaluates the non-offload-able HiveUDTF using vanilla Spark generator | -| spark.gluten.sql.columnar.physicalJoinOptimizationLevel | 12 | Fallback to row operators if there are several continuous joins. | -| spark.gluten.sql.columnar.physicalJoinOptimizeEnable | false | Enable or disable columnar physicalJoinOptimize. | -| spark.gluten.sql.columnar.preferStreamingAggregate | true | Velox backend supports `StreamingAggregate`. `StreamingAggregate` uses the less memory as it does not need to hold all groups in memory, so it could avoid spill. When true and the child output ordering satisfies the grouping key then Gluten will choose `StreamingAggregate` as the native operator. | -| spark.gluten.sql.columnar.project | true | Enable or disable columnar project. | -| spark.gluten.sql.columnar.project.collapse | true | Combines two columnar project operators into one and perform alias substitution | -| spark.gluten.sql.columnar.query.fallback.threshold | -1 | The threshold for whether query will fall back by counting the number of ColumnarToRow & vanilla leaf node. | -| spark.gluten.sql.columnar.range | true | Enable or disable columnar range. | -| spark.gluten.sql.columnar.replaceData | true | Enable or disable columnar v2 command replace data. | -| spark.gluten.sql.columnar.scanOnly | false | When enabled, only scan and the filter after scan will be offloaded to native. | -| spark.gluten.sql.columnar.shuffle | true | Enable or disable columnar shuffle. | -| spark.gluten.sql.columnar.shuffle.celeborn.fallback.enabled | true | If enabled, fall back to ColumnarShuffleManager when celeborn service is unavailable.Otherwise, throw an exception. | -| spark.gluten.sql.columnar.shuffle.celeborn.useRssSort | true | If true, use RSS sort implementation for Celeborn sort-based shuffle.If false, use Gluten's row-based sort implementation. Only valid when `spark.celeborn.client.spark.shuffle.writer` is set to `sort`. | -| spark.gluten.sql.columnar.shuffle.codec | <undefined> | By default, the supported codecs are lz4 and zstd. When spark.gluten.sql.columnar.shuffle.codecBackend=qat,the supported codecs are gzip and zstd. | +| spark.gluten.sql.columnar.partial.generate | true | Evaluates the non-offload-able HiveUDTF using vanilla Spark generator | +| spark.gluten.sql.columnar.partial.project | true | Break up one project node into 2 phases when some of the expressions are non offload-able. Phase one is a regular offloaded project transformer that evaluates the offload-able expressions in native, phase two preserves the output from phase one and evaluates the remaining non-offload-able expressions using vanilla Spark projections | +| spark.gluten.sql.columnar.physicalJoinOptimizationLevel | 12 | Fallback to row operators if there are several continuous joins. | +| spark.gluten.sql.columnar.physicalJoinOptimizeEnable | false | Enable or disable columnar physicalJoinOptimize. | +| spark.gluten.sql.columnar.preferStreamingAggregate | true | Velox backend supports `StreamingAggregate`. `StreamingAggregate` uses the less memory as it does not need to hold all groups in memory, so it could avoid spill. When true and the child output ordering satisfies the grouping key then Gluten will choose `StreamingAggregate` as the native operator. | +| spark.gluten.sql.columnar.project | true | Enable or disable columnar project. | +| spark.gluten.sql.columnar.project.collapse | true | Combines two columnar project operators into one and perform alias substitution | +| spark.gluten.sql.columnar.query.fallback.threshold | -1 | The threshold for whether query will fall back by counting the number of ColumnarToRow & vanilla leaf node. | +| spark.gluten.sql.columnar.range | true | Enable or disable columnar range. | +| spark.gluten.sql.columnar.replaceData | true | Enable or disable columnar v2 command replace data. | +| spark.gluten.sql.columnar.scanOnly | false | When enabled, only scan and the filter after scan will be offloaded to native. | +| spark.gluten.sql.columnar.shuffle | true | Enable or disable columnar shuffle. | +| spark.gluten.sql.columnar.shuffle.celeborn.fallback.enabled | true | If enabled, fall back to ColumnarShuffleManager when celeborn service is unavailable.Otherwise, throw an exception. | +| spark.gluten.sql.columnar.shuffle.celeborn.useRssSort | true | If true, use RSS sort implementation for Celeborn sort-based shuffle.If false, use Gluten's row-based sort implementation. Only valid when `spark.celeborn.client.spark.shuffle.writer` is set to `sort`. | +| spark.gluten.sql.columnar.shuffle.codec | <undefined> | By default, the supported codecs are lz4 and zstd. When spark.gluten.sql.columnar.shuffle.codecBackend=qat,the supported codecs are gzip and zstd. | | spark.gluten.sql.columnar.shuffle.codecBackend | <undefined> | | spark.gluten.sql.columnar.shuffle.compression.threshold | 100 | If number of rows in a batch falls below this threshold, will copy all buffers into one buffer to compress. | | spark.gluten.sql.columnar.shuffle.dictionary.enabled | false | Enable dictionary in hash-based shuffle. | diff --git a/docs/velox-configuration.md b/docs/velox-configuration.md index b5724f24e899..78123bde3fe7 100644 --- a/docs/velox-configuration.md +++ b/docs/velox-configuration.md @@ -22,6 +22,7 @@ nav_order: 16 | spark.gluten.sql.columnar.backend.velox.cacheEnabled | false | Enable Velox cache, default off. It's recommended to enablesoft-affinity as well when enable velox cache. | | spark.gluten.sql.columnar.backend.velox.cachePrefetchMinPct | 0 | Set prefetch cache min pct for velox file scan | | spark.gluten.sql.columnar.backend.velox.checkUsageLeak | true | Enable check memory usage leak. | +| spark.gluten.sql.columnar.backend.velox.cudf.enableTableScan | false | Enable cudf table scan | | spark.gluten.sql.columnar.backend.velox.cudf.memoryPercent | 50 | The initial percent of GPU memory to allocate for memory resource for one thread. | | spark.gluten.sql.columnar.backend.velox.cudf.memoryResource | async | GPU RMM memory resource. | | spark.gluten.sql.columnar.backend.velox.directorySizeGuess | 32KB | Deprecated, rename to spark.gluten.sql.columnar.backend.velox.footerEstimatedSize |