diff --git a/velox/connectors/CMakeLists.txt b/velox/connectors/CMakeLists.txt index 3fd17dde2d29..f0b50a5d1679 100644 --- a/velox/connectors/CMakeLists.txt +++ b/velox/connectors/CMakeLists.txt @@ -32,3 +32,5 @@ endif() if(${VELOX_BUILD_TESTING}) add_subdirectory(tests) endif() + +add_subdirectory(lakehouse) \ No newline at end of file diff --git a/velox/connectors/hive/HiveDataSource.cpp b/velox/connectors/hive/HiveDataSource.cpp index 7e5626396496..3cf742c64ac4 100644 --- a/velox/connectors/hive/HiveDataSource.cpp +++ b/velox/connectors/hive/HiveDataSource.cpp @@ -218,7 +218,9 @@ std::unique_ptr HiveDataSource::createSplitReader() { fsStats_, fileHandleFactory_, executor_, - scanSpec_); + scanSpec_, + expressionEvaluator_, + totalRemainingFilterTime_); } std::vector HiveDataSource::setupBucketConversion() { @@ -318,6 +320,7 @@ void HiveDataSource::addSplit(std::shared_ptr split) { } splitReader_ = createSplitReader(); + if (!bucketChannels.empty()) { splitReader_->setBucketConversion(std::move(bucketChannels)); } diff --git a/velox/connectors/hive/HiveDataSource.h b/velox/connectors/hive/HiveDataSource.h index 041783f2964c..e10228727871 100644 --- a/velox/connectors/hive/HiveDataSource.h +++ b/velox/connectors/hive/HiveDataSource.h @@ -159,7 +159,7 @@ class HiveDataSource : public DataSource { subfields_; common::SubfieldFilters filters_; std::shared_ptr metadataFilter_; - std::unique_ptr remainingFilterExprSet_; + std::shared_ptr remainingFilterExprSet_; RowVectorPtr emptyOutput_; dwio::common::RuntimeStatistics runtimeStats_; std::atomic totalRemainingFilterTime_{0}; diff --git a/velox/connectors/hive/SplitReader.cpp b/velox/connectors/hive/SplitReader.cpp index 7f4e3ad42899..e1ada3776939 100644 --- a/velox/connectors/hive/SplitReader.cpp +++ b/velox/connectors/hive/SplitReader.cpp @@ -84,7 +84,9 @@ std::unique_ptr SplitReader::create( const std::shared_ptr& fsStats, FileHandleFactory* fileHandleFactory, folly::Executor* executor, - const std::shared_ptr& scanSpec) { + const std::shared_ptr& scanSpec, + core::ExpressionEvaluator* expressionEvaluator, + std::atomic& totalRemainingFilterTime) { // Create the SplitReader based on hiveSplit->customSplitInfo["table_format"] if (hiveSplit->customSplitInfo.count("table_format") > 0 && hiveSplit->customSplitInfo["table_format"] == "hive-iceberg") { @@ -99,7 +101,9 @@ std::unique_ptr SplitReader::create( fsStats, fileHandleFactory, executor, - scanSpec); + scanSpec, + expressionEvaluator, + totalRemainingFilterTime); } else { return std::unique_ptr(new SplitReader( hiveSplit, diff --git a/velox/connectors/hive/SplitReader.h b/velox/connectors/hive/SplitReader.h index 1860b67d2b21..d0055082e7e5 100644 --- a/velox/connectors/hive/SplitReader.h +++ b/velox/connectors/hive/SplitReader.h @@ -67,7 +67,9 @@ class SplitReader { const std::shared_ptr& fsStats, FileHandleFactory* fileHandleFactory, folly::Executor* executor, - const std::shared_ptr& scanSpec); + const std::shared_ptr& scanSpec, + core::ExpressionEvaluator* expressionEvaluator, + std::atomic& totalRemainingFilterTime); virtual ~SplitReader() = default; diff --git a/velox/connectors/hive/iceberg/EqualityDeleteFileReader.cpp b/velox/connectors/hive/iceberg/EqualityDeleteFileReader.cpp new file mode 100644 index 000000000000..47b6bdf1b2e6 --- /dev/null +++ b/velox/connectors/hive/iceberg/EqualityDeleteFileReader.cpp @@ -0,0 +1,224 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "velox/connectors/hive/iceberg/EqualityDeleteFileReader.h" + +#include "velox/connectors/hive/HiveConnectorUtil.h" +#include "velox/connectors/hive/iceberg/FilterUtil.h" +#include "velox/connectors/hive/iceberg/IcebergDeleteFile.h" +#include "velox/core/Expressions.h" +#include "velox/dwio/common/ReaderFactory.h" + +using namespace facebook::velox::common; +using namespace facebook::velox::core; +using namespace facebook::velox::exec; + +namespace facebook::velox::connector::hive::iceberg { + +static constexpr const int kMaxBatchRows = 10'000; + +EqualityDeleteFileReader::EqualityDeleteFileReader( + const IcebergDeleteFile& deleteFile, + std::shared_ptr baseFileSchema, + FileHandleFactory* fileHandleFactory, + folly::Executor* executor, + const ConnectorQueryCtx* connectorQueryCtx, + const std::shared_ptr& hiveConfig, + const std::shared_ptr ioStats, + const std::shared_ptr& fsStats, + const std::string& connectorId) + : deleteFile_(deleteFile), + baseFileSchema_(baseFileSchema), + fileHandleFactory_(fileHandleFactory), + pool_(connectorQueryCtx->memoryPool()), + deleteSplit_(nullptr), + deleteRowReader_(nullptr) { + VELOX_CHECK_EQ(deleteFile_.content, FileContent::kEqualityDeletes); + + if (deleteFile_.recordCount == 0) { + return; + } + + // TODO: push down filter if previous delete file contains this one. E.g. + // previous equality delete file has a=1, and this file also contains + // columns a, then a!=1 can be pushed as a filter when reading this delete + // file. + + deleteSplit_ = std::make_shared( + connectorId, + deleteFile_.filePath, + deleteFile_.fileFormat, + 0, + deleteFile_.fileSizeInBytes); + + // Create the Reader and RowReader for the equality delete file + + dwio::common::ReaderOptions deleteReaderOpts(pool_); + configureReaderOptions( + hiveConfig, + connectorQueryCtx, + nullptr, + deleteSplit_, + {}, + deleteReaderOpts); + + const FileHandleKey fileHandleKey{ + .filename = deleteFile_.filePath, + .tokenProvider = connectorQueryCtx->fsTokenProvider()}; + auto deleteFileHandleCachePtr = fileHandleFactory_->generate(fileHandleKey); + auto deleteFileInput = createBufferedInput( + *deleteFileHandleCachePtr, + deleteReaderOpts, + connectorQueryCtx, + ioStats, + fsStats, + executor); + + auto deleteReader = + dwio::common::getReaderFactory(deleteReaderOpts.fileFormat()) + ->createReader(std::move(deleteFileInput), deleteReaderOpts); + + // For now, we assume only the delete columns are written in the delete file + deleteFileRowType_ = deleteReader->rowType(); + auto scanSpec = std::make_shared(""); + scanSpec->addAllChildFields(deleteFileRowType_->asRow()); + + dwio::common::RowReaderOptions deleteRowReaderOpts; + configureRowReaderOptions( + {}, + scanSpec, + nullptr, + deleteFileRowType_, + deleteSplit_, + hiveConfig, + connectorQueryCtx->sessionProperties(), + deleteRowReaderOpts); + + deleteRowReader_.reset(); + deleteRowReader_ = deleteReader->createRowReader(deleteRowReaderOpts); +} + +void EqualityDeleteFileReader::readDeleteValues( + SubfieldFilters& subfieldFilters, + std::vector& expressionInputs) { + VELOX_CHECK(deleteRowReader_); + VELOX_CHECK(deleteSplit_); + + if (!deleteValuesOutput_) { + deleteValuesOutput_ = BaseVector::create(deleteFileRowType_, 0, pool_); + } + + // TODO: verfiy if the field is an Iceberg RowId. Velox currently doesn't + // support pushing down filters to non-RowId types, i.e. sub-fields of Array + // or Map + if (deleteFileRowType_->size() == 1) { + // Construct the IN list filter that can be pushed down to the base file + // readers, then update the baseFileScanSpec. + buildDomainFilter(subfieldFilters); + } else { + // Build the filter functions that will be evaluated after all base file + // read is done + buildFilterFunctions(expressionInputs); + } + + deleteSplit_.reset(); +} + +void EqualityDeleteFileReader::buildDomainFilter( + SubfieldFilters& subfieldFilters) { + std::unique_ptr filter = std::make_unique(); + auto name = baseFileSchema_->childByFieldId(deleteFile_.equalityFieldIds[0]) + ->fullName(); + while (deleteRowReader_->next(kMaxBatchRows, deleteValuesOutput_)) { + if (deleteValuesOutput_->size() == 0) { + continue; + } + + deleteValuesOutput_->loadedVector(); + auto vector = + std::dynamic_pointer_cast(deleteValuesOutput_)->childAt(0); + + auto typeKind = vector->type()->kind(); + VELOX_CHECK( + typeKind != TypeKind::DOUBLE && typeKind != TypeKind::REAL, + "Iceberg does not allow DOUBLE or REAL columns as the equality delete columns: {} : {}", + name, + typeKind); + + auto notExistsFilter = + createNotInFilter(vector, 0, deleteValuesOutput_->size(), typeKind); + filter = filter->mergeWith(notExistsFilter.get()); + } + + if (filter->kind() != FilterKind::kAlwaysTrue) { + if (subfieldFilters.find(common::Subfield(name)) != subfieldFilters.end()) { + subfieldFilters[common::Subfield(name)] = + subfieldFilters[common::Subfield(name)]->mergeWith(filter.get()); + } else { + subfieldFilters[common::Subfield(name)] = std::move(filter); + } + } +} + +void EqualityDeleteFileReader::buildFilterFunctions( + std::vector& expressionInputs) { + auto numDeleteFields = deleteFileRowType_->size(); + VELOX_CHECK_GT( + numDeleteFields, + 0, + "Iceberg equality delete file should have at least one field."); + + // TODO: logical expression simplifications + while (deleteRowReader_->next(kMaxBatchRows, deleteValuesOutput_)) { + if (deleteValuesOutput_->size() == 0) { + continue; + } + + deleteValuesOutput_->loadedVector(); + auto rowVector = std::dynamic_pointer_cast(deleteValuesOutput_); + auto numDeletedValues = rowVector->childAt(0)->size(); + + for (int i = 0; i < numDeletedValues; i++) { + std::vector disconjunctInputs; + + for (int j = 0; j < numDeleteFields; j++) { + auto type = deleteFileRowType_->childAt(j); + auto name = + baseFileSchema_->childByFieldId(deleteFile_.equalityFieldIds[j]) + ->fullName(); + auto value = BaseVector::wrapInConstant(1, i, rowVector->childAt(j)); + + std::vector isNotEqualInputs; + isNotEqualInputs.push_back( + std::make_shared(type, name)); + isNotEqualInputs.push_back(std::make_shared(value)); + // TODO: generalize this to support different engines. Currently, only + // Presto "neq" is supported. Spark does not register the "neq" function + // but does support "not" and "equalto" functions. + auto isNotEqualExpr = + std::make_shared(BOOLEAN(), isNotEqualInputs, "neq"); + + disconjunctInputs.push_back(isNotEqualExpr); + } + + auto disconjunctNotEqualExpr = + std::make_shared(BOOLEAN(), disconjunctInputs, "or"); + expressionInputs.push_back(disconjunctNotEqualExpr); + } + } +} + +} // namespace facebook::velox::connector::hive::iceberg diff --git a/velox/connectors/hive/iceberg/EqualityDeleteFileReader.h b/velox/connectors/hive/iceberg/EqualityDeleteFileReader.h new file mode 100644 index 000000000000..99ce239b9f9d --- /dev/null +++ b/velox/connectors/hive/iceberg/EqualityDeleteFileReader.h @@ -0,0 +1,87 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#pragma once + +#include "velox/connectors/Connector.h" +#include "velox/connectors/hive/FileHandle.h" +#include "velox/connectors/hive/HiveConfig.h" +#include "velox/connectors/hive/HiveConnectorSplit.h" +#include "velox/dwio/common/Reader.h" +#include "velox/expression/Expr.h" + +namespace facebook::velox::connector::hive::iceberg { + +class IcebergDeleteFile; + +using SubfieldFilters = + std::unordered_map>; + +class EqualityDeleteFileReader { + public: + EqualityDeleteFileReader( + const IcebergDeleteFile& deleteFile, + std::shared_ptr baseFileSchema, + FileHandleFactory* fileHandleFactory, + folly::Executor* executor, + const ConnectorQueryCtx* connectorQueryCtx, + const std::shared_ptr& hiveConfig, + const std::shared_ptr ioStats, + const std::shared_ptr& fsStats, + const std::string& connectorId); + + /// Reads the delete values from the equality delete file, and interprets them + /// as filters for the base file reader. + /// + /// @subfieldFilters The built SubfieldFilter that can be pushed down to the + /// base file RowReader, when the equality delete file only contains one + /// single subfield of Iceberg RowId type. + /// @typedExpressions The built TypedExpr that will be evaluated by the + /// connector DataSource after rows are read from the base file RowReader. + void readDeleteValues( + SubfieldFilters& subfieldFilters, + std::vector& typedExpressions); + + private: + void buildDomainFilter(SubfieldFilters& subfieldFilters); + + void buildFilterFunctions(std::vector& expressionInputs); + + // The equality delete file to read + const IcebergDeleteFile& deleteFile_; + // The schema of the base file in terms of TypeWithId tree. In addition to the + // existing fields that were included in the base file ScanSpec, it also + // contains the extra fields that are in the equality delete file but not + // in the ScanSpec of the base file + const std::shared_ptr baseFileSchema_; + + // The cache factory of the file handles, which can be used to return the file + // handle of the delete file. + FileHandleFactory* const fileHandleFactory_; + memory::MemoryPool* const pool_; + + // The split of the equality delete file to be processed by the delete file + // RowReader. + std::shared_ptr deleteSplit_; + // The RowType of the equality delete file + RowTypePtr deleteFileRowType_; + // The RowReader to read the equality delete file + std::unique_ptr deleteRowReader_; + // The output vector to hold the delete values + VectorPtr deleteValuesOutput_; +}; + +} // namespace facebook::velox::connector::hive::iceberg diff --git a/velox/connectors/hive/iceberg/FilterUtil.cpp b/velox/connectors/hive/iceberg/FilterUtil.cpp new file mode 100644 index 000000000000..7ef26f5e18db --- /dev/null +++ b/velox/connectors/hive/iceberg/FilterUtil.cpp @@ -0,0 +1,87 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#include "velox/connectors/hive/iceberg/FilterUtil.h" + +namespace facebook::velox::connector::hive::iceberg { + +using namespace facebook::velox::exec; +using namespace facebook::velox::core; + +template +std::unique_ptr createNegatedBigintValuesFilter( + const VectorPtr& valuesVector, + vector_size_t offset, + vector_size_t size) { + auto valuesPair = + common::deDuplicateValues(valuesVector, offset, size); + + const auto& values = valuesPair.first; + bool hasNull = valuesPair.second; + + return common::createNegatedBigintValues(values, !hasNull); +} + +std::unique_ptr createNotInFilter( + const VectorPtr& elements, + vector_size_t offset, + vector_size_t size, + TypeKind type) { + std::unique_ptr filter; + switch (type) { + case TypeKind::HUGEINT: + // TODO: createNegatedHugeintValuesFilter is not implemented yet. + VELOX_NYI("createNegatedHugeintValuesFilter is not implemented yet"); + case TypeKind::BIGINT: + filter = createNegatedBigintValuesFilter(elements, offset, size); + break; + case TypeKind::INTEGER: + filter = createNegatedBigintValuesFilter(elements, offset, size); + break; + case TypeKind::SMALLINT: + filter = createNegatedBigintValuesFilter(elements, offset, size); + break; + case TypeKind::TINYINT: + filter = createNegatedBigintValuesFilter(elements, offset, size); + break; + case TypeKind::BOOLEAN: + // Hack: using BIGINT filter for bool, which is essentially "int1_t". + filter = createNegatedBigintValuesFilter(elements, offset, size); + break; + case TypeKind::TIMESTAMP: + filter = + createNegatedBigintValuesFilter(elements, offset, size); + break; + case TypeKind::VARCHAR: + case TypeKind::VARBINARY: + // TODO: createNegatedBytesValuesFilter is not implemented yet. + VELOX_NYI("createNegatedBytesValuesFilter is not implemented yet"); + case TypeKind::REAL: + case TypeKind::DOUBLE: + case TypeKind::UNKNOWN: + [[fallthrough]]; + case TypeKind::ARRAY: + [[fallthrough]]; + case TypeKind::MAP: + [[fallthrough]]; + case TypeKind::ROW: + [[fallthrough]]; + default: + VELOX_USER_FAIL( + "Iceberg equality delete column cannot be of type ", type); + } + return filter; +} +} // namespace facebook::velox::connector::hive::iceberg diff --git a/velox/connectors/hive/iceberg/FilterUtil.h b/velox/connectors/hive/iceberg/FilterUtil.h new file mode 100644 index 000000000000..96b8255511c2 --- /dev/null +++ b/velox/connectors/hive/iceberg/FilterUtil.h @@ -0,0 +1,30 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#pragma once + +#include "velox/expression/Expr.h" +#include "velox/type/Filter.h" +#include "velox/vector/ComplexVector.h" + +namespace facebook::velox::connector::hive::iceberg { +std::unique_ptr createNotInFilter( + const VectorPtr& elements, + vector_size_t offset, + vector_size_t size, + TypeKind type); + +} // namespace facebook::velox::connector::hive::iceberg diff --git a/velox/connectors/hive/iceberg/IcebergDeleteFile.cpp b/velox/connectors/hive/iceberg/IcebergDeleteFile.cpp new file mode 100644 index 000000000000..4237785e8ca9 --- /dev/null +++ b/velox/connectors/hive/iceberg/IcebergDeleteFile.cpp @@ -0,0 +1,40 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "velox/connectors/hive/iceberg/IcebergDeleteFile.h" + +namespace facebook::velox::connector::hive::iceberg { + +fmt::underlying_t format_as(FileContent f) { + return fmt::underlying(f); +} + +} // namespace facebook::velox::connector::hive::iceberg diff --git a/velox/connectors/hive/iceberg/IcebergDeleteFile.h b/velox/connectors/hive/iceberg/IcebergDeleteFile.h index 2f9206dfc264..995578e129c6 100644 --- a/velox/connectors/hive/iceberg/IcebergDeleteFile.h +++ b/velox/connectors/hive/iceberg/IcebergDeleteFile.h @@ -29,6 +29,9 @@ enum class FileContent { kEqualityDeletes, }; +/// Formatter for fmt. +fmt::underlying_t format_as(FileContent f); + struct IcebergDeleteFile { FileContent content; const std::string filePath; diff --git a/velox/connectors/hive/iceberg/IcebergSplitReader.cpp b/velox/connectors/hive/iceberg/IcebergSplitReader.cpp index 63e19d16a71f..dc9055fdcd53 100644 --- a/velox/connectors/hive/iceberg/IcebergSplitReader.cpp +++ b/velox/connectors/hive/iceberg/IcebergSplitReader.cpp @@ -16,6 +16,7 @@ #include "velox/connectors/hive/iceberg/IcebergSplitReader.h" +#include "velox/connectors/hive/iceberg/EqualityDeleteFileReader.h" #include "velox/connectors/hive/iceberg/IcebergDeleteFile.h" #include "velox/connectors/hive/iceberg/IcebergSplit.h" #include "velox/dwio/common/BufferUtil.h" @@ -35,7 +36,9 @@ IcebergSplitReader::IcebergSplitReader( const std::shared_ptr& fsStats, FileHandleFactory* const fileHandleFactory, folly::Executor* executor, - const std::shared_ptr& scanSpec) + const std::shared_ptr& scanSpec, + core::ExpressionEvaluator* expressionEvaluator, + std::atomic& totalRemainingFilterTime) : SplitReader( hiveSplit, hiveTableHandle, @@ -50,7 +53,12 @@ IcebergSplitReader::IcebergSplitReader( scanSpec), baseReadOffset_(0), splitOffset_(0), - deleteBitmap_(nullptr) {} + deleteBitmap_(nullptr), + deleteExprSet_(nullptr), + expressionEvaluator_(expressionEvaluator), + totalRemainingFilterMs_(totalRemainingFilterTime) {} + +IcebergSplitReader::~IcebergSplitReader() {} void IcebergSplitReader::prepareSplit( std::shared_ptr metadataFilter, @@ -61,20 +69,81 @@ void IcebergSplitReader::prepareSplit( } auto rowType = getAdaptedRowType(); + std::shared_ptr icebergSplit = + std::dynamic_pointer_cast(hiveSplit_); + const auto& deleteFiles = icebergSplit->deleteFiles; + std::unordered_set equalityFieldIds; + for (const auto& deleteFile : deleteFiles) { + if (deleteFile.content == FileContent::kEqualityDeletes && + deleteFile.recordCount > 0) { + equalityFieldIds.insert( + deleteFile.equalityFieldIds.begin(), + deleteFile.equalityFieldIds.end()); + } + } + + // checkIfSplitIsEmpty needs to use the base reader's schemaWithId_. For that + // we need to update the base RowReader to include these extra fields from the + // equality delete file first, so that the schemaWithId_ of the base file is + // updated when we call baseFileSchema() later. + baseReader_->setRequiredExtraFieldIds(equalityFieldIds); + if (checkIfSplitIsEmpty(runtimeStats)) { VELOX_CHECK(emptySplit_); return; } + // Process the equality delete files to update the scan spec and remaining + // filters. It needs to be done after creating the Reader and before creating + // the RowReader. + + SubfieldFilters subfieldFilters; + std::vector conjunctInputs; + + for (const auto& deleteFile : deleteFiles) { + if (deleteFile.content == FileContent::kEqualityDeletes && + deleteFile.recordCount > 0) { + // TODO: build cache of to avoid repeating file + // parsing across partitions. Within a single partition, the splits should + // be with the same equality delete files and only need to be parsed once. + auto equalityDeleteReader = std::make_unique( + deleteFile, + baseFileSchema(), + fileHandleFactory_, + executor_, + connectorQueryCtx_, + hiveConfig_, + ioStats_, + fsStats_, + hiveSplit_->connectorId); + equalityDeleteReader->readDeleteValues(subfieldFilters, conjunctInputs); + } + } + + if (!subfieldFilters.empty()) { + for (const auto& [key, filter] : subfieldFilters) { + auto childSpec = scanSpec_->getOrCreateChild(key, true); + childSpec->addFilter(*filter); + childSpec->setHasTempFilter(true); + childSpec->setSubscript(scanSpec_->children().size() - 1); + } + } + + if (!conjunctInputs.empty()) { + core::TypedExprPtr expression = + std::make_shared(BOOLEAN(), conjunctInputs, "and"); + deleteExprSet_ = expressionEvaluator_->compile(expression); + VELOX_CHECK_EQ(deleteExprSet_->size(), 1); + } + createRowReader(std::move(metadataFilter), std::move(rowType)); - std::shared_ptr icebergSplit = - std::dynamic_pointer_cast(hiveSplit_); baseReadOffset_ = 0; splitOffset_ = baseRowReader_->nextRowNumber(); - positionalDeleteFileReaders_.clear(); - const auto& deleteFiles = icebergSplit->deleteFiles; + // Create the positional deletes file readers. They need to be created after + // the RowReader is created. + positionalDeleteFileReaders_.clear(); for (const auto& deleteFile : deleteFiles) { if (deleteFile.content == FileContent::kPositionalDeletes) { if (deleteFile.recordCount > 0) { @@ -92,12 +161,16 @@ void IcebergSplitReader::prepareSplit( splitOffset_, hiveSplit_->connectorId)); } - } else { - VELOX_NYI(); } } } +std::shared_ptr +IcebergSplitReader::baseFileSchema() { + VELOX_CHECK_NOT_NULL(baseReader_.get()); + return baseReader_->typeWithId(); +} + uint64_t IcebergSplitReader::next(uint64_t size, VectorPtr& output) { Mutation mutation; mutation.randomSkip = baseReaderOpts_.randomSkip().get(); @@ -136,8 +209,34 @@ uint64_t IcebergSplitReader::next(uint64_t size, VectorPtr& output) { : nullptr; auto rowsScanned = baseRowReader_->next(actualSize, output, &mutation); + + // Evaluate the remaining filter deleteExprSet_ for every batch and update the + // output vector if it reduces any rows. + if (deleteExprSet_) { + auto filterStartMs = getCurrentTimeMs(); + + filterRows_.resize(output->size()); + auto rowVector = std::dynamic_pointer_cast(output); + expressionEvaluator_->evaluate( + deleteExprSet_.get(), filterRows_, *rowVector, filterResult_); + auto numRemainingRows = exec::processFilterResults( + filterResult_, filterRows_, filterEvalCtx_, pool_); + + if (numRemainingRows < output->size()) { + output = exec::wrap( + numRemainingRows, filterEvalCtx_.selectedIndices, rowVector); + } + + totalRemainingFilterMs_.fetch_add( + (getCurrentTimeMs() - filterStartMs), std::memory_order_relaxed); + } + baseReadOffset_ += rowsScanned; + if (rowsScanned == 0) { + scanSpec_->deleteTempNodes(); + } + return rowsScanned; } diff --git a/velox/connectors/hive/iceberg/IcebergSplitReader.h b/velox/connectors/hive/iceberg/IcebergSplitReader.h index 4b3c6b901048..a984b3770621 100644 --- a/velox/connectors/hive/iceberg/IcebergSplitReader.h +++ b/velox/connectors/hive/iceberg/IcebergSplitReader.h @@ -19,6 +19,7 @@ #include "velox/connectors/Connector.h" #include "velox/connectors/hive/SplitReader.h" #include "velox/connectors/hive/iceberg/PositionalDeleteFileReader.h" +#include "velox/exec/OperatorUtils.h" namespace facebook::velox::connector::hive::iceberg { @@ -37,9 +38,11 @@ class IcebergSplitReader : public SplitReader { const std::shared_ptr& fsStats, FileHandleFactory* fileHandleFactory, folly::Executor* executor, - const std::shared_ptr& scanSpec); + const std::shared_ptr& scanSpec, + core::ExpressionEvaluator* expressionEvaluator, + std::atomic& totalRemainingFilterTime); - ~IcebergSplitReader() override = default; + ~IcebergSplitReader() override; void prepareSplit( std::shared_ptr metadataFilter, @@ -47,6 +50,8 @@ class IcebergSplitReader : public SplitReader { uint64_t next(uint64_t size, VectorPtr& output) override; + std::shared_ptr baseFileSchema(); + private: // The read offset to the beginning of the split in number of rows for the // current batch for the base data file @@ -56,5 +61,14 @@ class IcebergSplitReader : public SplitReader { std::list> positionalDeleteFileReaders_; BufferPtr deleteBitmap_; + + std::unique_ptr deleteExprSet_; + core::ExpressionEvaluator* expressionEvaluator_; + std::atomic& totalRemainingFilterMs_; + + // Reusable memory for remaining filter evaluation. + VectorPtr filterResult_; + SelectivityVector filterRows_; + exec::FilterEvalCtx filterEvalCtx_; }; } // namespace facebook::velox::connector::hive::iceberg diff --git a/velox/connectors/hive/iceberg/tests/IcebergReadTest.cpp b/velox/connectors/hive/iceberg/tests/IcebergReadTest.cpp index 8bf51afb25ed..46544873bfaf 100644 --- a/velox/connectors/hive/iceberg/tests/IcebergReadTest.cpp +++ b/velox/connectors/hive/iceberg/tests/IcebergReadTest.cpp @@ -208,7 +208,7 @@ class HiveIcebergTest : public HiveConnectorTestBase { IcebergDeleteFile icebergDeleteFile( FileContent::kPositionalDeletes, deleteFilePath, - fileFomat_, + fileFormat_, deleteFilePaths[deleteFileName].first, testing::internal::GetFileSize( std::fopen(deleteFilePath.c_str(), "r"))); @@ -223,7 +223,7 @@ class HiveIcebergTest : public HiveConnectorTestBase { std::string duckdbSql = getDuckDBQuery(rowGroupSizesForFiles, deleteFilesForBaseDatafiles); - auto plan = tableScanNode(); + auto plan = tableScanNode(rowType_); auto task = HiveConnectorTestBase::assertQuery( plan, splits, duckdbSql, numPrefetchSplits); @@ -234,6 +234,125 @@ class HiveIcebergTest : public HiveConnectorTestBase { ASSERT_TRUE(it->second.peakMemoryBytes > 0); } + void assertEqualityDeletes( + const std::unordered_map>>& + equalityDeleteVectorMap, + const std::unordered_map>& + equalityFieldIdsMap, + std::string duckDbSql = "", + std::vector dataVectors = {}) { + VELOX_CHECK_EQ(equalityDeleteVectorMap.size(), equalityFieldIdsMap.size()); + // We will create data vectors with numColumns number of columns that is the + // max field Id in equalityFieldIds + int32_t numDataColumns = 0; + + for (auto it = equalityFieldIdsMap.begin(); it != equalityFieldIdsMap.end(); + ++it) { + auto equalityFieldIds = it->second; + auto currentMax = + *std::max_element(equalityFieldIds.begin(), equalityFieldIds.end()); + numDataColumns = std::max(numDataColumns, currentMax); + } + + VELOX_CHECK_GT(numDataColumns, 0); + VELOX_CHECK_GE(numDataColumns, equalityDeleteVectorMap.size()); + VELOX_CHECK_GT(equalityDeleteVectorMap.size(), 0); + + VELOX_CHECK_LE(equalityFieldIdsMap.size(), numDataColumns); + + std::shared_ptr dataFilePath = + writeDataFiles(rowCount, numDataColumns, 1, dataVectors)[0]; + + std::vector deleteFiles; + std::string predicates = ""; + unsigned long numDeletedValues = 0; + + std::vector> deleteFilePaths; + for (auto it = equalityFieldIdsMap.begin(); + it != equalityFieldIdsMap.end();) { + auto equalityFieldIds = it->second; + auto equalityDeleteVector = equalityDeleteVectorMap.at(it->first); + VELOX_CHECK_GT(equalityDeleteVector.size(), 0); + numDeletedValues = + std::max(numDeletedValues, equalityDeleteVector[0].size()); + deleteFilePaths.push_back(writeEqualityDeleteFile(equalityDeleteVector)); + IcebergDeleteFile deleteFile( + FileContent::kEqualityDeletes, + deleteFilePaths.back()->getPath(), + fileFormat_, + equalityDeleteVector[0].size(), + testing::internal::GetFileSize( + std::fopen(deleteFilePaths.back()->getPath().c_str(), "r")), + equalityFieldIds); + deleteFiles.push_back(deleteFile); + predicates += makePredicates(equalityDeleteVector, equalityFieldIds); + ++it; + if (it != equalityFieldIdsMap.end()) { + predicates += " AND "; + } + } + + // The default split count is 1. + auto icebergSplits = + makeIcebergSplits(dataFilePath->getPath(), deleteFiles); + + // If the caller passed in a query, use that. + if (duckDbSql == "") { + // Select all columns + duckDbSql = "SELECT * FROM tmp "; + if (numDeletedValues > 0) { + duckDbSql += fmt::format("WHERE {}", predicates); + } + } + + assertEqualityDeletes( + icebergSplits.back(), + !dataVectors.empty() ? asRowType(dataVectors[0]->type()) : rowType_, + duckDbSql); + + // Select a column that's not in the filter columns + if (numDataColumns > 1 && + equalityDeleteVectorMap.at(0).size() < numDataColumns) { + std::string duckDbQuery = "SELECT c0 FROM tmp"; + if (numDeletedValues > 0) { + duckDbQuery += fmt::format(" WHERE {}", predicates); + } + + std::vector names({"c0"}); + std::vector types(1, BIGINT()); + assertEqualityDeletes( + icebergSplits.back(), + std::make_shared(std::move(names), std::move(types)), + duckDbQuery); + } + } + + std::vector makeSequenceValues(int32_t numRows, int8_t repeat = 1) { + VELOX_CHECK_GT(repeat, 0); + + auto maxValue = std::ceil((double)numRows / repeat); + std::vector values; + values.reserve(numRows); + for (int32_t i = 0; i < maxValue; i++) { + for (int8_t j = 0; j < repeat; j++) { + values.push_back(i); + } + } + values.resize(numRows); + return values; + } + + std::vector makeRandomDeleteValues(int32_t maxRowNumber) { + std::mt19937 gen{0}; + std::vector deleteRows; + for (int i = 0; i < maxRowNumber; i++) { + if (folly::Random::rand32(0, 10, gen) > 8) { + deleteRows.push_back(i); + } + } + return deleteRows; + } + const static int rowCount = 20000; protected: @@ -259,7 +378,7 @@ class HiveIcebergTest : public HiveConnectorTestBase { splits.emplace_back(std::make_shared( kHiveConnectorId, dataFilePath, - fileFomat_, + fileFormat_, i * splitSize, splitSize, partitionKeys, @@ -273,6 +392,20 @@ class HiveIcebergTest : public HiveConnectorTestBase { return splits; } + void assertEqualityDeletes( + std::shared_ptr split, + RowTypePtr outputRowType, + const std::string& duckDbSql) { + auto plan = tableScanNode(outputRowType); + auto task = OperatorTestBase::assertQuery(plan, {split}, duckDbSql); + + auto planStats = toPlanStats(task->taskStats()); + auto scanNodeId = plan->id(); + auto it = planStats.find(scanNodeId); + ASSERT_TRUE(it != planStats.end()); + ASSERT_TRUE(it->second.peakMemoryBytes > 0); + } + private: std::map> writeDataFiles( std::map> rowGroupSizesForFiles) { @@ -494,17 +627,151 @@ class HiveIcebergTest : public HiveConnectorTestBase { }); } - core::PlanNodePtr tableScanNode() { - return PlanBuilder(pool_.get()).tableScan(rowType_).planNode(); + core::PlanNodePtr tableScanNode(RowTypePtr outputRowType) { + return PlanBuilder(pool_.get()).tableScan(outputRowType).planNode(); } - dwio::common::FileFormat fileFomat_{dwio::common::FileFormat::DWRF}; + std::string makePredicates( + const std::vector>& equalityDeleteVector, + const std::vector& equalityFieldIds) { + std::string predicates(""); + int32_t numDataColumns = + *std::max_element(equalityFieldIds.begin(), equalityFieldIds.end()); + + VELOX_CHECK_GT(numDataColumns, 0); + VELOX_CHECK_GE(numDataColumns, equalityDeleteVector.size()); + VELOX_CHECK_GT(equalityDeleteVector.size(), 0); + + auto numDeletedValues = equalityDeleteVector[0].size(); + + if (numDeletedValues == 0) { + return predicates; + } + + // If all values for a column are deleted, just return an always-false + // predicate + for (auto i = 0; i < equalityDeleteVector.size(); i++) { + auto equalityFieldId = equalityFieldIds[i]; + auto deleteValues = equalityDeleteVector[i]; + + auto lastIter = std::unique(deleteValues.begin(), deleteValues.end()); + auto numDistinctValues = lastIter - deleteValues.begin(); + auto minValue = 1; + auto maxValue = *std::max_element(deleteValues.begin(), lastIter); + if (maxValue - minValue + 1 == numDistinctValues && + maxValue == (rowCount - 1) / equalityFieldId) { + return "1 = 0"; + } + } + + if (equalityDeleteVector.size() == 1) { + std::string name = fmt::format("c{}", equalityFieldIds[0] - 1); + predicates = fmt::format( + "{} NOT IN ({})", name, makeNotInList({equalityDeleteVector[0]})); + } else { + for (int i = 0; i < numDeletedValues; i++) { + std::string oneRow(""); + for (int j = 0; j < equalityFieldIds.size(); j++) { + std::string name = fmt::format("c{}", equalityFieldIds[j] - 1); + std::string predicate = + fmt::format("({} <> {})", name, equalityDeleteVector[j][i]); + + oneRow = oneRow == "" ? predicate + : fmt::format("({} OR {})", oneRow, predicate); + } + + predicates = predicates == "" + ? oneRow + : fmt::format("{} AND {}", predicates, oneRow); + } + } + return predicates; + } - RowTypePtr rowType_{ROW({"c0"}, {BIGINT()})}; std::shared_ptr pathColumn_ = IcebergMetadataColumn::icebergDeleteFilePathColumn(); std::shared_ptr posColumn_ = IcebergMetadataColumn::icebergDeletePosColumn(); + + protected: + RowTypePtr rowType_{ROW({"c0"}, {BIGINT()})}; + dwio::common::FileFormat fileFormat_{dwio::common::FileFormat::DWRF}; + + std::shared_ptr writeEqualityDeleteFile( + const std::vector>& equalityDeleteVector) { + std::vector names; + std::vector vectors; + for (int i = 0; i < equalityDeleteVector.size(); i++) { + names.push_back(fmt::format("c{}", i)); + vectors.push_back(makeFlatVector(equalityDeleteVector[i])); + } + + RowVectorPtr deleteFileVectors = makeRowVector(names, vectors); + + auto deleteFilePath = TempFilePath::create(); + writeToFile(deleteFilePath->getPath(), deleteFileVectors); + + return deleteFilePath; + } + + std::vector> writeDataFiles( + uint64_t numRows, + int32_t numColumns = 1, + int32_t splitCount = 1, + std::vector dataVectors = {}) { + if (dataVectors.empty()) { + dataVectors = makeVectors(splitCount, numRows, numColumns); + } + VELOX_CHECK_EQ(dataVectors.size(), splitCount); + + std::vector> dataFilePaths; + dataFilePaths.reserve(splitCount); + for (auto i = 0; i < splitCount; i++) { + dataFilePaths.emplace_back(TempFilePath::create()); + writeToFile(dataFilePaths.back()->getPath(), dataVectors[i]); + } + + createDuckDbTable(dataVectors); + return dataFilePaths; + } + + std::vector + makeVectors(int32_t count, int32_t rowsPerVector, int32_t numColumns = 1) { + std::vector types(numColumns, BIGINT()); + std::vector names; + for (int j = 0; j < numColumns; j++) { + names.push_back(fmt::format("c{}", j)); + } + + std::vector rowVectors; + for (int i = 0; i < count; i++) { + std::vector vectors; + + // Create the column values like below: + // c0 c1 c2 + // 0 0 0 + // 1 0 0 + // 2 1 0 + // 3 1 1 + // 4 2 1 + // 5 2 1 + // 6 3 2 + // ... + // In the first column c0, the values are continuously increasing and not + // repeating. In the second column c1, the values are continuously + // increasing and each value repeats once. And so on. + for (int j = 0; j < numColumns; j++) { + auto data = makeSequenceValues(rowsPerVector, j + 1); + vectors.push_back(vectorMaker_.flatVector(data)); + } + + rowVectors.push_back(makeRowVector(names, vectors)); + } + + rowType_ = std::make_shared(std::move(names), std::move(types)); + + return rowVectors; + } }; /// This test creates one single data file and one delete file. The parameter @@ -774,4 +1041,232 @@ TEST_F(HiveIcebergTest, testPartitionedRead) { HiveConnectorTestBase::assertQuery(plan, splits, "SELECT 0, '2018-04-06'"); } + +// Delete values from a single column file +TEST_F(HiveIcebergTest, equalityDeletesSingleFileColumn1) { + folly::SingletonVault::singleton()->registrationComplete(); + + std::unordered_map> equalityFieldIdsMap; + std::unordered_map>> + equalityDeleteVectorMap; + equalityFieldIdsMap.insert({0, {1}}); + + // Delete row 0, 1, 2, 3 from the first batch out of two. + equalityDeleteVectorMap.insert({0, {{0, 1, 2, 3}}}); + assertEqualityDeletes(equalityDeleteVectorMap, equalityFieldIdsMap); + + // Delete the first and last row in each batch (10000 rows per batch) + equalityDeleteVectorMap.clear(); + equalityDeleteVectorMap.insert({0, {{0, 9999, 10000, 19999}}}); + assertEqualityDeletes(equalityDeleteVectorMap, equalityFieldIdsMap); + + // Delete several rows in the second batch (10000 rows per batch) + equalityDeleteVectorMap.clear(); + equalityDeleteVectorMap.insert({0, {{10000, 10002, 19999}}}); + assertEqualityDeletes(equalityDeleteVectorMap, equalityFieldIdsMap); + + // Delete random rows + equalityDeleteVectorMap.clear(); + equalityDeleteVectorMap.insert({0, {makeRandomDeleteValues(rowCount)}}); + assertEqualityDeletes(equalityDeleteVectorMap, equalityFieldIdsMap); + + // Delete 0 rows + equalityDeleteVectorMap.clear(); + equalityDeleteVectorMap.insert({0, {{}}}); + assertEqualityDeletes(equalityDeleteVectorMap, equalityFieldIdsMap); + + // Delete all rows + equalityDeleteVectorMap.insert({0, {makeSequenceValues(rowCount)}}); + assertEqualityDeletes(equalityDeleteVectorMap, equalityFieldIdsMap); + + // Delete rows that don't exist + equalityDeleteVectorMap.clear(); + equalityDeleteVectorMap.insert({0, {{20000, 29999}}}); + assertEqualityDeletes(equalityDeleteVectorMap, equalityFieldIdsMap); +} + +// Delete values from the second column in a 2-column file +// +// c1 c2 +// 0 0 +// 1 0 +// 2 1 +// 3 1 +// 4 2 +// ... ... +// 19999 9999 +TEST_F(HiveIcebergTest, equalityDeletesSingleFileColumn2) { + folly::SingletonVault::singleton()->registrationComplete(); + + std::unordered_map> equalityFieldIdsMap; + std::unordered_map>> + equalityDeleteVectorMap; + equalityFieldIdsMap.insert({0, {2}}); + + // Delete values 0, 1, 2, 3 from the second column + equalityDeleteVectorMap.insert({0, {{0, 1, 2, 3}}}); + assertEqualityDeletes(equalityDeleteVectorMap, equalityFieldIdsMap); + + // Delete the smallest value 0 and the largest value 9999 from the second + // column, which has the range [0, 9999] + equalityDeleteVectorMap.clear(); + equalityDeleteVectorMap.insert({0, {{0, 9999}}}); + assertEqualityDeletes(equalityDeleteVectorMap, equalityFieldIdsMap); + + // Delete non-existent values from the second column + equalityDeleteVectorMap.clear(); + equalityDeleteVectorMap.insert({0, {{10000, 10002, 19999}}}); + assertEqualityDeletes(equalityDeleteVectorMap, equalityFieldIdsMap); + + // Delete random rows from the second column + equalityDeleteVectorMap.clear(); + equalityDeleteVectorMap.insert({0, {makeSequenceValues(rowCount)}}); + assertEqualityDeletes(equalityDeleteVectorMap, equalityFieldIdsMap); + + // Delete 0 values + equalityDeleteVectorMap.clear(); + equalityDeleteVectorMap.insert({0, {{}}}); + assertEqualityDeletes(equalityDeleteVectorMap, equalityFieldIdsMap); + + // Delete all values + equalityDeleteVectorMap.clear(); + equalityDeleteVectorMap.insert({0, {makeSequenceValues(rowCount / 2)}}); + assertEqualityDeletes(equalityDeleteVectorMap, equalityFieldIdsMap); +} + +// Delete values from 2 columns with the following data: +// +// c1 c2 +// 0 0 +// 1 0 +// 2 1 +// 3 1 +// 4 2 +// ... ... +// 19999 9999 +TEST_F(HiveIcebergTest, equalityDeletesSingleFileMultipleColumns) { + folly::SingletonVault::singleton()->registrationComplete(); + + std::unordered_map> equalityFieldIdsMap; + std::unordered_map>> + equalityDeleteVectorMap; + equalityFieldIdsMap.insert({0, {1, 2}}); + + // Delete rows 0, 1 + equalityDeleteVectorMap.insert({0, {{0, 1}, {0, 0}}}); + assertEqualityDeletes(equalityDeleteVectorMap, equalityFieldIdsMap); + + // Delete rows 0, 2, 4, 6 + equalityDeleteVectorMap.clear(); + equalityDeleteVectorMap.insert({0, {{0, 2, 4, 6}, {0, 1, 2, 3}}}); + assertEqualityDeletes(equalityDeleteVectorMap, equalityFieldIdsMap); + + // Delete the last row + equalityDeleteVectorMap.clear(); + equalityDeleteVectorMap.insert({0, {{19999}, {9999}}}); + assertEqualityDeletes(equalityDeleteVectorMap, equalityFieldIdsMap); + + // Delete non-existent values + equalityDeleteVectorMap.clear(); + equalityDeleteVectorMap.insert({0, {{20000, 30000}, {10000, 1500}}}); + assertEqualityDeletes(equalityDeleteVectorMap, equalityFieldIdsMap); + + // Delete 0 values + equalityDeleteVectorMap.clear(); + equalityDeleteVectorMap.insert({0, {{}, {}}}); + assertEqualityDeletes(equalityDeleteVectorMap, equalityFieldIdsMap); + + // Delete all values + equalityDeleteVectorMap.clear(); + equalityDeleteVectorMap.insert( + {0, {makeSequenceValues(rowCount), makeSequenceValues(rowCount, 2)}}); + assertEqualityDeletes( + equalityDeleteVectorMap, + equalityFieldIdsMap, + "SELECT * FROM tmp WHERE 1 = 0"); +} + +TEST_F(HiveIcebergTest, equalityDeletesMultipleFiles) { + folly::SingletonVault::singleton()->registrationComplete(); + + std::unordered_map> equalityFieldIdsMap; + std::unordered_map>> + equalityDeleteVectorMap; + equalityFieldIdsMap.insert({{0, {1}}, {1, {2}}}); + + // Delete rows {0, 1} from c0, {2, 3} from c1, with two equality delete files + equalityDeleteVectorMap.insert({{0, {{0, 1}}}, {1, {{2, 3}}}}); + assertEqualityDeletes(equalityDeleteVectorMap, equalityFieldIdsMap); + + // Delete using 3 equality delete files + equalityFieldIdsMap.insert({{2, {3}}}); + equalityDeleteVectorMap.insert({{0, {{0, 1}}}, {1, {{2, 3}}}, {2, {{4, 5}}}}); + assertEqualityDeletes(equalityDeleteVectorMap, equalityFieldIdsMap); + + // Delete 0 values + equalityDeleteVectorMap.clear(); + equalityDeleteVectorMap.insert({{0, {{}}}, {1, {{}}}, {2, {{}}}}); + assertEqualityDeletes(equalityDeleteVectorMap, equalityFieldIdsMap); + + // Delete all values + equalityDeleteVectorMap.clear(); + equalityDeleteVectorMap.insert( + {{0, {makeSequenceValues(rowCount)}}, + {1, {makeSequenceValues(rowCount)}}, + {2, {makeSequenceValues(rowCount)}}}); + assertEqualityDeletes( + equalityDeleteVectorMap, + equalityFieldIdsMap, + "SELECT * FROM tmp WHERE 1 = 0"); +} + +TEST_F(HiveIcebergTest, TestSubFieldEqualityDelete) { + folly::SingletonVault::singleton()->registrationComplete(); + + // Write the base file + std::shared_ptr dataFilePath = TempFilePath::create(); + std::vector dataVectors = {makeRowVector( + {"c_bigint", "c_row"}, + {makeFlatVector(20, [](auto row) { return row + 1; }), + makeRowVector( + {"c0", "c1", "c2"}, + {makeFlatVector(20, [](auto row) { return row + 1; }), + makeFlatVector(20, [](auto row) { return row + 1; }), + makeFlatVector(20, [](auto row) { return row + 1; })})})}; + int32_t numDataColumns = 1; + dataFilePath = writeDataFiles(rowCount, numDataColumns, 1, dataVectors)[0]; + + // Write the delete file. Equality delete field is c_row.c1 + std::vector deleteFiles; + // Delete rows {0, 1} from c_row.c1, whose schema Id is 4 + std::vector deleteDataVectors = {makeRowVector( + {"c1"}, {makeFlatVector(2, [](auto row) { return row + 1; })})}; + + std::vector> deleteFilePaths; + auto equalityFieldIds = std::vector({4}); + auto deleteFilePath = TempFilePath::create(); + writeToFile(deleteFilePath->getPath(), deleteDataVectors.back()); + deleteFilePaths.push_back(deleteFilePath); + IcebergDeleteFile deleteFile( + FileContent::kEqualityDeletes, + deleteFilePaths.back()->getPath(), + fileFormat_, + 2, + testing::internal::GetFileSize( + std::fopen(deleteFilePaths.back()->getPath().c_str(), "r")), + equalityFieldIds); + deleteFiles.push_back(deleteFile); + + auto icebergSplits = makeIcebergSplits(dataFilePath->getPath(), deleteFiles); + + // Select both c_bigint and c_row column columns + std::string duckDbSql = "SELECT * FROM tmp WHERE c_row.c0 not in (1, 2)"; + assertEqualityDeletes( + icebergSplits.back(), asRowType(dataVectors[0]->type()), duckDbSql); + + // SELECT only c_bigint column + duckDbSql = "SELECT c_bigint FROM tmp WHERE c_row.c0 not in (1, 2)"; + assertEqualityDeletes( + icebergSplits.back(), ROW({"c_bigint"}, {BIGINT()}), duckDbSql); +} } // namespace facebook::velox::connector::hive::iceberg diff --git a/velox/connectors/hive/iceberg/tests/IcebergSplitReaderBenchmark.cpp b/velox/connectors/hive/iceberg/tests/IcebergSplitReaderBenchmark.cpp index 2c2c26297fd2..d835d7239b71 100644 --- a/velox/connectors/hive/iceberg/tests/IcebergSplitReaderBenchmark.cpp +++ b/velox/connectors/hive/iceberg/tests/IcebergSplitReaderBenchmark.cpp @@ -15,8 +15,13 @@ */ #include "velox/connectors/hive/iceberg/tests/IcebergSplitReaderBenchmark.h" + #include +#include + +#include "velox/vector/tests/utils/VectorMaker.h" + using namespace facebook::velox; using namespace facebook::velox::dwio; using namespace facebook::velox::dwio::common; @@ -330,9 +335,14 @@ void IcebergSplitReaderBenchmark::readSingleColumn( suspender.dismiss(); + auto ioExecutor = std::make_unique(3); + std::shared_ptr remainingFilterExprSet{nullptr}; + std::atomic totalRemainingFilterMs; + uint64_t resultSize = 0; for (std::shared_ptr split : splits) { scanSpec->resetCachedValues(true); + std::unique_ptr icebergSplitReader = std::make_unique( split, @@ -344,8 +354,10 @@ void IcebergSplitReaderBenchmark::readSingleColumn( ioStats, fsStats, &fileHandleFactory, - nullptr, - scanSpec); + ioExecutor.get(), + scanSpec, + connectorQueryCtx_->expressionEvaluator(), + totalRemainingFilterMs); std::shared_ptr randomSkip; icebergSplitReader->configureReaderOptions(randomSkip); diff --git a/velox/connectors/lakehouse/CMakeLists.txt b/velox/connectors/lakehouse/CMakeLists.txt new file mode 100644 index 000000000000..082233132678 --- /dev/null +++ b/velox/connectors/lakehouse/CMakeLists.txt @@ -0,0 +1,16 @@ +# Copyright (c) Facebook, Inc. and its affiliates. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +add_subdirectory(common) +add_subdirectory(iceberg) diff --git a/velox/connectors/lakehouse/common/CMakeLists.txt b/velox/connectors/lakehouse/common/CMakeLists.txt new file mode 100644 index 000000000000..2357d40a4c1b --- /dev/null +++ b/velox/connectors/lakehouse/common/CMakeLists.txt @@ -0,0 +1,45 @@ +# Copyright (c) Facebook, Inc. and its affiliates. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +velox_add_library(velox_lakehouse_hive_config OBJECT HiveConfig.cpp) +velox_link_libraries(velox_lakehouse_hive_config velox_core velox_exception) + +velox_add_library( + velox_connector_lakehouse_common + OBJECT + FileHandle.cpp + HiveConfig.cpp + HiveConnector.cpp + HiveConnectorUtil.cpp + HiveConnectorSplit.cpp + HiveDataSink.cpp + HiveDataSource.cpp + HivePartitionUtil.cpp + PartitionIdGenerator.cpp + SplitReader.cpp + TableHandle.cpp) + +velox_link_libraries( + velox_connector_lakehouse_common + PUBLIC velox_lakehouse_iceberg_splitreader + PRIVATE velox_common_io velox_connector velox_dwio_catalog_fbhive + velox_lakehouse_hive_partition_function) + +velox_add_library(velox_lakehouse_hive_partition_function HivePartitionFunction.cpp) + +velox_link_libraries(velox_lakehouse_hive_partition_function velox_core velox_exec) + +if(VELOX_BUILD_TESTING) + add_subdirectory(tests) +endif() diff --git a/velox/connectors/lakehouse/common/FileHandle.cpp b/velox/connectors/lakehouse/common/FileHandle.cpp new file mode 100644 index 000000000000..e37e093d76aa --- /dev/null +++ b/velox/connectors/lakehouse/common/FileHandle.cpp @@ -0,0 +1,78 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "velox/connectors/lakehouse/common/FileHandle.h" +#include "velox/common/base/Counters.h" +#include "velox/common/base/StatsReporter.h" +#include "velox/common/file/FileSystems.h" +#include "velox/common/time/Timer.h" + +#include + +namespace facebook::velox::connector::lakehouse::common { + +uint64_t FileHandleSizer::operator()(const FileHandle& fileHandle) { + // TODO: add to support variable file cache size support when the file system + // underneath supports. + return 1; +} + +namespace { +// The group tracking is at the level of the directory, i.e. Hive partition. +std::string groupName(const std::string& filename) { + const char* slash = strrchr(filename.c_str(), '/'); + return slash ? std::string(filename.data(), slash - filename.data()) + : filename; +} +} // namespace + +std::unique_ptr FileHandleGenerator::operator()( + const FileHandleKey& key, + const FileProperties* properties, + filesystems::File::IoStats* stats) { + // We have seen cases where drivers are stuck when creating file handles. + // Adding a trace here to spot this more easily in future. + process::TraceContext trace("FileHandleGenerator::operator()"); + uint64_t elapsedTimeUs{0}; + std::unique_ptr fileHandle; + { + MicrosecondTimer timer(&elapsedTimeUs); + fileHandle = std::make_unique(); + filesystems::FileOptions options; + options.stats = stats; + options.tokenProvider = key.tokenProvider; + if (properties) { + options.fileSize = properties->fileSize; + options.readRangeHint = properties->readRangeHint; + options.extraFileInfo = properties->extraFileInfo; + } + const auto& filename = key.filename; + fileHandle->file = filesystems::getFileSystem(filename, properties_) + ->openFileForRead(filename, options); + fileHandle->uuid = StringIdLease(fileIds(), filename); + fileHandle->groupId = StringIdLease(fileIds(), groupName(filename)); + VLOG(1) << "Generating file handle for: " << filename + << " uuid: " << fileHandle->uuid.id(); + } + RECORD_HISTOGRAM_METRIC_VALUE( + kMetricHiveFileHandleGenerateLatencyMs, elapsedTimeUs / 1000); + // TODO: build the hash map/etc per file type -- presumably after reading + // the appropriate magic number from the file, or perhaps we include the file + // type in the file handle key. + return fileHandle; +} + +} // namespace facebook::velox::connector::lakehouse::common diff --git a/velox/connectors/lakehouse/common/FileHandle.h b/velox/connectors/lakehouse/common/FileHandle.h new file mode 100644 index 000000000000..8616f5f5efcb --- /dev/null +++ b/velox/connectors/lakehouse/common/FileHandle.h @@ -0,0 +1,130 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// A FileHandle is a File pointer plus some (optional, file-type-dependent) +// extra information for speeding up loading columnar data. For example, when +// we open a file we might build a hash map saying what region(s) on disk +// correspond to a given column in a given stripe. +// +// The FileHandle will normally be used in conjunction with a CachedFactory +// to speed up queries that hit the same files repeatedly; see the +// FileHandleCache and FileHandleFactory. + +#pragma once + +#include "velox/common/base/BitUtil.h" +#include "velox/common/caching/CachedFactory.h" +#include "velox/common/caching/FileIds.h" +#include "velox/common/config/Config.h" +#include "velox/common/file/File.h" +#include "velox/common/file/TokenProvider.h" +#include "velox/connectors/lakehouse/common/FileProperties.h" + +namespace facebook::velox::connector::lakehouse::common { + +// See the file comment. +struct FileHandle { + std::shared_ptr file; + + // Each time we make a new FileHandle we assign it a uuid and use that id as + // the identifier in downstream data caching structures. This saves a lot of + // memory compared to using the filename as the identifier. + StringIdLease uuid; + + // Id for the group of files this belongs to, e.g. its + // directory. Used for coarse granularity access tracking, for + // example to decide placing on SSD. + StringIdLease groupId; + + // We'll want to have a hash map here to record the identifier->byte range + // mappings. Different formats may have different identifiers, so we may need + // a union of maps. For example in orc you need 3 integers (I think, to be + // confirmed with xldb): the row bundle, the node, and the sequence. For the + // first diff we'll not include the map. +}; + +/// Estimates the memory usage of a FileHandle object. +struct FileHandleSizer { + uint64_t operator()(const FileHandle& a); +}; + +struct FileHandleKey { + std::string filename; + std::shared_ptr tokenProvider{nullptr}; + + bool operator==(const FileHandleKey& other) const { + if (filename != other.filename) { + return false; + } + + if (tokenProvider == other.tokenProvider) { + return true; + } + + if (!tokenProvider || !other.tokenProvider) { + return false; + } + + return tokenProvider->equals(*other.tokenProvider); + } +}; + +} // namespace facebook::velox::connector::lakehouse::common + +namespace std { +template <> +struct hash { + size_t operator()(const facebook::velox::connector::lakehouse::common::FileHandleKey& key) const noexcept { + size_t filenameHash = std::hash()(key.filename); + return key.tokenProvider ? facebook::velox::bits::hashMix( + filenameHash, key.tokenProvider->hash()) + : filenameHash; + } +}; +} // namespace std + +namespace facebook::velox::connector::lakehouse::common { +using FileHandleCache = + SimpleLRUCache; + +// Creates FileHandles via the Generator interface the CachedFactory requires. +class FileHandleGenerator { + public: + FileHandleGenerator() {} + FileHandleGenerator(std::shared_ptr properties) + : properties_(std::move(properties)) {} + std::unique_ptr operator()( + const FileHandleKey& filename, + const FileProperties* properties, + filesystems::File::IoStats* stats); + + private: + const std::shared_ptr properties_; +}; + +using FileHandleFactory = CachedFactory< + FileHandleKey, + FileHandle, + FileHandleGenerator, + FileProperties, + filesystems::File::IoStats, + FileHandleSizer>; + +using FileHandleCachedPtr = CachedPtr; + +using FileHandleCacheStats = SimpleLRUCacheStats; + +} // namespace facebook::velox::connector::lakehouse::common diff --git a/velox/connectors/lakehouse/common/FileProperties.h b/velox/connectors/lakehouse/common/FileProperties.h new file mode 100644 index 000000000000..4239f812df56 --- /dev/null +++ b/velox/connectors/lakehouse/common/FileProperties.h @@ -0,0 +1,39 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// A FileHandle is a File pointer plus some (optional, file-type-dependent) +// extra information for speeding up loading columnar data. For example, when +// we open a file we might build a hash map saying what region(s) on disk +// correspond to a given column in a given stripe. +// +// The FileHandle will normally be used in conjunction with a CachedFactory +// to speed up queries that hit the same files repeatedly; see the +// FileHandleCache and FileHandleFactory. + +#pragma once + +#include + +namespace facebook::velox::connector::lakehouse::common { + +struct FileProperties { + std::optional fileSize; + std::optional modificationTime; + std::optional readRangeHint{std::nullopt}; + std::shared_ptr extraFileInfo{nullptr}; +}; + +} // namespace facebook::velox::connector::lakehouse::common diff --git a/velox/connectors/lakehouse/common/HiveConfig.cpp b/velox/connectors/lakehouse/common/HiveConfig.cpp new file mode 100644 index 000000000000..eb727e011395 --- /dev/null +++ b/velox/connectors/lakehouse/common/HiveConfig.cpp @@ -0,0 +1,248 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "velox/connectors/lakehouse/common/HiveConfig.h" +#include "velox/common/config/Config.h" +#include "velox/core/QueryConfig.h" + +#include + +namespace facebook::velox::connector::lakehouse::common { + +namespace { + +HiveConfig::InsertExistingPartitionsBehavior +stringToInsertExistingPartitionsBehavior(const std::string& strValue) { + auto upperValue = boost::algorithm::to_upper_copy(strValue); + if (upperValue == "ERROR") { + return HiveConfig ::InsertExistingPartitionsBehavior::kError; + } + if (upperValue == "OVERWRITE") { + return HiveConfig ::InsertExistingPartitionsBehavior::kOverwrite; + } + VELOX_UNSUPPORTED( + "Unsupported insert existing partitions behavior: {}.", strValue); +} + +} // namespace + +// static +std::string HiveConfig ::insertExistingPartitionsBehaviorString( + InsertExistingPartitionsBehavior behavior) { + switch (behavior) { + case InsertExistingPartitionsBehavior::kError: + return "ERROR"; + case InsertExistingPartitionsBehavior::kOverwrite: + return "OVERWRITE"; + default: + return fmt::format("UNKNOWN BEHAVIOR {}", static_cast(behavior)); + } +} + +HiveConfig::InsertExistingPartitionsBehavior +HiveConfig::insertExistingPartitionsBehavior( + const config::ConfigBase* session) const { + return stringToInsertExistingPartitionsBehavior(session->get( + kInsertExistingPartitionsBehaviorSession, + config_->get(kInsertExistingPartitionsBehavior, "ERROR"))); +} + +uint32_t HiveConfig ::maxPartitionsPerWriters( + const config::ConfigBase* session) const { + return session->get( + kMaxPartitionsPerWritersSession, + config_->get(kMaxPartitionsPerWriters, 128)); +} + +uint32_t HiveConfig ::maxBucketCount(const config::ConfigBase* session) const { + return session->get( + kMaxBucketCountSession, config_->get(kMaxBucketCount, 100'000)); +} + +bool HiveConfig ::immutablePartitions() const { + return config_->get(kImmutablePartitions, false); +} + +std::string HiveConfig ::gcsEndpoint() const { + return config_->get(kGcsEndpoint, std::string("")); +} + +std::string HiveConfig ::gcsCredentialsPath() const { + return config_->get(kGcsCredentialsPath, std::string("")); +} + +std::optional HiveConfig ::gcsMaxRetryCount() const { + return static_cast>(config_->get(kGcsMaxRetryCount)); +} + +std::optional HiveConfig ::gcsMaxRetryTime() const { + return static_cast>( + config_->get(kGcsMaxRetryTime)); +} + +bool HiveConfig ::isOrcUseColumnNames(const config::ConfigBase* session) const { + return session->get( + kOrcUseColumnNamesSession, config_->get(kOrcUseColumnNames, false)); +} + +bool HiveConfig ::isParquetUseColumnNames( + const config::ConfigBase* session) const { + return session->get( + kParquetUseColumnNamesSession, + config_->get(kParquetUseColumnNames, false)); +} + +bool HiveConfig ::isFileColumnNamesReadAsLowerCase( + const config::ConfigBase* session) const { + return session->get( + kFileColumnNamesReadAsLowerCaseSession, + config_->get(kFileColumnNamesReadAsLowerCase, false)); +} + +bool HiveConfig ::isPartitionPathAsLowerCase( + const config::ConfigBase* session) const { + return session->get(kPartitionPathAsLowerCaseSession, true); +} + +bool HiveConfig ::allowNullPartitionKeys( + const config::ConfigBase* session) const { + return session->get( + kAllowNullPartitionKeysSession, + config_->get(kAllowNullPartitionKeys, true)); +} + +bool HiveConfig ::ignoreMissingFiles(const config::ConfigBase* session) const { + return session->get(kIgnoreMissingFilesSession, false); +} + +int64_t HiveConfig ::maxCoalescedBytes(const config::ConfigBase* session) const { + return session->get( + kMaxCoalescedBytesSession, + config_->get(kMaxCoalescedBytes, 128 << 20)); // 128MB +} + +int32_t HiveConfig ::maxCoalescedDistanceBytes( + const config::ConfigBase* session) const { + const auto distance = config::toCapacity( + session->get( + kMaxCoalescedDistanceSession, + config_->get(kMaxCoalescedDistance, "512kB")), + config::CapacityUnit::BYTE); + VELOX_USER_CHECK_LE( + distance, + std::numeric_limits::max(), + "The max merge distance to combine read requests must be less than 2GB." + " Got {} bytes.", + distance); + return int32_t(distance); +} + +int32_t HiveConfig ::prefetchRowGroups() const { + return config_->get(kPrefetchRowGroups, 1); +} + +int32_t HiveConfig ::loadQuantum(const config::ConfigBase* session) const { + return session->get( + kLoadQuantumSession, config_->get(kLoadQuantum, 8 << 20)); +} + +int32_t HiveConfig ::numCacheFileHandles() const { + return config_->get(kNumCacheFileHandles, 20'000); +} + +uint64_t HiveConfig ::fileHandleExpirationDurationMs() const { + return config_->get(kFileHandleExpirationDurationMs, 0); +} + +bool HiveConfig ::isFileHandleCacheEnabled() const { + return config_->get(kEnableFileHandleCache, true); +} + +std::string HiveConfig ::writeFileCreateConfig() const { + return config_->get(kWriteFileCreateConfig, ""); +} + +uint32_t HiveConfig ::sortWriterMaxOutputRows( + const config::ConfigBase* session) const { + return session->get( + kSortWriterMaxOutputRowsSession, + config_->get(kSortWriterMaxOutputRows, 1024)); +} + +uint64_t HiveConfig ::sortWriterMaxOutputBytes( + const config::ConfigBase* session) const { + return config::toCapacity( + session->get( + kSortWriterMaxOutputBytesSession, + config_->get(kSortWriterMaxOutputBytes, "10MB")), + config::CapacityUnit::BYTE); +} + +uint64_t HiveConfig ::sortWriterFinishTimeSliceLimitMs( + const config::ConfigBase* session) const { + return session->get( + kSortWriterFinishTimeSliceLimitMsSession, + config_->get(kSortWriterFinishTimeSliceLimitMs, 5'000)); +} + +uint64_t HiveConfig ::footerEstimatedSize() const { + return config_->get(kFooterEstimatedSize, 256UL << 10); +} + +uint64_t HiveConfig ::filePreloadThreshold() const { + return config_->get(kFilePreloadThreshold, 8UL << 20); +} + +uint8_t HiveConfig ::readTimestampUnit(const config::ConfigBase* session) const { + const auto unit = session->get( + kReadTimestampUnitSession, + config_->get(kReadTimestampUnit, 3 /*milli*/)); + VELOX_CHECK( + unit == 3 || unit == 6 /*micro*/ || unit == 9 /*nano*/, + "Invalid timestamp unit."); + return unit; +} + +bool HiveConfig ::readTimestampPartitionValueAsLocalTime( + const config::ConfigBase* session) const { + return session->get( + kReadTimestampPartitionValueAsLocalTimeSession, + config_->get(kReadTimestampPartitionValueAsLocalTime, true)); +} + +bool HiveConfig ::readStatsBasedFilterReorderDisabled( + const config::ConfigBase* session) const { + return session->get( + kReadStatsBasedFilterReorderDisabledSession, + config_->get(kReadStatsBasedFilterReorderDisabled, false)); +} + +bool HiveConfig ::isRequestedTypeCheckEnabled( + const config::ConfigBase* session) const { + return session->get( + kEnableRequestedTypeCheckSession, + config_->get(kEnableRequestedTypeCheck, true)); +} + +std::string HiveConfig ::hiveLocalDataPath() const { + return config_->get(kLocalDataPath, ""); +} + +std::string HiveConfig ::hiveLocalFileFormat() const { + return config_->get(kLocalFileFormat, ""); +} + +} // namespace facebook::velox::connector::lakehouse::common diff --git a/velox/connectors/lakehouse/common/HiveConfig.h b/velox/connectors/lakehouse/common/HiveConfig.h new file mode 100644 index 000000000000..6f51a516758a --- /dev/null +++ b/velox/connectors/lakehouse/common/HiveConfig.h @@ -0,0 +1,294 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#pragma once + +#include +#include +#include "velox/common/base/Exceptions.h" + +namespace facebook::velox::config { +class ConfigBase; +} + +namespace facebook::velox::connector::lakehouse::common { + +/// Hive connector configs. +class HiveConfig { + public: + enum class InsertExistingPartitionsBehavior { + kError, + kOverwrite, + }; + + static std::string insertExistingPartitionsBehaviorString( + InsertExistingPartitionsBehavior behavior); + + /// Behavior on insert into existing partitions. + static constexpr const char* kInsertExistingPartitionsBehaviorSession = + "insert_existing_partitions_behavior"; + static constexpr const char* kInsertExistingPartitionsBehavior = + "insert-existing-partitions-behavior"; + + /// Maximum number of (bucketed) partitions per a single table writer + /// instance. + static constexpr const char* kMaxPartitionsPerWriters = + "max-partitions-per-writers"; + static constexpr const char* kMaxPartitionsPerWritersSession = + "max_partitions_per_writers"; + + /// Maximum number of buckets allowed to output by the table writers. + static constexpr const char* kMaxBucketCount = "hive.max-bucket-count"; + static constexpr const char* kMaxBucketCountSession = "hive.max_bucket_count"; + + /// Whether new data can be inserted into an unpartition table. + /// Velox currently does not support appending data to existing partitions. + static constexpr const char* kImmutablePartitions = + "hive.immutable-partitions"; + + /// The GCS storage endpoint server. + static constexpr const char* kGcsEndpoint = "hive.gcs.endpoint"; + + /// The GCS service account configuration JSON key file. + static constexpr const char* kGcsCredentialsPath = + "hive.gcs.json-key-file-path"; + + /// The GCS maximum retry counter of transient errors. + static constexpr const char* kGcsMaxRetryCount = "hive.gcs.max-retry-count"; + + /// The GCS maximum time allowed to retry transient errors. + static constexpr const char* kGcsMaxRetryTime = "hive.gcs.max-retry-time"; + + /// Maps table field names to file field names using names, not indices. + // TODO: remove hive_orc_use_column_names since it doesn't exist in presto, + // right now this is only used for testing. + static constexpr const char* kOrcUseColumnNames = "hive.orc.use-column-names"; + static constexpr const char* kOrcUseColumnNamesSession = + "hive_orc_use_column_names"; + + /// Maps table field names to file field names using names, not indices. + static constexpr const char* kParquetUseColumnNames = + "hive.parquet.use-column-names"; + static constexpr const char* kParquetUseColumnNamesSession = + "parquet_use_column_names"; + + /// Reads the source file column name as lower case. + static constexpr const char* kFileColumnNamesReadAsLowerCase = + "file-column-names-read-as-lower-case"; + static constexpr const char* kFileColumnNamesReadAsLowerCaseSession = + "file_column_names_read_as_lower_case"; + + static constexpr const char* kPartitionPathAsLowerCaseSession = + "partition_path_as_lower_case"; + + static constexpr const char* kAllowNullPartitionKeys = + "allow-null-partition-keys"; + static constexpr const char* kAllowNullPartitionKeysSession = + "allow_null_partition_keys"; + + static constexpr const char* kIgnoreMissingFilesSession = + "ignore_missing_files"; + + /// The max coalesce bytes for a request. + static constexpr const char* kMaxCoalescedBytes = "max-coalesced-bytes"; + static constexpr const char* kMaxCoalescedBytesSession = + "max-coalesced-bytes"; + + /// The max merge distance to combine read requests. + /// Note: The session property name differs from the constant name for + /// backward compatibility with Presto. + static constexpr const char* kMaxCoalescedDistance = "max-coalesced-distance"; + static constexpr const char* kMaxCoalescedDistanceSession = + "orc_max_merge_distance"; + + /// The number of prefetch rowgroups + static constexpr const char* kPrefetchRowGroups = "prefetch-rowgroups"; + + /// The total size in bytes for a direct coalesce request. Up to 8MB load + /// quantum size is supported when SSD cache is enabled. + static constexpr const char* kLoadQuantum = "load-quantum"; + static constexpr const char* kLoadQuantumSession = "load-quantum"; + + /// Maximum number of entries in the file handle cache. + static constexpr const char* kNumCacheFileHandles = "num_cached_file_handles"; + + /// Expiration time in ms for a file handle in the cache. A value of 0 + /// means cache will not evict the handle after kFileHandleExprationDurationMs + /// has passed. + static constexpr const char* kFileHandleExpirationDurationMs = + "file-handle-expiration-duration-ms"; + + /// Enable file handle cache. + static constexpr const char* kEnableFileHandleCache = + "file-handle-cache-enabled"; + + /// The size in bytes to be fetched with Meta data together, used when the + /// data after meta data will be used later. Optimization to decrease small IO + /// request + static constexpr const char* kFooterEstimatedSize = "footer-estimated-size"; + + /// The threshold of file size in bytes when the whole file is fetched with + /// meta data together. Optimization to decrease the small IO requests + static constexpr const char* kFilePreloadThreshold = "file-preload-threshold"; + + /// Config used to create write files. This config is provided to underlying + /// file system through hive connector and data sink. The config is free form. + /// The form should be defined by the underlying file system. + static constexpr const char* kWriteFileCreateConfig = + "hive.write_file_create_config"; + + /// Maximum number of rows for sort writer in one batch of output. + static constexpr const char* kSortWriterMaxOutputRows = + "sort-writer-max-output-rows"; + static constexpr const char* kSortWriterMaxOutputRowsSession = + "sort_writer_max_output_rows"; + + /// Maximum bytes for sort writer in one batch of output. + static constexpr const char* kSortWriterMaxOutputBytes = + "sort-writer-max-output-bytes"; + static constexpr const char* kSortWriterMaxOutputBytesSession = + "sort_writer_max_output_bytes"; + + /// Sort Writer will exit finish() method after this many milliseconds even if + /// it has not completed its work yet. Zero means no time limit. + static constexpr const char* kSortWriterFinishTimeSliceLimitMs = + "sort-writer_finish_time_slice_limit_ms"; + static constexpr const char* kSortWriterFinishTimeSliceLimitMsSession = + "sort_writer_finish_time_slice_limit_ms"; + + // The unit for reading timestamps from files. + static constexpr const char* kReadTimestampUnit = + "hive.reader.timestamp-unit"; + static constexpr const char* kReadTimestampUnitSession = + "hive.reader.timestamp_unit"; + + static constexpr const char* kReadTimestampPartitionValueAsLocalTime = + "hive.reader.timestamp-partition-value-as-local-time"; + static constexpr const char* kReadTimestampPartitionValueAsLocalTimeSession = + "hive.reader.timestamp_partition_value_as_local_time"; + + static constexpr const char* kReadStatsBasedFilterReorderDisabled = + "stats-based-filter-reorder-disabled"; + static constexpr const char* kReadStatsBasedFilterReorderDisabledSession = + "stats_based_filter_reorder_disabled"; + + static constexpr const char* kLocalDataPath = "hive_local_data_path"; + static constexpr const char* kLocalFileFormat = "hive_local_file_format"; + + static constexpr const char* kEnableRequestedTypeCheck = + "enable-requested-type-check"; + static constexpr const char* kEnableRequestedTypeCheckSession = + "enable_requested_type_check"; + + InsertExistingPartitionsBehavior insertExistingPartitionsBehavior( + const config::ConfigBase* session) const; + + uint32_t maxPartitionsPerWriters(const config::ConfigBase* session) const; + + uint32_t maxBucketCount(const config::ConfigBase* session) const; + + bool immutablePartitions() const; + + std::string gcsEndpoint() const; + + std::string gcsCredentialsPath() const; + + std::optional gcsMaxRetryCount() const; + + std::optional gcsMaxRetryTime() const; + + bool isOrcUseColumnNames(const config::ConfigBase* session) const; + + bool isParquetUseColumnNames(const config::ConfigBase* session) const; + + bool isFileColumnNamesReadAsLowerCase( + const config::ConfigBase* session) const; + + bool isPartitionPathAsLowerCase(const config::ConfigBase* session) const; + + bool allowNullPartitionKeys(const config::ConfigBase* session) const; + + bool ignoreMissingFiles(const config::ConfigBase* session) const; + + int64_t maxCoalescedBytes(const config::ConfigBase* session) const; + + int32_t maxCoalescedDistanceBytes(const config::ConfigBase* session) const; + + int32_t prefetchRowGroups() const; + + int32_t loadQuantum(const config::ConfigBase* session) const; + + int32_t numCacheFileHandles() const; + + uint64_t fileHandleExpirationDurationMs() const; + + bool isFileHandleCacheEnabled() const; + + uint64_t fileWriterFlushThresholdBytes() const; + + std::string writeFileCreateConfig() const; + + uint32_t sortWriterMaxOutputRows(const config::ConfigBase* session) const; + + uint64_t sortWriterMaxOutputBytes(const config::ConfigBase* session) const; + + uint64_t sortWriterFinishTimeSliceLimitMs( + const config::ConfigBase* session) const; + + uint64_t footerEstimatedSize() const; + + uint64_t filePreloadThreshold() const; + + // Returns the timestamp unit used when reading timestamps from files. + uint8_t readTimestampUnit(const config::ConfigBase* session) const; + + // Whether to read timestamp partition value as local time. If false, read as + // UTC. + bool readTimestampPartitionValueAsLocalTime( + const config::ConfigBase* session) const; + + /// Returns true if the stats based filter reorder for read is disabled. + bool readStatsBasedFilterReorderDisabled( + const config::ConfigBase* session) const; + + /// Whether to enable requested type check in the ReaderBase::convertType. + /// Returns true by default. + bool isRequestedTypeCheckEnabled(const config::ConfigBase* session) const; + + /// Returns the file system path containing local data. If non-empty, + /// initializes LocalHiveConnectorMetadata to provide metadata for the tables + /// in the directory. + std::string hiveLocalDataPath() const; + + /// Returns the name of the file format to use in interpreting the contents of + /// hiveLocalDataPath(). + std::string hiveLocalFileFormat() const; + + HiveConfig(std::shared_ptr config) { + VELOX_CHECK_NOT_NULL( + config, "Config is null for HiveConfig initialization"); + config_ = std::move(config); + // TODO: add sanity check + } + + const std::shared_ptr& config() const { + return config_; + } + + private: + std::shared_ptr config_; +}; + +} // namespace facebook::velox::connector::lakehouse::common diff --git a/velox/connectors/lakehouse/common/HiveConnector.cpp b/velox/connectors/lakehouse/common/HiveConnector.cpp new file mode 100644 index 000000000000..88b1f0300c0b --- /dev/null +++ b/velox/connectors/lakehouse/common/HiveConnector.cpp @@ -0,0 +1,214 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "velox/connectors/lakehouse/common/HiveConnector.h" + +#include "velox/connectors/lakehouse/common/HiveConfig.h" +#include "velox/connectors/lakehouse/common/HiveDataSink.h" +#include "velox/connectors/lakehouse/common/HiveDataSource.h" +#include "velox/connectors/lakehouse/common/HivePartitionFunction.h" +#include "velox/connectors/lakehouse/iceberg/IcebergDataSink.h" + +#include +#include + +using namespace facebook::velox::exec; + +namespace facebook::velox::connector::lakehouse::common { + +namespace { +std::vector>& +hiveConnectorMetadataFactories() { + static std::vector> factories; + return factories; +} +} // namespace + +HiveConnector::HiveConnector( + const std::string& id, + std::shared_ptr config, + folly::Executor* executor) + : Connector(id), + hiveConfig_(std::make_shared(config)), + fileHandleFactory_( + hiveConfig_->isFileHandleCacheEnabled() + ? std::make_unique>( + hiveConfig_->numCacheFileHandles()) + : nullptr, + std::make_unique(config)), + executor_(executor) { + if (hiveConfig_->isFileHandleCacheEnabled()) { + LOG(INFO) << "Hive connector " << connectorId() + << " created with maximum of " + << hiveConfig_->numCacheFileHandles() + << " cached file handles with expiration of " + << hiveConfig_->fileHandleExpirationDurationMs() << "ms."; + } else { + LOG(INFO) << "Hive connector " << connectorId() + << " created with file handle cache disabled"; + } + for (auto& factory : hiveConnectorMetadataFactories()) { + metadata_ = factory->create(this); + if (metadata_ != nullptr) { + break; + } + } +} + +std::unique_ptr HiveConnector::createDataSource( + const RowTypePtr& outputType, + const ConnectorTableHandlePtr& tableHandle, + const std::unordered_map& columnHandles, + ConnectorQueryCtx* connectorQueryCtx) { + return std::make_unique( + outputType, + tableHandle, + columnHandles, + &fileHandleFactory_, + executor_, + connectorQueryCtx, + hiveConfig_); +} + +std::unique_ptr HiveConnector::createDataSink( + RowTypePtr inputType, + ConnectorInsertTableHandlePtr connectorInsertTableHandle, + ConnectorQueryCtx* connectorQueryCtx, + CommitStrategy commitStrategy) { + if (auto icebergInsertHandle = + std::dynamic_pointer_cast( + connectorInsertTableHandle)) { + return std::make_unique( + inputType, + icebergInsertHandle, + connectorQueryCtx, + commitStrategy, + hiveConfig_); + } else { + auto hiveInsertHandle = + std::dynamic_pointer_cast( + connectorInsertTableHandle); + + VELOX_CHECK_NOT_NULL( + hiveInsertHandle, "Hive connector expecting hive write handle!"); + return std::make_unique( + inputType, + hiveInsertHandle, + connectorQueryCtx, + commitStrategy, + hiveConfig_); + } +} + +std::unique_ptr HivePartitionFunctionSpec::create( + int numPartitions, + bool localExchange) const { + std::vector bucketToPartitions; + if (bucketToPartition_.empty()) { + // NOTE: if hive partition function spec doesn't specify bucket to partition + // mapping, then we do round-robin mapping based on the actual number of + // partitions. + bucketToPartitions.resize(numBuckets_); + for (int bucket = 0; bucket < numBuckets_; ++bucket) { + bucketToPartitions[bucket] = bucket % numPartitions; + } + if (localExchange) { + // Shuffle the map from bucket to partition for local exchange so we don't + // use the same map for remote shuffle. + std::shuffle( + bucketToPartitions.begin(), + bucketToPartitions.end(), + std::mt19937{0}); + } + } + return std::make_unique( + numBuckets_, + bucketToPartition_.empty() ? std::move(bucketToPartitions) + : bucketToPartition_, + channels_, + constValues_); +} + +std::string HivePartitionFunctionSpec::toString() const { + std::ostringstream keys; + size_t constIndex = 0; + for (auto i = 0; i < channels_.size(); ++i) { + if (i > 0) { + keys << ", "; + } + auto channel = channels_[i]; + if (channel == kConstantChannel) { + keys << "\"" << constValues_[constIndex++]->toString(0) << "\""; + } else { + keys << channel; + } + } + + return fmt::format("HIVE(({}) buckets: {})", keys.str(), numBuckets_); +} + +folly::dynamic HivePartitionFunctionSpec::serialize() const { + folly::dynamic obj = folly::dynamic::object; + obj["name"] = "HivePartitionFunctionSpec"; + obj["numBuckets"] = ISerializable::serialize(numBuckets_); + obj["bucketToPartition"] = ISerializable::serialize(bucketToPartition_); + obj["keys"] = ISerializable::serialize(channels_); + std::vector constValueExprs; + constValueExprs.reserve(constValues_.size()); + for (const auto& value : constValues_) { + constValueExprs.emplace_back(value); + } + obj["constants"] = ISerializable::serialize(constValueExprs); + return obj; +} + +// static +core::PartitionFunctionSpecPtr HivePartitionFunctionSpec::deserialize( + const folly::dynamic& obj, + void* context) { + std::vector channels = + ISerializable::deserialize>( + obj["keys"], context); + const auto constTypedValues = + ISerializable::deserialize>( + obj["constants"], context); + std::vector constValues; + constValues.reserve(constTypedValues.size()); + auto* pool = static_cast(context); + for (const auto& value : constTypedValues) { + constValues.emplace_back(value->toConstantVector(pool)); + } + return std::make_shared( + ISerializable::deserialize(obj["numBuckets"], context), + ISerializable::deserialize>( + obj["bucketToPartition"], context), + std::move(channels), + std::move(constValues)); +} + +void registerHivePartitionFunctionSerDe() { + auto& registry = DeserializationWithContextRegistryForSharedPtr(); + registry.Register( + "HivePartitionFunctionSpec", HivePartitionFunctionSpec::deserialize); +} + +bool registerHiveConnectorMetadataFactory( + std::unique_ptr factory) { + hiveConnectorMetadataFactories().push_back(std::move(factory)); + return true; +} + +} // namespace facebook::velox::connector::lakehouse::common diff --git a/velox/connectors/lakehouse/common/HiveConnector.h b/velox/connectors/lakehouse/common/HiveConnector.h new file mode 100644 index 000000000000..04f476ee0d88 --- /dev/null +++ b/velox/connectors/lakehouse/common/HiveConnector.h @@ -0,0 +1,174 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#pragma once + +#include "velox/connectors/Connector.h" +#include "velox/connectors/lakehouse/common/FileHandle.h" +#include "velox/connectors/lakehouse/common/HiveConfig.h" +#include "velox/core/PlanNode.h" + +namespace facebook::velox::dwio::common { +class DataSink; +class DataSource; +} // namespace facebook::velox::dwio::common + +namespace facebook::velox::connector::lakehouse::common { + +class HiveConnector : public Connector { + public: + HiveConnector( + const std::string& id, + std::shared_ptr config, + folly::Executor* executor); + + const std::shared_ptr& connectorConfig() + const override { + return hiveConfig_->config(); + } + + bool canAddDynamicFilter() const override { + return true; + } + + ConnectorMetadata* metadata() const override { + VELOX_CHECK_NOT_NULL(metadata_); + return metadata_.get(); + } + + std::unique_ptr createDataSource( + const RowTypePtr& outputType, + const ConnectorTableHandlePtr& tableHandle, + const connector::ColumnHandleMap& columnHandles, + ConnectorQueryCtx* connectorQueryCtx) override; + + bool supportsSplitPreload() override { + return true; + } + + std::unique_ptr createDataSink( + RowTypePtr inputType, + ConnectorInsertTableHandlePtr connectorInsertTableHandle, + ConnectorQueryCtx* connectorQueryCtx, + CommitStrategy commitStrategy) override; + + folly::Executor* executor() const override { + return executor_; + } + + FileHandleCacheStats fileHandleCacheStats() { + return fileHandleFactory_.cacheStats(); + } + + // NOTE: this is to clear file handle cache which might affect performance, + // and is only used for operational purposes. + FileHandleCacheStats clearFileHandleCache() { + return fileHandleFactory_.clearCache(); + } + + protected: + const std::shared_ptr hiveConfig_; + FileHandleFactory fileHandleFactory_; + folly::Executor* executor_; + std::shared_ptr metadata_; +}; + +class HiveConnectorFactory : public ConnectorFactory { + public: + static constexpr const char* kHiveConnectorName = "hive"; + + HiveConnectorFactory() : ConnectorFactory(kHiveConnectorName) {} + + explicit HiveConnectorFactory(const char* connectorName) + : ConnectorFactory(connectorName) {} + + std::shared_ptr newConnector( + const std::string& id, + std::shared_ptr config, + folly::Executor* ioExecutor = nullptr, + folly::Executor* cpuExecutor = nullptr) override { + return std::make_shared(id, config, ioExecutor); + } +}; + +class HivePartitionFunctionSpec : public core::PartitionFunctionSpec { + public: + HivePartitionFunctionSpec( + int numBuckets, + std::vector bucketToPartition, + std::vector channels, + std::vector constValues) + : numBuckets_(numBuckets), + bucketToPartition_(std::move(bucketToPartition)), + channels_(std::move(channels)), + constValues_(std::move(constValues)) {} + + /// The constructor without 'bucketToPartition' input is used in case that + /// we don't know the actual number of partitions until we create the + /// partition function instance. The hive partition function spec then builds + /// a bucket to partition map based on the actual number of partitions with + /// round-robin partitioning scheme to create the function instance. For + /// instance, when we create the local partition node with hive bucket + /// function to support multiple table writer drivers, we don't know the the + /// actual number of table writer drivers until start the task. + HivePartitionFunctionSpec( + int numBuckets, + std::vector channels, + std::vector constValues) + : HivePartitionFunctionSpec( + numBuckets, + {}, + std::move(channels), + std::move(constValues)) {} + + std::unique_ptr create( + int numPartitions, + bool localExchange) const override; + + std::string toString() const override; + + folly::dynamic serialize() const override; + + static core::PartitionFunctionSpecPtr deserialize( + const folly::dynamic& obj, + void* context); + + private: + const int numBuckets_; + const std::vector bucketToPartition_; + const std::vector channels_; + const std::vector constValues_; +}; + +void registerHivePartitionFunctionSerDe(); + +/// Hook for connecting metadata functions to a HiveConnector. Each registered +/// factory is called after initializing a HiveConnector until one of these +/// returns a ConnectorMetadata instance. +class HiveConnectorMetadataFactory { + public: + virtual ~HiveConnectorMetadataFactory() = default; + + /// Returns a ConnectorMetadata to complete'hiveConnector' if 'this' + /// recognizes a data source, e.g. local file system or remote metadata + /// service associated to configs in 'hiveConnector'. + virtual std::shared_ptr create( + HiveConnector* connector) = 0; +}; + +bool registerHiveConnectorMetadataFactory( + std::unique_ptr); + +} // namespace facebook::velox::connector::lakehouse::common diff --git a/velox/connectors/lakehouse/common/HiveConnectorSplit.cpp b/velox/connectors/lakehouse/common/HiveConnectorSplit.cpp new file mode 100644 index 000000000000..3b011244a3fc --- /dev/null +++ b/velox/connectors/lakehouse/common/HiveConnectorSplit.cpp @@ -0,0 +1,225 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "velox/connectors/lakehouse/common/HiveConnectorSplit.h" + +namespace facebook::velox::connector::lakehouse::common { + +std::string HiveConnectorSplit::toString() const { + if (tableBucketNumber.has_value()) { + return fmt::format( + "Hive: {} {} - {} {}", + filePath, + start, + length, + tableBucketNumber.value()); + } + return fmt::format("Hive: {} {} - {}", filePath, start, length); +} + +uint64_t HiveConnectorSplit::size() const { + return length; +} + +std::string HiveConnectorSplit::getFileName() const { + const auto i = filePath.rfind('/'); + return i == std::string::npos ? filePath : filePath.substr(i + 1); +} + +folly::dynamic HiveConnectorSplit::serialize() const { + folly::dynamic obj = folly::dynamic::object; + obj["name"] = "HiveConnectorSplit"; + obj["connectorId"] = connectorId; + obj["splitWeight"] = splitWeight; + obj["cacheable"] = cacheable; + obj["filePath"] = filePath; + obj["fileFormat"] = dwio::common::toString(fileFormat); + obj["start"] = start; + obj["length"] = length; + + folly::dynamic partitionKeysObj = folly::dynamic::object; + for (const auto& [key, value] : partitionKeys) { + partitionKeysObj[key] = + value.has_value() ? folly::dynamic(value.value()) : nullptr; + } + obj["partitionKeys"] = partitionKeysObj; + + obj["tableBucketNumber"] = tableBucketNumber.has_value() + ? folly::dynamic(tableBucketNumber.value()) + : nullptr; + + if (bucketConversion.has_value()) { + folly::dynamic bucketConversionObj = folly::dynamic::object; + bucketConversionObj["tableBucketCount"] = + bucketConversion->tableBucketCount; + bucketConversionObj["partitionBucketCount"] = + bucketConversion->partitionBucketCount; + folly::dynamic bucketColumnHandlesArray = folly::dynamic::array; + for (const auto& handle : bucketConversion->bucketColumnHandles) { + bucketColumnHandlesArray.push_back(handle->serialize()); + } + bucketConversionObj["bucketColumnHandles"] = bucketColumnHandlesArray; + obj["bucketConversion"] = bucketConversionObj; + } else { + obj["bucketConversion"] = nullptr; + } + + folly::dynamic customSplitInfoObj = folly::dynamic::object; + for (const auto& [key, value] : customSplitInfo) { + customSplitInfoObj[key] = value; + } + obj["customSplitInfo"] = customSplitInfoObj; + obj["extraFileInfo"] = + extraFileInfo == nullptr ? nullptr : folly::dynamic(*extraFileInfo); + + folly::dynamic serdeParametersObj = folly::dynamic::object; + for (const auto& [key, value] : serdeParameters) { + serdeParametersObj[key] = value; + } + obj["serdeParameters"] = serdeParametersObj; + + folly::dynamic infoColumnsObj = folly::dynamic::object; + for (const auto& [key, value] : infoColumns) { + infoColumnsObj[key] = value; + } + obj["infoColumns"] = infoColumnsObj; + + if (properties.has_value()) { + folly::dynamic propertiesObj = folly::dynamic::object; + propertiesObj["fileSize"] = properties->fileSize.has_value() + ? folly::dynamic(properties->fileSize.value()) + : nullptr; + propertiesObj["modificationTime"] = properties->modificationTime.has_value() + ? folly::dynamic(properties->modificationTime.value()) + : nullptr; + obj["properties"] = propertiesObj; + } + + if (rowIdProperties.has_value()) { + folly::dynamic rowIdObj = folly::dynamic::object; + rowIdObj["metadataVersion"] = rowIdProperties->metadataVersion; + rowIdObj["partitionId"] = rowIdProperties->partitionId; + rowIdObj["tableGuid"] = rowIdProperties->tableGuid; + obj["rowIdProperties"] = rowIdObj; + } + + return obj; +} + +// static +std::shared_ptr HiveConnectorSplit::create( + const folly::dynamic& obj) { + const auto connectorId = obj["connectorId"].asString(); + const auto splitWeight = obj["splitWeight"].asInt(); + const bool cacheable = obj["cacheable"].asBool(); + const auto filePath = obj["filePath"].asString(); + const auto fileFormat = + dwio::common::toFileFormat(obj["fileFormat"].asString()); + const auto start = static_cast(obj["start"].asInt()); + const auto length = static_cast(obj["length"].asInt()); + + std::unordered_map> partitionKeys; + for (const auto& [key, value] : obj["partitionKeys"].items()) { + partitionKeys[key.asString()] = value.isNull() + ? std::nullopt + : std::optional(value.asString()); + } + + const auto tableBucketNumber = obj["tableBucketNumber"].isNull() + ? std::nullopt + : std::optional(obj["tableBucketNumber"].asInt()); + + std::optional bucketConversion = std::nullopt; + if (obj.count("bucketConversion") && !obj["bucketConversion"].isNull()) { + const auto& bucketConversionObj = obj["bucketConversion"]; + std::vector> bucketColumnHandles; + for (const auto& bucketColumnHandleObj : + bucketConversionObj["bucketColumnHandles"]) { + bucketColumnHandles.push_back(std::const_pointer_cast( + ISerializable::deserialize(bucketColumnHandleObj))); + } + bucketConversion = HiveBucketConversion{ + .tableBucketCount = static_cast( + bucketConversionObj["tableBucketCount"].asInt()), + .partitionBucketCount = static_cast( + bucketConversionObj["partitionBucketCount"].asInt()), + .bucketColumnHandles = bucketColumnHandles}; + } + + std::unordered_map customSplitInfo; + for (const auto& [key, value] : obj["customSplitInfo"].items()) { + customSplitInfo[key.asString()] = value.asString(); + } + + std::shared_ptr extraFileInfo = obj["extraFileInfo"].isNull() + ? nullptr + : std::make_shared(obj["extraFileInfo"].asString()); + std::unordered_map serdeParameters; + for (const auto& [key, value] : obj["serdeParameters"].items()) { + serdeParameters[key.asString()] = value.asString(); + } + + std::unordered_map infoColumns; + for (const auto& [key, value] : obj["infoColumns"].items()) { + infoColumns[key.asString()] = value.asString(); + } + + std::optional properties = std::nullopt; + const auto& propertiesObj = obj.getDefault("properties", nullptr); + if (propertiesObj != nullptr) { + properties = FileProperties{ + .fileSize = propertiesObj["fileSize"].isNull() + ? std::nullopt + : std::optional(propertiesObj["fileSize"].asInt()), + .modificationTime = propertiesObj["modificationTime"].isNull() + ? std::nullopt + : std::optional(propertiesObj["modificationTime"].asInt())}; + } + + std::optional rowIdProperties = std::nullopt; + const auto& rowIdObj = obj.getDefault("rowIdProperties", nullptr); + if (rowIdObj != nullptr) { + rowIdProperties = RowIdProperties{ + .metadataVersion = rowIdObj["metadataVersion"].asInt(), + .partitionId = rowIdObj["partitionId"].asInt(), + .tableGuid = rowIdObj["tableGuid"].asString()}; + } + + return std::make_shared( + connectorId, + filePath, + fileFormat, + start, + length, + partitionKeys, + tableBucketNumber, + customSplitInfo, + extraFileInfo, + serdeParameters, + splitWeight, + cacheable, + infoColumns, + properties, + rowIdProperties, + bucketConversion); +} + +// static +void HiveConnectorSplit::registerSerDe() { + auto& registry = DeserializationRegistryForSharedPtr(); + registry.Register("HiveConnectorSplit", HiveConnectorSplit::create); +} +} // namespace facebook::velox::connector::lakehouse::common diff --git a/velox/connectors/lakehouse/common/HiveConnectorSplit.h b/velox/connectors/lakehouse/common/HiveConnectorSplit.h new file mode 100644 index 000000000000..dca2dbe31be6 --- /dev/null +++ b/velox/connectors/lakehouse/common/HiveConnectorSplit.h @@ -0,0 +1,255 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#pragma once + +#include +#include +#include "velox/connectors/Connector.h" +#include "velox/connectors/lakehouse/common/FileProperties.h" +#include "velox/connectors/lakehouse/common/TableHandle.h" +#include "velox/dwio/common/Options.h" + +namespace facebook::velox::connector::lakehouse::common { + +/// A bucket conversion that should happen on the split. This happens when we +/// increase the bucket count of a table, but the old partitions are still +/// generated using the old bucket count, so that multiple new buckets can exist +/// in the same file, and we need to apply extra filter when we read these files +/// to make sure we read the rows corresponding to the selected bucket number +/// only. +struct HiveBucketConversion { + int32_t tableBucketCount; + int32_t partitionBucketCount; + std::vector> bucketColumnHandles; +}; + +struct RowIdProperties { + int64_t metadataVersion; + int64_t partitionId; + std::string tableGuid; +}; + +struct HiveConnectorSplit : public connector::ConnectorSplit { + const std::string filePath; + dwio::common::FileFormat fileFormat; + const uint64_t start; + const uint64_t length; + + /// Mapping from partition keys to values. Values are specified as strings + /// formatted the same way as CAST(x as VARCHAR). Null values are specified as + /// std::nullopt. Date values must be formatted using ISO 8601 as YYYY-MM-DD. + /// All scalar types and date type are supported. + const std::unordered_map> + partitionKeys; + std::optional tableBucketNumber; + std::unordered_map customSplitInfo; + std::shared_ptr extraFileInfo; + // Parameters that are provided as the serialization options. + std::unordered_map serdeParameters; + + /// These represent columns like $file_size, $file_modified_time that are + /// associated with the HiveSplit. + std::unordered_map infoColumns; + + /// These represent file properties like file size that are used while opening + /// the file handle. + std::optional properties; + + std::optional rowIdProperties; + + std::optional bucketConversion; + + HiveConnectorSplit( + const std::string& connectorId, + const std::string& _filePath, + dwio::common::FileFormat _fileFormat, + uint64_t _start = 0, + uint64_t _length = std::numeric_limits::max(), + const std::unordered_map>& + _partitionKeys = {}, + std::optional _tableBucketNumber = std::nullopt, + const std::unordered_map& _customSplitInfo = {}, + const std::shared_ptr& _extraFileInfo = {}, + const std::unordered_map& _serdeParameters = {}, + int64_t splitWeight = 0, + bool cacheable = true, + const std::unordered_map& _infoColumns = {}, + std::optional _properties = std::nullopt, + std::optional _rowIdProperties = std::nullopt, + const std::optional& _bucketConversion = + std::nullopt) + : ConnectorSplit(connectorId, splitWeight, cacheable), + filePath(_filePath), + fileFormat(_fileFormat), + start(_start), + length(_length), + partitionKeys(_partitionKeys), + tableBucketNumber(_tableBucketNumber), + customSplitInfo(_customSplitInfo), + extraFileInfo(_extraFileInfo), + serdeParameters(_serdeParameters), + infoColumns(_infoColumns), + properties(_properties), + rowIdProperties(_rowIdProperties), + bucketConversion(_bucketConversion) {} + + ~HiveConnectorSplit() = default; + + uint64_t size() const override; + + std::string toString() const override; + + std::string getFileName() const; + + folly::dynamic serialize() const override; + + static std::shared_ptr create(const folly::dynamic& obj); + + static void registerSerDe(); +}; + +class HiveConnectorSplitBuilder { + public: + explicit HiveConnectorSplitBuilder(std::string filePath) + : filePath_{std::move(filePath)} { + infoColumns_["$path"] = filePath_; + } + + HiveConnectorSplitBuilder& start(uint64_t start) { + start_ = start; + return *this; + } + + HiveConnectorSplitBuilder& length(uint64_t length) { + length_ = length; + return *this; + } + + HiveConnectorSplitBuilder& splitWeight(int64_t splitWeight) { + splitWeight_ = splitWeight; + return *this; + } + + HiveConnectorSplitBuilder& cacheable(bool cacheable) { + cacheable_ = cacheable; + return *this; + } + + HiveConnectorSplitBuilder& fileFormat(dwio::common::FileFormat format) { + fileFormat_ = format; + return *this; + } + + HiveConnectorSplitBuilder& infoColumn( + const std::string& name, + const std::string& value) { + infoColumns_.emplace(std::move(name), std::move(value)); + return *this; + } + + HiveConnectorSplitBuilder& partitionKey( + std::string name, + std::optional value) { + partitionKeys_.emplace(std::move(name), std::move(value)); + return *this; + } + + HiveConnectorSplitBuilder& tableBucketNumber(int32_t bucket) { + tableBucketNumber_ = bucket; + infoColumns_["$bucket"] = std::to_string(bucket); + return *this; + } + + HiveConnectorSplitBuilder& bucketConversion( + const HiveBucketConversion& bucketConversion) { + bucketConversion_ = bucketConversion; + return *this; + } + + HiveConnectorSplitBuilder& customSplitInfo( + const std::unordered_map& customSplitInfo) { + customSplitInfo_ = customSplitInfo; + return *this; + } + + HiveConnectorSplitBuilder& extraFileInfo( + const std::shared_ptr& extraFileInfo) { + extraFileInfo_ = extraFileInfo; + return *this; + } + + HiveConnectorSplitBuilder& serdeParameters( + const std::unordered_map& serdeParameters) { + serdeParameters_ = serdeParameters; + return *this; + } + + HiveConnectorSplitBuilder& connectorId(const std::string& connectorId) { + connectorId_ = connectorId; + return *this; + } + + HiveConnectorSplitBuilder& fileProperties(FileProperties fileProperties) { + fileProperties_ = fileProperties; + return *this; + } + + HiveConnectorSplitBuilder& rowIdProperties( + const RowIdProperties& rowIdProperties) { + rowIdProperties_ = rowIdProperties; + return *this; + } + + std::shared_ptr build() const { + return std::make_shared( + connectorId_, + filePath_, + fileFormat_, + start_, + length_, + partitionKeys_, + tableBucketNumber_, + customSplitInfo_, + extraFileInfo_, + serdeParameters_, + splitWeight_, + cacheable_, + infoColumns_, + fileProperties_, + rowIdProperties_, + bucketConversion_); + } + + private: + const std::string filePath_; + dwio::common::FileFormat fileFormat_{dwio::common::FileFormat::DWRF}; + uint64_t start_{0}; + uint64_t length_{std::numeric_limits::max()}; + std::unordered_map> partitionKeys_; + std::optional tableBucketNumber_; + std::optional bucketConversion_; + std::unordered_map customSplitInfo_ = {}; + std::shared_ptr extraFileInfo_ = {}; + std::unordered_map serdeParameters_ = {}; + std::unordered_map infoColumns_ = {}; + std::string connectorId_; + int64_t splitWeight_{0}; + bool cacheable_{true}; + std::optional fileProperties_; + std::optional rowIdProperties_ = std::nullopt; +}; + +} // namespace facebook::velox::connector::lakehouse::common diff --git a/velox/connectors/lakehouse/common/HiveConnectorUtil.cpp b/velox/connectors/lakehouse/common/HiveConnectorUtil.cpp new file mode 100644 index 000000000000..e3f17cf16685 --- /dev/null +++ b/velox/connectors/lakehouse/common/HiveConnectorUtil.cpp @@ -0,0 +1,937 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "velox/connectors/lakehouse/common/HiveConnectorUtil.h" + +#include "velox/connectors/lakehouse/common/HiveConfig.h" +#include "velox/connectors/lakehouse/common/HiveConnectorSplit.h" +#include "velox/dwio/common/CachedBufferedInput.h" +#include "velox/dwio/common/DirectBufferedInput.h" +#include "velox/expression/Expr.h" +#include "velox/expression/ExprToSubfieldFilter.h" + +#include +#include +#include + +namespace facebook::velox::connector::lakehouse::common { +namespace { + +struct SubfieldSpec { + const velox::common::Subfield* subfield; + bool filterOnly; +}; + +template +void deduplicate(std::vector& values) { + std::sort(values.begin(), values.end()); + values.erase(std::unique(values.begin(), values.end()), values.end()); +} + +// Floating point map key subscripts are truncated toward 0 in Presto. For +// example given `a' as a map with floating point key, if user queries a[0.99], +// Presto coordinator will generate a required subfield a[0]; for a[-1.99] it +// will generate a[-1]; for anything larger than 9223372036854775807, it +// generates a[9223372036854775807]; for anything smaller than +// -9223372036854775808 it generates a[-9223372036854775808]. +template +std::unique_ptr makeFloatingPointMapKeyFilter( + const std::vector& subscripts) { + std::vector> filters; + for (auto subscript : subscripts) { + T lower = subscript; + T upper = subscript; + bool lowerUnbounded = subscript == std::numeric_limits::min(); + bool upperUnbounded = subscript == std::numeric_limits::max(); + bool lowerExclusive = false; + bool upperExclusive = false; + if (lower <= 0 && !lowerUnbounded) { + if (lower > subscript - 1) { + lower = subscript - 1; + } else { + lower = std::nextafter(lower, -std::numeric_limits::infinity()); + } + lowerExclusive = true; + } + if (upper >= 0 && !upperUnbounded) { + if (upper < subscript + 1) { + upper = subscript + 1; + } else { + upper = std::nextafter(upper, std::numeric_limits::infinity()); + } + upperExclusive = true; + } + if (lowerUnbounded && upperUnbounded) { + continue; + } + filters.push_back(std::make_unique>( + lower, + lowerUnbounded, + lowerExclusive, + upper, + upperUnbounded, + upperExclusive, + false)); + } + if (filters.size() == 1) { + return std::move(filters[0]); + } + return std::make_unique(std::move(filters), false); +} + +// Recursively add subfields to scan spec. +void addSubfields( + const Type& type, + std::vector& subfields, + int level, + memory::MemoryPool* pool, + velox::common::ScanSpec& spec) { + int newSize = 0; + for (int i = 0; i < subfields.size(); ++i) { + if (level < subfields[i].subfield->path().size()) { + subfields[newSize++] = subfields[i]; + } else if (!subfields[i].filterOnly) { + spec.addAllChildFields(type); + return; + } + } + subfields.resize(newSize); + switch (type.kind()) { + case TypeKind::ROW: { + folly::F14FastMap> required; + for (auto& subfield : subfields) { + auto* element = subfield.subfield->path()[level].get(); + auto* nestedField = + dynamic_cast(element); + VELOX_CHECK( + nestedField, + "Unsupported for row subfields pruning: {}", + element->toString()); + required[nestedField->name()].push_back(subfield); + } + auto& rowType = type.asRow(); + for (int i = 0; i < rowType.size(); ++i) { + auto& childName = rowType.nameOf(i); + auto& childType = rowType.childAt(i); + auto* child = spec.addField(childName, i); + auto it = required.find(childName); + if (it == required.end()) { + child->setConstantValue( + BaseVector::createNullConstant(childType, 1, pool)); + } else { + addSubfields(*childType, it->second, level + 1, pool, *child); + } + } + break; + } + case TypeKind::MAP: { + auto& keyType = type.childAt(0); + auto* keys = spec.addMapKeyFieldRecursively(*keyType); + addSubfields( + *type.childAt(1), + subfields, + level + 1, + pool, + *spec.addMapValueField()); + if (subfields.empty()) { + return; + } + bool stringKey = keyType->isVarchar() || keyType->isVarbinary(); + std::vector stringSubscripts; + std::vector longSubscripts; + for (auto& subfield : subfields) { + auto* element = subfield.subfield->path()[level].get(); + if (dynamic_cast(element)) { + return; + } + if (stringKey) { + auto* subscript = + dynamic_cast(element); + VELOX_CHECK( + subscript, + "Unsupported for string map pruning: {}", + element->toString()); + stringSubscripts.push_back(subscript->index()); + } else { + auto* subscript = + dynamic_cast(element); + VELOX_CHECK( + subscript, + "Unsupported for long map pruning: {}", + element->toString()); + longSubscripts.push_back(subscript->index()); + } + } + std::unique_ptr filter; + if (stringKey) { + deduplicate(stringSubscripts); + filter = std::make_unique(stringSubscripts, false); + spec.setFlatMapFeatureSelection(std::move(stringSubscripts)); + } else { + deduplicate(longSubscripts); + if (keyType->isReal()) { + filter = makeFloatingPointMapKeyFilter(longSubscripts); + } else if (keyType->isDouble()) { + filter = makeFloatingPointMapKeyFilter(longSubscripts); + } else { + filter = velox::common::createBigintValues(longSubscripts, false); + } + std::vector features; + for (auto num : longSubscripts) { + features.push_back(std::to_string(num)); + } + spec.setFlatMapFeatureSelection(std::move(features)); + } + keys->setFilter(std::move(filter)); + break; + } + case TypeKind::ARRAY: { + addSubfields( + *type.childAt(0), + subfields, + level + 1, + pool, + *spec.addArrayElementField()); + if (subfields.empty()) { + return; + } + constexpr long kMaxIndex = std::numeric_limits::max(); + long maxIndex = -1; + for (auto& subfield : subfields) { + auto* element = subfield.subfield->path()[level].get(); + if (dynamic_cast(element)) { + return; + } + auto* subscript = + dynamic_cast(element); + VELOX_CHECK( + subscript, + "Unsupported for array pruning: {}", + element->toString()); + VELOX_USER_CHECK_GT( + subscript->index(), + 0, + "Non-positive array subscript cannot be push down"); + maxIndex = std::max(maxIndex, std::min(kMaxIndex, subscript->index())); + } + spec.setMaxArrayElementsCount(maxIndex); + break; + } + default: + break; + } +} + +inline uint8_t parseDelimiter(const std::string& delim) { + for (char const& ch : delim) { + if (!std::isdigit(ch)) { + return delim[0]; + } + } + return stoi(delim); +} + +inline bool isSynthesizedColumn( + const std::string& name, + const std::unordered_map& infoColumns) { + return infoColumns.count(name) != 0; +} + +bool isSpecialColumn( + const std::string& name, + const std::optional& specialName) { + return specialName.has_value() && name == *specialName; +} + +} // namespace + +const std::string& getColumnName(const velox::common::Subfield& subfield) { + VELOX_CHECK_GT(subfield.path().size(), 0); + auto* field = dynamic_cast( + subfield.path()[0].get()); + VELOX_CHECK_NOT_NULL(field); + return field->name(); +} + +void checkColumnNameLowerCase(const TypePtr& type) { + switch (type->kind()) { + case TypeKind::ARRAY: + checkColumnNameLowerCase(type->asArray().elementType()); + break; + case TypeKind::MAP: { + checkColumnNameLowerCase(type->asMap().keyType()); + checkColumnNameLowerCase(type->asMap().valueType()); + + } break; + case TypeKind::ROW: { + for (const auto& outputName : type->asRow().names()) { + VELOX_CHECK( + !std::any_of(outputName.begin(), outputName.end(), isupper)); + } + for (auto& childType : type->asRow().children()) { + checkColumnNameLowerCase(childType); + } + } break; + default: + VLOG(1) << "No need to check type lowercase mode" << type->toString(); + } +} + +void checkColumnNameLowerCase( + const velox::common::SubfieldFilters& filters, + const std::unordered_map& infoColumns) { + for (const auto& filterIt : filters) { + const auto name = filterIt.first.toString(); + if (isSynthesizedColumn(name, infoColumns)) { + continue; + } + const auto& path = filterIt.first.path(); + + for (int i = 0; i < path.size(); ++i) { + auto* nestedField = + dynamic_cast(path[i].get()); + if (nestedField == nullptr) { + continue; + } + VELOX_CHECK(!std::any_of( + nestedField->name().begin(), nestedField->name().end(), isupper)); + } + } +} + +void checkColumnNameLowerCase(const core::TypedExprPtr& typeExpr) { + if (typeExpr == nullptr) { + return; + } + checkColumnNameLowerCase(typeExpr->type()); + for (auto& type : typeExpr->inputs()) { + checkColumnNameLowerCase(type); + } +} + +namespace { + +void processFieldSpec( + const RowTypePtr& dataColumns, + const TypePtr& outputType, + velox::common::ScanSpec& fieldSpec) { + fieldSpec.visit(*outputType, [](const Type& type, velox::common::ScanSpec& spec) { + if (type.isMap() && !spec.isConstant()) { + auto* keys = spec.childByName(velox::common::ScanSpec::kMapKeysFieldName); + VELOX_CHECK_NOT_NULL(keys); + if (keys->filter()) { + VELOX_CHECK(!keys->filter()->testNull()); + } else { + keys->setFilter(std::make_shared()); + } + } + }); + if (dataColumns) { + auto i = dataColumns->getChildIdxIfExists(fieldSpec.fieldName()); + if (i.has_value()) { + if (dataColumns->childAt(*i)->isMap() && outputType->isRow()) { + fieldSpec.setFlatMapAsStruct(true); + } + } + } +} + +} // namespace + +std::shared_ptr makeScanSpec( + const RowTypePtr& rowType, + const folly::F14FastMap>& + outputSubfields, + const velox::common::SubfieldFilters& filters, + const RowTypePtr& dataColumns, + const std::unordered_map& partitionKeys, + const std::unordered_map& infoColumns, + const SpecialColumnNames& specialColumns, + bool disableStatsBasedFilterReorder, + memory::MemoryPool* pool) { + auto spec = std::make_shared("root"); + folly::F14FastMap> + filterSubfields; + std::vector subfieldSpecs; + for (auto& [subfield, _] : filters) { + if (auto name = subfield.toString(); + !isSynthesizedColumn(name, infoColumns) && + partitionKeys.count(name) == 0) { + VELOX_CHECK(!isSpecialColumn(name, specialColumns.rowIndex)); + VELOX_CHECK(!isSpecialColumn(name, specialColumns.rowId)); + filterSubfields[getColumnName(subfield)].push_back(&subfield); + } + } + + // Process columns that will be projected out. + for (int i = 0; i < rowType->size(); ++i) { + auto& name = rowType->nameOf(i); + auto& type = rowType->childAt(i); + if (isSpecialColumn(name, specialColumns.rowIndex)) { + VELOX_CHECK(type->isBigint()); + auto* fieldSpec = spec->addField(name, i); + fieldSpec->setColumnType(velox::common::ScanSpec::ColumnType::kRowIndex); + continue; + } + if (isSpecialColumn(name, specialColumns.rowId)) { + VELOX_CHECK(type->isRow() && type->size() == 5); + auto& rowIdType = type->asRow(); + auto* fieldSpec = spec->addFieldRecursively(name, rowIdType, i); + fieldSpec->setColumnType(velox::common::ScanSpec::ColumnType::kComposite); + fieldSpec->childByName(rowIdType.nameOf(0)) + ->setColumnType(velox::common::ScanSpec::ColumnType::kRowIndex); + continue; + } + auto it = outputSubfields.find(name); + if (it == outputSubfields.end()) { + auto* fieldSpec = spec->addFieldRecursively(name, *type, i); + processFieldSpec(dataColumns, type, *fieldSpec); + filterSubfields.erase(name); + continue; + } + for (auto* subfield : it->second) { + subfieldSpecs.push_back({subfield, false}); + } + it = filterSubfields.find(name); + if (it != filterSubfields.end()) { + for (auto* subfield : it->second) { + subfieldSpecs.push_back({subfield, true}); + } + filterSubfields.erase(it); + } + auto* fieldSpec = spec->addField(name, i); + addSubfields(*type, subfieldSpecs, 1, pool, *fieldSpec); + processFieldSpec(dataColumns, type, *fieldSpec); + subfieldSpecs.clear(); + } + + // Now process the columns that will not be projected out. + if (!filterSubfields.empty()) { + VELOX_CHECK_NOT_NULL(dataColumns); + for (auto& [fieldName, subfields] : filterSubfields) { + for (auto* subfield : subfields) { + subfieldSpecs.push_back({subfield, true}); + } + auto& type = dataColumns->findChild(fieldName); + auto* fieldSpec = spec->getOrCreateChild(fieldName); + addSubfields(*type, subfieldSpecs, 1, pool, *fieldSpec); + processFieldSpec(dataColumns, type, *fieldSpec); + subfieldSpecs.clear(); + } + } + + for (auto& pair : filters) { + const auto name = pair.first.toString(); + // SelectiveColumnReader doesn't support constant columns with filters, + // hence, we can't have a filter for a $path or $bucket column. + // + // Unfortunately, Presto happens to specify a filter for $path, $file_size, + // $file_modified_time or $bucket column. This filter is redundant and needs + // to be removed. + // TODO Remove this check when Presto is fixed to not specify a filter + // on $path and $bucket column. + if (isSynthesizedColumn(name, infoColumns)) { + continue; + } + auto fieldSpec = spec->getOrCreateChild(pair.first); + VELOX_CHECK_NULL(spec->filter()); + fieldSpec->setFilter(pair.second); + } + + if (disableStatsBasedFilterReorder) { + spec->disableStatsBasedFilterReorder(); + } + return spec; +} + +std::unique_ptr parseSerdeParameters( + const std::unordered_map& serdeParameters, + const std::unordered_map& tableParameters) { + auto fieldIt = serdeParameters.find(dwio::common::SerDeOptions::kFieldDelim); + if (fieldIt == serdeParameters.end()) { + fieldIt = serdeParameters.find("serialization.format"); + } + auto collectionIt = + serdeParameters.find(dwio::common::SerDeOptions::kCollectionDelim); + if (collectionIt == serdeParameters.end()) { + // For collection delimiter, Hive 1.x, 2.x uses "colelction.delim", but + // Hive 3.x uses "collection.delim". + // See: https://issues.apache.org/jira/browse/HIVE-16922) + collectionIt = serdeParameters.find("colelction.delim"); + } + auto mapKeyIt = + serdeParameters.find(dwio::common::SerDeOptions::kMapKeyDelim); + + auto escapeCharIt = + serdeParameters.find(dwio::common::SerDeOptions::kEscapeChar); + + auto nullStringIt = tableParameters.find( + dwio::common::TableParameter::kSerializationNullFormat); + + if (fieldIt == serdeParameters.end() && + collectionIt == serdeParameters.end() && + mapKeyIt == serdeParameters.end() && + escapeCharIt == serdeParameters.end() && + nullStringIt == tableParameters.end()) { + return nullptr; + } + + uint8_t fieldDelim = '\1'; + uint8_t collectionDelim = '\2'; + uint8_t mapKeyDelim = '\3'; + if (fieldIt != serdeParameters.end()) { + fieldDelim = parseDelimiter(fieldIt->second); + } + if (collectionIt != serdeParameters.end()) { + collectionDelim = parseDelimiter(collectionIt->second); + } + if (mapKeyIt != serdeParameters.end()) { + mapKeyDelim = parseDelimiter(mapKeyIt->second); + } + + // If escape character is specified then we use it, unless it is empty - in + // which case we default to '\\'. + // If escape character is not specified (not in the map) we turn escaping off. + // Logic is based on apache hive java code: + // https://github.com/apache/hive/blob/3f6f940af3f60cc28834268e5d7f5612e3b13c30/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazySerDeParameters.java#L105-L108 + uint8_t escapeChar = '\\'; + const bool hasEscapeChar = (escapeCharIt != serdeParameters.end()); + if (hasEscapeChar) { + if (!escapeCharIt->second.empty()) { + // If delim is convertible to uint8_t then we use it as character code, + // otherwise we use the 1st character of the string. + escapeChar = folly::tryTo(escapeCharIt->second) + .value_or(escapeCharIt->second[0]); + } + } + + auto serDeOptions = hasEscapeChar + ? std::make_unique( + fieldDelim, collectionDelim, mapKeyDelim, escapeChar, true) + : std::make_unique( + fieldDelim, collectionDelim, mapKeyDelim); + if (nullStringIt != tableParameters.end()) { + serDeOptions->nullString = nullStringIt->second; + } + return serDeOptions; +} + +void configureReaderOptions( + const std::shared_ptr& hiveConfig, + const ConnectorQueryCtx* connectorQueryCtx, + const std::shared_ptr& hiveTableHandle, + const std::shared_ptr& hiveSplit, + dwio::common::ReaderOptions& readerOptions) { + configureReaderOptions( + hiveConfig, + connectorQueryCtx, + hiveTableHandle->dataColumns(), + hiveSplit, + hiveTableHandle->tableParameters(), + readerOptions); +} + +void configureReaderOptions( + const std::shared_ptr& hiveConfig, + const ConnectorQueryCtx* connectorQueryCtx, + const RowTypePtr& fileSchema, + const std::shared_ptr& hiveSplit, + const std::unordered_map& tableParameters, + dwio::common::ReaderOptions& readerOptions) { + auto sessionProperties = connectorQueryCtx->sessionProperties(); + readerOptions.setLoadQuantum(hiveConfig->loadQuantum(sessionProperties)); + readerOptions.setMaxCoalesceBytes( + hiveConfig->maxCoalescedBytes(sessionProperties)); + readerOptions.setMaxCoalesceDistance( + hiveConfig->maxCoalescedDistanceBytes(sessionProperties)); + readerOptions.setFileColumnNamesReadAsLowerCase( + hiveConfig->isFileColumnNamesReadAsLowerCase(sessionProperties)); + readerOptions.setAllowEmptyFile(true); + bool useColumnNamesForColumnMapping = false; + switch (hiveSplit->fileFormat) { + case dwio::common::FileFormat::DWRF: + case dwio::common::FileFormat::ORC: { + useColumnNamesForColumnMapping = + hiveConfig->isOrcUseColumnNames(sessionProperties); + break; + } + case dwio::common::FileFormat::PARQUET: { + useColumnNamesForColumnMapping = + hiveConfig->isParquetUseColumnNames(sessionProperties); + break; + } + default: + useColumnNamesForColumnMapping = false; + } + + readerOptions.setUseColumnNamesForColumnMapping( + useColumnNamesForColumnMapping); + readerOptions.setFileSchema(fileSchema); + readerOptions.setFooterEstimatedSize(hiveConfig->footerEstimatedSize()); + readerOptions.setFilePreloadThreshold(hiveConfig->filePreloadThreshold()); + readerOptions.setPrefetchRowGroups(hiveConfig->prefetchRowGroups()); + readerOptions.setNoCacheRetention(!hiveSplit->cacheable); + const auto& sessionTzName = connectorQueryCtx->sessionTimezone(); + if (!sessionTzName.empty()) { + const auto timezone = tz::locateZone(sessionTzName); + readerOptions.setSessionTimezone(timezone); + } + readerOptions.setAdjustTimestampToTimezone( + connectorQueryCtx->adjustTimestampToTimezone()); + readerOptions.setSelectiveNimbleReaderEnabled( + connectorQueryCtx->selectiveNimbleReaderEnabled()); + + if (readerOptions.fileFormat() != dwio::common::FileFormat::UNKNOWN) { + VELOX_CHECK( + readerOptions.fileFormat() == hiveSplit->fileFormat, + "HiveDataSource received splits of different formats: {} and {}", + dwio::common::toString(readerOptions.fileFormat()), + dwio::common::toString(hiveSplit->fileFormat)); + } else { + auto serDeOptions = + parseSerdeParameters(hiveSplit->serdeParameters, tableParameters); + if (serDeOptions) { + readerOptions.setSerDeOptions(*serDeOptions); + } + + readerOptions.setFileFormat(hiveSplit->fileFormat); +// readerOptions.setEnableRequestedTypeCheck( +// hiveConfig->isRequestedTypeCheckEnabled(sessionProperties)); + } +} + +void configureRowReaderOptions( + const std::unordered_map& tableParameters, + const std::shared_ptr& scanSpec, + std::shared_ptr metadataFilter, + const RowTypePtr& rowType, + const std::shared_ptr& hiveSplit, + const std::shared_ptr& hiveConfig, + const config::ConfigBase* sessionProperties, + dwio::common::RowReaderOptions& rowReaderOptions) { + auto skipRowsIt = + tableParameters.find(dwio::common::TableParameter::kSkipHeaderLineCount); + if (skipRowsIt != tableParameters.end()) { + rowReaderOptions.setSkipRows(folly::to(skipRowsIt->second)); + } + rowReaderOptions.setScanSpec(scanSpec); + rowReaderOptions.setMetadataFilter(std::move(metadataFilter)); + rowReaderOptions.setRequestedType(rowType); + rowReaderOptions.range(hiveSplit->start, hiveSplit->length); + if (hiveConfig && sessionProperties) { + rowReaderOptions.setTimestampPrecision(static_cast( + hiveConfig->readTimestampUnit(sessionProperties))); + } + rowReaderOptions.setSerdeParameters(hiveSplit->serdeParameters); +} + +namespace { + +bool applyPartitionFilter( + const TypePtr& type, + const std::string& partitionValue, + bool isPartitionDateDaysSinceEpoch, + const velox::common::Filter* filter, + bool asLocalTime) { + if (type->isDate()) { + int32_t result = 0; + // days_since_epoch partition values are integers in string format. Eg. + // Iceberg partition values. + if (isPartitionDateDaysSinceEpoch) { + result = folly::to(partitionValue); + } else { + result = DATE()->toDays(static_cast(partitionValue)); + } + return applyFilter(*filter, result); + } + + switch (type->kind()) { + case TypeKind::BIGINT: + case TypeKind::INTEGER: + case TypeKind::SMALLINT: + case TypeKind::TINYINT: { + return applyFilter(*filter, folly::to(partitionValue)); + } + case TypeKind::REAL: + case TypeKind::DOUBLE: { + return applyFilter(*filter, folly::to(partitionValue)); + } + case TypeKind::BOOLEAN: { + return applyFilter(*filter, folly::to(partitionValue)); + } + case TypeKind::TIMESTAMP: { + auto result = util::fromTimestampString( + StringView(partitionValue), util::TimestampParseMode::kPrestoCast); + VELOX_CHECK(!result.hasError()); + if (asLocalTime) { + result.value().toGMT(Timestamp::defaultTimezone()); + } + return applyFilter(*filter, result.value()); + } + case TypeKind::VARCHAR: { + return applyFilter(*filter, partitionValue); + } + default: + VELOX_FAIL( + "Bad type {} for partition value: {}", type->kind(), partitionValue); + } +} + +} // namespace + +bool testFilters( + const velox::common::ScanSpec* scanSpec, + const dwio::common::Reader* reader, + const std::string& filePath, + const std::unordered_map>& + partitionKeys, + const std::unordered_map& + partitionKeysHandle, + bool asLocalTime) { + const auto totalRows = reader->numberOfRows(); + const auto& fileTypeWithId = reader->typeWithId(); + const auto& rowType = reader->rowType(); + for (const auto& child : scanSpec->children()) { + if (child->filter()) { + const auto& name = child->fieldName(); + auto iter = partitionKeys.find(name); + // By design, the partition key columns for Iceberg tables are included in + // the data files to facilitate partition transform and partition + // evolution, so we need to test both cases. + if (!rowType->containsChild(name) || iter != partitionKeys.end()) { + if (iter != partitionKeys.end() && iter->second.has_value()) { + const auto handlesIter = partitionKeysHandle.find(name); + VELOX_CHECK(handlesIter != partitionKeysHandle.end()); + + // This is a non-null partition key + return applyPartitionFilter( + handlesIter->second->dataType(), + iter->second.value(), + handlesIter->second->isPartitionDateValueDaysSinceEpoch(), + child->filter(), + asLocalTime); + } + // Column is missing, most likely due to schema evolution. Or it's a + // partition key but the partition value is NULL. + if (child->filter()->isDeterministic() && + !child->filter()->testNull()) { + VLOG(1) << "Skipping " << filePath + << " because the filter testNull() failed for column " + << child->fieldName(); + return false; + } + } else { + const auto& typeWithId = fileTypeWithId->childByName(name); + const auto columnStats = reader->columnStatistics(typeWithId->id()); + if (columnStats != nullptr && + !testFilter( + child->filter(), + columnStats.get(), + totalRows.value(), + typeWithId->type())) { + VLOG(1) << "Skipping " << filePath + << " based on stats and filter for column " + << child->fieldName(); + return false; + } + } + } + } + + return true; +} + +std::unique_ptr createBufferedInput( + const FileHandle& fileHandle, + const dwio::common::ReaderOptions& readerOpts, + const ConnectorQueryCtx* connectorQueryCtx, + std::shared_ptr ioStats, + std::shared_ptr fsStats, + folly::Executor* executor) { + if (connectorQueryCtx->cache()) { + return std::make_unique( + fileHandle.file, + dwio::common::MetricsLog::voidLog(), + fileHandle.uuid, + connectorQueryCtx->cache(), + Connector::getTracker( + connectorQueryCtx->scanId(), readerOpts.loadQuantum()), + fileHandle.groupId, + ioStats, + std::move(fsStats), + executor, + readerOpts); + } + if (readerOpts.fileFormat() == dwio::common::FileFormat::NIMBLE) { + // Nimble streams (in case of single chunk) are compressed as whole and need + // to be fully fetched in order to do decompression, so there is no point to + // fetch them by quanta. Just use BufferedInput to fetch streams as whole + // to reduce memory footprint. + return std::make_unique( + fileHandle.file, + readerOpts.memoryPool(), + dwio::common::MetricsLog::voidLog(), + ioStats.get(), + fsStats.get()); + } + return std::make_unique( + fileHandle.file, + dwio::common::MetricsLog::voidLog(), + fileHandle.uuid, + Connector::getTracker( + connectorQueryCtx->scanId(), readerOpts.loadQuantum()), + fileHandle.groupId, + std::move(ioStats), + std::move(fsStats), + executor, + readerOpts); +} + +namespace { + +core::CallTypedExprPtr replaceInputs( + const core::CallTypedExpr* call, + std::vector&& inputs) { + return std::make_shared( + call->type(), std::move(inputs), call->name()); +} + +bool endWith(const std::string& str, const char* suffix) { + int len = strlen(suffix); + if (str.size() < len) { + return false; + } + for (int i = 0, j = str.size() - len; i < len; ++i, ++j) { + if (str[j] != suffix[i]) { + return false; + } + } + return true; +} + +bool isNotExpr( + const core::TypedExprPtr& expr, + const core::CallTypedExpr* call, + core::ExpressionEvaluator* evaluator) { + if (!endWith(call->name(), "not")) { + return false; + } + auto exprs = evaluator->compile(expr); + VELOX_CHECK_EQ(exprs->size(), 1); + auto& compiled = exprs->expr(0); + return compiled->vectorFunction() && + compiled->vectorFunction()->getCanonicalName() == + exec::FunctionCanonicalName::kNot; +} + +double getPrestoSampleRate( + const core::TypedExprPtr& expr, + const core::CallTypedExpr* call, + core::ExpressionEvaluator* evaluator) { + if (!endWith(call->name(), "lt")) { + return -1; + } + VELOX_CHECK_EQ(call->inputs().size(), 2); + auto exprs = evaluator->compile(expr); + VELOX_CHECK_EQ(exprs->size(), 1); + auto& lt = exprs->expr(0); + if (!(lt->vectorFunction() && + lt->vectorFunction()->getCanonicalName() == + exec::FunctionCanonicalName::kLt)) { + return -1; + } + auto& rand = lt->inputs()[0]; + if (!(rand->inputs().empty() && rand->vectorFunction() && + rand->vectorFunction()->getCanonicalName() == + exec::FunctionCanonicalName::kRand)) { + return -1; + } + auto* rate = + dynamic_cast(call->inputs()[1].get()); + if (!(rate && rate->type()->kind() == TypeKind::DOUBLE)) { + return -1; + } + return std::max(0.0, std::min(1.0, rate->value().value())); +} + +} // namespace + +core::TypedExprPtr extractFiltersFromRemainingFilter( + const core::TypedExprPtr& expr, + core::ExpressionEvaluator* evaluator, + bool negated, + velox::common::SubfieldFilters& filters, + double& sampleRate) { + auto* call = dynamic_cast(expr.get()); + if (call == nullptr) { + return expr; + } + velox::common::Filter* oldFilter = nullptr; + try { + velox::common::Subfield subfield; + if (auto filter = exec::ExprToSubfieldFilterParser::getInstance() + ->leafCallToSubfieldFilter( + *call, subfield, evaluator, negated)) { + if (auto it = filters.find(subfield); it != filters.end()) { + oldFilter = it->second.get(); + filter = filter->mergeWith(oldFilter); + } + filters.insert_or_assign(std::move(subfield), std::move(filter)); + return nullptr; + } + } catch (const VeloxException&) { + LOG(WARNING) << "Unexpected failure when extracting filter for: " + << expr->toString(); + if (oldFilter) { + LOG(WARNING) << "Merging with " << oldFilter->toString(); + } + } + + if (isNotExpr(expr, call, evaluator)) { + auto inner = extractFiltersFromRemainingFilter( + call->inputs()[0], evaluator, !negated, filters, sampleRate); + return inner ? replaceInputs(call, {inner}) : nullptr; + } + + if ((call->name() == "and" && !negated) || + (call->name() == "or" && negated)) { + auto lhs = extractFiltersFromRemainingFilter( + call->inputs()[0], evaluator, negated, filters, sampleRate); + auto rhs = extractFiltersFromRemainingFilter( + call->inputs()[1], evaluator, negated, filters, sampleRate); + if (!lhs) { + return rhs; + } + if (!rhs) { + return lhs; + } + return replaceInputs(call, {lhs, rhs}); + } + if (!negated) { + double rate = getPrestoSampleRate(expr, call, evaluator); + if (rate != -1) { + sampleRate *= rate; + return nullptr; + } + } + return expr; +} + +std::string makeUuid() { + return boost::lexical_cast(boost::uuids::random_generator()()); +} + +} // namespace facebook::velox::connector::lakehouse::common diff --git a/velox/connectors/lakehouse/common/HiveConnectorUtil.h b/velox/connectors/lakehouse/common/HiveConnectorUtil.h new file mode 100644 index 000000000000..18ff5bac4318 --- /dev/null +++ b/velox/connectors/lakehouse/common/HiveConnectorUtil.h @@ -0,0 +1,119 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#pragma once +#include +#include + +#include "velox/connectors/Connector.h" +#include "velox/connectors/lakehouse/common/FileHandle.h" +#include "velox/dwio/common/BufferedInput.h" +#include "velox/dwio/common/Reader.h" + +namespace facebook::velox::connector::lakehouse::common { + +class HiveColumnHandle; +class HiveTableHandle; +class HiveConfig; +struct HiveConnectorSplit; + +const std::string& getColumnName(const velox::common::Subfield& subfield); + +void checkColumnNameLowerCase(const std::shared_ptr& type); + +void checkColumnNameLowerCase( + const velox::common::SubfieldFilters& filters, + const std::unordered_map< + std::string, + std::shared_ptr>& infoColumns); + +void checkColumnNameLowerCase(const core::TypedExprPtr& typeExpr); + +struct SpecialColumnNames { + std::optional rowIndex; + std::optional rowId; +}; + +std::shared_ptr makeScanSpec( + const RowTypePtr& rowType, + const folly::F14FastMap>& + outputSubfields, + const velox::common::SubfieldFilters& filters, + const RowTypePtr& dataColumns, + const std::unordered_map< + std::string, + std::shared_ptr>& partitionKeys, + const std::unordered_map< + std::string, + std::shared_ptr>& infoColumns, + const SpecialColumnNames& specialColumns, + bool disableStatsBasedFilterReorder, + memory::MemoryPool* pool); + +void configureReaderOptions( + const std::shared_ptr& config, + const ConnectorQueryCtx* connectorQueryCtx, + const std::shared_ptr& hiveTableHandle, + const std::shared_ptr& hiveSplit, + dwio::common::ReaderOptions& readerOptions); + +void configureReaderOptions( + const std::shared_ptr& hiveConfig, + const ConnectorQueryCtx* connectorQueryCtx, + const RowTypePtr& fileSchema, + const std::shared_ptr& hiveSplit, + const std::unordered_map& tableParameters, + dwio::common::ReaderOptions& readerOptions); + +void configureRowReaderOptions( + const std::unordered_map& tableParameters, + const std::shared_ptr& scanSpec, + std::shared_ptr metadataFilter, + const RowTypePtr& rowType, + const std::shared_ptr& hiveSplit, + const std::shared_ptr& hiveConfig, + const config::ConfigBase* sessionProperties, + dwio::common::RowReaderOptions& rowReaderOptions); + +bool testFilters( + const velox::common::ScanSpec* scanSpec, + const dwio::common::Reader* reader, + const std::string& filePath, + const std::unordered_map>& + partitionKey, + const std::unordered_map< + std::string, + std::shared_ptr>& partitionKeysHandle, + bool asLocalTime); + +std::unique_ptr createBufferedInput( + const FileHandle& fileHandle, + const dwio::common::ReaderOptions& readerOpts, + const ConnectorQueryCtx* connectorQueryCtx, + std::shared_ptr ioStats, + std::shared_ptr fsStats, + folly::Executor* executor); + +core::TypedExprPtr extractFiltersFromRemainingFilter( + const core::TypedExprPtr& expr, + core::ExpressionEvaluator* evaluator, + bool negated, + velox::common::SubfieldFilters& filters, + double& sampleRate); + +std::string makeUuid(); + +} // namespace facebook::velox::connector::lakehouse::common diff --git a/velox/connectors/lakehouse/common/HiveDataSink.cpp b/velox/connectors/lakehouse/common/HiveDataSink.cpp new file mode 100644 index 000000000000..beff09e863a9 --- /dev/null +++ b/velox/connectors/lakehouse/common/HiveDataSink.cpp @@ -0,0 +1,1302 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "velox/connectors/lakehouse/common/HiveDataSink.h" + +#include "velox/common/base/Counters.h" +#include "velox/common/base/Fs.h" +#include "velox/common/base/StatsReporter.h" +#include "velox/common/testutil/TestValue.h" +#include "velox/connectors/lakehouse/common/HiveConfig.h" +#include "velox/connectors/lakehouse/common/HiveConnectorUtil.h" +#include "velox/connectors/lakehouse/common/HivePartitionFunction.h" +#include "velox/connectors/lakehouse/common/TableHandle.h" +#include "velox/core/ITypedExpr.h" +#include "velox/dwio/common/Options.h" +#include "velox/dwio/common/SortingWriter.h" +#include "velox/exec/OperatorUtils.h" +#include "velox/exec/SortBuffer.h" + +using facebook::velox::common::testutil::TestValue; + +namespace facebook::velox::connector::lakehouse::common { +namespace { +#define WRITER_NON_RECLAIMABLE_SECTION_GUARD(index) \ + memory::NonReclaimableSectionGuard nonReclaimableGuard( \ + writerInfo_[(index)]->nonReclaimableSectionHolder.get()) + +// Returns the type of non-partition data columns. +RowTypePtr getNonPartitionTypes( + const std::vector& dataCols, + const RowTypePtr& inputType) { + std::vector childNames; + std::vector childTypes; + const auto& dataSize = dataCols.size(); + childNames.reserve(dataSize); + childTypes.reserve(dataSize); + for (int dataCol : dataCols) { + childNames.push_back(inputType->nameOf(dataCol)); + childTypes.push_back(inputType->childAt(dataCol)); + } + + return ROW(std::move(childNames), std::move(childTypes)); +} + +// Filters out partition columns if there is any. +RowVectorPtr makeDataInput( + const std::vector& dataCols, + const RowVectorPtr& input) { + std::vector childVectors; + childVectors.reserve(dataCols.size()); + for (int dataCol : dataCols) { + childVectors.push_back(input->childAt(dataCol)); + } + + return std::make_shared( + input->pool(), + getNonPartitionTypes(dataCols, asRowType(input->type())), + input->nulls(), + input->size(), + std::move(childVectors), + input->getNullCount()); +} + +// Returns a subset of column indices corresponding to partition keys. +std::vector getPartitionChannels( + const std::shared_ptr& insertTableHandle) { + std::vector channels; + + for (column_index_t i = 0; i < insertTableHandle->inputColumns().size(); + i++) { + if (insertTableHandle->inputColumns()[i]->isPartitionKey()) { + channels.push_back(i); + } + } + + return channels; +} + +// Returns the column indices of non-partition data columns. +std::vector getNonPartitionChannels( + const std::shared_ptr& insertTableHandle) { + std::vector dataChannels; + + for (column_index_t i = 0; i < insertTableHandle->inputColumns().size(); + i++) { + if (!insertTableHandle->inputColumns()[i]->isPartitionKey()) { + dataChannels.push_back(i); + } + } + + return dataChannels; +} + +std::string makePartitionDirectory( + const std::string& tableDirectory, + const std::optional& partitionSubdirectory) { + if (partitionSubdirectory.has_value()) { + return fs::path(tableDirectory) / partitionSubdirectory.value(); + } + return tableDirectory; +} + +std::unordered_map tableTypeNames() { + return { + {LocationHandle::TableType::kNew, "kNew"}, + {LocationHandle::TableType::kExisting, "kExisting"}, + }; +} + +template +std::unordered_map invertMap(const std::unordered_map& mapping) { + std::unordered_map inverted; + for (const auto& [key, value] : mapping) { + inverted.emplace(value, key); + } + return inverted; +} + +std::unique_ptr createBucketFunction( + const HiveBucketProperty& bucketProperty, + const RowTypePtr& inputType) { + const auto& bucketedBy = bucketProperty.bucketedBy(); + const auto& bucketedTypes = bucketProperty.bucketedTypes(); + std::vector bucketedByChannels; + bucketedByChannels.reserve(bucketedBy.size()); + for (int32_t i = 0; i < bucketedBy.size(); ++i) { + const auto& bucketColumn = bucketedBy[i]; + const auto& bucketType = bucketedTypes[i]; + const auto inputChannel = inputType->getChildIdx(bucketColumn); + if (FOLLY_UNLIKELY( + !inputType->childAt(inputChannel)->equivalent(*bucketType))) { + VELOX_USER_FAIL( + "Input column {} type {} doesn't match bucket type {}", + inputType->nameOf(inputChannel), + inputType->childAt(inputChannel)->toString(), + bucketType->toString()); + } + bucketedByChannels.push_back(inputChannel); + } + return std::make_unique( + bucketProperty.bucketCount(), bucketedByChannels); +} + +std::string computeBucketedFileName( + const std::string& queryId, + uint32_t maxBucketCount, + uint32_t bucket) { + const uint32_t kMaxBucketCountPadding = + std::to_string(maxBucketCount - 1).size(); + const std::string bucketValueStr = std::to_string(bucket); + return fmt::format( + "0{:0>{}}_0_{}", bucketValueStr, kMaxBucketCountPadding, queryId); +} + +std::shared_ptr createSinkPool( + const std::shared_ptr& writerPool) { + return writerPool->addLeafChild(fmt::format("{}.sink", writerPool->name())); +} + +std::shared_ptr createSortPool( + const std::shared_ptr& writerPool) { + return writerPool->addLeafChild(fmt::format("{}.sort", writerPool->name())); +} + +uint64_t getFinishTimeSliceLimitMsFromHiveConfig( + const std::shared_ptr& config, + const config::ConfigBase* sessions) { + const uint64_t flushTimeSliceLimitMsFromConfig = + config->sortWriterFinishTimeSliceLimitMs(sessions); + // NOTE: if the flush time slice limit is set to 0, then we treat it as no + // limit. + return flushTimeSliceLimitMsFromConfig == 0 + ? std::numeric_limits::max() + : flushTimeSliceLimitMsFromConfig; +} + +FOLLY_ALWAYS_INLINE int32_t +getBucketCount(const HiveBucketProperty* bucketProperty) { + return bucketProperty == nullptr ? 0 : bucketProperty->bucketCount(); +} +} // namespace + +const HiveWriterId& HiveWriterId::unpartitionedId() { + static const HiveWriterId writerId{0}; + return writerId; +} + +std::string HiveWriterId::toString() const { + if (partitionId.has_value() && bucketId.has_value()) { + return fmt::format("part[{}.{}]", partitionId.value(), bucketId.value()); + } + + if (partitionId.has_value() && !bucketId.has_value()) { + return fmt::format("part[{}]", partitionId.value()); + } + + // This WriterId is used to add an identifier in the MemoryPools. This could + // indicate unpart, but the bucket number needs to be disambiguated. So + // creating a new label using bucket. + if (!partitionId.has_value() && bucketId.has_value()) { + return fmt::format("bucket[{}]", bucketId.value()); + } + + return "unpart"; +} + +const std::string LocationHandle::tableTypeName( + LocationHandle::TableType type) { + static const auto tableTypes = tableTypeNames(); + return tableTypes.at(type); +} + +LocationHandle::TableType LocationHandle::tableTypeFromName( + const std::string& name) { + static const auto nameTableTypes = invertMap(tableTypeNames()); + return nameTableTypes.at(name); +} + +HiveSortingColumn::HiveSortingColumn( + const std::string& sortColumn, + const core::SortOrder& sortOrder) + : sortColumn_(sortColumn), sortOrder_(sortOrder) { + VELOX_USER_CHECK(!sortColumn_.empty(), "hive sort column must be set"); + + if (FOLLY_UNLIKELY( + (sortOrder_.isAscending() && !sortOrder_.isNullsFirst()) || + (!sortOrder_.isAscending() && sortOrder_.isNullsFirst()))) { + VELOX_USER_FAIL("Bad hive sort order: {}", toString()); + } +} + +folly::dynamic HiveSortingColumn::serialize() const { + folly::dynamic obj = folly::dynamic::object; + obj["name"] = "HiveSortingColumn"; + obj["columnName"] = sortColumn_; + obj["sortOrder"] = sortOrder_.serialize(); + return obj; +} + +std::shared_ptr HiveSortingColumn::deserialize( + const folly::dynamic& obj, + void* context) { + const std::string columnName = obj["columnName"].asString(); + const auto sortOrder = core::SortOrder::deserialize(obj["sortOrder"]); + return std::make_shared(columnName, sortOrder); +} + +std::string HiveSortingColumn::toString() const { + return fmt::format( + "[COLUMN[{}] ORDER[{}]]", sortColumn_, sortOrder_.toString()); +} + +void HiveSortingColumn::registerSerDe() { + auto& registry = DeserializationWithContextRegistryForSharedPtr(); + registry.Register("HiveSortingColumn", HiveSortingColumn::deserialize); +} + +HiveBucketProperty::HiveBucketProperty( + Kind kind, + int32_t bucketCount, + const std::vector& bucketedBy, + const std::vector& bucketTypes, + const std::vector>& sortedBy) + : kind_(kind), + bucketCount_(bucketCount), + bucketedBy_(bucketedBy), + bucketTypes_(bucketTypes), + sortedBy_(sortedBy) { + validate(); +} + +void HiveBucketProperty::validate() const { + VELOX_USER_CHECK_GT(bucketCount_, 0, "Hive bucket count can't be zero"); + VELOX_USER_CHECK(!bucketedBy_.empty(), "Hive bucket columns must be set"); + VELOX_USER_CHECK_EQ( + bucketedBy_.size(), + bucketTypes_.size(), + "The number of hive bucket columns and types do not match {}", + toString()); +} + +std::string HiveBucketProperty::kindString(Kind kind) { + switch (kind) { + case Kind::kHiveCompatible: + return "HIVE_COMPATIBLE"; + case Kind::kPrestoNative: + return "PRESTO_NATIVE"; + default: + return fmt::format("UNKNOWN {}", static_cast(kind)); + } +} + +folly::dynamic HiveBucketProperty::serialize() const { + folly::dynamic obj = folly::dynamic::object; + obj["name"] = "HiveBucketProperty"; + obj["kind"] = static_cast(kind_); + obj["bucketCount"] = bucketCount_; + obj["bucketedBy"] = ISerializable::serialize(bucketedBy_); + obj["bucketedTypes"] = ISerializable::serialize(bucketTypes_); + obj["sortedBy"] = ISerializable::serialize(sortedBy_); + return obj; +} + +std::shared_ptr HiveBucketProperty::deserialize( + const folly::dynamic& obj, + void* context) { + const Kind kind = static_cast(obj["kind"].asInt()); + const int32_t bucketCount = obj["bucketCount"].asInt(); + const auto buckectedBy = + ISerializable::deserialize>(obj["bucketedBy"]); + const auto bucketedTypes = ISerializable::deserialize>( + obj["bucketedTypes"], context); + const auto sortedBy = + ISerializable::deserialize>( + obj["sortedBy"], context); + return std::make_shared( + kind, bucketCount, buckectedBy, bucketedTypes, sortedBy); +} + +void HiveBucketProperty::registerSerDe() { + auto& registry = DeserializationWithContextRegistryForSharedPtr(); + registry.Register("HiveBucketProperty", HiveBucketProperty::deserialize); +} + +std::string HiveBucketProperty::toString() const { + std::stringstream out; + out << "\nHiveBucketProperty[<" << kind_ << " " << bucketCount_ << ">\n"; + out << "\tBucket Columns:\n"; + for (const auto& column : bucketedBy_) { + out << "\t\t" << column << "\n"; + } + out << "\tBucket Types:\n"; + for (const auto& type : bucketTypes_) { + out << "\t\t" << type->toString() << "\n"; + } + if (!sortedBy_.empty()) { + out << "\tSortedBy Columns:\n"; + for (const auto& sortColum : sortedBy_) { + out << "\t\t" << sortColum->toString() << "\n"; + } + } + out << "]\n"; + return out.str(); +} + +HiveDataSink::HiveDataSink( + RowTypePtr inputType, + std::shared_ptr insertTableHandle, + const ConnectorQueryCtx* connectorQueryCtx, + CommitStrategy commitStrategy, + const std::shared_ptr& hiveConfig) + : HiveDataSink( + inputType, + insertTableHandle, + connectorQueryCtx, + commitStrategy, + hiveConfig, + getBucketCount(insertTableHandle->bucketProperty()), + getBucketCount(insertTableHandle->bucketProperty()) > 0 + ? createBucketFunction( + *insertTableHandle->bucketProperty(), + inputType) + : nullptr, + getNonPartitionChannels(insertTableHandle)) {} + +HiveDataSink::HiveDataSink( + RowTypePtr inputType, + std::shared_ptr insertTableHandle, + const ConnectorQueryCtx* connectorQueryCtx, + CommitStrategy commitStrategy, + const std::shared_ptr& hiveConfig, + uint32_t bucketCount, + std::unique_ptr bucketFunction, + const std::vector& dataChannels) + : inputType_(std::move(inputType)), + insertTableHandle_(std::move(insertTableHandle)), + connectorQueryCtx_(connectorQueryCtx), + commitStrategy_(commitStrategy), + hiveConfig_(hiveConfig), + updateMode_(getUpdateMode()), + maxOpenWriters_(hiveConfig_->maxPartitionsPerWriters( + connectorQueryCtx->sessionProperties())), + partitionChannels_(getPartitionChannels(insertTableHandle_)), + partitionIdGenerator_( + !partitionChannels_.empty() + ? std::make_unique( + inputType_, + partitionChannels_, + maxOpenWriters_, + connectorQueryCtx_->memoryPool(), + hiveConfig_->isPartitionPathAsLowerCase( + connectorQueryCtx->sessionProperties())) + : nullptr), + dataChannels_(dataChannels), + bucketCount_(static_cast(bucketCount)), + bucketFunction_(std::move(bucketFunction)), + writerFactory_( + dwio::common::getWriterFactory(insertTableHandle_->storageFormat())), + spillConfig_(connectorQueryCtx->spillConfig()), + sortWriterFinishTimeSliceLimitMs_(getFinishTimeSliceLimitMsFromHiveConfig( + hiveConfig_, + connectorQueryCtx->sessionProperties())), + fileNameGenerator_(insertTableHandle_->fileNameGenerator()) { + if (isBucketed()) { + VELOX_USER_CHECK_LT( + bucketCount_, + hiveConfig_->maxBucketCount(connectorQueryCtx->sessionProperties()), + "bucketCount exceeds the limit"); + } + VELOX_USER_CHECK( + (commitStrategy_ == CommitStrategy::kNoCommit) || + (commitStrategy_ == CommitStrategy::kTaskCommit), + "Unsupported commit strategy: {}", + commitStrategyToString(commitStrategy_)); + + if (insertTableHandle_->ensureFiles()) { + VELOX_CHECK( + !isPartitioned() && !isBucketed(), + "ensureFiles is not supported with bucketing or partition keys in the data"); + ensureWriter(HiveWriterId::unpartitionedId()); + } + + if (!isBucketed()) { + return; + } + const auto& sortedProperty = insertTableHandle_->bucketProperty()->sortedBy(); + if (!sortedProperty.empty()) { + sortColumnIndices_.reserve(sortedProperty.size()); + sortCompareFlags_.reserve(sortedProperty.size()); + for (int i = 0; i < sortedProperty.size(); ++i) { + auto columnIndex = + getNonPartitionTypes(dataChannels_, inputType_) + ->getChildIdxIfExists(sortedProperty.at(i)->sortColumn()); + if (columnIndex.has_value()) { + sortColumnIndices_.push_back(columnIndex.value()); + sortCompareFlags_.push_back( + {sortedProperty.at(i)->sortOrder().isNullsFirst(), + sortedProperty.at(i)->sortOrder().isAscending(), + false, + CompareFlags::NullHandlingMode::kNullAsValue}); + } + } + } +} + +bool HiveDataSink::canReclaim() const { + // Currently, we only support memory reclaim on dwrf file writer. + return (spillConfig_ != nullptr) && + (insertTableHandle_->storageFormat() == dwio::common::FileFormat::DWRF); +} + +void HiveDataSink::appendData(RowVectorPtr input) { + checkRunning(); + + // Lazy load all the input columns. + input->loadedVector(); + + // Write to unpartitioned (and unbucketed) table. + if (!isPartitioned() && !isBucketed()) { + const auto index = ensureWriter(HiveWriterId::unpartitionedId()); + write(index, input); + return; + } + + // Compute partition and bucket numbers. + computePartitionAndBucketIds(input); + + splitInputRowsAndEnsureWriters(input); + + // All inputs belong to a single non-bucketed partition. The partition id + // must be zero. + if (!isBucketed() && partitionIdGenerator_->numPartitions() == 1) { + const auto index = ensureWriter(HiveWriterId{0}); + write(index, input); + return; + } + + for (auto index = 0; index < writers_.size(); ++index) { + const vector_size_t partitionSize = partitionSizes_[index]; + if (partitionSize == 0) { + continue; + } + + RowVectorPtr writerInput = partitionSize == input->size() + ? input + : exec::wrap(partitionSize, partitionRows_[index], input); + write(index, writerInput); + } +} + +void HiveDataSink::write(size_t index, RowVectorPtr input) { + WRITER_NON_RECLAIMABLE_SECTION_GUARD(index); + auto dataInput = makeDataInput(dataChannels_, input); + + writers_[index]->write(dataInput); + writerInfo_[index]->inputSizeInBytes += dataInput->estimateFlatSize(); + writerInfo_[index]->numWrittenRows += dataInput->size(); +} + +std::string HiveDataSink::stateString(State state) { + switch (state) { + case State::kRunning: + return "RUNNING"; + case State::kFinishing: + return "FLUSHING"; + case State::kClosed: + return "CLOSED"; + case State::kAborted: + return "ABORTED"; + default: + VELOX_UNREACHABLE("BAD STATE: {}", static_cast(state)); + } +} + +void HiveDataSink::computePartitionAndBucketIds(const RowVectorPtr& input) { + VELOX_CHECK(isPartitioned() || isBucketed()); + if (isPartitioned()) { + if (!hiveConfig_->allowNullPartitionKeys( + connectorQueryCtx_->sessionProperties())) { + // Check that there are no nulls in the partition keys. + for (auto& partitionIdx : partitionChannels_) { + auto col = input->childAt(partitionIdx); + if (col->mayHaveNulls()) { + for (auto i = 0; i < col->size(); ++i) { + VELOX_USER_CHECK( + !col->isNullAt(i), + "Partition key must not be null: {}", + input->type()->asRow().nameOf(partitionIdx)); + } + } + } + } + partitionIdGenerator_->run(input, partitionIds_); + } + + if (isBucketed()) { + bucketFunction_->partition(*input, bucketIds_); + } +} + +DataSink::Stats HiveDataSink::stats() const { + Stats stats; + if (state_ == State::kAborted) { + return stats; + } + + int64_t numWrittenBytes{0}; + int64_t writeIOTimeUs{0}; + for (const auto& ioStats : ioStats_) { + numWrittenBytes += ioStats->rawBytesWritten(); + writeIOTimeUs += ioStats->writeIOTimeUs(); + } + stats.numWrittenBytes = numWrittenBytes; + stats.writeIOTimeUs = writeIOTimeUs; + + if (state_ != State::kClosed) { + return stats; + } + + stats.numWrittenFiles = writers_.size(); + for (int i = 0; i < writerInfo_.size(); ++i) { + const auto& info = writerInfo_.at(i); + VELOX_CHECK_NOT_NULL(info); + const auto spillStats = info->spillStats->rlock(); + if (!spillStats->empty()) { + stats.spillStats += *spillStats; + } + } + return stats; +} + +std::shared_ptr HiveDataSink::createWriterPool( + const HiveWriterId& writerId) { + auto* connectorPool = connectorQueryCtx_->connectorMemoryPool(); + return connectorPool->addAggregateChild( + fmt::format("{}.{}", connectorPool->name(), writerId.toString())); +} + +void HiveDataSink::setMemoryReclaimers( + HiveWriterInfo* writerInfo, + io::IoStatistics* ioStats) { + auto* connectorPool = connectorQueryCtx_->connectorMemoryPool(); + if (connectorPool->reclaimer() == nullptr) { + return; + } + writerInfo->writerPool->setReclaimer( + WriterReclaimer::create(this, writerInfo, ioStats)); + writerInfo->sinkPool->setReclaimer(exec::MemoryReclaimer::create()); + // NOTE: we set the memory reclaimer for sort pool when we construct the sort + // writer. +} + +void HiveDataSink::setState(State newState) { + checkStateTransition(state_, newState); + state_ = newState; +} + +/// Validates the state transition from 'oldState' to 'newState'. +void HiveDataSink::checkStateTransition(State oldState, State newState) { + switch (oldState) { + case State::kRunning: + if (newState == State::kAborted || newState == State::kFinishing) { + return; + } + break; + case State::kFinishing: + if (newState == State::kAborted || newState == State::kClosed || + // The finishing state is reentry state if we yield in the middle of + // finish processing if a single run takes too long. + newState == State::kFinishing) { + return; + } + [[fallthrough]]; + case State::kAborted: + case State::kClosed: + default: + break; + } + VELOX_FAIL("Unexpected state transition from {} to {}", oldState, newState); +} + +bool HiveDataSink::finish() { + // Flush is reentry state. + setState(State::kFinishing); + + // As for now, only sorted writer needs flush buffered data. For non-sorted + // writer, data is directly written to the underlying file writer. + if (!sortWrite()) { + return true; + } + + // TODO: we might refactor to move the data sorting logic into hive data sink. + const uint64_t startTimeMs = getCurrentTimeMs(); + for (auto i = 0; i < writers_.size(); ++i) { + WRITER_NON_RECLAIMABLE_SECTION_GUARD(i); + if (!writers_[i]->finish()) { + return false; + } + if (getCurrentTimeMs() - startTimeMs > sortWriterFinishTimeSliceLimitMs_) { + return false; + } + } + return true; +} + +std::vector HiveDataSink::close() { + setState(State::kClosed); + closeInternal(); + return commitMessage(); +} + +std::vector HiveDataSink::commitMessage() const { + std::vector partitionUpdates; + partitionUpdates.reserve(writerInfo_.size()); + for (int i = 0; i < writerInfo_.size(); ++i) { + const auto& info = writerInfo_.at(i); + VELOX_CHECK_NOT_NULL(info); + // clang-format off + auto partitionUpdateJson = folly::toJson( + folly::dynamic::object + ("name", info->writerParameters.partitionName().value_or("")) + ("updateMode", + HiveWriterParameters::updateModeToString( + info->writerParameters.updateMode())) + ("writePath", info->writerParameters.writeDirectory()) + ("targetPath", info->writerParameters.targetDirectory()) + ("fileWriteInfos", folly::dynamic::array( + folly::dynamic::object + ("writeFileName", info->writerParameters.writeFileName()) + ("targetFileName", info->writerParameters.targetFileName()) + ("fileSize", ioStats_.at(i)->rawBytesWritten()))) + ("rowCount", info->numWrittenRows) + ("inMemoryDataSizeInBytes", info->inputSizeInBytes) + ("onDiskDataSizeInBytes", ioStats_.at(i)->rawBytesWritten()) + ("containsNumberedFileNames", true)); + // clang-format on + partitionUpdates.push_back(partitionUpdateJson); + } + return partitionUpdates; +} + +void HiveDataSink::abort() { + setState(State::kAborted); + closeInternal(); +} + +void HiveDataSink::closeInternal() { + VELOX_CHECK_NE(state_, State::kRunning); + VELOX_CHECK_NE(state_, State::kFinishing); + + TestValue::adjust( + "facebook::velox::connector::lakehouse::common::HiveDataSink::closeInternal", this); + + if (state_ == State::kClosed) { + for (int i = 0; i < writers_.size(); ++i) { + WRITER_NON_RECLAIMABLE_SECTION_GUARD(i); + writers_[i]->close(); + } + } else { + for (int i = 0; i < writers_.size(); ++i) { + WRITER_NON_RECLAIMABLE_SECTION_GUARD(i); + writers_[i]->abort(); + } + } +} + +uint32_t HiveDataSink::ensureWriter(const HiveWriterId& id) { + auto it = writerIndexMap_.find(id); + if (it != writerIndexMap_.end()) { + return it->second; + } + return appendWriter(id); +} + +uint32_t HiveDataSink::appendWriter(const HiveWriterId& id) { + // Check max open writers. + VELOX_USER_CHECK_LE( + writers_.size(), maxOpenWriters_, "Exceeded open writer limit"); + VELOX_CHECK_EQ(writers_.size(), writerInfo_.size()); + VELOX_CHECK_EQ(writerIndexMap_.size(), writerInfo_.size()); + + std::optional partitionName = getPartitionName(id); + + // Without explicitly setting flush policy, the default memory based flush + // policy is used. + auto writerParameters = getWriterParameters(partitionName, id.bucketId); + const auto writePath = fs::path(writerParameters.writeDirectory()) / + writerParameters.writeFileName(); + auto writerPool = createWriterPool(id); + auto sinkPool = createSinkPool(writerPool); + std::shared_ptr sortPool{nullptr}; + if (sortWrite()) { + sortPool = createSortPool(writerPool); + } + writerInfo_.emplace_back(std::make_shared( + std::move(writerParameters), + std::move(writerPool), + std::move(sinkPool), + std::move(sortPool))); + ioStats_.emplace_back(std::make_shared()); + setMemoryReclaimers(writerInfo_.back().get(), ioStats_.back().get()); + + // Take the writer options provided by the user as a starting point, or + // allocate a new one. + auto options = insertTableHandle_->writerOptions(); + if (!options) { + options = writerFactory_->createWriterOptions(); + } + + const auto* connectorSessionProperties = + connectorQueryCtx_->sessionProperties(); + + // Only overwrite options in case they were not already provided. + if (options->schema == nullptr) { + options->schema = getNonPartitionTypes(dataChannels_, inputType_); + } + + if (options->memoryPool == nullptr) { + options->memoryPool = writerInfo_.back()->writerPool.get(); + } + + if (!options->compressionKind) { + options->compressionKind = insertTableHandle_->compressionKind(); + } + + if (options->spillConfig == nullptr && canReclaim()) { + options->spillConfig = spillConfig_; + } + + if (options->nonReclaimableSection == nullptr) { + options->nonReclaimableSection = + writerInfo_.back()->nonReclaimableSectionHolder.get(); + } + + if (options->memoryReclaimerFactory == nullptr || + options->memoryReclaimerFactory() == nullptr) { + options->memoryReclaimerFactory = []() { + return exec::MemoryReclaimer::create(); + }; + } + + if (options->serdeParameters.empty()) { + options->serdeParameters = std::map( + insertTableHandle_->serdeParameters().begin(), + insertTableHandle_->serdeParameters().end()); + } + + options->sessionTimezoneName = connectorQueryCtx_->sessionTimezone(); + options->adjustTimestampToTimezone = + connectorQueryCtx_->adjustTimestampToTimezone(); + options->processConfigs(*hiveConfig_->config(), *connectorSessionProperties); + + // Prevents the memory allocation during the writer creation. + WRITER_NON_RECLAIMABLE_SECTION_GUARD(writerInfo_.size() - 1); + auto writer = writerFactory_->createWriter( + dwio::common::FileSink::create( + writePath, + { + .bufferWrite = false, + .connectorProperties = hiveConfig_->config(), + .fileCreateConfig = hiveConfig_->writeFileCreateConfig(), + .pool = writerInfo_.back()->sinkPool.get(), + .metricLogger = dwio::common::MetricsLog::voidLog(), + .stats = ioStats_.back().get(), + }), + options); + writer = maybeCreateBucketSortWriter(std::move(writer)); + writers_.emplace_back(std::move(writer)); + + extendBuffersForPartitionedTables(); + + writerIndexMap_.emplace(id, writers_.size() - 1); + return writerIndexMap_[id]; +} + +std::optional HiveDataSink::getPartitionName( + const HiveWriterId& id) const { + std::optional partitionName; + if (isPartitioned()) { + partitionName = + partitionIdGenerator_->partitionName(id.partitionId.value()); + } + return partitionName; +} + +std::unique_ptr +HiveDataSink::maybeCreateBucketSortWriter( + std::unique_ptr writer) { + if (!sortWrite()) { + return writer; + } + auto* sortPool = writerInfo_.back()->sortPool.get(); + VELOX_CHECK_NOT_NULL(sortPool); + auto sortBuffer = std::make_unique( + getNonPartitionTypes(dataChannels_, inputType_), + sortColumnIndices_, + sortCompareFlags_, + sortPool, + writerInfo_.back()->nonReclaimableSectionHolder.get(), + connectorQueryCtx_->prefixSortConfig(), + spillConfig_, + writerInfo_.back()->spillStats.get()); + return std::make_unique( + std::move(writer), + std::move(sortBuffer), + hiveConfig_->sortWriterMaxOutputRows( + connectorQueryCtx_->sessionProperties()), + hiveConfig_->sortWriterMaxOutputBytes( + connectorQueryCtx_->sessionProperties()), + sortWriterFinishTimeSliceLimitMs_); +} + +void HiveDataSink::extendBuffersForPartitionedTables() { + // Extends the buffer used for partition rows calculations. + partitionSizes_.emplace_back(0); + partitionRows_.emplace_back(nullptr); + rawPartitionRows_.emplace_back(nullptr); +} + +HiveWriterId HiveDataSink::getWriterId(size_t row) const { + std::optional partitionId; + if (isPartitioned()) { + VELOX_CHECK_LT(partitionIds_[row], std::numeric_limits::max()); + partitionId = static_cast(partitionIds_[row]); + } + + std::optional bucketId; + if (isBucketed()) { + bucketId = bucketIds_[row]; + } + return HiveWriterId{partitionId, bucketId}; +} + +void HiveDataSink::updatePartitionRows( + uint32_t index, + vector_size_t numRows, + vector_size_t row) { + VELOX_DCHECK_LT(index, partitionSizes_.size()); + VELOX_DCHECK_EQ(partitionSizes_.size(), partitionRows_.size()); + VELOX_DCHECK_EQ(partitionRows_.size(), rawPartitionRows_.size()); + if (FOLLY_UNLIKELY(partitionRows_[index] == nullptr) || + (partitionRows_[index]->capacity() < numRows * sizeof(vector_size_t))) { + partitionRows_[index] = + allocateIndices(numRows, connectorQueryCtx_->memoryPool()); + rawPartitionRows_[index] = + partitionRows_[index]->asMutable(); + } + rawPartitionRows_[index][partitionSizes_[index]] = row; + ++partitionSizes_[index]; +} + +void HiveDataSink::splitInputRowsAndEnsureWriters(RowVectorPtr /* input */) { + VELOX_CHECK(isPartitioned() || isBucketed()); + if (isBucketed() && isPartitioned()) { + VELOX_CHECK_EQ(bucketIds_.size(), partitionIds_.size()); + } + + std::fill(partitionSizes_.begin(), partitionSizes_.end(), 0); + + const auto numRows = + isPartitioned() ? partitionIds_.size() : bucketIds_.size(); + for (auto row = 0; row < numRows; ++row) { + const auto id = getWriterId(row); + const uint32_t index = ensureWriter(id); + + updatePartitionRows(index, numRows, row); + } + + for (uint32_t i = 0; i < partitionSizes_.size(); ++i) { + if (partitionSizes_[i] != 0) { + VELOX_CHECK_NOT_NULL(partitionRows_[i]); + partitionRows_[i]->setSize(partitionSizes_[i] * sizeof(vector_size_t)); + } + } +} + +std::string HiveDataSink::makePartitionDirectory( + const std::string& tableDirectory, + const std::optional& partitionSubdirectory) const { + if (partitionSubdirectory.has_value()) { + return fs::path(tableDirectory) / partitionSubdirectory.value(); + } + return tableDirectory; +} + +HiveWriterParameters HiveDataSink::getWriterParameters( + const std::optional& partition, + std::optional bucketId) const { + auto [targetFileName, writeFileName] = getWriterFileNames(bucketId); + + return HiveWriterParameters{ + updateMode_, + partition, + targetFileName, + makePartitionDirectory( + insertTableHandle_->locationHandle()->targetPath(), partition), + writeFileName, + makePartitionDirectory( + insertTableHandle_->locationHandle()->writePath(), partition)}; +} + +std::pair HiveDataSink::getWriterFileNames( + std::optional bucketId) const { + if (auto hiveInsertFileNameGenerator = + std::dynamic_pointer_cast( + fileNameGenerator_)) { + return hiveInsertFileNameGenerator->gen( + bucketId, + insertTableHandle_, + *connectorQueryCtx_, + hiveConfig_, + isCommitRequired()); + } + + return fileNameGenerator_->gen( + bucketId, insertTableHandle_, *connectorQueryCtx_, isCommitRequired()); +} + +std::pair HiveInsertFileNameGenerator::gen( + std::optional bucketId, + const std::shared_ptr insertTableHandle, + const ConnectorQueryCtx& connectorQueryCtx, + bool commitRequired) const { + auto defaultHiveConfig = + std::make_shared(std::make_shared( + std::unordered_map())); + + return this->gen( + bucketId, + insertTableHandle, + connectorQueryCtx, + defaultHiveConfig, + commitRequired); +} + +std::pair HiveInsertFileNameGenerator::gen( + std::optional bucketId, + const std::shared_ptr insertTableHandle, + const ConnectorQueryCtx& connectorQueryCtx, + const std::shared_ptr& hiveConfig, + bool commitRequired) const { + auto targetFileName = insertTableHandle->locationHandle()->targetFileName(); + const bool generateFileName = targetFileName.empty(); + if (bucketId.has_value()) { + VELOX_CHECK(generateFileName); + // TODO: add hive.file_renaming_enabled support. + targetFileName = computeBucketedFileName( + connectorQueryCtx.queryId(), + hiveConfig->maxBucketCount(connectorQueryCtx.sessionProperties()), + bucketId.value()); + } else if (generateFileName) { + // targetFileName includes planNodeId and Uuid. As a result, different + // table writers run by the same task driver or the same table writer + // run in different task tries would have different targetFileNames. + targetFileName = fmt::format( + "{}_{}_{}_{}", + connectorQueryCtx.taskId(), + connectorQueryCtx.driverId(), + connectorQueryCtx.planNodeId(), + makeUuid()); + } + VELOX_CHECK(!targetFileName.empty()); + const std::string writeFileName = commitRequired + ? fmt::format(".tmp.velox.{}_{}", targetFileName, makeUuid()) + : targetFileName; + if (generateFileName && + insertTableHandle->storageFormat() == dwio::common::FileFormat::PARQUET) { + return { + fmt::format("{}{}", targetFileName, ".parquet"), + fmt::format("{}{}", writeFileName, ".parquet")}; + } + return {targetFileName, writeFileName}; +} + +folly::dynamic HiveInsertFileNameGenerator::serialize() const { + folly::dynamic obj = folly::dynamic::object; + obj["name"] = "HiveInsertFileNameGenerator"; + return obj; +} + +std::shared_ptr +HiveInsertFileNameGenerator::deserialize( + const folly::dynamic& /* obj */, + void* /* context */) { + return std::make_shared(); +} + +void HiveInsertFileNameGenerator::registerSerDe() { + auto& registry = DeserializationWithContextRegistryForSharedPtr(); + registry.Register( + "HiveInsertFileNameGenerator", HiveInsertFileNameGenerator::deserialize); +} + +std::string HiveInsertFileNameGenerator::toString() const { + return "HiveInsertFileNameGenerator"; +} + +HiveWriterParameters::UpdateMode HiveDataSink::getUpdateMode() const { + if (insertTableHandle_->isExistingTable()) { + if (insertTableHandle_->isPartitioned()) { + const auto insertBehavior = hiveConfig_->insertExistingPartitionsBehavior( + connectorQueryCtx_->sessionProperties()); + switch (insertBehavior) { + case common::HiveConfig::InsertExistingPartitionsBehavior::kOverwrite: + return HiveWriterParameters::UpdateMode::kOverwrite; + case common::HiveConfig::InsertExistingPartitionsBehavior::kError: + return HiveWriterParameters::UpdateMode::kNew; + default: + VELOX_UNSUPPORTED( + "Unsupported insert existing partitions behavior: {}", + common::HiveConfig::insertExistingPartitionsBehaviorString( + insertBehavior)); + } + } else { + if (hiveConfig_->immutablePartitions()) { + VELOX_USER_FAIL("Unpartitioned Hive tables are immutable."); + } + return HiveWriterParameters::UpdateMode::kAppend; + } + } else { + return HiveWriterParameters::UpdateMode::kNew; + } +} + +bool HiveInsertTableHandle::isPartitioned() const { + return std::any_of( + inputColumns_.begin(), inputColumns_.end(), [](auto column) { + return column->isPartitionKey(); + }); +} + +const HiveBucketProperty* HiveInsertTableHandle::bucketProperty() const { + return bucketProperty_.get(); +} + +bool HiveInsertTableHandle::isBucketed() const { + return bucketProperty() != nullptr; +} + +bool HiveInsertTableHandle::isExistingTable() const { + return locationHandle_->tableType() == LocationHandle::TableType::kExisting; +} + +folly::dynamic HiveInsertTableHandle::serialize() const { + folly::dynamic obj = folly::dynamic::object; + obj["name"] = "HiveInsertTableHandle"; + folly::dynamic arr = folly::dynamic::array; + for (const auto& ic : inputColumns_) { + arr.push_back(ic->serialize()); + } + + obj["inputColumns"] = arr; + obj["locationHandle"] = locationHandle_->serialize(); + obj["tableStorageFormat"] = dwio::common::toString(storageFormat_); + + if (bucketProperty_) { + obj["bucketProperty"] = bucketProperty_->serialize(); + } + + if (compressionKind_.has_value()) { + obj["compressionKind"] = velox::common::compressionKindToString(*compressionKind_); + } + + folly::dynamic params = folly::dynamic::object; + for (const auto& [key, value] : serdeParameters_) { + params[key] = value; + } + obj["serdeParameters"] = params; + obj["ensureFiles"] = ensureFiles_; + obj["fileNameGenerator"] = fileNameGenerator_->serialize(); + return obj; +} + +HiveInsertTableHandlePtr HiveInsertTableHandle::create( + const folly::dynamic& obj) { + auto inputColumns = ISerializable::deserialize>( + obj["inputColumns"]); + auto locationHandle = + ISerializable::deserialize(obj["locationHandle"]); + auto storageFormat = + dwio::common::toFileFormat(obj["tableStorageFormat"].asString()); + + std::optional compressionKind = std::nullopt; + if (obj.count("compressionKind") > 0) { + compressionKind = + velox::common::stringToCompressionKind(obj["compressionKind"].asString()); + } + + std::shared_ptr bucketProperty; + if (obj.count("bucketProperty") > 0) { + bucketProperty = + ISerializable::deserialize(obj["bucketProperty"]); + } + + std::unordered_map serdeParameters; + for (const auto& pair : obj["serdeParameters"].items()) { + serdeParameters.emplace(pair.first.asString(), pair.second.asString()); + } + + bool ensureFiles = obj["ensureFiles"].asBool(); + + auto fileNameGenerator = + ISerializable::deserialize(obj["fileNameGenerator"]); + return std::make_shared( + inputColumns, + locationHandle, + storageFormat, + bucketProperty, + compressionKind, + serdeParameters, + nullptr, // writerOptions is not serializable + ensureFiles, + fileNameGenerator); +} + +void HiveInsertTableHandle::registerSerDe() { + auto& registry = DeserializationRegistryForSharedPtr(); + registry.Register("HiveInsertTableHandle", HiveInsertTableHandle::create); +} + +std::string HiveInsertTableHandle::toString() const { + std::ostringstream out; + out << "HiveInsertTableHandle [" << dwio::common::toString(storageFormat_); + if (compressionKind_.has_value()) { + out << " " << velox::common::compressionKindToString(compressionKind_.value()); + } else { + out << " none"; + } + out << "], [inputColumns: ["; + for (const auto& i : inputColumns_) { + out << " " << i->toString(); + } + out << " ], locationHandle: " << locationHandle_->toString(); + if (bucketProperty_) { + out << ", bucketProperty: " << bucketProperty_->toString(); + } + + if (serdeParameters_.size() > 0) { + std::map sortedSerdeParams( + serdeParameters_.begin(), serdeParameters_.end()); + out << ", serdeParameters: "; + for (const auto& [key, value] : sortedSerdeParams) { + out << "[" << key << ", " << value << "] "; + } + } + out << ", fileNameGenerator: " << fileNameGenerator_->toString(); + out << "]"; + return out.str(); +} + +std::string LocationHandle::toString() const { + return fmt::format( + "LocationHandle [targetPath: {}, writePath: {}, tableType: {}, tableFileName: {}]", + targetPath_, + writePath_, + tableTypeName(tableType_), + targetFileName_); +} + +void LocationHandle::registerSerDe() { + auto& registry = DeserializationRegistryForSharedPtr(); + registry.Register("LocationHandle", LocationHandle::create); +} + +folly::dynamic LocationHandle::serialize() const { + folly::dynamic obj = folly::dynamic::object; + obj["name"] = "LocationHandle"; + obj["targetPath"] = targetPath_; + obj["writePath"] = writePath_; + obj["tableType"] = tableTypeName(tableType_); + obj["targetFileName"] = targetFileName_; + return obj; +} + +LocationHandlePtr LocationHandle::create(const folly::dynamic& obj) { + auto targetPath = obj["targetPath"].asString(); + auto writePath = obj["writePath"].asString(); + auto tableType = tableTypeFromName(obj["tableType"].asString()); + auto targetFileName = obj["targetFileName"].asString(); + return std::make_shared( + targetPath, writePath, tableType, targetFileName); +} + +std::unique_ptr HiveDataSink::WriterReclaimer::create( + HiveDataSink* dataSink, + HiveWriterInfo* writerInfo, + io::IoStatistics* ioStats) { + return std::unique_ptr( + new HiveDataSink::WriterReclaimer(dataSink, writerInfo, ioStats)); +} + +bool HiveDataSink::WriterReclaimer::reclaimableBytes( + const memory::MemoryPool& pool, + uint64_t& reclaimableBytes) const { + VELOX_CHECK_EQ(pool.name(), writerInfo_->writerPool->name()); + reclaimableBytes = 0; + if (!dataSink_->canReclaim()) { + return false; + } + return exec::MemoryReclaimer::reclaimableBytes(pool, reclaimableBytes); +} + +uint64_t HiveDataSink::WriterReclaimer::reclaim( + memory::MemoryPool* pool, + uint64_t targetBytes, + uint64_t maxWaitMs, + memory::MemoryReclaimer::Stats& stats) { + VELOX_CHECK_EQ(pool->name(), writerInfo_->writerPool->name()); + if (!dataSink_->canReclaim()) { + return 0; + } + + if (*writerInfo_->nonReclaimableSectionHolder.get()) { + RECORD_METRIC_VALUE(kMetricMemoryNonReclaimableCount); + LOG(WARNING) << "Can't reclaim from hive writer pool " << pool->name() + << " which is under non-reclaimable section, " + << " reserved memory: " + << succinctBytes(pool->reservedBytes()); + ++stats.numNonReclaimableAttempts; + return 0; + } + + const uint64_t memoryUsageBeforeReclaim = pool->reservedBytes(); + const std::string memoryUsageTreeBeforeReclaim = pool->treeMemoryUsage(); + const auto writtenBytesBeforeReclaim = ioStats_->rawBytesWritten(); + const auto reclaimedBytes = + exec::MemoryReclaimer::reclaim(pool, targetBytes, maxWaitMs, stats); + const auto earlyFlushedRawBytes = + ioStats_->rawBytesWritten() - writtenBytesBeforeReclaim; + addThreadLocalRuntimeStat( + kEarlyFlushedRawBytes, + RuntimeCounter(earlyFlushedRawBytes, RuntimeCounter::Unit::kBytes)); + if (earlyFlushedRawBytes > 0) { + RECORD_METRIC_VALUE( + kMetricFileWriterEarlyFlushedRawBytes, earlyFlushedRawBytes); + } + const uint64_t memoryUsageAfterReclaim = pool->reservedBytes(); + if (memoryUsageAfterReclaim > memoryUsageBeforeReclaim) { + VELOX_FAIL( + "Unexpected memory growth after memory reclaim from {}, the memory usage before reclaim: {}, after reclaim: {}\nThe memory tree usage before reclaim:\n{}\nThe memory tree usage after reclaim:\n{}", + pool->name(), + succinctBytes(memoryUsageBeforeReclaim), + succinctBytes(memoryUsageAfterReclaim), + memoryUsageTreeBeforeReclaim, + pool->treeMemoryUsage()); + } + return reclaimedBytes; +} +} // namespace facebook::velox::connector::lakehouse::common diff --git a/velox/connectors/lakehouse/common/HiveDataSink.h b/velox/connectors/lakehouse/common/HiveDataSink.h new file mode 100644 index 000000000000..be35f1864f6e --- /dev/null +++ b/velox/connectors/lakehouse/common/HiveDataSink.h @@ -0,0 +1,755 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#pragma once + +#include "velox/common/compression/Compression.h" +#include "velox/connectors/Connector.h" +#include "velox/connectors/lakehouse/common/HiveConfig.h" +#include "velox/connectors/lakehouse/common/PartitionIdGenerator.h" +#include "velox/connectors/lakehouse/common/TableHandle.h" +#include "velox/dwio/common/Options.h" +#include "velox/dwio/common/Writer.h" +#include "velox/dwio/common/WriterFactory.h" +#include "velox/exec/MemoryReclaimer.h" + +namespace facebook::velox::connector::lakehouse::common { + +class LocationHandle; +using LocationHandlePtr = std::shared_ptr; + +/// Location related properties of the Hive table to be written. +class LocationHandle : public ISerializable { + public: + enum class TableType { + /// Write to a new table to be created. + kNew, + /// Write to an existing table. + kExisting, + }; + + LocationHandle( + std::string targetPath, + std::string writePath, + TableType tableType, + std::string targetFileName = "") + : targetPath_(std::move(targetPath)), + targetFileName_(std::move(targetFileName)), + writePath_(std::move(writePath)), + tableType_(tableType) {} + + const std::string& targetPath() const { + return targetPath_; + } + + const std::string& targetFileName() const { + return targetFileName_; + } + + const std::string& writePath() const { + return writePath_; + } + + TableType tableType() const { + return tableType_; + } + + std::string toString() const; + + static void registerSerDe(); + + folly::dynamic serialize() const override; + + static LocationHandlePtr create(const folly::dynamic& obj); + + static const std::string tableTypeName(LocationHandle::TableType type); + + static LocationHandle::TableType tableTypeFromName(const std::string& name); + + private: + // Target directory path. + const std::string targetPath_; + // If non-empty, use this name instead of generating our own. + const std::string targetFileName_; + // Staging directory path. + const std::string writePath_; + // Whether the table to be written is new, already existing or temporary. + const TableType tableType_; +}; + +class HiveSortingColumn : public ISerializable { + public: + HiveSortingColumn( + const std::string& sortColumn, + const core::SortOrder& sortOrder); + + const std::string& sortColumn() const { + return sortColumn_; + } + + core::SortOrder sortOrder() const { + return sortOrder_; + } + + folly::dynamic serialize() const override; + + static std::shared_ptr deserialize( + const folly::dynamic& obj, + void* context); + + std::string toString() const; + + static void registerSerDe(); + + private: + const std::string sortColumn_; + const core::SortOrder sortOrder_; +}; + +class HiveBucketProperty : public ISerializable { + public: + enum class Kind { kHiveCompatible, kPrestoNative }; + + HiveBucketProperty( + Kind kind, + int32_t bucketCount, + const std::vector& bucketedBy, + const std::vector& bucketedTypes, + const std::vector>& sortedBy); + + Kind kind() const { + return kind_; + } + + static std::string kindString(Kind kind); + + /// Returns the number of bucket count. + int32_t bucketCount() const { + return bucketCount_; + } + + /// Returns the bucketed by column names. + const std::vector& bucketedBy() const { + return bucketedBy_; + } + + /// Returns the bucketed by column types. + const std::vector& bucketedTypes() const { + return bucketTypes_; + } + + /// Returns the hive sorting columns if not empty. + const std::vector>& sortedBy() + const { + return sortedBy_; + } + + folly::dynamic serialize() const override; + + static std::shared_ptr deserialize( + const folly::dynamic& obj, + void* context); + + bool operator==(const HiveBucketProperty& other) const { + return true; + } + + static void registerSerDe(); + + std::string toString() const; + + private: + void validate() const; + + const Kind kind_; + const int32_t bucketCount_; + const std::vector bucketedBy_; + const std::vector bucketTypes_; + const std::vector> sortedBy_; +}; + +FOLLY_ALWAYS_INLINE std::ostream& operator<<( + std::ostream& os, + HiveBucketProperty::Kind kind) { + os << HiveBucketProperty::kindString(kind); + return os; +} + +class HiveInsertTableHandle; +using HiveInsertTableHandlePtr = std::shared_ptr; + +class FileNameGenerator : public ISerializable { + public: + virtual ~FileNameGenerator() = default; + + virtual std::pair gen( + std::optional bucketId, + const std::shared_ptr insertTableHandle, + const ConnectorQueryCtx& connectorQueryCtx, + bool commitRequired) const = 0; + + virtual std::string toString() const = 0; +}; + +class HiveInsertFileNameGenerator : public FileNameGenerator { + public: + HiveInsertFileNameGenerator() {} + + std::pair gen( + std::optional bucketId, + const std::shared_ptr insertTableHandle, + const ConnectorQueryCtx& connectorQueryCtx, + bool commitRequired) const override; + + /// Version of file generation that takes hiveConfig into account when + /// generating file names + std::pair gen( + std::optional bucketId, + const std::shared_ptr insertTableHandle, + const ConnectorQueryCtx& connectorQueryCtx, + const std::shared_ptr& hiveConfig, + bool commitRequired) const; + + static void registerSerDe(); + + folly::dynamic serialize() const override; + + static std::shared_ptr deserialize( + const folly::dynamic& obj, + void* context); + + std::string toString() const override; +}; + +/// Represents a request for Hive write. +class HiveInsertTableHandle : public ConnectorInsertTableHandle { + public: + HiveInsertTableHandle( + std::vector> inputColumns, + std::shared_ptr locationHandle, + dwio::common::FileFormat storageFormat = dwio::common::FileFormat::DWRF, + std::shared_ptr bucketProperty = nullptr, + std::optional compressionKind = {}, + const std::unordered_map& serdeParameters = {}, + const std::shared_ptr& writerOptions = + nullptr, + // When this option is set the HiveDataSink will always write a file even + // if there's no data. This is useful when the table is bucketed, but the + // engine handles ensuring a 1 to 1 mapping from task to bucket. + const bool ensureFiles = false, + std::shared_ptr fileNameGenerator = + std::make_shared()) + : inputColumns_(std::move(inputColumns)), + locationHandle_(std::move(locationHandle)), + storageFormat_(storageFormat), + bucketProperty_(std::move(bucketProperty)), + compressionKind_(compressionKind), + serdeParameters_(serdeParameters), + writerOptions_(writerOptions), + ensureFiles_(ensureFiles), + fileNameGenerator_(std::move(fileNameGenerator)) { + if (compressionKind.has_value()) { + VELOX_CHECK( + compressionKind.value() != velox::common::CompressionKind_MAX, + "Unsupported compression type: CompressionKind_MAX"); + } + + if (ensureFiles_) { + // If ensureFiles is set and either the bucketProperty is set or some + // partition keys are in the data, there is not a 1:1 mapping from Task to + // files so we can't proactively create writers. + VELOX_CHECK( + bucketProperty_ == nullptr || bucketProperty_->bucketCount() == 0, + "ensureFiles is not supported with bucketing"); + + for (const auto& inputColumn : inputColumns_) { + VELOX_CHECK( + !inputColumn->isPartitionKey(), + "ensureFiles is not supported with partition keys in the data"); + } + } + } + + virtual ~HiveInsertTableHandle() = default; + + const std::vector>& inputColumns() + const { + return inputColumns_; + } + + const std::shared_ptr& locationHandle() const { + return locationHandle_; + } + + std::optional compressionKind() const { + return compressionKind_; + } + + dwio::common::FileFormat storageFormat() const { + return storageFormat_; + } + + const std::unordered_map& serdeParameters() const { + return serdeParameters_; + } + + const std::shared_ptr& writerOptions() const { + return writerOptions_; + } + + bool ensureFiles() const { + return ensureFiles_; + } + + const std::shared_ptr& fileNameGenerator() const { + return fileNameGenerator_; + } + + bool supportsMultiThreading() const override { + return true; + } + + bool isPartitioned() const; + + bool isBucketed() const; + + const HiveBucketProperty* bucketProperty() const; + + bool isExistingTable() const; + + folly::dynamic serialize() const override; + + static HiveInsertTableHandlePtr create(const folly::dynamic& obj); + + static void registerSerDe(); + + std::string toString() const override; + + private: + const std::vector> inputColumns_; + const std::shared_ptr locationHandle_; + const dwio::common::FileFormat storageFormat_; + const std::shared_ptr bucketProperty_; + const std::optional compressionKind_; + const std::unordered_map serdeParameters_; + const std::shared_ptr writerOptions_; + const bool ensureFiles_; + const std::shared_ptr fileNameGenerator_; +}; + +/// Parameters for Hive writers. +class HiveWriterParameters { + public: + enum class UpdateMode { + kNew, // Write files to a new directory. + kOverwrite, // Overwrite an existing directory. + // Append mode is currently only supported for unpartitioned tables. + kAppend, // Append to an unpartitioned table. + }; + + /// @param updateMode Write the files to a new directory, or append to an + /// existing directory or overwrite an existing directory. + /// @param partitionName Partition name in the typical Hive style, which is + /// also the partition subdirectory part of the partition path. + /// @param targetFileName The final name of a file after committing. + /// @param targetDirectory The final directory that a file should be in after + /// committing. + /// @param writeFileName The temporary name of the file that a running writer + /// writes to. If a running writer writes directory to the target file, set + /// writeFileName to targetFileName by default. + /// @param writeDirectory The temporary directory that a running writer writes + /// to. If a running writer writes directory to the target directory, set + /// writeDirectory to targetDirectory by default. + HiveWriterParameters( + UpdateMode updateMode, + std::optional partitionName, + std::string targetFileName, + std::string targetDirectory, + std::optional writeFileName = std::nullopt, + std::optional writeDirectory = std::nullopt) + : updateMode_(updateMode), + partitionName_(std::move(partitionName)), + targetFileName_(std::move(targetFileName)), + targetDirectory_(std::move(targetDirectory)), + writeFileName_(writeFileName.value_or(targetFileName_)), + writeDirectory_(writeDirectory.value_or(targetDirectory_)) {} + + UpdateMode updateMode() const { + return updateMode_; + } + + static std::string updateModeToString(UpdateMode updateMode) { + switch (updateMode) { + case UpdateMode::kNew: + return "NEW"; + case UpdateMode::kOverwrite: + return "OVERWRITE"; + case UpdateMode::kAppend: + return "APPEND"; + default: + VELOX_UNSUPPORTED("Unsupported update mode."); + } + } + + const std::optional& partitionName() const { + return partitionName_; + } + + const std::string& targetFileName() const { + return targetFileName_; + } + + const std::string& writeFileName() const { + return writeFileName_; + } + + const std::string& targetDirectory() const { + return targetDirectory_; + } + + const std::string& writeDirectory() const { + return writeDirectory_; + } + + private: + const UpdateMode updateMode_; + const std::optional partitionName_; + const std::string targetFileName_; + const std::string targetDirectory_; + const std::string writeFileName_; + const std::string writeDirectory_; +}; + +struct HiveWriterInfo { + HiveWriterInfo( + HiveWriterParameters parameters, + std::shared_ptr _writerPool, + std::shared_ptr _sinkPool, + std::shared_ptr _sortPool) + : writerParameters(std::move(parameters)), + nonReclaimableSectionHolder(new tsan_atomic(false)), + spillStats(std::make_unique>()), + writerPool(std::move(_writerPool)), + sinkPool(std::move(_sinkPool)), + sortPool(std::move(_sortPool)) {} + + const HiveWriterParameters writerParameters; + const std::unique_ptr> nonReclaimableSectionHolder; + /// Collects the spill stats from sort writer if the spilling has been + /// triggered. + const std::unique_ptr> spillStats; + const std::shared_ptr writerPool; + const std::shared_ptr sinkPool; + const std::shared_ptr sortPool; + int64_t numWrittenRows = 0; + int64_t inputSizeInBytes = 0; +}; + +/// Identifies a hive writer. +struct HiveWriterId { + std::optional partitionId{std::nullopt}; + std::optional bucketId{std::nullopt}; + + HiveWriterId() = default; + + HiveWriterId( + std::optional _partitionId, + std::optional _bucketId = std::nullopt) + : partitionId(_partitionId), bucketId(_bucketId) {} + + /// Returns the special writer id for the un-partitioned (and non-bucketed) + /// table. + static const HiveWriterId& unpartitionedId(); + + std::string toString() const; + + bool operator==(const HiveWriterId& other) const { + return std::tie(partitionId, bucketId) == + std::tie(other.partitionId, other.bucketId); + } +}; + +struct HiveWriterIdHasher { + std::size_t operator()(const HiveWriterId& id) const { + return bits::hashMix( + id.partitionId.value_or(std::numeric_limits::max()), + id.bucketId.value_or(std::numeric_limits::max())); + } +}; + +struct HiveWriterIdEq { + bool operator()(const HiveWriterId& lhs, const HiveWriterId& rhs) const { + return lhs == rhs; + } +}; + +class HiveDataSink : public DataSink { + public: + /// The list of runtime stats reported by hive data sink + static constexpr const char* kEarlyFlushedRawBytes = "earlyFlushedRawBytes"; + + /// Defines the execution states of a hive data sink running internally. + enum class State { + /// The data sink accepts new append data in this state. + kRunning = 0, + /// The data sink flushes any buffered data to the underlying file writer + /// but no more data can be appended. + kFinishing = 1, + /// The data sink is aborted on error and no more data can be appended. + kAborted = 2, + /// The data sink is closed on error and no more data can be appended. + kClosed = 3 + }; + static std::string stateString(State state); + + HiveDataSink( + RowTypePtr inputType, + std::shared_ptr insertTableHandle, + const ConnectorQueryCtx* connectorQueryCtx, + CommitStrategy commitStrategy, + const std::shared_ptr& hiveConfig); + + HiveDataSink( + RowTypePtr inputType, + std::shared_ptr insertTableHandle, + const ConnectorQueryCtx* connectorQueryCtx, + CommitStrategy commitStrategy, + const std::shared_ptr& hiveConfig, + uint32_t bucketCount, + std::unique_ptr bucketFunction, + const std::vector& dataChannels); + + void appendData(RowVectorPtr input) override; + + bool finish() override; + + Stats stats() const override; + + std::vector close() override; + + void abort() override; + + bool canReclaim() const; + + protected: + // Validates the state transition from 'oldState' to 'newState'. + void checkStateTransition(State oldState, State newState); + + void setState(State newState); + + virtual std::vector commitMessage() const; + + class WriterReclaimer : public exec::MemoryReclaimer { + public: + static std::unique_ptr create( + HiveDataSink* dataSink, + HiveWriterInfo* writerInfo, + io::IoStatistics* ioStats); + + bool reclaimableBytes( + const memory::MemoryPool& pool, + uint64_t& reclaimableBytes) const override; + + uint64_t reclaim( + memory::MemoryPool* pool, + uint64_t targetBytes, + uint64_t maxWaitMs, + memory::MemoryReclaimer::Stats& stats) override; + + private: + WriterReclaimer( + HiveDataSink* dataSink, + HiveWriterInfo* writerInfo, + io::IoStatistics* ioStats) + : exec::MemoryReclaimer(0), + dataSink_(dataSink), + writerInfo_(writerInfo), + ioStats_(ioStats) { + VELOX_CHECK_NOT_NULL(dataSink_); + VELOX_CHECK_NOT_NULL(writerInfo_); + VELOX_CHECK_NOT_NULL(ioStats_); + } + + HiveDataSink* const dataSink_; + HiveWriterInfo* const writerInfo_; + io::IoStatistics* const ioStats_; + }; + + FOLLY_ALWAYS_INLINE bool sortWrite() const { + return !sortColumnIndices_.empty(); + } + + // Returns true if the table is partitioned. + FOLLY_ALWAYS_INLINE bool isPartitioned() const { + return partitionIdGenerator_ != nullptr; + } + + // Returns true if the table is bucketed. + FOLLY_ALWAYS_INLINE bool isBucketed() const { + return bucketCount_ != 0; + } + + FOLLY_ALWAYS_INLINE bool isCommitRequired() const { + return commitStrategy_ != CommitStrategy::kNoCommit; + } + + std::shared_ptr createWriterPool( + const HiveWriterId& writerId); + + void setMemoryReclaimers( + HiveWriterInfo* writerInfo, + io::IoStatistics* ioStats); + + // Compute the partition id and bucket id for each row in 'input'. + void computePartitionAndBucketIds(const RowVectorPtr& input); + + // Get the hive writer id corresponding to the row + // from partitionIds and bucketIds. + FOLLY_ALWAYS_INLINE HiveWriterId getWriterId(size_t row) const; + + // Computes the number of input rows as well as the actual input row indices + // to each corresponding (bucketed) partition based on the partition and + // bucket ids calculated by 'computePartitionAndBucketIds'. The function also + // ensures that there is a writer created for each (bucketed) partition. + virtual void splitInputRowsAndEnsureWriters(RowVectorPtr input); + + // Makes sure the writer is created for the given writer id. The function + // returns the corresponding index in 'writers_'. + uint32_t ensureWriter(const HiveWriterId& id); + + // Appends a new writer for the given 'id'. The function returns the index of + // the newly created writer in 'writers_'. + uint32_t appendWriter(const HiveWriterId& id); + + virtual std::optional getPartitionName( + const HiveWriterId& id) const; + + std::unique_ptr + maybeCreateBucketSortWriter( + std::unique_ptr writer); + + std::string makePartitionDirectory( + const std::string& tableDirectory, + const std::optional& partitionSubdirectory) const; + + void + updatePartitionRows(uint32_t index, vector_size_t numRows, vector_size_t row); + + void extendBuffersForPartitionedTables(); + + HiveWriterParameters getWriterParameters( + const std::optional& partition, + std::optional bucketId) const; + + // Gets write and target file names for a writer based on the table commit + // strategy as well as table partitioned type. If commit is not required, the + // write file and target file has the same name. If not, add a temp file + // prefix to the target file for write file name. The coordinator (or driver + // for Presto on spark) will rename the write file to target file to commit + // the table write when update the metadata store. If it is a bucketed table, + // the file name encodes the corresponding bucket id. + std::pair getWriterFileNames( + std::optional bucketId) const; + + HiveWriterParameters::UpdateMode getUpdateMode() const; + + FOLLY_ALWAYS_INLINE void checkRunning() const { + VELOX_CHECK_EQ(state_, State::kRunning, "Hive data sink is not running"); + } + + // Invoked to write 'input' to the specified file writer. + void write(size_t index, RowVectorPtr input); + + void closeInternal(); + + const RowTypePtr inputType_; + const std::shared_ptr insertTableHandle_; + const ConnectorQueryCtx* const connectorQueryCtx_; + const CommitStrategy commitStrategy_; + const std::shared_ptr hiveConfig_; + const HiveWriterParameters::UpdateMode updateMode_; + const uint32_t maxOpenWriters_; + const std::vector partitionChannels_; + const std::unique_ptr partitionIdGenerator_; + // Indices of dataChannel are stored in ascending order + const std::vector dataChannels_; + const int32_t bucketCount_{0}; + const std::unique_ptr bucketFunction_; + const std::shared_ptr writerFactory_; + const velox::common::SpillConfig* const spillConfig_; + const uint64_t sortWriterFinishTimeSliceLimitMs_{0}; + + std::vector sortColumnIndices_; + std::vector sortCompareFlags_; + + State state_{State::kRunning}; + + tsan_atomic nonReclaimableSection_{false}; + + // The map from writer id to the writer index in 'writers_' and 'writerInfo_'. + folly::F14FastMap + writerIndexMap_; + + // Below are structures for partitions from all inputs. writerInfo_ and + // writers_ are both indexed by partitionId. + std::vector> writerInfo_; + std::vector> writers_; + // IO statistics collected for each writer. + std::vector> ioStats_; + + // Below are structures updated when processing current input. partitionIds_ + // are indexed by the row of input_. partitionRows_, rawPartitionRows_ and + // partitionSizes_ are indexed by partitionId. + raw_vector partitionIds_; + std::vector partitionRows_; + std::vector rawPartitionRows_; + std::vector partitionSizes_; + + // Reusable buffers for bucket id calculations. + std::vector bucketIds_; + + // Strategy for naming writer files + std::shared_ptr fileNameGenerator_; +}; + +FOLLY_ALWAYS_INLINE std::ostream& operator<<( + std::ostream& os, + HiveDataSink::State state) { + os << HiveDataSink::stateString(state); + return os; +} +} // namespace facebook::velox::connector::lakehouse::common + +template <> +struct fmt::formatter + : formatter { + auto format( + facebook::velox::connector::lakehouse::common::HiveDataSink::State s, + format_context& ctx) const { + return formatter::format( + facebook::velox::connector::lakehouse::common::HiveDataSink::stateString(s), ctx); + } +}; + +template <> +struct fmt::formatter< + facebook::velox::connector::lakehouse::common::LocationHandle::TableType> + : formatter { + auto format( + facebook::velox::connector::lakehouse::common::LocationHandle::TableType s, + format_context& ctx) const { + return formatter::format(static_cast(s), ctx); + } +}; diff --git a/velox/connectors/lakehouse/common/HiveDataSource.cpp b/velox/connectors/lakehouse/common/HiveDataSource.cpp new file mode 100644 index 000000000000..e999e65f0ade --- /dev/null +++ b/velox/connectors/lakehouse/common/HiveDataSource.cpp @@ -0,0 +1,572 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "velox/connectors/lakehouse/common/HiveDataSource.h" + +#include +#include +#include + +#include "velox/common/testutil/TestValue.h" +#include "velox/connectors/lakehouse/common/HiveConfig.h" +#include "velox/dwio/common/ReaderFactory.h" +#include "velox/expression/FieldReference.h" + +using facebook::velox::common::testutil::TestValue; + +namespace facebook::velox::connector::lakehouse::common { + +class HiveTableHandle; +class HiveColumnHandle; + +namespace { + +bool isMember( + const std::vector& fields, + const exec::FieldReference& field) { + return std::find(fields.begin(), fields.end(), &field) != fields.end(); +} + +bool shouldEagerlyMaterialize( + const exec::Expr& remainingFilter, + const exec::FieldReference& field) { + if (!remainingFilter.evaluatesArgumentsOnNonIncreasingSelection()) { + return true; + } + for (auto& input : remainingFilter.inputs()) { + if (isMember(input->distinctFields(), field) && input->hasConditionals()) { + return true; + } + } + return false; +} + +} // namespace + +HiveDataSource::HiveDataSource( + const RowTypePtr& outputType, + const connector::ConnectorTableHandlePtr& tableHandle, + const connector::ColumnHandleMap& columnHandles, + FileHandleFactory* fileHandleFactory, + folly::Executor* executor, + const ConnectorQueryCtx* connectorQueryCtx, + const std::shared_ptr& hiveConfig) + : fileHandleFactory_(fileHandleFactory), + executor_(executor), + connectorQueryCtx_(connectorQueryCtx), + hiveConfig_(hiveConfig), + pool_(connectorQueryCtx->memoryPool()), + outputType_(outputType), + expressionEvaluator_(connectorQueryCtx->expressionEvaluator()) { + // Column handled keyed on the column alias, the name used in the query. + for (const auto& [canonicalizedName, columnHandle] : columnHandles) { + auto handle = + std::dynamic_pointer_cast(columnHandle); + VELOX_CHECK_NOT_NULL( + handle, + "ColumnHandle must be an instance of HiveColumnHandle for {}", + canonicalizedName); + switch (handle->columnType()) { + case HiveColumnHandle::ColumnType::kRegular: + break; + case HiveColumnHandle::ColumnType::kPartitionKey: + partitionKeys_.emplace(handle->name(), handle); + break; + case HiveColumnHandle::ColumnType::kSynthesized: + infoColumns_.emplace(handle->name(), handle); + break; + case HiveColumnHandle::ColumnType::kRowIndex: + specialColumns_.rowIndex = handle->name(); + break; + case HiveColumnHandle::ColumnType::kRowId: + specialColumns_.rowId = handle->name(); + break; + } + } + + std::vector readColumnNames; + auto readColumnTypes = outputType_->children(); + for (const auto& outputName : outputType_->names()) { + auto it = columnHandles.find(outputName); + VELOX_CHECK( + it != columnHandles.end(), + "ColumnHandle is missing for output column: {}", + outputName); + + auto* handle = static_cast(it->second.get()); + readColumnNames.push_back(handle->name()); + for (auto& subfield : handle->requiredSubfields()) { + VELOX_USER_CHECK_EQ( + getColumnName(subfield), + handle->name(), + "Required subfield does not match column name"); + subfields_[handle->name()].push_back(&subfield); + } + } + + hiveTableHandle_ = + std::dynamic_pointer_cast(tableHandle); + VELOX_CHECK_NOT_NULL( + hiveTableHandle_, "TableHandle must be an instance of HiveTableHandle"); + if (hiveConfig_->isFileColumnNamesReadAsLowerCase( + connectorQueryCtx->sessionProperties())) { + checkColumnNameLowerCase(outputType_); + checkColumnNameLowerCase(hiveTableHandle_->subfieldFilters(), infoColumns_); + checkColumnNameLowerCase(hiveTableHandle_->remainingFilter()); + } + + for (const auto& [k, v] : hiveTableHandle_->subfieldFilters()) { + filters_.emplace(k.clone(), v); + } + double sampleRate = 1; + auto remainingFilter = extractFiltersFromRemainingFilter( + hiveTableHandle_->remainingFilter(), + expressionEvaluator_, + false, + filters_, + sampleRate); + if (sampleRate != 1) { + randomSkip_ = std::make_shared(sampleRate); + } + + if (remainingFilter) { + remainingFilterExprSet_ = expressionEvaluator_->compile(remainingFilter); + auto& remainingFilterExpr = remainingFilterExprSet_->expr(0); + folly::F14FastMap columnNames; + for (int i = 0; i < readColumnNames.size(); ++i) { + columnNames[readColumnNames[i]] = i; + } + for (auto& input : remainingFilterExpr->distinctFields()) { + auto it = columnNames.find(input->field()); + if (it != columnNames.end()) { + if (shouldEagerlyMaterialize(*remainingFilterExpr, *input)) { + multiReferencedFields_.push_back(it->second); + } + continue; + } + // Remaining filter may reference columns that are not used otherwise, + // e.g. are not being projected out and are not used in range filters. + // Make sure to add these columns to readerOutputType_. + readColumnNames.push_back(input->field()); + readColumnTypes.push_back(input->type()); + } + remainingFilterSubfields_ = remainingFilterExpr->extractSubfields(); + if (VLOG_IS_ON(1)) { + VLOG(1) << fmt::format( + "Extracted subfields from remaining filter: [{}]", + fmt::join(remainingFilterSubfields_, ", ")); + } + for (auto& subfield : remainingFilterSubfields_) { + const auto& name = getColumnName(subfield); + auto it = subfields_.find(name); + if (it != subfields_.end()) { + // Some subfields of the column are already projected out, we append the + // remainingFilter subfield + it->second.push_back(&subfield); + } else if (columnNames.count(name) == 0) { + // remainingFilter subfield's column is not projected out, we add the + // column and append the subfield + subfields_[name].push_back(&subfield); + } + } + } + + readerOutputType_ = + ROW(std::move(readColumnNames), std::move(readColumnTypes)); + scanSpec_ = makeScanSpec( + readerOutputType_, + subfields_, + filters_, + hiveTableHandle_->dataColumns(), + partitionKeys_, + infoColumns_, + specialColumns_, + hiveConfig_->readStatsBasedFilterReorderDisabled( + connectorQueryCtx_->sessionProperties()), + pool_); + if (remainingFilter) { + metadataFilter_ = std::make_shared( + *scanSpec_, *remainingFilter, expressionEvaluator_); + } + + ioStats_ = std::make_shared(); + fsStats_ = std::make_shared(); +} + +std::unique_ptr HiveDataSource::createSplitReader() { + return SplitReader::create( + split_, + hiveTableHandle_, + &partitionKeys_, + connectorQueryCtx_, + hiveConfig_, + readerOutputType_, + ioStats_, + fsStats_, + fileHandleFactory_, + executor_, + scanSpec_, + expressionEvaluator_, + totalRemainingFilterTime_); +} + +std::vector HiveDataSource::setupBucketConversion() { + VELOX_CHECK_NE( + split_->bucketConversion->tableBucketCount, + split_->bucketConversion->partitionBucketCount); + VELOX_CHECK(split_->tableBucketNumber.has_value()); + VELOX_CHECK_NOT_NULL(hiveTableHandle_->dataColumns()); + ++numBucketConversion_; + bool rebuildScanSpec = false; + std::vector names; + std::vector types; + std::vector bucketChannels; + for (auto& handle : split_->bucketConversion->bucketColumnHandles) { + VELOX_CHECK(handle->columnType() == HiveColumnHandle::ColumnType::kRegular); + if (subfields_.erase(handle->name()) > 0) { + rebuildScanSpec = true; + } + auto index = readerOutputType_->getChildIdxIfExists(handle->name()); + if (!index.has_value()) { + if (names.empty()) { + names = readerOutputType_->names(); + types = readerOutputType_->children(); + } + index = names.size(); + names.push_back(handle->name()); + types.push_back( + hiveTableHandle_->dataColumns()->findChild(handle->name())); + rebuildScanSpec = true; + } + bucketChannels.push_back(*index); + } + if (!names.empty()) { + readerOutputType_ = ROW(std::move(names), std::move(types)); + } + if (rebuildScanSpec) { + auto newScanSpec = makeScanSpec( + readerOutputType_, + subfields_, + filters_, + hiveTableHandle_->dataColumns(), + partitionKeys_, + infoColumns_, + specialColumns_, + hiveConfig_->readStatsBasedFilterReorderDisabled( + connectorQueryCtx_->sessionProperties()), + pool_); + newScanSpec->moveAdaptationFrom(*scanSpec_); + scanSpec_ = std::move(newScanSpec); + } + return bucketChannels; +} + +void HiveDataSource::setupRowIdColumn() { + VELOX_CHECK(split_->rowIdProperties.has_value()); + const auto& props = *split_->rowIdProperties; + auto* rowId = scanSpec_->childByName(*specialColumns_.rowId); + VELOX_CHECK_NOT_NULL(rowId); + auto& rowIdType = + readerOutputType_->findChild(*specialColumns_.rowId)->asRow(); + auto rowGroupId = split_->getFileName(); + rowId->childByName(rowIdType.nameOf(1)) + ->setConstantValue( + StringView(rowGroupId), VARCHAR(), connectorQueryCtx_->memoryPool()); + rowId->childByName(rowIdType.nameOf(2)) + ->setConstantValue( + props.metadataVersion, BIGINT(), connectorQueryCtx_->memoryPool()); + rowId->childByName(rowIdType.nameOf(3)) + ->setConstantValue( + props.partitionId, BIGINT(), connectorQueryCtx_->memoryPool()); + rowId->childByName(rowIdType.nameOf(4)) + ->setConstantValue( + StringView(props.tableGuid), + VARCHAR(), + connectorQueryCtx_->memoryPool()); +} + +void HiveDataSource::addSplit(std::shared_ptr split) { + VELOX_CHECK_NULL( + split_, + "Previous split has not been processed yet. Call next to process the split."); + split_ = std::dynamic_pointer_cast(split); + VELOX_CHECK_NOT_NULL(split_, "Wrong type of split"); + + VLOG(1) << "Adding split " << split_->toString(); + + if (splitReader_) { + splitReader_.reset(); + } + + std::vector bucketChannels; + if (split_->bucketConversion.has_value()) { + bucketChannels = setupBucketConversion(); + } + if (specialColumns_.rowId.has_value()) { + setupRowIdColumn(); + } + + splitReader_ = createSplitReader(); + + if (!bucketChannels.empty()) { + splitReader_->setBucketConversion(std::move(bucketChannels)); + } + // Split reader subclasses may need to use the reader options in prepareSplit + // so we initialize it beforehand. + splitReader_->configureReaderOptions(randomSkip_); + splitReader_->prepareSplit(metadataFilter_, runtimeStats_); + readerOutputType_ = splitReader_->readerOutputType(); +} + +std::optional HiveDataSource::next( + uint64_t size, + velox::ContinueFuture& /*future*/) { + VELOX_CHECK(split_ != nullptr, "No split to process. Call addSplit first."); + VELOX_CHECK_NOT_NULL(splitReader_, "No split reader present"); + + TestValue::adjust( + "facebook::velox::connector::lakehouse::common::HiveDataSource::next", this); + + if (splitReader_->emptySplit()) { + resetSplit(); + return nullptr; + } + + // Bucket conversion or delta update could add extra column to reader output. + auto needsExtraColumn = [&] { + return output_->asUnchecked()->childrenSize() < + readerOutputType_->size(); + }; + if (!output_ || needsExtraColumn()) { + output_ = BaseVector::create(readerOutputType_, 0, pool_); + } + + const auto rowsScanned = splitReader_->next(size, output_); + completedRows_ += rowsScanned; + if (rowsScanned == 0) { + splitReader_->updateRuntimeStats(runtimeStats_); + resetSplit(); + return nullptr; + } + + VELOX_CHECK( + !output_->mayHaveNulls(), "Top-level row vector cannot have nulls"); + auto rowsRemaining = output_->size(); + if (rowsRemaining == 0) { + // no rows passed the pushed down filters. + return getEmptyOutput(); + } + + auto rowVector = std::dynamic_pointer_cast(output_); + + // In case there is a remaining filter that excludes some but not all + // rows, collect the indices of the passing rows. If there is no filter, + // or it passes on all rows, leave this as null and let exec::wrap skip + // wrapping the results. + BufferPtr remainingIndices; + filterRows_.resize(rowVector->size()); + + if (remainingFilterExprSet_) { + rowsRemaining = evaluateRemainingFilter(rowVector); + VELOX_CHECK_LE(rowsRemaining, rowsScanned); + if (rowsRemaining == 0) { + // No rows passed the remaining filter. + return getEmptyOutput(); + } + + if (rowsRemaining < rowVector->size()) { + // Some, but not all rows passed the remaining filter. + remainingIndices = filterEvalCtx_.selectedIndices; + } + } + + if (outputType_->size() == 0) { + return exec::wrap(rowsRemaining, remainingIndices, rowVector); + } + + std::vector outputColumns; + outputColumns.reserve(outputType_->size()); + for (int i = 0; i < outputType_->size(); ++i) { + auto& child = rowVector->childAt(i); + if (remainingIndices) { + // Disable dictionary values caching in expression eval so that we + // don't need to reallocate the result for every batch. + child->disableMemo(); + } + outputColumns.emplace_back( + exec::wrapChild(rowsRemaining, remainingIndices, child)); + } + + return std::make_shared( + pool_, outputType_, BufferPtr(nullptr), rowsRemaining, outputColumns); +} + +void HiveDataSource::addDynamicFilter( + column_index_t outputChannel, + const std::shared_ptr& filter) { + auto& fieldSpec = scanSpec_->getChildByChannel(outputChannel); + fieldSpec.setFilter(filter); + scanSpec_->resetCachedValues(true); + if (splitReader_) { + splitReader_->resetFilterCaches(); + } +} + +std::unordered_map HiveDataSource::runtimeStats() { + auto res = runtimeStats_.toMap(); + res.insert( + {{"numPrefetch", RuntimeCounter(ioStats_->prefetch().count())}, + {"prefetchBytes", + RuntimeCounter( + ioStats_->prefetch().sum(), RuntimeCounter::Unit::kBytes)}, + {"totalScanTime", + RuntimeCounter( + ioStats_->totalScanTime(), RuntimeCounter::Unit::kNanos)}, + {"totalRemainingFilterTime", + RuntimeCounter( + totalRemainingFilterTime_.load(std::memory_order_relaxed), + RuntimeCounter::Unit::kNanos)}, + {"ioWaitWallNanos", + RuntimeCounter( + ioStats_->queryThreadIoLatency().sum() * 1000, + RuntimeCounter::Unit::kNanos)}, + {"maxSingleIoWaitWallNanos", + RuntimeCounter( + ioStats_->queryThreadIoLatency().max() * 1000, + RuntimeCounter::Unit::kNanos)}, + {"overreadBytes", + RuntimeCounter( + ioStats_->rawOverreadBytes(), RuntimeCounter::Unit::kBytes)}}); + if (ioStats_->read().count() > 0) { + res.insert({"numStorageRead", RuntimeCounter(ioStats_->read().count())}); + res.insert( + {"storageReadBytes", + RuntimeCounter(ioStats_->read().sum(), RuntimeCounter::Unit::kBytes)}); + } + if (ioStats_->ssdRead().count() > 0) { + res.insert({"numLocalRead", RuntimeCounter(ioStats_->ssdRead().count())}); + res.insert( + {"localReadBytes", + RuntimeCounter( + ioStats_->ssdRead().sum(), RuntimeCounter::Unit::kBytes)}); + } + if (ioStats_->ramHit().count() > 0) { + res.insert({"numRamRead", RuntimeCounter(ioStats_->ramHit().count())}); + res.insert( + {"ramReadBytes", + RuntimeCounter( + ioStats_->ramHit().sum(), RuntimeCounter::Unit::kBytes)}); + } + if (numBucketConversion_ > 0) { + res.insert({"numBucketConversion", RuntimeCounter(numBucketConversion_)}); + } + + const auto fsStats = fsStats_->stats(); + for (const auto& storageStats : fsStats) { + res.emplace( + storageStats.first, + RuntimeCounter(storageStats.second.sum, storageStats.second.unit)); + } + return res; +} + +void HiveDataSource::setFromDataSource( + std::unique_ptr sourceUnique) { + auto source = dynamic_cast(sourceUnique.get()); + VELOX_CHECK_NOT_NULL(source, "Bad DataSource type"); + + split_ = std::move(source->split_); + runtimeStats_.skippedSplits += source->runtimeStats_.skippedSplits; + runtimeStats_.processedSplits += source->runtimeStats_.processedSplits; + runtimeStats_.skippedSplitBytes += source->runtimeStats_.skippedSplitBytes; + readerOutputType_ = std::move(source->readerOutputType_); + source->scanSpec_->moveAdaptationFrom(*scanSpec_); + scanSpec_ = std::move(source->scanSpec_); + splitReader_ = std::move(source->splitReader_); + splitReader_->setConnectorQueryCtx(connectorQueryCtx_); + // New io will be accounted on the stats of 'source'. Add the existing + // balance to that. + source->ioStats_->merge(*ioStats_); + ioStats_ = std::move(source->ioStats_); + source->fsStats_->merge(*fsStats_); + fsStats_ = std::move(source->fsStats_); + + numBucketConversion_ += source->numBucketConversion_; +} + +int64_t HiveDataSource::estimatedRowSize() { + if (!splitReader_) { + return kUnknownRowSize; + } + return splitReader_->estimatedRowSize(); +} + +vector_size_t HiveDataSource::evaluateRemainingFilter(RowVectorPtr& rowVector) { + for (auto fieldIndex : multiReferencedFields_) { + LazyVector::ensureLoadedRows( + rowVector->childAt(fieldIndex), + filterRows_, + filterLazyDecoded_, + filterLazyBaseRows_); + } + uint64_t filterTimeUs{0}; + vector_size_t rowsRemaining{0}; + { + MicrosecondTimer timer(&filterTimeUs); + expressionEvaluator_->evaluate( + remainingFilterExprSet_.get(), filterRows_, *rowVector, filterResult_); + rowsRemaining = exec::processFilterResults( + filterResult_, filterRows_, filterEvalCtx_, pool_); + } + totalRemainingFilterTime_.fetch_add( + filterTimeUs * 1000, std::memory_order_relaxed); + return rowsRemaining; +} + +void HiveDataSource::resetSplit() { + split_.reset(); + splitReader_->resetSplit(); + // Keep readers around to hold adaptation. +} + +HiveDataSource::WaveDelegateHookFunction HiveDataSource::waveDelegateHook_; + +std::shared_ptr HiveDataSource::toWaveDataSource() { + VELOX_CHECK_NOT_NULL(waveDelegateHook_); + if (!waveDataSource_) { + waveDataSource_ = waveDelegateHook_( + hiveTableHandle_, + scanSpec_, + readerOutputType_, + &partitionKeys_, + fileHandleFactory_, + executor_, + connectorQueryCtx_, + hiveConfig_, + ioStats_, + remainingFilterExprSet_.get(), + metadataFilter_); + } + return waveDataSource_; +} + +// static +void HiveDataSource::registerWaveDelegateHook(WaveDelegateHookFunction hook) { + waveDelegateHook_ = hook; +} +std::shared_ptr toWaveDataSource(); + +} // namespace facebook::velox::connector::lakehouse::common diff --git a/velox/connectors/lakehouse/common/HiveDataSource.h b/velox/connectors/lakehouse/common/HiveDataSource.h new file mode 100644 index 000000000000..e4626f7bf483 --- /dev/null +++ b/velox/connectors/lakehouse/common/HiveDataSource.h @@ -0,0 +1,187 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#pragma once + +#include "velox/common/base/RandomUtil.h" +#include "velox/common/file/FileSystems.h" +#include "velox/common/io/IoStatistics.h" +#include "velox/connectors/Connector.h" +#include "velox/connectors/lakehouse/common/FileHandle.h" +#include "velox/connectors/lakehouse/common/HiveConnectorSplit.h" +#include "velox/connectors/lakehouse/common/HiveConnectorUtil.h" +#include "velox/connectors/lakehouse/common/SplitReader.h" +#include "velox/connectors/lakehouse/common/TableHandle.h" +#include "velox/dwio/common/Statistics.h" +#include "velox/exec/OperatorUtils.h" +#include "velox/expression/Expr.h" + +namespace facebook::velox::connector::lakehouse::common { + +class HiveConfig; + +class HiveDataSource : public DataSource { + public: + HiveDataSource( + const RowTypePtr& outputType, + const connector::ConnectorTableHandlePtr& tableHandle, + const connector::ColumnHandleMap& columnHandles, + FileHandleFactory* fileHandleFactory, + folly::Executor* executor, + const ConnectorQueryCtx* connectorQueryCtx, + const std::shared_ptr& hiveConfig); + + void addSplit(std::shared_ptr split) override; + + std::optional next(uint64_t size, velox::ContinueFuture& future) + override; + + void addDynamicFilter( + column_index_t outputChannel, + const std::shared_ptr& filter) override; + + uint64_t getCompletedBytes() override { + return ioStats_->rawBytesRead(); + } + + uint64_t getCompletedRows() override { + return completedRows_; + } + + std::unordered_map runtimeStats() override; + + bool allPrefetchIssued() const override { + return splitReader_ && splitReader_->allPrefetchIssued(); + } + + void setFromDataSource(std::unique_ptr sourceUnique) override; + + int64_t estimatedRowSize() override; + + const velox::common::SubfieldFilters* getFilters() const override { + return &filters_; + } + + std::shared_ptr toWaveDataSource() override; + + using WaveDelegateHookFunction = + std::function( + const HiveTableHandlePtr& hiveTableHandle, + const std::shared_ptr& scanSpec, + const RowTypePtr& readerOutputType, + std::unordered_map* partitionKeys, + FileHandleFactory* fileHandleFactory, + folly::Executor* executor, + const ConnectorQueryCtx* connectorQueryCtx, + const std::shared_ptr& hiveConfig, + const std::shared_ptr& ioStats, + const exec::ExprSet* remainingFilter, + std::shared_ptr metadataFilter)>; + + static WaveDelegateHookFunction waveDelegateHook_; + + static void registerWaveDelegateHook(WaveDelegateHookFunction hook); + + const ConnectorQueryCtx* testingConnectorQueryCtx() const { + return connectorQueryCtx_; + } + + protected: + virtual std::unique_ptr createSplitReader(); + + FileHandleFactory* const fileHandleFactory_; + folly::Executor* const executor_; + const ConnectorQueryCtx* const connectorQueryCtx_; + const std::shared_ptr hiveConfig_; + memory::MemoryPool* const pool_; + + std::shared_ptr split_; + HiveTableHandlePtr hiveTableHandle_; + std::shared_ptr scanSpec_; + VectorPtr output_; + std::unique_ptr splitReader_; + + // Output type from file reader. This is different from outputType_ that it + // contains column names before assignment, and columns that only used in + // remaining filter. + RowTypePtr readerOutputType_; + + // Column handles for the partition key columns keyed on partition key column + // name. + std::unordered_map partitionKeys_; + + std::shared_ptr ioStats_; + std::shared_ptr fsStats_; + + private: + std::vector setupBucketConversion(); + + void setupRowIdColumn(); + + // Evaluates remainingFilter_ on the specified vector. Returns number of rows + // passed. Populates filterEvalCtx_.selectedIndices and selectedBits if only + // some rows passed the filter. If none or all rows passed + // filterEvalCtx_.selectedIndices and selectedBits are not updated. + vector_size_t evaluateRemainingFilter(RowVectorPtr& rowVector); + + // Clear split_ after split has been fully processed. Keep readers around to + // hold adaptation. + void resetSplit(); + + const RowVectorPtr& getEmptyOutput() { + if (!emptyOutput_) { + emptyOutput_ = RowVector::createEmpty(outputType_, pool_); + } + return emptyOutput_; + } + + // The row type for the data source output, not including filter-only columns + const RowTypePtr outputType_; + core::ExpressionEvaluator* const expressionEvaluator_; + + // Column handles for the Split info columns keyed on their column names. + std::unordered_map infoColumns_; + SpecialColumnNames specialColumns_{}; + std::vector remainingFilterSubfields_; + folly::F14FastMap> + subfields_; + velox::common::SubfieldFilters filters_; + std::shared_ptr metadataFilter_; + std::shared_ptr remainingFilterExprSet_; + RowVectorPtr emptyOutput_; + dwio::common::RuntimeStatistics runtimeStats_; + std::atomic totalRemainingFilterTime_{0}; + uint64_t completedRows_ = 0; + + // Field indices referenced in both remaining filter and output type. These + // columns need to be materialized eagerly to avoid missing values in output. + std::vector multiReferencedFields_; + + std::shared_ptr randomSkip_; + + int64_t numBucketConversion_ = 0; + + // Reusable memory for remaining filter evaluation. + VectorPtr filterResult_; + SelectivityVector filterRows_; + DecodedVector filterLazyDecoded_; + SelectivityVector filterLazyBaseRows_; + exec::FilterEvalCtx filterEvalCtx_; + + // Remembers the WaveDataSource. Successive calls to toWaveDataSource() will + // return the same. + std::shared_ptr waveDataSource_; +}; +} // namespace facebook::velox::connector::lakehouse::common diff --git a/velox/connectors/lakehouse/common/HivePartitionFunction.cpp b/velox/connectors/lakehouse/common/HivePartitionFunction.cpp new file mode 100644 index 000000000000..d0563d7f10c7 --- /dev/null +++ b/velox/connectors/lakehouse/common/HivePartitionFunction.cpp @@ -0,0 +1,579 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#include "velox/connectors/lakehouse/common/HivePartitionFunction.h" + +namespace facebook::velox::connector::lakehouse::common { + +namespace { +void mergeHash(bool mix, uint32_t oneHash, uint32_t& aggregateHash) { + aggregateHash = mix ? aggregateHash * 31 + oneHash : oneHash; +} + +int32_t hashInt64(int64_t value) { + return ((*reinterpret_cast(&value)) >> 32) ^ value; +} + +#if defined(__has_feature) +#if __has_feature(__address_sanitizer__) +__attribute__((no_sanitize("integer"))) +#endif +#endif +uint32_t +hashBytes(StringView bytes, int32_t initialValue) { + uint32_t hash = initialValue; + auto* data = bytes.data(); + for (auto i = 0; i < bytes.size(); ++i) { + hash = hash * 31 + *reinterpret_cast(data + i); + } + return hash; +} + +int32_t hashTimestamp(const Timestamp& ts) { + return hashInt64((ts.getSeconds() << 30) | ts.getNanos()); +} + +template +inline uint32_t hashOne(typename TypeTraits::NativeType /* value */) { + VELOX_UNSUPPORTED( + "Hive partitioning function doesn't support {} type", + TypeTraits::name); + return 0; // Make compiler happy. +} + +template <> +inline uint32_t hashOne(bool value) { + return value ? 1 : 0; +} + +template <> +inline uint32_t hashOne(int8_t value) { + return static_cast(value); +} + +template <> +inline uint32_t hashOne(int16_t value) { + return static_cast(value); +} + +template <> +inline uint32_t hashOne(int32_t value) { + return static_cast(value); +} + +template <> +inline uint32_t hashOne(float value) { + return static_cast(*reinterpret_cast(&value)); +} + +template <> +inline uint32_t hashOne(int64_t value) { + return hashInt64(value); +} + +template <> +inline uint32_t hashOne(double value) { + return hashInt64(*reinterpret_cast(&value)); +} + +template <> +inline uint32_t hashOne(StringView value) { + return hashBytes(value, 0); +} + +template <> +inline uint32_t hashOne(StringView value) { + return hashBytes(value, 0); +} + +template <> +inline uint32_t hashOne(Timestamp value) { + return hashTimestamp(value); +} + +template <> +inline uint32_t hashOne(UnknownValue /*value*/) { + VELOX_FAIL("Unknown values cannot be non-NULL"); +} + +template +void hashPrimitive( + const DecodedVector& values, + const SelectivityVector& rows, + bool mix, + std::vector& hashes) { + if (rows.isAllSelected()) { + // The compiler seems to be a little fickle with optimizations. + // Although rows.applyToSelected should do roughly the same thing, doing + // this here along with assigning rows.size() to a variable seems to help + // the compiler to inline hashOne showing a 50% performance improvement in + // benchmarks. + vector_size_t numRows = rows.size(); + for (auto i = 0; i < numRows; ++i) { + const uint32_t hash = values.isNullAt(i) + ? 0 + : hashOne( + values.valueAt::NativeType>(i)); + mergeHash(mix, hash, hashes[i]); + } + } else { + rows.applyToSelected([&](auto row) INLINE_LAMBDA { + const uint32_t hash = values.isNullAt(row) + ? 0 + : hashOne( + values.valueAt::NativeType>(row)); + mergeHash(mix, hash, hashes[row]); + }); + } +} + +void hashPrecomputed( + uint32_t precomputedHash, + vector_size_t numRows, + bool mix, + std::vector& hashes) { + for (auto i = 0; i < numRows; ++i) { + hashes[i] = mix ? hashes[i] * 31 + precomputedHash : precomputedHash; + } +} +} // namespace + +template <> +void HivePartitionFunction::hashTyped( + const DecodedVector& values, + const SelectivityVector& rows, + bool mix, + std::vector& hashes, + size_t /* poolIndex */) { + hashPrimitive(values, rows, mix, hashes); +} + +template <> +void HivePartitionFunction::hashTyped( + const DecodedVector& values, + const SelectivityVector& rows, + bool mix, + std::vector& hashes, + size_t /* poolIndex */) { + hashPrimitive(values, rows, mix, hashes); +} + +template <> +void HivePartitionFunction::hashTyped( + const DecodedVector& values, + const SelectivityVector& rows, + bool mix, + std::vector& hashes, + size_t /* poolIndex */) { + hashPrimitive(values, rows, mix, hashes); +} + +template <> +void HivePartitionFunction::hashTyped( + const DecodedVector& values, + const SelectivityVector& rows, + bool mix, + std::vector& hashes, + size_t /* poolIndex */) { + hashPrimitive(values, rows, mix, hashes); +} + +template <> +void HivePartitionFunction::hashTyped( + const DecodedVector& values, + const SelectivityVector& rows, + bool mix, + std::vector& hashes, + size_t /* poolIndex */) { + hashPrimitive(values, rows, mix, hashes); +} + +template <> +void HivePartitionFunction::hashTyped( + const DecodedVector& values, + const SelectivityVector& rows, + bool mix, + std::vector& hashes, + size_t /* poolIndex */) { + hashPrimitive(values, rows, mix, hashes); +} + +template <> +void HivePartitionFunction::hashTyped( + const DecodedVector& values, + const SelectivityVector& rows, + bool mix, + std::vector& hashes, + size_t /* poolIndex */) { + hashPrimitive(values, rows, mix, hashes); +} + +template <> +void HivePartitionFunction::hashTyped( + const DecodedVector& values, + const SelectivityVector& rows, + bool mix, + std::vector& hashes, + size_t /* poolIndex */) { + hashPrimitive(values, rows, mix, hashes); +} + +template <> +void HivePartitionFunction::hashTyped( + const DecodedVector& values, + const SelectivityVector& rows, + bool mix, + std::vector& hashes, + size_t /* poolIndex */) { + hashPrimitive(values, rows, mix, hashes); +} + +template <> +void HivePartitionFunction::hashTyped( + const DecodedVector& values, + const SelectivityVector& rows, + bool mix, + std::vector& hashes, + size_t /* poolIndex */) { + hashPrimitive(values, rows, mix, hashes); +} + +template <> +void HivePartitionFunction::hashTyped( + const DecodedVector& values, + const SelectivityVector& rows, + bool mix, + std::vector& hashes, + size_t /* poolIndex */) { + hashPrimitive(values, rows, mix, hashes); +} + +template <> +void HivePartitionFunction::hashTyped( + const DecodedVector& /*values*/, + const SelectivityVector& /*rows*/, + bool /*mix*/, + std::vector& /*hashes*/, + size_t /* poolIndex */) { + VELOX_UNSUPPORTED("Hive partitioning function doesn't support OPAQUE type"); +} + +template <> +void HivePartitionFunction::hashTyped( + const DecodedVector& values, + const SelectivityVector& rows, + bool mix, + std::vector& hashes, + size_t poolIndex) { + auto& elementsDecoded = getDecodedVector(poolIndex); + auto& elementsRows = getRows(poolIndex); + auto& elementsHashes = getHashes(poolIndex); + + const auto* arrayVector = values.base()->as(); + const vector_size_t elementsSize = arrayVector->elements()->size(); + elementsRows.resizeFill(elementsSize, false); + elementsHashes.resize(elementsSize); + + rows.applyToSelected([&](auto row) { + if (!values.isNullAt(row)) { + const auto index = values.index(row); + const auto offset = arrayVector->offsetAt(index); + const auto length = arrayVector->sizeAt(index); + + elementsRows.setValidRange(offset, offset + length, true); + } + }); + + elementsRows.updateBounds(); + + elementsDecoded.decode(*arrayVector->elements(), elementsRows); + + hash( + elementsDecoded, + elementsDecoded.base()->typeKind(), + elementsRows, + false, + elementsHashes, + poolIndex + 1); + + rows.applyToSelected([&](auto row) { + uint32_t hash = 0; + + if (!values.isNullAt(row)) { + const auto index = values.index(row); + const auto offset = arrayVector->offsetAt(index); + const auto length = arrayVector->sizeAt(index); + + for (size_t i = offset; i < offset + length; ++i) { + mergeHash(true, elementsHashes[i], hash); + } + } + + mergeHash(mix, hash, hashes[row]); + }); +} + +template <> +void HivePartitionFunction::hashTyped( + const DecodedVector& values, + const SelectivityVector& rows, + bool mix, + std::vector& hashes, + size_t poolIndex) { + auto& valuesDecoded = getDecodedVector(poolIndex); + auto& keysDecoded = getDecodedVector(poolIndex + 1); + auto& elementsRows = getRows(poolIndex); + auto& valuesHashes = getHashes(poolIndex); + auto& keysHashes = getHashes(poolIndex + 1); + + const auto* mapVector = values.base()->as(); + const vector_size_t elementsSize = mapVector->mapKeys()->size(); + elementsRows.resizeFill(elementsSize, false); + keysHashes.resize(elementsSize); + valuesHashes.resize(elementsSize); + + rows.applyToSelected([&](auto row) { + if (!values.isNullAt(row)) { + const auto index = values.index(row); + const auto offset = mapVector->offsetAt(index); + const auto length = mapVector->sizeAt(index); + + elementsRows.setValidRange(offset, offset + length, true); + } + }); + + elementsRows.updateBounds(); + + keysDecoded.decode(*mapVector->mapKeys(), elementsRows); + valuesDecoded.decode(*mapVector->mapValues(), elementsRows); + + hash( + keysDecoded, + keysDecoded.base()->typeKind(), + elementsRows, + false, + keysHashes, + poolIndex + 2); + + hash( + valuesDecoded, + valuesDecoded.base()->typeKind(), + elementsRows, + false, + valuesHashes, + poolIndex + 2); + + rows.applyToSelected([&](auto row) { + uint32_t hash = 0; + + if (!values.isNullAt(row)) { + const auto index = values.index(row); + const auto offset = mapVector->offsetAt(index); + const auto length = mapVector->sizeAt(index); + + for (size_t i = offset; i < offset + length; ++i) { + hash += keysHashes[i] ^ valuesHashes[i]; + } + } + + mergeHash(mix, hash, hashes[row]); + }); +} + +template <> +void HivePartitionFunction::hashTyped( + const DecodedVector& values, + const SelectivityVector& rows, + bool mix, + std::vector& hashes, + size_t poolIndex) { + auto& childDecodedVector = getDecodedVector(poolIndex); + auto& childRows = getRows(poolIndex); + auto& childHashes = getHashes(poolIndex); + + const auto* rowVector = values.base()->as(); + childRows.resizeFill(rowVector->size(), false); + childHashes.resize(rowVector->size()); + + rows.applyToSelected([&](auto row) { + if (!values.isNullAt(row)) { + childRows.setValid(values.index(row), true); + } + }); + + childRows.updateBounds(); + + for (vector_size_t i = 0; i < rowVector->childrenSize(); ++i) { + auto& child = rowVector->childAt(i); + childDecodedVector.decode(*child, childRows); + hash( + childDecodedVector, + child->typeKind(), + childRows, + i > 0, + childHashes, + poolIndex + 1); + } + + rows.applyToSelected([&](auto row) { + mergeHash( + mix, + values.isNullAt(row) ? 0 : childHashes[values.index(row)], + hashes[row]); + }); +} + +void HivePartitionFunction::hash( + const DecodedVector& values, + TypeKind typeKind, + const SelectivityVector& rows, + bool mix, + std::vector& hashes, + size_t poolIndex) { + // This function mirrors the behavior of function hashCode in + // HIVE-12025 ba83fd7bff + // serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspectorUtils.java + // https://github.com/apache/hive/blob/ba83fd7bff/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspectorUtils.java + + // HIVE-7148 proposed change to bucketing hash algorithms. If that + // gets implemented, this function will need to change + // significantly. + + VELOX_DYNAMIC_TYPE_DISPATCH_ALL( + hashTyped, typeKind, values, rows, mix, hashes, poolIndex); +} + +HivePartitionFunction::HivePartitionFunction( + int numBuckets, + std::vector bucketToPartition, + std::vector keyChannels, + const std::vector& constValues) + : numBuckets_{numBuckets}, + bucketToPartition_{bucketToPartition}, + keyChannels_{std::move(keyChannels)} { + precomputedHashes_.resize(keyChannels_.size()); + size_t constChannel{0}; + for (auto i = 0; i < keyChannels_.size(); ++i) { + if (keyChannels_[i] == kConstantChannel) { + precompute(*(constValues[constChannel++]), i); + } + } +} + +std::optional HivePartitionFunction::partition( + const RowVector& input, + std::vector& partitions) { + const auto numRows = input.size(); + + auto& decodedVector = getDecodedVector(); + auto& rows = getRows(); + auto& hashes = getHashes(); + rows.resizeFill(numRows, true); + if (numRows > hashes.size()) { + hashes.resize(numRows); + } + partitions.resize(numRows); + for (auto i = 0; i < keyChannels_.size(); ++i) { + if (keyChannels_[i] != kConstantChannel) { + const auto& keyVector = input.childAt(keyChannels_[i]); + decodedVector.decode(*keyVector, rows); + hash(decodedVector, keyVector->typeKind(), rows, i > 0, hashes, 1); + } else { + hashPrecomputed(precomputedHashes_[i], numRows, i > 0, hashes); + } + } + + static const int32_t kInt32Max = std::numeric_limits::max(); + + if (bucketToPartition_.empty()) { + // NOTE: if bucket to partition mapping is empty, then we do + // identical mapping. + for (auto i = 0; i < numRows; ++i) { + partitions[i] = (hashes[i] & kInt32Max) % numBuckets_; + } + } else { + for (auto i = 0; i < numRows; ++i) { + partitions[i] = + bucketToPartition_[((hashes[i] & kInt32Max) % numBuckets_)]; + } + } + + return std::nullopt; +} + +void HivePartitionFunction::precompute( + const BaseVector& value, + size_t channelIndex) { + if (value.isNullAt(0)) { + precomputedHashes_[channelIndex] = 0; + return; + } + + const SelectivityVector rows(1, true); + DecodedVector& decodedVector = getDecodedVector(); + decodedVector.decode(value, rows); + + std::vector hashes{1}; + hash(decodedVector, value.typeKind(), rows, false, hashes, 1); + precomputedHashes_[channelIndex] = hashes[0]; +} + +DecodedVector& HivePartitionFunction::getDecodedVector(size_t poolIndex) { + while (poolIndex >= decodedVectorsPool_.size()) { + decodedVectorsPool_.push_back(std::make_unique()); + } + + return *decodedVectorsPool_[poolIndex]; +} + +SelectivityVector& HivePartitionFunction::getRows(size_t poolIndex) { + while (poolIndex >= rowsPool_.size()) { + rowsPool_.push_back(std::make_unique()); + } + + return *rowsPool_[poolIndex]; +} + +std::vector& HivePartitionFunction::getHashes(size_t poolIndex) { + while (poolIndex >= hashesPool_.size()) { + hashesPool_.push_back(std::make_unique>()); + } + + return *hashesPool_[poolIndex]; +} + +HiveIdentityPartitionFunction::HiveIdentityPartitionFunction( + int numBuckets, + column_index_t keyChannel) + : numBuckets_(numBuckets), keyChannel_(keyChannel) {} + +std::optional HiveIdentityPartitionFunction::partition( + const RowVector& input, + std::vector& partitions) { + const auto& keyVector = input.childAt(keyChannel_); + VELOX_CHECK( + keyVector->typeKind() == TypeKind::BIGINT, + "Only BIGINT type is supported for identity partition function"); + decodedVector_->decode(*keyVector); + + const auto numRows = input.size(); + partitions.resize(numRows); + for (auto i = 0; i < numRows; i++) { + partitions[i] = decodedVector_->valueAt(i) % numBuckets_; + } + return std::nullopt; +} + +} // namespace facebook::velox::connector::lakehouse::common diff --git a/velox/connectors/lakehouse/common/HivePartitionFunction.h b/velox/connectors/lakehouse/common/HivePartitionFunction.h new file mode 100644 index 000000000000..4d6566e45ea7 --- /dev/null +++ b/velox/connectors/lakehouse/common/HivePartitionFunction.h @@ -0,0 +1,109 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#pragma once + +#include "velox/core/PlanNode.h" +#include "velox/vector/DecodedVector.h" + +namespace facebook::velox::connector::lakehouse::common { + +class HivePartitionFunction : public core::PartitionFunction { + public: + HivePartitionFunction( + int numBuckets, + std::vector bucketToPartition, + std::vector keyChannels, + const std::vector& constValues = {}); + + HivePartitionFunction( + int numBuckets, + std::vector keyChannels, + const std::vector& constValues = {}) + : HivePartitionFunction( + numBuckets, + {}, + std::move(keyChannels), + constValues) {} + + ~HivePartitionFunction() override = default; + + std::optional partition( + const RowVector& input, + std::vector& partitions) override; + + const std::vector& testingBucketToPartition() const { + return bucketToPartition_; + } + + private: + // Precompute single value hive hash for a constant partition key. + void precompute(const BaseVector& value, size_t column_index_t); + + void hash( + const DecodedVector& values, + TypeKind typeKind, + const SelectivityVector& rows, + bool mix, + std::vector& hashes, + size_t poolIndex); + + template + void hashTyped( + const DecodedVector& /* values */, + const SelectivityVector& /* rows */, + bool /* mix */, + std::vector& /* hashes */, + size_t /* poolIndex */) { + VELOX_UNSUPPORTED( + "Hive partitioning function doesn't support {} type", + TypeTraits::name); + } + + // Helper functions to retrieve reusable memory from pools. + DecodedVector& getDecodedVector(size_t poolIndex = 0); + SelectivityVector& getRows(size_t poolIndex = 0); + std::vector& getHashes(size_t poolIndex = 0); + + const int numBuckets_; + const std::vector bucketToPartition_; + const std::vector keyChannels_; + + // Pools of reusable memory. + std::vector>> hashesPool_; + std::vector> rowsPool_; + std::vector> decodedVectorsPool_; + // Precomputed hashes for constant partition keys (one per key). + std::vector precomputedHashes_; +}; + +// PartitionFunction implementation which uses the value extracted +// from a column channel as the partition index. +class HiveIdentityPartitionFunction : public core::PartitionFunction { + public: + HiveIdentityPartitionFunction(int numBuckets, column_index_t keyChannel); + + std::optional partition( + const RowVector& input, + std::vector& partitions) override; + + private: + const int numBuckets_; + const column_index_t keyChannel_; + std::unique_ptr decodedVector_ = + std::make_unique(); +}; + +} // namespace facebook::velox::connector::lakehouse::common diff --git a/velox/connectors/lakehouse/common/HivePartitionUtil.cpp b/velox/connectors/lakehouse/common/HivePartitionUtil.cpp new file mode 100644 index 000000000000..49a315d7f78f --- /dev/null +++ b/velox/connectors/lakehouse/common/HivePartitionUtil.cpp @@ -0,0 +1,119 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "velox/connectors/lakehouse/common/HivePartitionUtil.h" +#include "velox/vector/SimpleVector.h" + +namespace facebook::velox::connector::lakehouse::common { + +#define PARTITION_TYPE_DISPATCH(TEMPLATE_FUNC, typeKind, ...) \ + [&]() { \ + switch (typeKind) { \ + case TypeKind::BOOLEAN: \ + case TypeKind::TINYINT: \ + case TypeKind::SMALLINT: \ + case TypeKind::INTEGER: \ + case TypeKind::BIGINT: \ + case TypeKind::VARCHAR: \ + case TypeKind::VARBINARY: \ + case TypeKind::TIMESTAMP: \ + return VELOX_DYNAMIC_SCALAR_TYPE_DISPATCH( \ + TEMPLATE_FUNC, typeKind, __VA_ARGS__); \ + default: \ + VELOX_UNSUPPORTED( \ + "Unsupported partition type: {}", mapTypeKindToName(typeKind)); \ + } \ + }() + +namespace { +template +inline std::string makePartitionValueString(T value) { + return folly::to(value); +} + +template <> +inline std::string makePartitionValueString(bool value) { + return value ? "true" : "false"; +} + +template <> +inline std::string makePartitionValueString(Timestamp value) { + value.toTimezone(Timestamp::defaultTimezone()); + TimestampToStringOptions options; + options.dateTimeSeparator = ' '; + // Set the precision to milliseconds, and enable the skipTrailingZeros match + // the timestamp precision and truncation behavior of Presto. + options.precision = TimestampPrecision::kMilliseconds; + options.skipTrailingZeros = true; + + auto result = value.toString(options); + + // Presto's java.sql.Timestamp.toString() always keeps at least one decimal + // place even when all fractional seconds are zero. + // If skipTrailingZeros removed all fractional digits, add back ".0" to match + // Presto's behavior. + if (auto dotPos = result.find_last_of('.'); dotPos == std::string::npos) { + // No decimal point found, add ".0" + result += ".0"; + } + + return result; +} + +template +std::pair makePartitionKeyValueString( + const BaseVector* partitionVector, + vector_size_t row, + const std::string& name, + bool isDate, + const std::string& nullValueName) { + using T = typename TypeTraits::NativeType; + if (partitionVector->as>()->isNullAt(row)) { + return std::make_pair(name, nullValueName); + } + if (isDate) { + return std::make_pair( + name, + DATE()->toString( + partitionVector->as>()->valueAt(row))); + } + return std::make_pair( + name, + makePartitionValueString( + partitionVector->as>()->valueAt(row))); +} + +} // namespace + +std::vector> extractPartitionKeyValues( + const RowVectorPtr& partitionsVector, + vector_size_t row, + const std::string& nullValueName) { + std::vector> partitionKeyValues; + for (auto i = 0; i < partitionsVector->childrenSize(); i++) { + partitionKeyValues.push_back(PARTITION_TYPE_DISPATCH( + makePartitionKeyValueString, + partitionsVector->childAt(i)->typeKind(), + partitionsVector->childAt(i)->loadedVector(), + row, + asRowType(partitionsVector->type())->nameOf(i), + partitionsVector->childAt(i)->type()->isDate(), + nullValueName)); + } + return partitionKeyValues; +} + +} // namespace facebook::velox::connector::lakehouse::common diff --git a/velox/connectors/lakehouse/common/HivePartitionUtil.h b/velox/connectors/lakehouse/common/HivePartitionUtil.h new file mode 100644 index 000000000000..6c9527242d6b --- /dev/null +++ b/velox/connectors/lakehouse/common/HivePartitionUtil.h @@ -0,0 +1,27 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#pragma once + +#include "velox/vector/ComplexVector.h" + +namespace facebook::velox::connector::lakehouse::common { + +std::vector> extractPartitionKeyValues( + const RowVectorPtr& partitionsVector, + vector_size_t row, + const std::string& nullValueName = ""); + +} // namespace facebook::velox::connector::lakehouse::common diff --git a/velox/connectors/lakehouse/common/PartitionIdGenerator.cpp b/velox/connectors/lakehouse/common/PartitionIdGenerator.cpp new file mode 100644 index 000000000000..eab5d3bacb63 --- /dev/null +++ b/velox/connectors/lakehouse/common/PartitionIdGenerator.cpp @@ -0,0 +1,185 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "velox/connectors/lakehouse/common/PartitionIdGenerator.h" + +#include "velox/connectors/lakehouse/common/HivePartitionUtil.h" +#include "velox/dwio/catalog/fbhive/FileUtils.h" + +using namespace facebook::velox::dwio::catalog::fbhive; + +namespace facebook::velox::connector::lakehouse::common { + +PartitionIdGenerator::PartitionIdGenerator( + const RowTypePtr& inputType, + std::vector partitionChannels, + uint32_t maxPartitions, + memory::MemoryPool* pool, + bool partitionPathAsLowerCase) + : pool_(pool), + partitionChannels_(std::move(partitionChannels)), + maxPartitions_(maxPartitions), + partitionPathAsLowerCase_(partitionPathAsLowerCase) { + VELOX_USER_CHECK( + !partitionChannels_.empty(), "There must be at least one partition key."); + for (auto channel : partitionChannels_) { + hashers_.emplace_back( + exec::VectorHasher::create(inputType->childAt(channel), channel)); + } + + std::vector partitionKeyTypes; + std::vector partitionKeyNames; + for (auto channel : partitionChannels_) { + VELOX_USER_CHECK( + exec::VectorHasher::typeKindSupportsValueIds( + inputType->childAt(channel)->kind()), + "Unsupported partition type: {}.", + inputType->childAt(channel)->toString()); + partitionKeyTypes.push_back(inputType->childAt(channel)); + partitionKeyNames.push_back(inputType->nameOf(channel)); + } + + partitionValues_ = BaseVector::create( + ROW(std::move(partitionKeyNames), std::move(partitionKeyTypes)), + maxPartitions_, + pool); + for (auto& key : partitionValues_->children()) { + key->resize(maxPartitions_); + } +} + +void PartitionIdGenerator::run( + const RowVectorPtr& input, + raw_vector& result) { + const auto numRows = input->size(); + result.resize(numRows); + + // Compute value IDs using VectorHashers and store these in 'result'. + computeValueIds(input, result); + + // Convert value IDs in 'result' into partition IDs using partitionIds + // mapping. Update 'result' in place. + + // TODO Optimize common use case where all records belong to the same + // partition. VectorHashers keep track of the number of unique values, hence, + // we can find out if there is only one unique value for each partition key. + for (auto i = 0; i < numRows; ++i) { + auto valueId = result[i]; + auto it = partitionIds_.find(valueId); + if (it != partitionIds_.end()) { + result[i] = it->second; + } else { + uint64_t nextPartitionId = partitionIds_.size(); + VELOX_USER_CHECK_LT( + nextPartitionId, + maxPartitions_, + "Exceeded limit of {} distinct partitions.", + maxPartitions_); + + partitionIds_.emplace(valueId, nextPartitionId); + savePartitionValues(nextPartitionId, input, i); + + result[i] = nextPartitionId; + } + } +} + +std::string PartitionIdGenerator::partitionName( + uint64_t partitionId, + const std::string& nullValueName) const { + return FileUtils::makePartName( + extractPartitionKeyValues(partitionValues_, partitionId, nullValueName), + partitionPathAsLowerCase_); +} + +void PartitionIdGenerator::computeValueIds( + const RowVectorPtr& input, + raw_vector& valueIds) { + allRows_.resize(input->size()); + allRows_.setAll(); + + bool rehash = false; + for (auto& hasher : hashers_) { + // NOTE: for boolean column type, computeValueIds() always returns true and + // this might cause problem in case of multiple boolean partition columns as + // we might not set the multiplier properly. + auto partitionVector = input->childAt(hasher->channel())->loadedVector(); + hasher->decode(*partitionVector, allRows_); + if (!hasher->computeValueIds(allRows_, valueIds)) { + rehash = true; + } + } + + if (!rehash && hasMultiplierSet_) { + return; + } + + uint64_t multiplier = 1; + for (auto& hasher : hashers_) { + hasMultiplierSet_ = true; + multiplier = hasher->typeKind() == TypeKind::BOOLEAN + ? hasher->enableValueRange(multiplier, 50) + : hasher->enableValueIds(multiplier, 50); + + VELOX_CHECK_NE( + multiplier, + exec::VectorHasher::kRangeTooLarge, + "Number of requested IDs is out of range."); + } + + for (auto& hasher : hashers_) { + const bool ok = hasher->computeValueIds(allRows_, valueIds); + VELOX_CHECK(ok); + } + + updateValueToPartitionIdMapping(); +} + +void PartitionIdGenerator::updateValueToPartitionIdMapping() { + if (partitionIds_.empty()) { + return; + } + + const auto numPartitions = partitionIds_.size(); + + partitionIds_.clear(); + + raw_vector newValueIds(numPartitions, pool_); + SelectivityVector rows(numPartitions); + for (auto i = 0; i < hashers_.size(); ++i) { + auto& hasher = hashers_[i]; + hasher->decode(*partitionValues_->childAt(i), rows); + const bool ok = hasher->computeValueIds(rows, newValueIds); + VELOX_CHECK(ok); + } + + for (auto i = 0; i < numPartitions; ++i) { + partitionIds_.emplace(newValueIds[i], i); + } +} + +void PartitionIdGenerator::savePartitionValues( + uint64_t partitionId, + const RowVectorPtr& input, + vector_size_t row) { + for (auto i = 0; i < partitionChannels_.size(); ++i) { + auto channel = partitionChannels_[i]; + partitionValues_->childAt(i)->copy( + input->childAt(channel).get(), partitionId, row, 1); + } +} + +} // namespace facebook::velox::connector::lakehouse::common diff --git a/velox/connectors/lakehouse/common/PartitionIdGenerator.h b/velox/connectors/lakehouse/common/PartitionIdGenerator.h new file mode 100644 index 000000000000..6ba17449805d --- /dev/null +++ b/velox/connectors/lakehouse/common/PartitionIdGenerator.h @@ -0,0 +1,102 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#pragma once + +#include "velox/exec/VectorHasher.h" + +namespace facebook::velox::connector::lakehouse::common { +/// Generate sequential integer IDs for distinct partition values, which could +/// be used as vector index. +class PartitionIdGenerator { + public: + /// @param inputType RowType of the input. + /// @param partitionChannels Channels of partition keys in the input + /// RowVector. + /// @param maxPartitions The max number of distinct partitions. + /// @param pool Memory pool. Used to allocate memory for storing unique + /// partition key values. + /// @param partitionPathAsLowerCase Used to control whether the partition path + /// need to convert to lower case. + PartitionIdGenerator( + const RowTypePtr& inputType, + std::vector partitionChannels, + uint32_t maxPartitions, + memory::MemoryPool* pool, + bool partitionPathAsLowerCase); + + /// Generate sequential partition IDs for input vector. + /// @param input Input RowVector. + /// @param result Generated integer IDs indexed by input row number. + void run(const RowVectorPtr& input, raw_vector& result); + + /// Return the total number of distinct partitions processed so far. + uint64_t numPartitions() const { + return partitionIds_.size(); + } + + /// Return partition name for the given partition id in the typical Hive + /// style. It is derived from the partitionValues_ at index partitionId. + /// Partition keys appear in the order of partition columns in the table + /// schema. + std::string partitionName( + uint64_t partitionId, + const std::string& nullValueName = "") const; + + private: + static constexpr const int32_t kHasherReservePct = 20; + + // Computes value IDs using VectorHashers for all rows in 'input'. + void computeValueIds( + const RowVectorPtr& input, + raw_vector& valueIds); + + // In case of rehash (when value IDs produced by VectorHashers change), we + // update value id for pre-existing partitions while keeping partition ids. + // This method rebuilds 'partitionIds_' by re-calculating the value ids using + // updated 'hashers_'. + void updateValueToPartitionIdMapping(); + + // Copies partition values of 'row' from 'input' into 'partitionId' row in + // 'partitionValues_'. + void savePartitionValues( + uint64_t partitionId, + const RowVectorPtr& input, + vector_size_t row); + + memory::MemoryPool* const pool_; + + const std::vector partitionChannels_; + + const uint32_t maxPartitions_; + + const bool partitionPathAsLowerCase_; + + std::vector> hashers_; + bool hasMultiplierSet_ = false; + + // A mapping from value ID produced by VectorHashers to a partition ID. + std::unordered_map partitionIds_; + + // A vector holding unique partition key values. One row per partition. Row + // numbers match partition IDs. + RowVectorPtr partitionValues_; + + // All rows are set valid to compute partition IDs for all input rows. + SelectivityVector allRows_; +}; + +} // namespace facebook::velox::connector::lakehouse::common diff --git a/velox/connectors/lakehouse/common/SplitReader.cpp b/velox/connectors/lakehouse/common/SplitReader.cpp new file mode 100644 index 000000000000..38783cfc8f59 --- /dev/null +++ b/velox/connectors/lakehouse/common/SplitReader.cpp @@ -0,0 +1,483 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "velox/connectors/lakehouse/common/SplitReader.h" + +#include "velox/common/caching/CacheTTLController.h" +#include "velox/connectors/lakehouse/common/HiveConfig.h" +#include "velox/connectors/lakehouse/common/HiveConnectorSplit.h" +#include "velox/connectors/lakehouse/common/HiveConnectorUtil.h" +#include "velox/connectors/lakehouse/common/TableHandle.h" +#include "velox/connectors/lakehouse/iceberg/IcebergSplitReader.h" +#include "velox/dwio/common/ReaderFactory.h" + +namespace facebook::velox::connector::lakehouse::common { +namespace { + +template +VectorPtr newConstantFromString( + const TypePtr& type, + const std::optional& value, + vector_size_t size, + velox::memory::MemoryPool* pool, + const std::string& sessionTimezone, + bool asLocalTime, + bool isPartitionDateDaysSinceEpoch = false) { + using T = typename TypeTraits::NativeType; + if (!value.has_value()) { + return std::make_shared>(pool, size, true, type, T()); + } + + if (type->isDate()) { + int32_t days = 0; + // For Iceberg, the date partition values are already in daysSinceEpoch + // form. + if (isPartitionDateDaysSinceEpoch) { + days = folly::to(value.value()); + } else { + days = DATE()->toDays(static_cast(value.value())); + } + return std::make_shared>( + pool, size, false, type, std::move(days)); + } + + if constexpr (std::is_same_v) { + return std::make_shared>( + pool, size, false, type, StringView(value.value())); + } else { + auto copy = velox::util::Converter::tryCast(value.value()) + .thenOrThrow(folly::identity, [&](const Status& status) { + VELOX_USER_FAIL("{}", status.message()); + }); + if constexpr (kind == TypeKind::TIMESTAMP) { + if (asLocalTime) { + copy.toGMT(Timestamp::defaultTimezone()); + } + } + return std::make_shared>( + pool, size, false, type, std::move(copy)); + } +} +} // namespace + +std::unique_ptr SplitReader::create( + const std::shared_ptr& hiveSplit, + const HiveTableHandlePtr& hiveTableHandle, + const std::unordered_map* partitionKeys, + const ConnectorQueryCtx* connectorQueryCtx, + const std::shared_ptr& hiveConfig, + const RowTypePtr& readerOutputType, + const std::shared_ptr& ioStats, + const std::shared_ptr& fsStats, + FileHandleFactory* fileHandleFactory, + folly::Executor* executor, + const std::shared_ptr& scanSpec, + core::ExpressionEvaluator* expressionEvaluator, + std::atomic& totalRemainingFilterTime) { + // Create the SplitReader based on hiveSplit->customSplitInfo["table_format"] + if (hiveSplit->customSplitInfo.count("table_format") > 0 && + hiveSplit->customSplitInfo["table_format"] == "hive-iceberg") { + return std::make_unique( + hiveSplit, + hiveTableHandle, + partitionKeys, + connectorQueryCtx, + hiveConfig, + readerOutputType, + ioStats, + fsStats, + fileHandleFactory, + executor, + scanSpec, + expressionEvaluator, + totalRemainingFilterTime); + } else { + return std::unique_ptr(new SplitReader( + hiveSplit, + hiveTableHandle, + partitionKeys, + connectorQueryCtx, + hiveConfig, + readerOutputType, + ioStats, + fsStats, + fileHandleFactory, + executor, + scanSpec)); + } +} + +SplitReader::SplitReader( + const std::shared_ptr& hiveSplit, + const HiveTableHandlePtr& hiveTableHandle, + const std::unordered_map* partitionKeys, + const ConnectorQueryCtx* connectorQueryCtx, + const std::shared_ptr& hiveConfig, + const RowTypePtr& readerOutputType, + const std::shared_ptr& ioStats, + const std::shared_ptr& fsStats, + FileHandleFactory* fileHandleFactory, + folly::Executor* executor, + const std::shared_ptr& scanSpec) + : hiveSplit_(hiveSplit), + hiveTableHandle_(hiveTableHandle), + partitionKeys_(partitionKeys), + connectorQueryCtx_(connectorQueryCtx), + hiveConfig_(hiveConfig), + readerOutputType_(readerOutputType), + ioStats_(ioStats), + fsStats_(fsStats), + fileHandleFactory_(fileHandleFactory), + executor_(executor), + pool_(connectorQueryCtx->memoryPool()), + scanSpec_(scanSpec), + baseReaderOpts_(connectorQueryCtx->memoryPool()), + emptySplit_(false) {} + +void SplitReader::configureReaderOptions( + std::shared_ptr randomSkip) { + lakehouse::common::configureReaderOptions( + hiveConfig_, + connectorQueryCtx_, + hiveTableHandle_, + hiveSplit_, + baseReaderOpts_); + baseReaderOpts_.setRandomSkip(std::move(randomSkip)); + baseReaderOpts_.setScanSpec(scanSpec_); + baseReaderOpts_.setFileFormat(hiveSplit_->fileFormat); +} + +void SplitReader::prepareSplit( + std::shared_ptr metadataFilter, + dwio::common::RuntimeStatistics& runtimeStats) { + createReader(); + if (emptySplit_) { + return; + } + auto rowType = getAdaptedRowType(); + + if (checkIfSplitIsEmpty(runtimeStats)) { + VELOX_CHECK(emptySplit_); + return; + } + + createRowReader(std::move(metadataFilter), std::move(rowType)); +} + +void SplitReader::setBucketConversion( + std::vector bucketChannels) { + bucketChannels_ = {bucketChannels.begin(), bucketChannels.end()}; + partitionFunction_ = std::make_unique( + hiveSplit_->bucketConversion->tableBucketCount, + std::move(bucketChannels)); +} + +std::vector SplitReader::bucketConversionRows( + const RowVector& vector) { + partitions_.clear(); + partitionFunction_->partition(vector, partitions_); + const auto bucketToKeep = *hiveSplit_->tableBucketNumber; + const auto partitionBucketCount = + hiveSplit_->bucketConversion->partitionBucketCount; + std::vector ranges; + for (vector_size_t i = 0; i < vector.size(); ++i) { + VELOX_CHECK_EQ((partitions_[i] - bucketToKeep) % partitionBucketCount, 0); + if (partitions_[i] == bucketToKeep) { + auto& r = ranges.emplace_back(); + r.sourceIndex = i; + r.targetIndex = ranges.size() - 1; + r.count = 1; + } + } + return ranges; +} + +void SplitReader::applyBucketConversion( + VectorPtr& output, + const std::vector& ranges) { + auto filtered = + BaseVector::create(output->type(), ranges.size(), output->pool()); + filtered->copyRanges(output.get(), ranges); + output = std::move(filtered); +} + +uint64_t SplitReader::next(uint64_t size, VectorPtr& output) { + uint64_t numScanned; + if (!baseReaderOpts_.randomSkip()) { + numScanned = baseRowReader_->next(size, output); + } else { + dwio::common::Mutation mutation; + mutation.randomSkip = baseReaderOpts_.randomSkip().get(); + numScanned = baseRowReader_->next(size, output, &mutation); + } + if (numScanned > 0 && output->size() > 0 && partitionFunction_) { + applyBucketConversion( + output, bucketConversionRows(*output->asChecked())); + } + return numScanned; +} + +void SplitReader::resetFilterCaches() { + if (baseRowReader_) { + baseRowReader_->resetFilterCaches(); + } +} + +bool SplitReader::emptySplit() const { + return emptySplit_; +} + +void SplitReader::resetSplit() { + hiveSplit_.reset(); +} + +int64_t SplitReader::estimatedRowSize() const { + if (!baseRowReader_) { + return DataSource::kUnknownRowSize; + } + + const auto size = baseRowReader_->estimatedRowSize(); + return size.value_or(DataSource::kUnknownRowSize); +} + +void SplitReader::updateRuntimeStats( + dwio::common::RuntimeStatistics& stats) const { + if (baseRowReader_) { + baseRowReader_->updateRuntimeStats(stats); + } +} + +bool SplitReader::allPrefetchIssued() const { + return baseRowReader_ && baseRowReader_->allPrefetchIssued(); +} + +void SplitReader::setConnectorQueryCtx( + const ConnectorQueryCtx* connectorQueryCtx) { + connectorQueryCtx_ = connectorQueryCtx; +} + +std::string SplitReader::toString() const { + std::string partitionKeys; + std::for_each( + partitionKeys_->begin(), partitionKeys_->end(), [&](const auto& column) { + partitionKeys += " " + column.second->toString(); + }); + return fmt::format( + "SplitReader: hiveSplit_{} scanSpec_{} readerOutputType_{} partitionKeys_{} reader{} rowReader{}", + hiveSplit_->toString(), + scanSpec_->toString(), + readerOutputType_->toString(), + partitionKeys, + static_cast(baseReader_.get()), + static_cast(baseRowReader_.get())); +} + +void SplitReader::createReader() { + VELOX_CHECK_NE( + baseReaderOpts_.fileFormat(), dwio::common::FileFormat::UNKNOWN); + + FileHandleCachedPtr fileHandleCachePtr; + FileHandleKey fileHandleKey{ + .filename = hiveSplit_->filePath, + .tokenProvider = connectorQueryCtx_->fsTokenProvider()}; + try { + fileHandleCachePtr = fileHandleFactory_->generate( + fileHandleKey, + hiveSplit_->properties.has_value() ? &*hiveSplit_->properties : nullptr, + fsStats_ ? fsStats_.get() : nullptr); + VELOX_CHECK_NOT_NULL(fileHandleCachePtr.get()); + } catch (const VeloxRuntimeError& e) { + if (e.errorCode() == error_code::kFileNotFound && + hiveConfig_->ignoreMissingFiles( + connectorQueryCtx_->sessionProperties())) { + emptySplit_ = true; + return; + } + throw; + } + + // Here we keep adding new entries to CacheTTLController when new fileHandles + // are generated, if CacheTTLController was created. Creator of + // CacheTTLController needs to make sure a size control strategy was available + // such as removing aged out entries. + if (auto* cacheTTLController = cache::CacheTTLController::getInstance()) { + cacheTTLController->addOpenFileInfo(fileHandleCachePtr->uuid.id()); + } + auto baseFileInput = createBufferedInput( + *fileHandleCachePtr, + baseReaderOpts_, + connectorQueryCtx_, + ioStats_, + fsStats_, + executor_); + + baseReader_ = dwio::common::getReaderFactory(baseReaderOpts_.fileFormat()) + ->createReader(std::move(baseFileInput), baseReaderOpts_); + if (!baseReader_) { + emptySplit_ = true; + } +} + +RowTypePtr SplitReader::getAdaptedRowType() const { + auto& fileType = baseReader_->rowType(); + auto columnTypes = adaptColumns(fileType, baseReaderOpts_.fileSchema()); + auto columnNames = fileType->names(); + return ROW(std::move(columnNames), std::move(columnTypes)); +} + +bool SplitReader::filterOnStats( + dwio::common::RuntimeStatistics& runtimeStats) const { + if (testFilters( + scanSpec_.get(), + baseReader_.get(), + hiveSplit_->filePath, + hiveSplit_->partitionKeys, + *partitionKeys_, + hiveConfig_->readTimestampPartitionValueAsLocalTime( + connectorQueryCtx_->sessionProperties()))) { + ++runtimeStats.processedSplits; + return true; + } + ++runtimeStats.skippedSplits; + runtimeStats.skippedSplitBytes += hiveSplit_->length; + return false; +} + +bool SplitReader::checkIfSplitIsEmpty( + dwio::common::RuntimeStatistics& runtimeStats) { + // emptySplit_ may already be set if the data file is not found. In this case + // we don't need to test further. + if (emptySplit_) { + return true; + } + if (!baseReader_ || baseReader_->numberOfRows() == 0 || + !filterOnStats(runtimeStats)) { + emptySplit_ = true; + } + return emptySplit_; +} + +void SplitReader::createRowReader( + std::shared_ptr metadataFilter, + RowTypePtr rowType) { + VELOX_CHECK_NULL(baseRowReader_); + configureRowReaderOptions( + hiveTableHandle_->tableParameters(), + scanSpec_, + std::move(metadataFilter), + std::move(rowType), + hiveSplit_, + hiveConfig_, + connectorQueryCtx_->sessionProperties(), + baseRowReaderOpts_); + baseRowReader_ = baseReader_->createRowReader(baseRowReaderOpts_); +} + +std::vector SplitReader::adaptColumns( + const RowTypePtr& fileType, + const std::shared_ptr& tableSchema) const { + // Keep track of schema types for columns in file, used by ColumnSelector. + std::vector columnTypes = fileType->children(); + + auto& childrenSpecs = scanSpec_->children(); + for (size_t i = 0; i < childrenSpecs.size(); ++i) { + auto* childSpec = childrenSpecs[i].get(); + const std::string& fieldName = childSpec->fieldName(); + + if (auto it = hiveSplit_->partitionKeys.find(fieldName); + it != hiveSplit_->partitionKeys.end()) { + setPartitionValue(childSpec, fieldName, it->second); + } else if (auto iter = hiveSplit_->infoColumns.find(fieldName); + iter != hiveSplit_->infoColumns.end()) { + auto infoColumnType = + readerOutputType_->childAt(readerOutputType_->getChildIdx(fieldName)); + auto constant = VELOX_DYNAMIC_SCALAR_TYPE_DISPATCH_ALL( + newConstantFromString, + infoColumnType->kind(), + infoColumnType, + iter->second, + 1, + connectorQueryCtx_->memoryPool(), + connectorQueryCtx_->sessionTimezone(), + hiveConfig_->readTimestampPartitionValueAsLocalTime( + connectorQueryCtx_->sessionProperties())); + childSpec->setConstantValue(constant); + } else if ( + childSpec->columnType() == velox::common::ScanSpec::ColumnType::kRegular) { + auto fileTypeIdx = fileType->getChildIdxIfExists(fieldName); + if (!fileTypeIdx.has_value()) { + // Column is missing. Most likely due to schema evolution. + auto outputTypeIdx = readerOutputType_->getChildIdxIfExists(fieldName); + TypePtr fieldType; + if (outputTypeIdx.has_value()) { + // Field name exists in the user-specified output type. + fieldType = readerOutputType_->childAt(outputTypeIdx.value()); + } else { + VELOX_CHECK(tableSchema, "Unable to resolve column '{}'", fieldName); + fieldType = tableSchema->findChild(fieldName); + } + childSpec->setConstantValue(BaseVector::createNullConstant( + fieldType, 1, connectorQueryCtx_->memoryPool())); + } else { + // Column no longer missing, reset constant value set on the spec. + childSpec->setConstantValue(nullptr); + auto outputTypeIdx = readerOutputType_->getChildIdxIfExists(fieldName); + if (outputTypeIdx.has_value()) { + auto& outputType = readerOutputType_->childAt(*outputTypeIdx); + auto& columnType = columnTypes[*fileTypeIdx]; + if (childSpec->isFlatMapAsStruct()) { + // Flat map column read as struct. Leave the schema type as MAP. + VELOX_CHECK(outputType->isRow() && columnType->isMap()); + } else { + // We know the fieldName exists in the file, make the type at that + // position match what we expect in the output. + columnType = outputType; + } + } + } + } + } + + scanSpec_->resetCachedValues(false); + + return columnTypes; +} + +void SplitReader::setPartitionValue( + velox::common::ScanSpec* spec, + const std::string& partitionKey, + const std::optional& value) const { + auto it = partitionKeys_->find(partitionKey); + VELOX_CHECK( + it != partitionKeys_->end(), + "ColumnHandle is missing for partition key {}", + partitionKey); + auto type = it->second->dataType(); + auto constant = VELOX_DYNAMIC_SCALAR_TYPE_DISPATCH_ALL( + newConstantFromString, + type->kind(), + type, + value, + 1, + connectorQueryCtx_->memoryPool(), + connectorQueryCtx_->sessionTimezone(), + hiveConfig_->readTimestampPartitionValueAsLocalTime( + connectorQueryCtx_->sessionProperties()), + it->second->isPartitionDateValueDaysSinceEpoch()); + spec->setConstantValue(constant); +} + +} // namespace facebook::velox::connector::lakehouse::common diff --git a/velox/connectors/lakehouse/common/SplitReader.h b/velox/connectors/lakehouse/common/SplitReader.h new file mode 100644 index 000000000000..ab698fd4b52d --- /dev/null +++ b/velox/connectors/lakehouse/common/SplitReader.h @@ -0,0 +1,206 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#pragma once + +#include "velox/common/base/RandomUtil.h" +#include "velox/common/file/FileSystems.h" +#include "velox/connectors/lakehouse/common/FileHandle.h" +#include "velox/connectors/lakehouse/common/HivePartitionFunction.h" +#include "velox/dwio/common/Options.h" +#include "velox/dwio/common/Reader.h" + +namespace facebook::velox { +class BaseVector; +using VectorPtr = std::shared_ptr; +} // namespace facebook::velox + +namespace facebook::velox::common { +class MetadataFilter; +class ScanSpec; +} // namespace facebook::velox::common + +namespace facebook::velox::connector { +class ConnectorQueryCtx; +} // namespace facebook::velox::connector + +namespace facebook::velox::dwio::common { +struct RuntimeStatistics; +} // namespace facebook::velox::dwio::common + +namespace facebook::velox::memory { +class MemoryPool; +} + +namespace facebook::velox::connector::lakehouse::common { + +struct HiveConnectorSplit; +class HiveTableHandle; +class HiveColumnHandle; +class HiveConfig; + +class SplitReader { + public: + static std::unique_ptr create( + const std::shared_ptr& hiveSplit, + const std::shared_ptr& hiveTableHandle, + const std::unordered_map< + std::string, + std::shared_ptr>* partitionKeys, + const ConnectorQueryCtx* connectorQueryCtx, + const std::shared_ptr& hiveConfig, + const RowTypePtr& readerOutputType, + const std::shared_ptr& ioStats, + const std::shared_ptr& fsStats, + FileHandleFactory* fileHandleFactory, + folly::Executor* executor, + const std::shared_ptr& scanSpec, + core::ExpressionEvaluator* expressionEvaluator, + std::atomic& totalRemainingFilterTime); + + virtual ~SplitReader() = default; + + void configureReaderOptions( + std::shared_ptr randomSkip); + + /// This function is used by different table formats like Iceberg and Hudi to + /// do additional preparations before reading the split, e.g. Open delete + /// files or log files, and add column adapatations for metadata columns. It + /// would be called only once per incoming split + virtual void prepareSplit( + std::shared_ptr metadataFilter, + dwio::common::RuntimeStatistics& runtimeStats); + + virtual uint64_t next(uint64_t size, VectorPtr& output); + + void resetFilterCaches(); + + bool emptySplit() const; + + void resetSplit(); + + int64_t estimatedRowSize() const; + + void updateRuntimeStats(dwio::common::RuntimeStatistics& stats) const; + + bool allPrefetchIssued() const; + + void setConnectorQueryCtx(const ConnectorQueryCtx* connectorQueryCtx); + + void setBucketConversion(std::vector bucketChannels); + + const RowTypePtr& readerOutputType() const { + return readerOutputType_; + } + + std::string toString() const; + + protected: + SplitReader( + const std::shared_ptr& hiveSplit, + const std::shared_ptr& hiveTableHandle, + const std::unordered_map< + std::string, + std::shared_ptr>* partitionKeys, + const ConnectorQueryCtx* connectorQueryCtx, + const std::shared_ptr& hiveConfig, + const RowTypePtr& readerOutputType, + const std::shared_ptr& ioStats, + const std::shared_ptr& fsStats, + FileHandleFactory* fileHandleFactory, + folly::Executor* executor, + const std::shared_ptr& scanSpec); + + /// Create the dwio::common::Reader object baseReader_, which will be used to + /// read the data file's metadata and schema + void createReader(); + + // Adjust the scan spec according to the current split, then return the + // adapted row type. + RowTypePtr getAdaptedRowType() const; + + // Check if the filters pass on the column statistics. When delta update is + // present, the corresonding filter should be disabled before calling this + // function. + bool filterOnStats(dwio::common::RuntimeStatistics& runtimeStats) const; + + /// Check if the hiveSplit_ is empty. The split is considered empty when + /// 1) The data file is missing but the user chooses to ignore it + /// 2) The file does not contain any rows + /// 3) The data in the file does not pass the filters. The test is based on + /// the file metadata and partition key values + /// This function needs to be called after baseReader_ is created. + bool checkIfSplitIsEmpty(dwio::common::RuntimeStatistics& runtimeStats); + + /// Create the dwio::common::RowReader object baseRowReader_, which owns the + /// ColumnReaders that will be used to read the data + void createRowReader( + std::shared_ptr metadataFilter, + RowTypePtr rowType); + + const folly::F14FastSet& bucketChannels() const { + return bucketChannels_; + } + + std::vector bucketConversionRows( + const RowVector& vector); + + void applyBucketConversion( + VectorPtr& output, + const std::vector& ranges); + + private: + /// Different table formats may have different meatadata columns. + /// This function will be used to update the scanSpec for these columns. + std::vector adaptColumns( + const RowTypePtr& fileType, + const std::shared_ptr& tableSchema) const; + + void setPartitionValue( + velox::common::ScanSpec* spec, + const std::string& partitionKey, + const std::optional& value) const; + + protected: + std::shared_ptr hiveSplit_; + const std::shared_ptr hiveTableHandle_; + const std::unordered_map< + std::string, + std::shared_ptr>* const partitionKeys_; + const ConnectorQueryCtx* connectorQueryCtx_; + const std::shared_ptr hiveConfig_; + + RowTypePtr readerOutputType_; + const std::shared_ptr ioStats_; + const std::shared_ptr fsStats_; + FileHandleFactory* const fileHandleFactory_; + folly::Executor* const executor_; + memory::MemoryPool* const pool_; + + std::shared_ptr scanSpec_; + std::unique_ptr baseReader_; + std::unique_ptr baseRowReader_; + dwio::common::ReaderOptions baseReaderOpts_; + dwio::common::RowReaderOptions baseRowReaderOpts_; + bool emptySplit_; + + private: + folly::F14FastSet bucketChannels_; + std::unique_ptr partitionFunction_; + std::vector partitions_; +}; + +} // namespace facebook::velox::connector::lakehouse::common diff --git a/velox/connectors/lakehouse/common/TableHandle.cpp b/velox/connectors/lakehouse/common/TableHandle.cpp new file mode 100644 index 000000000000..60457908290d --- /dev/null +++ b/velox/connectors/lakehouse/common/TableHandle.cpp @@ -0,0 +1,244 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "velox/connectors/lakehouse/common/TableHandle.h" + +namespace facebook::velox::connector::lakehouse::common { + +namespace { +std::unordered_map +columnTypeNames() { + return { + {HiveColumnHandle::ColumnType::kPartitionKey, "PartitionKey"}, + {HiveColumnHandle::ColumnType::kRegular, "Regular"}, + {HiveColumnHandle::ColumnType::kSynthesized, "Synthesized"}, + {HiveColumnHandle::ColumnType::kRowIndex, "RowIndex"}, + }; +} + +template +std::unordered_map invertMap(const std::unordered_map& mapping) { + std::unordered_map inverted; + for (const auto& [key, value] : mapping) { + inverted.emplace(value, key); + } + return inverted; +} + +} // namespace + +std::string HiveColumnHandle::columnTypeName( + HiveColumnHandle::ColumnType type) { + static const auto ctNames = columnTypeNames(); + return ctNames.at(type); +} + +HiveColumnHandle::ColumnType HiveColumnHandle::columnTypeFromName( + const std::string& name) { + static const auto nameColumnTypes = invertMap(columnTypeNames()); + return nameColumnTypes.at(name); +} + +folly::dynamic HiveColumnHandle::serialize() const { + folly::dynamic obj = ColumnHandle::serializeBase("HiveColumnHandle"); + obj["hiveColumnHandleName"] = name_; + obj["columnType"] = columnTypeName(columnType_); + obj["dataType"] = dataType_->serialize(); + obj["hiveType"] = hiveType_->serialize(); + folly::dynamic requiredSubfields = folly::dynamic::array; + for (const auto& subfield : requiredSubfields_) { + requiredSubfields.push_back(subfield.toString()); + } + obj["requiredSubfields"] = requiredSubfields; + return obj; +} + +std::string HiveColumnHandle::toString() const { + std::ostringstream out; + out << fmt::format( + "HiveColumnHandle [name: {}, columnType: {}, dataType: {},", + name_, + columnTypeName(columnType_), + dataType_->toString()); + out << " requiredSubfields: ["; + for (const auto& subfield : requiredSubfields_) { + out << " " << subfield.toString(); + } + out << " ]]"; + return out.str(); +} + +ColumnHandlePtr HiveColumnHandle::create(const folly::dynamic& obj) { + auto name = obj["hiveColumnHandleName"].asString(); + auto columnType = columnTypeFromName(obj["columnType"].asString()); + auto dataType = ISerializable::deserialize(obj["dataType"]); + auto hiveType = ISerializable::deserialize(obj["hiveType"]); + + const auto& arr = obj["requiredSubfields"]; + std::vector requiredSubfields; + requiredSubfields.reserve(arr.size()); + for (auto& s : arr) { + requiredSubfields.emplace_back(s.asString()); + } + + return std::make_shared( + name, columnType, dataType, hiveType, std::move(requiredSubfields)); +} + +void HiveColumnHandle::registerSerDe() { + auto& registry = DeserializationRegistryForSharedPtr(); + registry.Register("HiveColumnHandle", HiveColumnHandle::create); +} + +HiveTableHandle::HiveTableHandle( + std::string connectorId, + const std::string& tableName, + bool filterPushdownEnabled, + velox::common::SubfieldFilters subfieldFilters, + const core::TypedExprPtr& remainingFilter, + const RowTypePtr& dataColumns, + const std::unordered_map& tableParameters) + : ConnectorTableHandle(std::move(connectorId)), + tableName_(tableName), + filterPushdownEnabled_(filterPushdownEnabled), + subfieldFilters_(std::move(subfieldFilters)), + remainingFilter_(remainingFilter), + dataColumns_(dataColumns), + tableParameters_(tableParameters) {} + +std::string HiveTableHandle::toString() const { + std::stringstream out; + out << "table: " << tableName_; + if (!subfieldFilters_.empty()) { + // Sort filters by subfield for deterministic output. + std::map orderedFilters; + for (const auto& [field, filter] : subfieldFilters_) { + orderedFilters[field.toString()] = filter.get(); + } + out << ", range filters: ["; + bool notFirstFilter = false; + for (const auto& [field, filter] : orderedFilters) { + if (notFirstFilter) { + out << ", "; + } + out << "(" << field << ", " << filter->toString() << ")"; + notFirstFilter = true; + } + out << "]"; + } + if (remainingFilter_) { + out << ", remaining filter: (" << remainingFilter_->toString() << ")"; + } + if (dataColumns_) { + out << ", data columns: " << dataColumns_->toString(); + } + if (!tableParameters_.empty()) { + std::map orderedTableParameters{ + tableParameters_.begin(), tableParameters_.end()}; + out << ", table parameters: ["; + bool firstParam = true; + for (const auto& param : orderedTableParameters) { + if (!firstParam) { + out << ", "; + } + out << param.first << ":" << param.second; + firstParam = false; + } + out << "]"; + } + return out.str(); +} + +folly::dynamic HiveTableHandle::serialize() const { + folly::dynamic obj = ConnectorTableHandle::serializeBase("HiveTableHandle"); + obj["tableName"] = tableName_; + obj["filterPushdownEnabled"] = filterPushdownEnabled_; + + folly::dynamic subfieldFilters = folly::dynamic::array; + for (const auto& [subfield, filter] : subfieldFilters_) { + folly::dynamic pair = folly::dynamic::object; + pair["subfield"] = subfield.toString(); + pair["filter"] = filter->serialize(); + subfieldFilters.push_back(pair); + } + + obj["subfieldFilters"] = subfieldFilters; + if (remainingFilter_) { + obj["remainingFilter"] = remainingFilter_->serialize(); + } + if (dataColumns_) { + obj["dataColumns"] = dataColumns_->serialize(); + } + folly::dynamic tableParameters = folly::dynamic::object; + for (const auto& param : tableParameters_) { + tableParameters[param.first] = param.second; + } + obj["tableParameters"] = tableParameters; + + return obj; +} + +ConnectorTableHandlePtr HiveTableHandle::create( + const folly::dynamic& obj, + void* context) { + auto connectorId = obj["connectorId"].asString(); + auto tableName = obj["tableName"].asString(); + auto filterPushdownEnabled = obj["filterPushdownEnabled"].asBool(); + + core::TypedExprPtr remainingFilter; + if (auto it = obj.find("remainingFilter"); it != obj.items().end()) { + remainingFilter = + ISerializable::deserialize(it->second, context); + } + + velox::common::SubfieldFilters subfieldFilters; + folly::dynamic subfieldFiltersObj = obj["subfieldFilters"]; + for (const auto& subfieldFilter : subfieldFiltersObj) { + velox::common::Subfield subfield(subfieldFilter["subfield"].asString()); + auto filter = + ISerializable::deserialize(subfieldFilter["filter"]); + subfieldFilters[velox::common::Subfield(std::move(subfield.path()))] = + filter->clone(); + } + + RowTypePtr dataColumns; + if (auto it = obj.find("dataColumns"); it != obj.items().end()) { + dataColumns = ISerializable::deserialize(it->second, context); + } + + std::unordered_map tableParameters{}; + const auto& tableParametersObj = obj["tableParameters"]; + for (const auto& key : tableParametersObj.keys()) { + const auto& value = tableParametersObj[key]; + tableParameters.emplace(key.asString(), value.asString()); + } + + return std::make_shared( + connectorId, + tableName, + filterPushdownEnabled, + std::move(subfieldFilters), + remainingFilter, + dataColumns, + tableParameters); +} + +void HiveTableHandle::registerSerDe() { + auto& registry = DeserializationWithContextRegistryForSharedPtr(); + registry.Register("HiveTableHandle", create); +} + +} // namespace facebook::velox::connector::lakehouse::common diff --git a/velox/connectors/lakehouse/common/TableHandle.h b/velox/connectors/lakehouse/common/TableHandle.h new file mode 100644 index 000000000000..e19426ac076a --- /dev/null +++ b/velox/connectors/lakehouse/common/TableHandle.h @@ -0,0 +1,200 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#pragma once + +#include "velox/connectors/Connector.h" +#include "velox/core/ITypedExpr.h" +#include "velox/type/Filter.h" +#include "velox/type/Subfield.h" +#include "velox/type/Type.h" + +namespace facebook::velox::connector::lakehouse::common { + +class HiveColumnHandle : public ColumnHandle { + public: + enum class ColumnType { + kPartitionKey, + kRegular, + kSynthesized, + /// A zero-based row number of type BIGINT auto-generated by the connector. + /// Rows numbers are unique within a single file only. + kRowIndex, + kRowId, + }; + + struct ColumnParseParameters { + enum PartitionDateValueFormat { + kISO8601, + kDaysSinceEpoch, + } partitionDateValueFormat; + }; + + /// NOTE: 'dataType' is the column type in target write table. 'hiveType' is + /// converted type of the corresponding column in source table which might not + /// be the same type, and the table scan needs to do data coercion if needs. + /// The table writer also needs to respect the type difference when processing + /// input data such as bucket id calculation. + HiveColumnHandle( + const std::string& name, + ColumnType columnType, + TypePtr dataType, + TypePtr hiveType, + std::vector requiredSubfields = {}, + ColumnParseParameters columnParseParameters = {}) + : name_(name), + columnType_(columnType), + dataType_(std::move(dataType)), + hiveType_(std::move(hiveType)), + requiredSubfields_(std::move(requiredSubfields)), + columnParseParameters_(columnParseParameters) { + VELOX_USER_CHECK( + dataType_->equivalent(*hiveType_), + "data type {} and hive type {} do not match", + dataType_->toString(), + hiveType_->toString()); + } + + const std::string& name() const override { + return name_; + } + + ColumnType columnType() const { + return columnType_; + } + + const TypePtr& dataType() const { + return dataType_; + } + + const TypePtr& hiveType() const { + return hiveType_; + } + + /// Applies to columns of complex types: arrays, maps and structs. When a + /// query uses only some of the subfields, the engine provides the complete + /// list of required subfields and the connector is free to prune the rest. + /// + /// Examples: + /// - SELECT a[1], b['x'], x.y FROM t + /// - SELECT a FROM t WHERE b['y'] > 10 + /// + /// Pruning a struct means populating some of the members with null values. + /// + /// Pruning a map means dropping keys not listed in the required subfields. + /// + /// Pruning arrays means dropping values with indices larger than maximum + /// required index. + const std::vector& requiredSubfields() const { + return requiredSubfields_; + } + + bool isPartitionKey() const { + return columnType_ == ColumnType::kPartitionKey; + } + + bool isPartitionDateValueDaysSinceEpoch() const { + return columnParseParameters_.partitionDateValueFormat == + ColumnParseParameters::kDaysSinceEpoch; + } + + std::string toString() const; + + folly::dynamic serialize() const override; + + static ColumnHandlePtr create(const folly::dynamic& obj); + + static std::string columnTypeName(HiveColumnHandle::ColumnType columnType); + + static HiveColumnHandle::ColumnType columnTypeFromName( + const std::string& name); + + static void registerSerDe(); + + private: + const std::string name_; + const ColumnType columnType_; + const TypePtr dataType_; + const TypePtr hiveType_; + const std::vector requiredSubfields_; + const ColumnParseParameters columnParseParameters_; +}; + +using HiveColumnHandlePtr = std::shared_ptr; +using HiveColumnHandleMap = + std::unordered_map; + +class HiveTableHandle : public ConnectorTableHandle { + public: + HiveTableHandle( + std::string connectorId, + const std::string& tableName, + bool filterPushdownEnabled, + velox::common::SubfieldFilters subfieldFilters, + const core::TypedExprPtr& remainingFilter, + const RowTypePtr& dataColumns = nullptr, + const std::unordered_map& tableParameters = {}); + + const std::string& tableName() const { + return tableName_; + } + + const std::string& name() const override { + return tableName(); + } + + bool isFilterPushdownEnabled() const { + return filterPushdownEnabled_; + } + + const velox::common::SubfieldFilters& subfieldFilters() const { + return subfieldFilters_; + } + + const core::TypedExprPtr& remainingFilter() const { + return remainingFilter_; + } + + // Schema of the table. Need this for reading TEXTFILE. + const RowTypePtr& dataColumns() const { + return dataColumns_; + } + + const std::unordered_map& tableParameters() const { + return tableParameters_; + } + + std::string toString() const override; + + folly::dynamic serialize() const override; + + static ConnectorTableHandlePtr create( + const folly::dynamic& obj, + void* context); + + static void registerSerDe(); + + private: + const std::string tableName_; + const bool filterPushdownEnabled_; + const velox::common::SubfieldFilters subfieldFilters_; + const core::TypedExprPtr remainingFilter_; + const RowTypePtr dataColumns_; + const std::unordered_map tableParameters_; +}; + +using HiveTableHandlePtr = std::shared_ptr; + +} // namespace facebook::velox::connector::lakehouse::common diff --git a/velox/connectors/lakehouse/common/tests/CMakeLists.txt b/velox/connectors/lakehouse/common/tests/CMakeLists.txt new file mode 100644 index 000000000000..1abe04ce73b3 --- /dev/null +++ b/velox/connectors/lakehouse/common/tests/CMakeLists.txt @@ -0,0 +1,19 @@ +# Copyright (c) Facebook, Inc. and its affiliates. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +add_library( + velox_lakehouse_common_test_lib HiveConnectorTestBase.cpp PlanBuilder.cpp) + +target_link_libraries( + velox_lakehouse_common_test_lib velox_exec_test_lib) diff --git a/velox/connectors/lakehouse/common/tests/HiveConnectorTestBase.cpp b/velox/connectors/lakehouse/common/tests/HiveConnectorTestBase.cpp new file mode 100644 index 000000000000..2a66b20f0f05 --- /dev/null +++ b/velox/connectors/lakehouse/common/tests/HiveConnectorTestBase.cpp @@ -0,0 +1,420 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "velox/connectors/lakehouse/common/tests/HiveConnectorTestBase.h" + +#include "velox/common/file/tests/FaultyFileSystem.h" +#include "velox/connectors/lakehouse/common/HiveConnector.h" +#include "velox/dwio/common/tests/utils/BatchMaker.h" +#include "velox/dwio/dwrf/RegisterDwrfReader.h" +#include "velox/dwio/dwrf/RegisterDwrfWriter.h" +#include "velox/dwio/dwrf/writer/Writer.h" +#include "velox/dwio/text/RegisterTextReader.h" +#include "velox/exec/tests/utils/AssertQueryBuilder.h" + +namespace facebook::velox::connector::lakehouse::common::test { + +HiveConnectorTestBase::HiveConnectorTestBase() { + filesystems::registerLocalFileSystem(); + tests::utils::registerFaultyFileSystem(); +} + +void HiveConnectorTestBase::SetUp() { + exec::test::OperatorTestBase::SetUp(); + connector::registerConnectorFactory(std::make_shared()); + auto hiveConnector = + connector::getConnectorFactory(HiveConnectorFactory::kHiveConnectorName) + ->newConnector( + kHiveConnectorId, + std::make_shared( + std::unordered_map()), + ioExecutor_.get()); + connector::registerConnector(hiveConnector); + dwio::common::registerFileSinks(); + dwrf::registerDwrfReaderFactory(); + dwrf::registerDwrfWriterFactory(); + text::registerTextReaderFactory(); +} + +void HiveConnectorTestBase::TearDown() { + // Make sure all pending loads are finished or cancelled before unregister + // connector. + ioExecutor_.reset(); + dwrf::unregisterDwrfReaderFactory(); + dwrf::unregisterDwrfWriterFactory(); + connector::unregisterConnector(kHiveConnectorId); + connector::unregisterConnectorFactory( + HiveConnectorFactory::kHiveConnectorName); + text::unregisterTextReaderFactory(); + exec::test::OperatorTestBase::TearDown(); +} + +void HiveConnectorTestBase::resetHiveConnector( + const std::shared_ptr& config) { + connector::unregisterConnector(kHiveConnectorId); + auto hiveConnector = + connector::getConnectorFactory(HiveConnectorFactory::kHiveConnectorName) + ->newConnector(kHiveConnectorId, config, ioExecutor_.get()); + connector::registerConnector(hiveConnector); +} + +void HiveConnectorTestBase::writeToFiles( + const std::vector& filePaths, + std::vector vectors) { + VELOX_CHECK_EQ(filePaths.size(), vectors.size()); + for (int i = 0; i < filePaths.size(); ++i) { + writeToFile(filePaths[i], std::vector{vectors[i]}); + } +} + +void HiveConnectorTestBase::writeToFile( + const std::string& filePath, + RowVectorPtr vector) { + writeToFile(filePath, std::vector{vector}); +} + +void HiveConnectorTestBase::writeToFile( + const std::string& filePath, + const std::vector& vectors, + std::shared_ptr config, + const std::function()>& + flushPolicyFactory) { + writeToFile( + filePath, + vectors, + std::move(config), + vectors[0]->type(), + flushPolicyFactory); +} + +void HiveConnectorTestBase::writeToFile( + const std::string& filePath, + const std::vector& vectors, + std::shared_ptr config, + const TypePtr& schema, + const std::function()>& + flushPolicyFactory) { + velox::dwrf::WriterOptions options; + options.config = config; + options.schema = schema; + auto fs = filesystems::getFileSystem(filePath, {}); + auto writeFile = fs->openFileForWrite( + filePath, + {.shouldCreateParentDirectories = true, + .shouldThrowOnFileAlreadyExists = false}); + auto sink = std::make_unique( + std::move(writeFile), filePath); + auto childPool = rootPool_->addAggregateChild("HiveConnectorTestBase.Writer"); + options.memoryPool = childPool.get(); + options.flushPolicyFactory = flushPolicyFactory; + + facebook::velox::dwrf::Writer writer{std::move(sink), options}; + for (size_t i = 0; i < vectors.size(); ++i) { + writer.write(vectors[i]); + } + writer.close(); +} + +void HiveConnectorTestBase::createDirectory(const std::string& directoryPath) { + auto fs = filesystems::getFileSystem(directoryPath, {}); + fs->mkdir(directoryPath); +} + +void HiveConnectorTestBase::removeDirectory(const std::string& directoryPath) { + auto fs = filesystems::getFileSystem(directoryPath, {}); + if (fs->exists(directoryPath)) { + fs->rmdir(directoryPath); + } +} + +void HiveConnectorTestBase::removeFile(const std::string& filePath) { + auto fs = filesystems::getFileSystem(filePath, {}); + if (fs->exists(filePath)) { + fs->remove(filePath); + } +} + +std::vector HiveConnectorTestBase::makeVectors( + const RowTypePtr& rowType, + int32_t numVectors, + int32_t rowsPerVector) { + std::vector vectors; + for (int32_t i = 0; i < numVectors; ++i) { + auto vector = std::dynamic_pointer_cast( + velox::test::BatchMaker::createBatch(rowType, rowsPerVector, *pool_)); + vectors.push_back(vector); + } + return vectors; +} + +std::shared_ptr HiveConnectorTestBase::assertQuery( + const core::PlanNodePtr& plan, + const std::vector>& filePaths, + const std::string& duckDbSql) { + return exec::test::OperatorTestBase::assertQuery( + plan, makeHiveConnectorSplits(filePaths), duckDbSql); +} + +std::shared_ptr HiveConnectorTestBase::assertQuery( + const core::PlanNodePtr& plan, + const std::vector>& splits, + const std::string& duckDbSql, + const int32_t numPrefetchSplit) { + return exec::test::AssertQueryBuilder(plan, duckDbQueryRunner_) + .config( + core::QueryConfig::kMaxSplitPreloadPerDriver, + std::to_string(numPrefetchSplit)) + .splits(splits) + .assertResults(duckDbSql); +} + +std::vector> +HiveConnectorTestBase::makeFilePaths(int count) { + std::vector> filePaths; + + filePaths.reserve(count); + for (auto i = 0; i < count; ++i) { + filePaths.emplace_back(exec::test::TempFilePath::create()); + } + return filePaths; +} + +std::vector> +HiveConnectorTestBase::makeHiveConnectorSplits( + const std::string& filePath, + uint32_t splitCount, + dwio::common::FileFormat format, + const std::optional< + std::unordered_map>>& + partitionKeys, + const std::optional>& + infoColumns) { + auto file = + filesystems::getFileSystem(filePath, nullptr)->openFileForRead(filePath); + const uint64_t fileSize = file->size(); + // Take the upper bound. + const uint64_t splitSize = std::ceil((fileSize) / splitCount); + std::vector> splits; + // Add all the splits. + for (uint32_t i = 0; i < splitCount; i++) { + auto splitBuilder = HiveConnectorSplitBuilder(filePath) + .fileFormat(format) + .start(i * splitSize) + .length(splitSize); + if (infoColumns.has_value()) { + for (const auto& infoColumn : infoColumns.value()) { + splitBuilder.infoColumn(infoColumn.first, infoColumn.second); + } + } + if (partitionKeys.has_value()) { + for (const auto& partitionKey : partitionKeys.value()) { + splitBuilder.partitionKey(partitionKey.first, partitionKey.second); + } + } + + auto split = splitBuilder.build(); + splits.push_back(std::move(split)); + } + return splits; +} + +std::unique_ptr +HiveConnectorTestBase::makeColumnHandle( + const std::string& name, + const TypePtr& type, + const std::vector& requiredSubfields) { + return makeColumnHandle(name, type, type, requiredSubfields); +} + +std::unique_ptr HiveConnectorTestBase::makeColumnHandle( + const std::string& name, + const TypePtr& dataType, + const TypePtr& hiveType, + const std::vector& requiredSubfields, + HiveColumnHandle::ColumnType columnType) { + std::vector subfields; + subfields.reserve(requiredSubfields.size()); + for (auto& path : requiredSubfields) { + subfields.emplace_back(path); + } + + return std::make_unique( + name, columnType, dataType, hiveType, std::move(subfields)); +} + +std::vector> +HiveConnectorTestBase::makeHiveConnectorSplits( + const std::vector>& filePaths) { + std::vector> splits; + splits.reserve(filePaths.size()); + for (const auto& filePath : filePaths) { + splits.push_back(makeHiveConnectorSplit( + filePath->getPath(), + filePath->fileSize(), + filePath->fileModifiedTime(), + 0, + std::numeric_limits::max())); + } + return splits; +} + +std::shared_ptr +HiveConnectorTestBase::makeHiveConnectorSplit( + const std::string& filePath, + uint64_t start, + uint64_t length, + int64_t splitWeight, + bool cacheable) { + return HiveConnectorSplitBuilder(filePath) + .start(start) + .length(length) + .splitWeight(splitWeight) + .cacheable(cacheable) + .build(); +} + +std::shared_ptr +HiveConnectorTestBase::makeHiveConnectorSplit( + const std::string& filePath, + int64_t fileSize, + int64_t fileModifiedTime, + uint64_t start, + uint64_t length) { + return HiveConnectorSplitBuilder(filePath) + .infoColumn("$file_size", fmt::format("{}", fileSize)) + .infoColumn("$file_modified_time", fmt::format("{}", fileModifiedTime)) + .start(start) + .length(length) + .build(); +} + +// static +std::shared_ptr +HiveConnectorTestBase::makeHiveInsertTableHandle( + const std::vector& tableColumnNames, + const std::vector& tableColumnTypes, + const std::vector& partitionedBy, + std::shared_ptr locationHandle, + const dwio::common::FileFormat tableStorageFormat, + const std::optional compressionKind, + const std::shared_ptr& writerOptions, + const bool ensureFiles) { + return makeHiveInsertTableHandle( + tableColumnNames, + tableColumnTypes, + partitionedBy, + nullptr, + std::move(locationHandle), + tableStorageFormat, + compressionKind, + {}, + writerOptions, + ensureFiles); +} + +// static +std::shared_ptr +HiveConnectorTestBase::makeHiveInsertTableHandle( + const std::vector& tableColumnNames, + const std::vector& tableColumnTypes, + const std::vector& partitionedBy, + std::shared_ptr bucketProperty, + std::shared_ptr locationHandle, + const dwio::common::FileFormat tableStorageFormat, + const std::optional compressionKind, + const std::unordered_map& serdeParameters, + const std::shared_ptr& writerOptions, + const bool ensureFiles) { + std::vector> columnHandles; + std::vector bucketedBy; + std::vector bucketedTypes; + std::vector> sortedBy; + if (bucketProperty != nullptr) { + bucketedBy = bucketProperty->bucketedBy(); + bucketedTypes = bucketProperty->bucketedTypes(); + sortedBy = bucketProperty->sortedBy(); + } + int32_t numPartitionColumns{0}; + int32_t numSortingColumns{0}; + int32_t numBucketColumns{0}; + for (int i = 0; i < tableColumnNames.size(); ++i) { + for (int j = 0; j < bucketedBy.size(); ++j) { + if (bucketedBy[j] == tableColumnNames[i]) { + ++numBucketColumns; + } + } + for (int j = 0; j < sortedBy.size(); ++j) { + if (sortedBy[j]->sortColumn() == tableColumnNames[i]) { + ++numSortingColumns; + } + } + if (std::find( + partitionedBy.cbegin(), + partitionedBy.cend(), + tableColumnNames.at(i)) != partitionedBy.cend()) { + ++numPartitionColumns; + columnHandles.push_back( + std::make_shared( + tableColumnNames.at(i), + HiveColumnHandle::ColumnType::kPartitionKey, + tableColumnTypes.at(i), + tableColumnTypes.at(i))); + } else { + columnHandles.push_back( + std::make_shared( + tableColumnNames.at(i), + HiveColumnHandle::ColumnType::kRegular, + tableColumnTypes.at(i), + tableColumnTypes.at(i))); + } + } + VELOX_CHECK_EQ(numPartitionColumns, partitionedBy.size()); + VELOX_CHECK_EQ(numBucketColumns, bucketedBy.size()); + VELOX_CHECK_EQ(numSortingColumns, sortedBy.size()); + + return std::make_shared( + columnHandles, + locationHandle, + tableStorageFormat, + bucketProperty, + compressionKind, + serdeParameters, + writerOptions, + ensureFiles); +} + +std::shared_ptr HiveConnectorTestBase::regularColumn( + const std::string& name, + const TypePtr& type) { + return std::make_shared( + name, HiveColumnHandle::ColumnType::kRegular, type, type); +} + +std::shared_ptr HiveConnectorTestBase::synthesizedColumn( + const std::string& name, + const TypePtr& type) { + return std::make_shared( + name, HiveColumnHandle::ColumnType::kSynthesized, type, type); +} + +std::shared_ptr HiveConnectorTestBase::partitionKey( + const std::string& name, + const TypePtr& type) { + return std::make_shared( + name, HiveColumnHandle::ColumnType::kPartitionKey, type, type); +} + +} // namespace facebook::velox::connector::lakehouse::common::test diff --git a/velox/connectors/lakehouse/common/tests/HiveConnectorTestBase.h b/velox/connectors/lakehouse/common/tests/HiveConnectorTestBase.h new file mode 100644 index 000000000000..7f6a010e4b31 --- /dev/null +++ b/velox/connectors/lakehouse/common/tests/HiveConnectorTestBase.h @@ -0,0 +1,251 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#pragma once + +#include "velox/connectors/lakehouse/common/HiveConnectorSplit.h" +#include "velox/connectors/lakehouse/common/HiveDataSink.h" +#include "velox/connectors/lakehouse/common/TableHandle.h" +#include "velox/dwio/dwrf/common/Config.h" +#include "velox/dwio/dwrf/writer/FlushPolicy.h" +#include "velox/exec/tests/utils/OperatorTestBase.h" +#include "velox/exec/tests/utils/TempFilePath.h" + +namespace facebook::velox::connector::lakehouse::common::test { + +static const std::string kHiveConnectorId = "test-hive"; + +class HiveConnectorTestBase : public exec::test::OperatorTestBase { + public: + HiveConnectorTestBase(); + + void SetUp() override; + void TearDown() override; + + void resetHiveConnector( + const std::shared_ptr& config); + + void writeToFiles( + const std::vector& filePaths, + std::vector vectors); + + void writeToFile(const std::string& filePath, RowVectorPtr vector); + + void writeToFile( + const std::string& filePath, + const std::vector& vectors, + std::shared_ptr config = + std::make_shared(), + const std::function()>& + flushPolicyFactory = nullptr); + + void writeToFile( + const std::string& filePath, + const std::vector& vectors, + std::shared_ptr config, + const TypePtr& schema, + const std::function()>& + flushPolicyFactory = nullptr); + + // Creates a directory using matching file system based on directoryPath. + // No throw when directory already exists. + void createDirectory(const std::string& directoryPath); + + // Removes a directory using matching file system based on directoryPath. + // No op when directory does not exist. + void removeDirectory(const std::string& directoryPath); + + // Removes a file using matching file system based on filePath. + // No op when file does not exist. + void removeFile(const std::string& filePath); + + std::vector makeVectors( + const RowTypePtr& rowType, + int32_t numVectors, + int32_t rowsPerVector); + + using exec::test::OperatorTestBase::assertQuery; + + /// Assumes plan has a single TableScan node. + std::shared_ptr assertQuery( + const core::PlanNodePtr& plan, + const std::vector>& filePaths, + const std::string& duckDbSql); + + std::shared_ptr assertQuery( + const core::PlanNodePtr& plan, + const std::vector>& splits, + const std::string& duckDbSql, + const int32_t numPrefetchSplit); + + static std::vector> makeFilePaths( + int count); + + static std::vector> + makeHiveConnectorSplits( + const std::vector>& filePaths); + + static std::shared_ptr makeHiveConnectorSplit( + const std::string& filePath, + uint64_t start = 0, + uint64_t length = std::numeric_limits::max(), + int64_t splitWeight = 0, + bool cacheable = true); + + static std::shared_ptr makeHiveConnectorSplit( + const std::string& filePath, + int64_t fileSize, + int64_t fileModifiedTime, + uint64_t start, + uint64_t length); + + /// Split file at path 'filePath' into 'splitCount' splits. If not local file, + /// file size can be given as 'externalSize'. + static std::vector> + makeHiveConnectorSplits( + const std::string& filePath, + uint32_t splitCount, + dwio::common::FileFormat format, + const std::optional< + std::unordered_map>>& + partitionKeys = {}, + const std::optional>& + infoColumns = {}); + + static std::shared_ptr makeTableHandle( + velox::common::SubfieldFilters subfieldFilters = {}, + const core::TypedExprPtr& remainingFilter = nullptr, + const std::string& tableName = "hive_table", + const RowTypePtr& dataColumns = nullptr, + bool filterPushdownEnabled = true, + const std::unordered_map& tableParameters = + {}) { + return std::make_shared( + kHiveConnectorId, + tableName, + filterPushdownEnabled, + std::move(subfieldFilters), + remainingFilter, + dataColumns, + tableParameters); + } + + /// @param name Column name. + /// @param type Column type. + /// @param Required subfields of this column. + static std::unique_ptr makeColumnHandle( + const std::string& name, + const TypePtr& type, + const std::vector& requiredSubfields); + + /// @param name Column name. + /// @param type Column type. + /// @param type Hive type. + /// @param Required subfields of this column. + static std::unique_ptr makeColumnHandle( + const std::string& name, + const TypePtr& dataType, + const TypePtr& hiveType, + const std::vector& requiredSubfields, + HiveColumnHandle::ColumnType columnType = + HiveColumnHandle::ColumnType::kRegular); + + /// @param targetDirectory Final directory of the target table after commit. + /// @param writeDirectory Write directory of the target table before commit. + /// @param tableType Whether to create a new table, insert into an existing + /// table, or write a temporary table. + /// @param writeMode How to write to the target directory. + static std::shared_ptr makeLocationHandle( + std::string targetDirectory, + std::optional writeDirectory = std::nullopt, + LocationHandle::TableType tableType = LocationHandle::TableType::kNew) { + return std::make_shared( + targetDirectory, writeDirectory.value_or(targetDirectory), tableType); + } + + /// Build a HiveInsertTableHandle. + /// @param tableColumnNames Column names of the target table. Corresponding + /// type of tableColumnNames[i] is tableColumnTypes[i]. + /// @param tableColumnTypes Column types of the target table. Corresponding + /// name of tableColumnTypes[i] is tableColumnNames[i]. + /// @param partitionedBy A list of partition columns of the target table. + /// @param bucketProperty if not nulll, specifies the property for a bucket + /// table. + /// @param locationHandle Location handle for the table write. + /// @param compressionKind compression algorithm to use for table write. + /// @param serdeParameters Table writer configuration parameters. + /// @param ensureFiles When this option is set the HiveDataSink will always + /// create a file even if there is no data. + static std::shared_ptr makeHiveInsertTableHandle( + const std::vector& tableColumnNames, + const std::vector& tableColumnTypes, + const std::vector& partitionedBy, + std::shared_ptr bucketProperty, + std::shared_ptr locationHandle, + const dwio::common::FileFormat tableStorageFormat = + dwio::common::FileFormat::DWRF, + const std::optional compressionKind = {}, + const std::unordered_map& serdeParameters = {}, + const std::shared_ptr& writerOptions = + nullptr, + const bool ensureFiles = false); + + static std::shared_ptr makeHiveInsertTableHandle( + const std::vector& tableColumnNames, + const std::vector& tableColumnTypes, + const std::vector& partitionedBy, + std::shared_ptr locationHandle, + const dwio::common::FileFormat tableStorageFormat = + dwio::common::FileFormat::DWRF, + const std::optional compressionKind = {}, + const std::shared_ptr& writerOptions = + nullptr, + const bool ensureFiles = false); + + static std::shared_ptr regularColumn( + const std::string& name, + const TypePtr& type); + + static std::shared_ptr partitionKey( + const std::string& name, + const TypePtr& type); + + static std::shared_ptr synthesizedColumn( + const std::string& name, + const TypePtr& type); + + static connector::ColumnHandleMap allRegularColumns( + const RowTypePtr& rowType) { + connector::ColumnHandleMap assignments; + assignments.reserve(rowType->size()); + for (uint32_t i = 0; i < rowType->size(); ++i) { + const auto& name = rowType->nameOf(i); + assignments[name] = regularColumn(name, rowType->childAt(i)); + } + return assignments; + } +}; + +/// Same as HiveConnectorBuilder, except that this defaults +/// connectorId to kHiveConnectorId. +class HiveConnectorSplitBuilder : public common::HiveConnectorSplitBuilder { + public: + explicit HiveConnectorSplitBuilder(std::string filePath) + : common::HiveConnectorSplitBuilder(std::move(filePath)) { + connectorId(kHiveConnectorId); + } +}; + +} // namespace facebook::velox::connector::lakehouse::common::test diff --git a/velox/connectors/lakehouse/common/tests/PlanBuilder.cpp b/velox/connectors/lakehouse/common/tests/PlanBuilder.cpp new file mode 100644 index 000000000000..cce7d69a631f --- /dev/null +++ b/velox/connectors/lakehouse/common/tests/PlanBuilder.cpp @@ -0,0 +1,2495 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "velox/connectors/lakehouse/common/tests/PlanBuilder.h" + +#include "velox/common/base/Status.h" +#include "velox/connectors/lakehouse/common/HiveConnector.h" +#include "velox/connectors/lakehouse/common/TableHandle.h" +#include "velox/core/FilterToExpression.h" +#include "velox/duckdb/conversion/DuckParser.h" +#include "velox/exec/Aggregate.h" +#include "velox/exec/HashPartitionFunction.h" +#include "velox/exec/RoundRobinPartitionFunction.h" +#include "velox/exec/TableWriter.h" +#include "velox/exec/WindowFunction.h" +#include "velox/expression/Expr.h" +#include "velox/expression/ExprToSubfieldFilter.h" +#include "velox/expression/FunctionCallToSpecialForm.h" +#include "velox/expression/FunctionSignature.h" +#include "velox/expression/SignatureBinder.h" +#include "velox/expression/VectorReaders.h" +#include "velox/parse/Expressions.h" +#include "velox/parse/TypeResolver.h" + +using namespace facebook::velox; +using namespace facebook::velox::exec; +using namespace facebook::velox::connector::lakehouse::common; + +namespace facebook::velox::connector::lakehouse::common::test { +namespace { + +core::TypedExprPtr parseExpr( + const std::string& text, + const RowTypePtr& rowType, + const parse::ParseOptions& options, + memory::MemoryPool* pool) { + auto untyped = parse::parseExpr(text, options); + return core::Expressions::inferTypes(untyped, rowType, pool); +} + +std::shared_ptr buildHiveBucketProperty( + const RowTypePtr rowType, + int32_t bucketCount, + const std::vector& bucketColumns, + const std::vector>& sortBy) { + std::vector bucketTypes; + bucketTypes.reserve(bucketColumns.size()); + for (const auto& bucketColumn : bucketColumns) { + bucketTypes.push_back(rowType->childAt(rowType->getChildIdx(bucketColumn))); + } + return std::make_shared( + HiveBucketProperty::Kind::kHiveCompatible, + bucketCount, + bucketColumns, + bucketTypes, + sortBy); +} +} // namespace + +PlanBuilder& PlanBuilder::tableScan( + const RowTypePtr& outputType, + const std::vector& subfieldFilters, + const std::string& remainingFilter, + const RowTypePtr& dataColumns, + const connector::ColumnHandleMap& assignments) { + return TableScanBuilder(*this) + .filtersAsNode(filtersAsNode_ ? planNodeIdGenerator_ : nullptr) + .outputType(outputType) + .assignments(assignments) + .dataColumns(dataColumns) + .subfieldFilters(subfieldFilters) + .remainingFilter(remainingFilter) + .endTableScan(); +} + +PlanBuilder& PlanBuilder::tableScan( + const std::string& tableName, + const RowTypePtr& outputType, + const std::unordered_map& columnAliases, + const std::vector& subfieldFilters, + const std::string& remainingFilter, + const RowTypePtr& dataColumns, + const connector::ColumnHandleMap& assignments) { + return TableScanBuilder(*this) + .filtersAsNode(filtersAsNode_ ? planNodeIdGenerator_ : nullptr) + .tableName(tableName) + .outputType(outputType) + .columnAliases(columnAliases) + .dataColumns(dataColumns) + + .subfieldFilters(subfieldFilters) + .remainingFilter(remainingFilter) + .assignments(assignments) + .endTableScan(); +} + +PlanBuilder& PlanBuilder::tableScanWithPushDown( + const RowTypePtr& outputType, + const PushdownConfig& pushdownConfig, + const RowTypePtr& dataColumns, + const connector::ColumnHandleMap& assignments) { + return TableScanBuilder(*this) + .filtersAsNode(filtersAsNode_ ? planNodeIdGenerator_ : nullptr) + .outputType(outputType) + .assignments(assignments) + .dataColumns(dataColumns) + .subfieldFiltersMap(pushdownConfig.subfieldFiltersMap) + .remainingFilter(pushdownConfig.remainingFilter) + .endTableScan(); +} + +PlanBuilder::TableScanBuilder& PlanBuilder::TableScanBuilder::subfieldFilters( + std::vector subfieldFilters) { + VELOX_CHECK(subfieldFiltersMap_.empty()); + + if (subfieldFilters.empty()) { + return *this; + } + + // Parse subfield filters + auto queryCtx = core::QueryCtx::create(); + exec::SimpleExpressionEvaluator evaluator(queryCtx.get(), planBuilder_.pool_); + const RowTypePtr& parseType = dataColumns_ ? dataColumns_ : outputType_; + + for (const auto& filter : subfieldFilters) { + auto untypedExpr = parse::parseExpr(filter, planBuilder_.options_); + + // Parse directly to subfieldFiltersMap_ + auto filterExpr = core::Expressions::inferTypes( + untypedExpr, parseType, planBuilder_.pool_); + auto [subfield, subfieldFilter] = + exec::toSubfieldFilter(filterExpr, &evaluator); + + auto it = columnAliases_.find(subfield.toString()); + if (it != columnAliases_.end()) { + subfield = velox::common::Subfield(it->second); + } + VELOX_CHECK_EQ( + subfieldFiltersMap_.count(subfield), + 0, + "Duplicate subfield: {}", + subfield.toString()); + + subfieldFiltersMap_[std::move(subfield)] = std::move(subfieldFilter); + } + return *this; +} + +PlanBuilder::TableScanBuilder& +PlanBuilder::TableScanBuilder::subfieldFiltersMap( + const velox::common::SubfieldFilters& filtersMap) { + for (const auto& [k, v] : filtersMap) { + subfieldFiltersMap_[k.clone()] = v->clone(); + } + return *this; +} + +PlanBuilder::TableScanBuilder& PlanBuilder::TableScanBuilder::remainingFilter( + std::string remainingFilter) { + if (!remainingFilter.empty()) { + remainingFilter_ = parse::parseExpr(remainingFilter, planBuilder_.options_); + } + return *this; +} + +namespace { +void addConjunct( + const core::TypedExprPtr& conjunct, + core::TypedExprPtr& conjunction) { + if (!conjunction) { + conjunction = conjunct; + } else { + conjunction = std::make_shared( + BOOLEAN(), + std::vector{conjunction, conjunct}, + "and"); + } +} +} // namespace + +core::PlanNodePtr PlanBuilder::TableScanBuilder::build(core::PlanNodeId id) { + VELOX_CHECK_NOT_NULL(outputType_, "outputType must be specified"); + std::unordered_map typedMapping; + bool hasAssignments = !(assignments_.empty()); + for (uint32_t i = 0; i < outputType_->size(); ++i) { + const auto& name = outputType_->nameOf(i); + const auto& type = outputType_->childAt(i); + + std::string hiveColumnName = name; + auto it = columnAliases_.find(name); + if (it != columnAliases_.end()) { + hiveColumnName = it->second; + typedMapping.emplace( + name, + std::make_shared(type, hiveColumnName)); + } + + if (!hasAssignments) { + assignments_.insert( + {name, + std::make_shared( + hiveColumnName, + HiveColumnHandle::ColumnType::kRegular, + type, + type)}); + } + } + + const RowTypePtr& parseType = dataColumns_ ? dataColumns_ : outputType_; + + core::TypedExprPtr filterNodeExpr; + + if (filtersAsNode_) { + for (const auto& [subfield, filter] : subfieldFiltersMap_) { + auto filterExpr = core::filterToExpr( + subfield, filter.get(), parseType, planBuilder_.pool_); + + addConjunct(filterExpr, filterNodeExpr); + } + + subfieldFiltersMap_.clear(); + } + + core::TypedExprPtr remainingFilterExpr; + if (remainingFilter_) { + remainingFilterExpr = core::Expressions::inferTypes( + remainingFilter_, parseType, planBuilder_.pool_) + ->rewriteInputNames(typedMapping); + if (filtersAsNode_) { + addConjunct(remainingFilterExpr, filterNodeExpr); + remainingFilterExpr = nullptr; + } + } + + if (!tableHandle_) { + tableHandle_ = std::make_shared( + connectorId_, + tableName_, + true, + std::move(subfieldFiltersMap_), + remainingFilterExpr, + dataColumns_); + } + core::PlanNodePtr result = std::make_shared( + id, outputType_, tableHandle_, assignments_); + + if (filtersAsNode_ && filterNodeExpr) { + auto filterId = planNodeIdGenerator_->next(); + result = + std::make_shared(filterId, filterNodeExpr, result); + } + return result; +} + +core::PlanNodePtr PlanBuilder::TableWriterBuilder::build(core::PlanNodeId id) { + auto upstreamNode = planBuilder_.planNode(); + VELOX_CHECK_NOT_NULL(upstreamNode, "TableWrite cannot be the source node"); + + // If outputType wasn't explicit specified, fallback to use the output of the + // upstream operator. + auto outputType = outputType_ ? outputType_ : upstreamNode->outputType(); + + // If insertHandle_ is not specified, build a HiveInsertTableHandle along with + // columnHandles, bucketProperty and locationHandle. + if (!insertHandle_) { + // Create column handles. + std::vector< + std::shared_ptr> + columnHandles; + for (auto i = 0; i < outputType->size(); ++i) { + const auto column = outputType->nameOf(i); + const bool isPartitionKey = + std::find(partitionBy_.begin(), partitionBy_.end(), column) != + partitionBy_.end(); + columnHandles.push_back( + std::make_shared( + column, + isPartitionKey ? connector::lakehouse::common::HiveColumnHandle:: + ColumnType::kPartitionKey + : connector::lakehouse::common::HiveColumnHandle:: + ColumnType::kRegular, + outputType->childAt(i), + outputType->childAt(i))); + } + + auto locationHandle = + std::make_shared( + outputDirectoryPath_, + outputDirectoryPath_, + connector::lakehouse::common::LocationHandle::TableType::kNew, + outputFileName_); + + std::shared_ptr bucketProperty; + if (bucketCount_ != 0) { + bucketProperty = buildHiveBucketProperty( + outputType, bucketCount_, bucketedBy_, sortBy_); + } + + auto hiveHandle = + std::make_shared( + columnHandles, + locationHandle, + fileFormat_, + bucketProperty, + compressionKind_, + serdeParameters_, + options_, + ensureFiles_); + + insertHandle_ = + std::make_shared(connectorId_, hiveHandle); + } + + std::shared_ptr aggregationNode; + if (!aggregates_.empty()) { + auto aggregatesAndNames = planBuilder_.createAggregateExpressionsAndNames( + aggregates_, {}, core::AggregationNode::Step::kPartial); + aggregationNode = std::make_shared( + planBuilder_.nextPlanNodeId(), + core::AggregationNode::Step::kPartial, + std::vector{}, // groupingKeys + std::vector{}, // preGroupedKeys + aggregatesAndNames.names, // ignoreNullKeys + aggregatesAndNames.aggregates, + false, + upstreamNode); + VELOX_CHECK_EQ( + aggregationNode->supportsBarrier(), aggregationNode->isPreGrouped()); + } + + const auto writeNode = std::make_shared( + id, + outputType, + outputType->names(), + aggregationNode, + insertHandle_, + false, + TableWriteTraits::outputType(aggregationNode), + commitStrategy_, + upstreamNode); + VELOX_CHECK(!writeNode->supportsBarrier()); + return writeNode; +} + +PlanBuilder& PlanBuilder::values( + const std::vector& values, + bool parallelizable, + size_t repeatTimes) { + VELOX_CHECK_NULL(planNode_, "Values must be the source node"); + auto valuesCopy = values; + planNode_ = std::make_shared( + nextPlanNodeId(), std::move(valuesCopy), parallelizable, repeatTimes); + VELOX_CHECK(!planNode_->supportsBarrier()); + return *this; +} + +PlanBuilder& PlanBuilder::traceScan( + const std::string& traceNodeDir, + uint32_t pipelineId, + std::vector driverIds, + const RowTypePtr& outputType) { + planNode_ = std::make_shared( + nextPlanNodeId(), + traceNodeDir, + pipelineId, + std::move(driverIds), + outputType); + VELOX_CHECK(!planNode_->supportsBarrier()); + return *this; +} + +PlanBuilder& PlanBuilder::exchange( + const RowTypePtr& outputType, + VectorSerde::Kind serdeKind) { + VELOX_CHECK_NULL(planNode_, "Exchange must be the source node"); + planNode_ = std::make_shared( + nextPlanNodeId(), outputType, serdeKind); + VELOX_CHECK(!planNode_->supportsBarrier()); + return *this; +} + +namespace { +std::pair< + std::vector>, + std::vector> +parseOrderByClauses( + const std::vector& keys, + const RowTypePtr& inputType, + memory::MemoryPool* pool) { + std::vector> sortingKeys; + std::vector sortingOrders; + for (const auto& key : keys) { + auto orderBy = parse::parseOrderByExpr(key); + auto typedExpr = + core::Expressions::inferTypes(orderBy.expr, inputType, pool); + + auto sortingKey = + std::dynamic_pointer_cast(typedExpr); + VELOX_CHECK_NOT_NULL( + sortingKey, + "ORDER BY clause must use a column name, not an expression: {}", + key); + sortingKeys.emplace_back(sortingKey); + sortingOrders.emplace_back(orderBy.ascending, orderBy.nullsFirst); + } + + return {sortingKeys, sortingOrders}; +} +} // namespace + +PlanBuilder& PlanBuilder::mergeExchange( + const RowTypePtr& outputType, + const std::vector& keys, + VectorSerde::Kind serdeKind) { + VELOX_CHECK_NULL(planNode_, "MergeExchange must be the source node"); + auto [sortingKeys, sortingOrders] = + parseOrderByClauses(keys, outputType, pool_); + + planNode_ = std::make_shared( + nextPlanNodeId(), outputType, sortingKeys, sortingOrders, serdeKind); + VELOX_CHECK(!planNode_->supportsBarrier()); + return *this; +} + +PlanBuilder& PlanBuilder::optionalProject( + const std::vector& optionalProjections) { + if (optionalProjections.empty()) { + return *this; + } + return project(optionalProjections); +} + +PlanBuilder& PlanBuilder::projectExpressions( + const std::vector>& projections) { + std::vector expressions; + std::vector projectNames; + for (auto i = 0; i < projections.size(); ++i) { + expressions.push_back(inferTypes(projections[i])); + if (projections[i]->alias().has_value()) { + projectNames.push_back(projections[i]->alias().value()); + } else if ( + auto fieldExpr = + dynamic_cast(projections[i].get())) { + projectNames.push_back(fieldExpr->name()); + } else { + projectNames.push_back(fmt::format("p{}", i)); + } + } + planNode_ = std::make_shared( + nextPlanNodeId(), + std::move(projectNames), + std::move(expressions), + planNode_); + VELOX_CHECK(planNode_->supportsBarrier()); + return *this; +} + +PlanBuilder& PlanBuilder::projectExpressions( + const std::vector>& projections) { + std::vector expressions; + std::vector projectNames; + for (auto i = 0; i < projections.size(); ++i) { + expressions.push_back(projections[i]); + if (auto fieldExpr = + dynamic_cast(projections[i].get())) { + projectNames.push_back(fieldExpr->name()); + } else { + projectNames.push_back(fmt::format("p{}", i)); + } + } + planNode_ = std::make_shared( + nextPlanNodeId(), + std::move(projectNames), + std::move(expressions), + planNode_); + VELOX_CHECK(planNode_->supportsBarrier()); + return *this; +} + +PlanBuilder& PlanBuilder::project(const std::vector& projections) { + VELOX_CHECK_NOT_NULL(planNode_, "Project cannot be the source node"); + std::vector> expressions; + expressions.reserve(projections.size()); + for (auto i = 0; i < projections.size(); ++i) { + expressions.push_back(parse::parseExpr(projections[i], options_)); + } + return projectExpressions(expressions); +} + +PlanBuilder& PlanBuilder::parallelProject( + const std::vector>& projectionGroups, + const std::vector& noLoadColumns) { + VELOX_CHECK_NOT_NULL(planNode_, "ParallelProject cannot be the source node"); + + std::vector names; + + std::vector> exprGroups; + exprGroups.reserve(projectionGroups.size()); + + size_t i = 0; + + for (const auto& group : projectionGroups) { + std::vector typedExprs; + typedExprs.reserve(group.size()); + + for (const auto& expr : group) { + const auto typedExpr = inferTypes(parse::parseExpr(expr, options_)); + typedExprs.push_back(typedExpr); + + if (auto fieldExpr = + dynamic_cast(typedExpr.get())) { + names.push_back(fieldExpr->name()); + } else { + names.push_back(fmt::format("p{}", i)); + } + + ++i; + } + exprGroups.push_back(std::move(typedExprs)); + } + + planNode_ = std::make_shared( + nextPlanNodeId(), + std::move(names), + std::move(exprGroups), + noLoadColumns, + planNode_); + + return *this; +} + +PlanBuilder& PlanBuilder::lazyDereference( + const std::vector& projections) { + VELOX_CHECK_NOT_NULL(planNode_, "LazyDeference cannot be the source node"); + std::vector expressions; + std::vector projectNames; + for (auto i = 0; i < projections.size(); ++i) { + auto expr = inferTypes(parse::parseExpr(projections[i], options_)); + expressions.push_back(expr); + if (auto* fieldExpr = + dynamic_cast(expr.get())) { + projectNames.push_back(fieldExpr->name()); + } else { + projectNames.push_back(fmt::format("p{}", i)); + } + } + planNode_ = std::make_shared( + nextPlanNodeId(), + std::move(projectNames), + std::move(expressions), + planNode_); + VELOX_CHECK(planNode_->supportsBarrier()); + return *this; +} + +PlanBuilder& PlanBuilder::appendColumns( + const std::vector& newColumns) { + VELOX_CHECK_NOT_NULL(planNode_, "Project cannot be the source node"); + std::vector allProjections = planNode_->outputType()->names(); + for (const auto& column : newColumns) { + allProjections.push_back(column); + } + + return project(allProjections); +} + +PlanBuilder& PlanBuilder::optionalFilter(const std::string& optionalFilter) { + if (optionalFilter.empty()) { + return *this; + } + return filter(optionalFilter); +} + +PlanBuilder& PlanBuilder::filter(const std::string& filter) { + VELOX_CHECK_NOT_NULL(planNode_, "Filter cannot be the source node"); + auto expr = parseExpr(filter, planNode_->outputType(), options_, pool_); + planNode_ = + std::make_shared(nextPlanNodeId(), expr, planNode_); + VELOX_CHECK(planNode_->supportsBarrier()); + return *this; +} + +PlanBuilder& PlanBuilder::tableWrite( + const std::string& outputDirectoryPath, + const dwio::common::FileFormat fileFormat, + const std::vector& aggregates, + const std::shared_ptr& options, + const std::string& outputFileName) { + return TableWriterBuilder(*this) + .outputDirectoryPath(outputDirectoryPath) + .outputFileName(outputFileName) + .fileFormat(fileFormat) + .aggregates(aggregates) + .options(options) + .endTableWriter(); +} + +PlanBuilder& PlanBuilder::tableWrite( + const std::string& outputDirectoryPath, + const std::vector& partitionBy, + const dwio::common::FileFormat fileFormat, + const std::vector& aggregates, + const std::shared_ptr& options) { + return TableWriterBuilder(*this) + .outputDirectoryPath(outputDirectoryPath) + .partitionBy(partitionBy) + .fileFormat(fileFormat) + .aggregates(aggregates) + .options(options) + .endTableWriter(); +} + +PlanBuilder& PlanBuilder::tableWrite( + const std::string& outputDirectoryPath, + const std::vector& partitionBy, + int32_t bucketCount, + const std::vector& bucketedBy, + const dwio::common::FileFormat fileFormat, + const std::vector& aggregates, + const std::shared_ptr& options) { + return TableWriterBuilder(*this) + .outputDirectoryPath(outputDirectoryPath) + .partitionBy(partitionBy) + .bucketCount(bucketCount) + .bucketedBy(bucketedBy) + .fileFormat(fileFormat) + .aggregates(aggregates) + .options(options) + .endTableWriter(); +} + +PlanBuilder& PlanBuilder::tableWrite( + const std::string& outputDirectoryPath, + const std::vector& partitionBy, + int32_t bucketCount, + const std::vector& bucketedBy, + const std::vector>& sortBy, + const dwio::common::FileFormat fileFormat, + const std::vector& aggregates, + const std::string_view& connectorId, + const std::unordered_map& serdeParameters, + const std::shared_ptr& options, + const std::string& outputFileName, + const velox::common::CompressionKind compressionKind, + const RowTypePtr& schema, + const bool ensureFiles, + const connector::CommitStrategy commitStrategy) { + return TableWriterBuilder(*this) + .outputDirectoryPath(outputDirectoryPath) + .outputFileName(outputFileName) + .outputType(schema) + .partitionBy(partitionBy) + .bucketCount(bucketCount) + .bucketedBy(bucketedBy) + .sortBy(sortBy) + .fileFormat(fileFormat) + .aggregates(aggregates) + .connectorId(connectorId) + .serdeParameters(serdeParameters) + .options(options) + .compressionKind(compressionKind) + .ensureFiles(ensureFiles) + .commitStrategy(commitStrategy) + .endTableWriter(); +} + +PlanBuilder& PlanBuilder::tableWriteMerge( + const core::AggregationNodePtr& aggregationNode) { + planNode_ = std::make_shared( + nextPlanNodeId(), + TableWriteTraits::outputType(aggregationNode), + aggregationNode, + planNode_); + VELOX_CHECK(!planNode_->supportsBarrier()); + return *this; +} + +namespace { + +std::string throwAggregateFunctionDoesntExist(const std::string& name) { + std::stringstream error; + error << "Aggregate function doesn't exist: " << name << "."; + exec::aggregateFunctions().withRLock([&](const auto& functionsMap) { + if (functionsMap.empty()) { + error << " Registry of aggregate functions is empty. " + "Make sure to register some aggregate functions."; + } + }); + VELOX_USER_FAIL(error.str()); +} + +std::string throwAggregateFunctionSignatureNotSupported( + const std::string& name, + const std::vector& types, + const std::vector>& + signatures) { + std::stringstream error; + error << "Aggregate function signature is not supported: " + << exec::toString(name, types) + << ". Supported signatures: " << toString(signatures) << "."; + VELOX_USER_FAIL(error.str()); +} + +TypePtr resolveAggregateType( + const std::string& aggregateName, + core::AggregationNode::Step step, + const std::vector& rawInputTypes, + bool nullOnFailure) { + if (auto signatures = exec::getAggregateFunctionSignatures(aggregateName)) { + for (const auto& signature : signatures.value()) { + exec::SignatureBinder binder(*signature, rawInputTypes); + if (binder.tryBind()) { + return binder.tryResolveType( + exec::isPartialOutput(step) ? signature->intermediateType() + : signature->returnType()); + } + } + + if (nullOnFailure) { + return nullptr; + } + + throwAggregateFunctionSignatureNotSupported( + aggregateName, rawInputTypes, signatures.value()); + } + + // We may be parsing lambda expression used in a lambda aggregate function. In + // this case, 'aggregateName' would refer to a scalar function. + // + // TODO Enhance the parser to allow for specifying separate resolver for + // lambda expressions. + if (auto type = + exec::resolveTypeForSpecialForm(aggregateName, rawInputTypes)) { + return type; + } + + if (auto type = parse::resolveScalarFunctionType( + aggregateName, rawInputTypes, true)) { + return type; + } + + if (nullOnFailure) { + return nullptr; + } + + throwAggregateFunctionDoesntExist(aggregateName); + return nullptr; +} + +class AggregateTypeResolver { + public: + explicit AggregateTypeResolver(core::AggregationNode::Step step) + : step_(step), previousHook_(core::Expressions::getResolverHook()) { + core::Expressions::setTypeResolverHook( + [&](const auto& inputs, const auto& expr, bool nullOnFailure) { + return resolveType(inputs, expr, nullOnFailure); + }); + } + + ~AggregateTypeResolver() { + core::Expressions::setTypeResolverHook(previousHook_); + } + + void setRawInputTypes(const std::vector& types) { + rawInputTypes_ = types; + } + + private: + TypePtr resolveType( + const std::vector& inputs, + const std::shared_ptr& expr, + bool nullOnFailure) const { + auto functionName = expr->name(); + + // Use raw input types (if available) to resolve intermediate and final + // result types. + if (exec::isRawInput(step_)) { + std::vector types; + for (auto& input : inputs) { + types.push_back(input->type()); + } + + return resolveAggregateType(functionName, step_, types, nullOnFailure); + } + + if (!rawInputTypes_.empty()) { + return resolveAggregateType( + functionName, step_, rawInputTypes_, nullOnFailure); + } + + if (!nullOnFailure) { + VELOX_USER_FAIL( + "Cannot resolve aggregation function return type without raw input types: {}", + functionName); + } + return nullptr; + } + + const core::AggregationNode::Step step_; + const core::Expressions::TypeResolverHook previousHook_; + std::vector rawInputTypes_; +}; + +} // namespace + +core::PlanNodePtr PlanBuilder::createIntermediateOrFinalAggregation( + core::AggregationNode::Step step, + const core::AggregationNode* partialAggNode) { + // Create intermediate or final aggregation using same grouping keys and same + // aggregate function names. + const auto& partialAggregates = partialAggNode->aggregates(); + const auto& groupingKeys = partialAggNode->groupingKeys(); + + auto numAggregates = partialAggregates.size(); + auto numGroupingKeys = groupingKeys.size(); + + std::vector aggregates; + aggregates.reserve(numAggregates); + for (auto i = 0; i < numAggregates; i++) { + // Resolve final or intermediate aggregation result type using raw input + // types for the partial aggregation. + auto name = partialAggregates[i].call->name(); + auto rawInputs = partialAggregates[i].call->inputs(); + + core::AggregationNode::Aggregate aggregate; + for (auto& rawInput : rawInputs) { + aggregate.rawInputTypes.push_back(rawInput->type()); + } + + auto type = + resolveAggregateType(name, step, aggregate.rawInputTypes, false); + std::vector inputs = {field(numGroupingKeys + i)}; + + // Add lambda inputs. + for (const auto& rawInput : rawInputs) { + if (rawInput->type()->kind() == TypeKind::FUNCTION) { + inputs.push_back(rawInput); + } + } + + aggregate.call = + std::make_shared(type, std::move(inputs), name); + aggregates.emplace_back(aggregate); + } + + auto aggregationNode = std::make_shared( + nextPlanNodeId(), + step, + groupingKeys, + partialAggNode->preGroupedKeys(), + partialAggNode->aggregateNames(), + aggregates, + partialAggNode->ignoreNullKeys(), + planNode_); + VELOX_CHECK_EQ( + aggregationNode->supportsBarrier(), aggregationNode->isPreGrouped()); + return aggregationNode; +} + +namespace { +/// Checks that specified plan node is a partial or intermediate aggregation or +/// local exchange over the same. Returns a pointer to core::AggregationNode. +const core::AggregationNode* findPartialAggregation( + const core::PlanNode* planNode) { + const core::AggregationNode* aggNode; + if (auto exchange = dynamic_cast(planNode)) { + aggNode = dynamic_cast( + exchange->sources()[0].get()); + } else if (auto merge = dynamic_cast(planNode)) { + aggNode = + dynamic_cast(merge->sources()[0].get()); + } else { + aggNode = dynamic_cast(planNode); + } + VELOX_CHECK_NOT_NULL( + aggNode, + "Current plan node must be one of: partial or intermediate aggregation, " + "local merge or exchange. Got: {}", + planNode->toString()); + VELOX_CHECK(exec::isPartialOutput(aggNode->step())); + return aggNode; +} +} // namespace + +PlanBuilder& PlanBuilder::intermediateAggregation() { + const auto* aggNode = findPartialAggregation(planNode_.get()); + VELOX_CHECK(exec::isRawInput(aggNode->step())); + + auto step = core::AggregationNode::Step::kIntermediate; + + planNode_ = createIntermediateOrFinalAggregation(step, aggNode); + return *this; +} + +PlanBuilder& PlanBuilder::finalAggregation() { + const auto* aggNode = findPartialAggregation(planNode_.get()); + + if (!exec::isRawInput(aggNode->step())) { + // If aggregation node is not the partial aggregation, keep looking again. + aggNode = findPartialAggregation(aggNode->sources()[0].get()); + VELOX_CHECK_NOT_NULL(aggNode); + } + + VELOX_CHECK(exec::isRawInput(aggNode->step())); + VELOX_CHECK(exec::isPartialOutput(aggNode->step())); + + auto step = core::AggregationNode::Step::kFinal; + + planNode_ = createIntermediateOrFinalAggregation(step, aggNode); + return *this; +} + +PlanBuilder::AggregatesAndNames PlanBuilder::createAggregateExpressionsAndNames( + const std::vector& aggregates, + const std::vector& masks, + core::AggregationNode::Step step, + const std::vector>& rawInputTypes) { + if (step == core::AggregationNode::Step::kPartial || + step == core::AggregationNode::Step::kSingle) { + VELOX_CHECK( + rawInputTypes.empty(), + "Do not provide raw inputs types for partial or single aggregation"); + } else { + VELOX_CHECK_EQ( + aggregates.size(), + rawInputTypes.size(), + "Do provide raw inputs types for final or intermediate aggregation"); + } + + std::vector aggs; + + AggregateTypeResolver resolver(step); + std::vector names; + aggs.reserve(aggregates.size()); + names.reserve(aggregates.size()); + + duckdb::ParseOptions options; + options.parseIntegerAsBigint = options_.parseIntegerAsBigint; + + for (auto i = 0; i < aggregates.size(); i++) { + auto& aggregate = aggregates[i]; + + if (!rawInputTypes.empty()) { + resolver.setRawInputTypes(rawInputTypes[i]); + } + + auto untypedExpr = duckdb::parseAggregateExpr(aggregate, options); + + core::AggregationNode::Aggregate agg; + + agg.call = std::dynamic_pointer_cast( + inferTypes(untypedExpr.expr)); + + if (step == core::AggregationNode::Step::kPartial || + step == core::AggregationNode::Step::kSingle) { + for (const auto& input : agg.call->inputs()) { + agg.rawInputTypes.push_back(input->type()); + } + } else { + agg.rawInputTypes = rawInputTypes[i]; + } + + if (untypedExpr.maskExpr != nullptr) { + auto maskExpr = + std::dynamic_pointer_cast( + inferTypes(untypedExpr.maskExpr)); + VELOX_CHECK_NOT_NULL( + maskExpr, + "FILTER clause must use a column name, not an expression: {}", + aggregate); + agg.mask = maskExpr; + } + + if (i < masks.size() && !masks[i].empty()) { + VELOX_CHECK_NULL( + agg.mask, + "Aggregation mask should be specified only once (either explicitly or using FILTER clause)"); + agg.mask = field(masks[i]); + } + + agg.distinct = untypedExpr.distinct; + + if (!untypedExpr.orderBy.empty()) { + auto* entry = exec::getAggregateFunctionEntry(agg.call->name()); + const auto& metadata = entry->metadata; + if (metadata.orderSensitive) { + VELOX_CHECK( + step == core::AggregationNode::Step::kSingle, + "Order sensitive aggregation over sorted inputs cannot be split " + "into partial and final: {}.", + aggregate); + } + } + + for (const auto& orderBy : untypedExpr.orderBy) { + auto sortingKey = + std::dynamic_pointer_cast( + inferTypes(orderBy.expr)); + VELOX_CHECK_NOT_NULL( + sortingKey, + "ORDER BY clause must use a column name, not an expression: {}", + aggregate); + + agg.sortingKeys.push_back(sortingKey); + agg.sortingOrders.emplace_back(orderBy.ascending, orderBy.nullsFirst); + } + + aggs.emplace_back(agg); + + if (untypedExpr.expr->alias().has_value()) { + names.push_back(untypedExpr.expr->alias().value()); + } else { + names.push_back(fmt::format("a{}", i)); + } + } + + return {aggs, names}; +} + +PlanBuilder& PlanBuilder::aggregation( + const std::vector& groupingKeys, + const std::vector& preGroupedKeys, + const std::vector& aggregates, + const std::vector& masks, + core::AggregationNode::Step step, + bool ignoreNullKeys, + const std::vector>& rawInputTypes) { + auto aggregatesAndNames = createAggregateExpressionsAndNames( + aggregates, masks, step, rawInputTypes); + + // If the aggregationNode is over a GroupId, then global grouping sets + // need to be populated. + std::vector globalGroupingSets; + std::optional groupId; + if (auto groupIdNode = + dynamic_cast(planNode_.get())) { + for (auto i = 0; i < groupIdNode->groupingSets().size(); i++) { + if (groupIdNode->groupingSets().at(i).empty()) { + globalGroupingSets.push_back(i); + } + } + + if (!globalGroupingSets.empty()) { + // GroupId is the last column of the GroupIdNode. + groupId = field(groupIdNode->outputType()->names().back()); + } + } + + auto aggregationNode = std::make_shared( + nextPlanNodeId(), + step, + fields(groupingKeys), + fields(preGroupedKeys), + aggregatesAndNames.names, + aggregatesAndNames.aggregates, + globalGroupingSets, + groupId, + ignoreNullKeys, + planNode_); + VELOX_CHECK_EQ( + aggregationNode->supportsBarrier(), aggregationNode->isPreGrouped()); + planNode_ = std::move(aggregationNode); + return *this; +} + +PlanBuilder& PlanBuilder::streamingAggregation( + const std::vector& groupingKeys, + const std::vector& aggregates, + const std::vector& masks, + core::AggregationNode::Step step, + bool ignoreNullKeys) { + auto aggregatesAndNames = + createAggregateExpressionsAndNames(aggregates, masks, step); + auto aggregationNode = std::make_shared( + nextPlanNodeId(), + step, + fields(groupingKeys), + fields(groupingKeys), + aggregatesAndNames.names, + aggregatesAndNames.aggregates, + ignoreNullKeys, + planNode_); + VELOX_CHECK_EQ( + aggregationNode->supportsBarrier(), aggregationNode->isPreGrouped()); + planNode_ = std::move(aggregationNode); + return *this; +} + +PlanBuilder& PlanBuilder::groupId( + const std::vector& groupingKeys, + const std::vector>& groupingSets, + const std::vector& aggregationInputs, + std::string groupIdName) { + std::vector groupingKeyInfos; + groupingKeyInfos.reserve(groupingKeys.size()); + for (const auto& groupingKey : groupingKeys) { + auto untypedExpr = parse::parseExpr(groupingKey, options_); + const auto* fieldAccessExpr = + dynamic_cast(untypedExpr.get()); + VELOX_USER_CHECK( + fieldAccessExpr, + "Grouping key {} is not valid projection", + groupingKey); + std::string inputField = fieldAccessExpr->name(); + std::string outputField = untypedExpr->alias().has_value() + ? + // This is a projection with a column alias with the format + // "input_col as output_col". + untypedExpr->alias().value() + : + // This is a projection without a column alias. + fieldAccessExpr->name(); + + core::GroupIdNode::GroupingKeyInfo keyInfos; + keyInfos.output = outputField; + keyInfos.input = field(inputField); + groupingKeyInfos.push_back(keyInfos); + } + + planNode_ = std::make_shared( + nextPlanNodeId(), + groupingSets, + std::move(groupingKeyInfos), + fields(aggregationInputs), + std::move(groupIdName), + planNode_); + VELOX_CHECK(!planNode_->supportsBarrier()); + return *this; +} + +namespace { +core::PlanNodePtr createLocalMergeNode( + const core::PlanNodeId& id, + const std::vector& keys, + std::vector sources, + memory::MemoryPool* pool) { + const auto& inputType = sources[0]->outputType(); + auto [sortingKeys, sortingOrders] = + parseOrderByClauses(keys, inputType, pool); + + return std::make_shared( + id, std::move(sortingKeys), std::move(sortingOrders), std::move(sources)); +} +} // namespace + +PlanBuilder& PlanBuilder::localMerge(const std::vector& keys) { + planNode_ = createLocalMergeNode(nextPlanNodeId(), keys, {planNode_}, pool_); + return *this; +} + +PlanBuilder& PlanBuilder::expand( + const std::vector>& projections) { + VELOX_CHECK(!projections.empty(), "projections must not be empty."); + const auto numColumns = projections[0].size(); + const auto numRows = projections.size(); + std::vector aliases; + aliases.reserve(numColumns); + + std::vector> projectExprs; + projectExprs.reserve(projections.size()); + + for (auto i = 0; i < numRows; i++) { + std::vector projectExpr; + VELOX_CHECK_EQ(numColumns, projections[i].size()); + for (auto j = 0; j < numColumns; j++) { + auto untypedExpression = parse::parseExpr(projections[i][j], options_); + auto typedExpression = inferTypes(untypedExpression); + + if (i == 0) { + if (untypedExpression->alias().has_value()) { + aliases.push_back(untypedExpression->alias().value()); + } else { + auto fieldExpr = dynamic_cast( + untypedExpression.get()); + VELOX_CHECK_NOT_NULL(fieldExpr); + aliases.push_back(fieldExpr->name()); + } + projectExpr.push_back(typedExpression); + } else { + // The types of values in 2nd and subsequent rows must much types in the + // 1st row. + const auto& expectedType = projectExprs[0][j]->type(); + if (typedExpression->type()->equivalent(*expectedType)) { + projectExpr.push_back(typedExpression); + } else { + auto constantExpr = + dynamic_cast(untypedExpression.get()); + VELOX_CHECK_NOT_NULL(constantExpr); + VELOX_CHECK(constantExpr->value().isNull()); + projectExpr.push_back( + std::make_shared( + expectedType, variant::null(expectedType->kind()))); + } + } + } + projectExprs.push_back(projectExpr); + } + + planNode_ = std::make_shared( + nextPlanNodeId(), projectExprs, std::move(aliases), planNode_); + VELOX_CHECK(!planNode_->supportsBarrier()); + return *this; +} + +PlanBuilder& PlanBuilder::localMerge( + const std::vector& keys, + std::vector sources) { + VELOX_CHECK_NULL(planNode_, "localMerge() must be the first call"); + VELOX_CHECK_GE( + sources.size(), 1, "localMerge() requires at least one source"); + + planNode_ = + createLocalMergeNode(nextPlanNodeId(), keys, std::move(sources), pool_); + return *this; +} + +PlanBuilder& PlanBuilder::orderBy( + const std::vector& keys, + bool isPartial) { + VELOX_CHECK_NOT_NULL(planNode_, "OrderBy cannot be the source node"); + auto [sortingKeys, sortingOrders] = + parseOrderByClauses(keys, planNode_->outputType(), pool_); + + planNode_ = std::make_shared( + nextPlanNodeId(), sortingKeys, sortingOrders, isPartial, planNode_); + VELOX_CHECK(!planNode_->supportsBarrier()); + return *this; +} + +PlanBuilder& PlanBuilder::topN( + const std::vector& keys, + int32_t count, + bool isPartial) { + VELOX_CHECK_NOT_NULL(planNode_, "TopN cannot be the source node"); + auto [sortingKeys, sortingOrders] = + parseOrderByClauses(keys, planNode_->outputType(), pool_); + planNode_ = std::make_shared( + nextPlanNodeId(), + sortingKeys, + sortingOrders, + count, + isPartial, + planNode_); + VELOX_CHECK(!planNode_->supportsBarrier()); + return *this; +} + +PlanBuilder& PlanBuilder::limit(int64_t offset, int64_t count, bool isPartial) { + planNode_ = std::make_shared( + nextPlanNodeId(), offset, count, isPartial, planNode_); + VELOX_CHECK(planNode_->supportsBarrier()); + return *this; +} + +PlanBuilder& PlanBuilder::enforceSingleRow() { + planNode_ = + std::make_shared(nextPlanNodeId(), planNode_); + return *this; +} + +PlanBuilder& PlanBuilder::assignUniqueId( + const std::string& idName, + const int32_t taskUniqueId) { + planNode_ = std::make_shared( + nextPlanNodeId(), idName, taskUniqueId, planNode_); + VELOX_CHECK(planNode_->supportsBarrier()); + return *this; +} + +namespace { +core::PartitionFunctionSpecPtr createPartitionFunctionSpec( + const RowTypePtr& inputType, + const std::vector& keys, + memory::MemoryPool* pool) { + if (keys.empty()) { + return std::make_shared(); + } else { + std::vector keyIndices; + keyIndices.reserve(keys.size()); + + std::vector constValues; + constValues.reserve(keys.size()); + + for (const auto& key : keys) { + if (auto field = + std::dynamic_pointer_cast( + key)) { + keyIndices.push_back(inputType->getChildIdx(field->name())); + } else if ( + auto constant = + std::dynamic_pointer_cast(key)) { + keyIndices.push_back(kConstantChannel); + constValues.push_back(constant->toConstantVector(pool)); + } else { + VELOX_UNREACHABLE(); + } + } + return std::make_shared( + inputType, std::move(keyIndices), std::move(constValues)); + } +} + +RowTypePtr concat(const RowTypePtr& a, const RowTypePtr& b) { + std::vector names = a->names(); + std::vector types = a->children(); + names.insert(names.end(), b->names().begin(), b->names().end()); + types.insert(types.end(), b->children().begin(), b->children().end()); + return ROW(std::move(names), std::move(types)); +} + +RowTypePtr extract( + const RowTypePtr& type, + const std::vector& childNames) { + std::vector names = childNames; + + std::vector types; + types.reserve(childNames.size()); + for (const auto& name : childNames) { + types.emplace_back(type->findChild(name)); + } + return ROW(std::move(names), std::move(types)); +} + +// Rename columns in the given row type. +RowTypePtr rename( + const RowTypePtr& type, + const std::vector& newNames) { + VELOX_CHECK_EQ( + type->size(), + newNames.size(), + "Number of types and new type names should be the same"); + std::vector names{newNames}; + std::vector types{type->children()}; + return ROW(std::move(names), std::move(types)); +} + +core::PlanNodePtr createLocalPartitionNode( + const core::PlanNodeId& planNodeId, + const std::vector& keys, + bool scaleWriter, + const std::vector& sources, + memory::MemoryPool* pool) { + auto partitionFunctionFactory = + createPartitionFunctionSpec(sources[0]->outputType(), keys, pool); + return std::make_shared( + planNodeId, + keys.empty() ? core::LocalPartitionNode::Type::kGather + : core::LocalPartitionNode::Type::kRepartition, + scaleWriter, + partitionFunctionFactory, + sources); +} +} // namespace + +PlanBuilder& PlanBuilder::partitionedOutput( + const std::vector& keys, + int numPartitions, + const std::vector& outputLayout, + VectorSerde::Kind serdeKind) { + return partitionedOutput(keys, numPartitions, false, outputLayout, serdeKind); +} + +PlanBuilder& PlanBuilder::partitionedOutput( + const std::vector& keys, + int numPartitions, + bool replicateNullsAndAny, + const std::vector& outputLayout, + VectorSerde::Kind serdeKind) { + VELOX_CHECK_NOT_NULL( + planNode_, "PartitionedOutput cannot be the source node"); + + auto keyExprs = exprs(keys, planNode_->outputType()); + return partitionedOutput( + keys, + numPartitions, + replicateNullsAndAny, + createPartitionFunctionSpec(planNode_->outputType(), keyExprs, pool_), + outputLayout, + serdeKind); +} + +PlanBuilder& PlanBuilder::partitionedOutput( + const std::vector& keys, + int numPartitions, + bool replicateNullsAndAny, + core::PartitionFunctionSpecPtr partitionFunctionSpec, + const std::vector& outputLayout, + VectorSerde::Kind serdeKind) { + VELOX_CHECK_NOT_NULL( + planNode_, "PartitionedOutput cannot be the source node"); + auto outputType = outputLayout.empty() + ? planNode_->outputType() + : extract(planNode_->outputType(), outputLayout); + planNode_ = std::make_shared( + nextPlanNodeId(), + core::PartitionedOutputNode::Kind::kPartitioned, + exprs(keys, planNode_->outputType()), + numPartitions, + replicateNullsAndAny, + std::move(partitionFunctionSpec), + outputType, + serdeKind, + planNode_); + VELOX_CHECK(!planNode_->supportsBarrier()); + return *this; +} + +PlanBuilder& PlanBuilder::partitionedOutputBroadcast( + const std::vector& outputLayout, + VectorSerde::Kind serdeKind) { + VELOX_CHECK_NOT_NULL( + planNode_, "PartitionedOutput cannot be the source node"); + auto outputType = outputLayout.empty() + ? planNode_->outputType() + : extract(planNode_->outputType(), outputLayout); + planNode_ = core::PartitionedOutputNode::broadcast( + nextPlanNodeId(), 1, outputType, serdeKind, planNode_); + VELOX_CHECK(!planNode_->supportsBarrier()); + return *this; +} + +PlanBuilder& PlanBuilder::partitionedOutputArbitrary( + const std::vector& outputLayout, + VectorSerde::Kind serdeKind) { + VELOX_CHECK_NOT_NULL( + planNode_, "PartitionedOutput cannot be the source node"); + auto outputType = outputLayout.empty() + ? planNode_->outputType() + : extract(planNode_->outputType(), outputLayout); + planNode_ = core::PartitionedOutputNode::arbitrary( + nextPlanNodeId(), outputType, serdeKind, planNode_); + VELOX_CHECK(!planNode_->supportsBarrier()); + return *this; +} + +PlanBuilder& PlanBuilder::localPartition( + const std::vector& keys, + const std::vector& sources) { + VELOX_CHECK_NULL(planNode_, "localPartition() must be the first call"); + planNode_ = createLocalPartitionNode( + nextPlanNodeId(), + exprs(keys, sources[0]->outputType()), + /*scaleWriter=*/false, + sources, + pool_); + VELOX_CHECK(planNode_->supportsBarrier()); + return *this; +} + +PlanBuilder& PlanBuilder::localPartition(const std::vector& keys) { + planNode_ = createLocalPartitionNode( + nextPlanNodeId(), + exprs(keys, planNode_->outputType()), + /*scaleWriter=*/false, + {planNode_}, + pool_); + VELOX_CHECK(planNode_->supportsBarrier()); + return *this; +} + +PlanBuilder& PlanBuilder::scaleWriterlocalPartition( + const std::vector& keys) { + std::vector keyIndices; + keyIndices.reserve(keys.size()); + for (const auto& key : keys) { + keyIndices.push_back(planNode_->outputType()->getChildIdx(key)); + } + auto hivePartitionFunctionFactory = + std::make_shared( + 1009, keyIndices, std::vector{}); + planNode_ = std::make_shared( + nextPlanNodeId(), + core::LocalPartitionNode::Type::kRepartition, + true, + hivePartitionFunctionFactory, + std::vector{planNode_}); + VELOX_CHECK(!planNode_->supportsBarrier()); + return *this; +} + +PlanBuilder& PlanBuilder::localPartition( + int numBuckets, + const std::vector& bucketChannels, + const std::vector& constValues) { + auto hivePartitionFunctionFactory = + std::make_shared( + numBuckets, bucketChannels, constValues); + planNode_ = std::make_shared( + nextPlanNodeId(), + core::LocalPartitionNode::Type::kRepartition, + /*scaleWriter=*/false, + std::move(hivePartitionFunctionFactory), + std::vector{planNode_}); + VELOX_CHECK(planNode_->supportsBarrier()); + return *this; +} + +PlanBuilder& PlanBuilder::localPartitionByBucket( + const std::shared_ptr& + bucketProperty) { + VELOX_CHECK_NOT_NULL(planNode_, "LocalPartition cannot be the source node"); + std::vector bucketChannels; + for (const auto& bucketColumn : bucketProperty->bucketedBy()) { + bucketChannels.push_back( + planNode_->outputType()->getChildIdx(bucketColumn)); + } + auto hivePartitionFunctionFactory = + std::make_shared( + bucketProperty->bucketCount(), + bucketChannels, + std::vector{}); + planNode_ = std::make_shared( + nextPlanNodeId(), + core::LocalPartitionNode::Type::kRepartition, + /*scaleWriter=*/false, + std::move(hivePartitionFunctionFactory), + std::vector{planNode_}); + VELOX_CHECK(planNode_->supportsBarrier()); + return *this; +} + +namespace { +core::PlanNodePtr createLocalPartitionRoundRobinNode( + const core::PlanNodeId& planNodeId, + bool scaleWriter, + const std::vector& sources) { + return std::make_shared( + planNodeId, + core::LocalPartitionNode::Type::kRepartition, + scaleWriter, + std::make_shared(), + sources); +} +} // namespace + +PlanBuilder& PlanBuilder::localPartitionRoundRobin( + const std::vector& sources) { + VELOX_CHECK_NULL( + planNode_, "localPartitionRoundRobin() must be the first call"); + planNode_ = createLocalPartitionRoundRobinNode( + nextPlanNodeId(), /*scaleWriter=*/false, sources); + VELOX_CHECK(planNode_->supportsBarrier()); + return *this; +} + +PlanBuilder& PlanBuilder::localPartitionRoundRobin() { + planNode_ = createLocalPartitionRoundRobinNode( + nextPlanNodeId(), /*scaleWriter=*/false, {planNode_}); + VELOX_CHECK(planNode_->supportsBarrier()); + return *this; +} + +PlanBuilder& PlanBuilder::scaleWriterlocalPartitionRoundRobin() { + planNode_ = createLocalPartitionRoundRobinNode( + nextPlanNodeId(), /*scaleWriter=*/true, {planNode_}); + VELOX_CHECK(!planNode_->supportsBarrier()); + return *this; +} + +namespace { +class RoundRobinRowPartitionFunction : public core::PartitionFunction { + public: + explicit RoundRobinRowPartitionFunction(int numPartitions) + : numPartitions_{numPartitions} {} + + std::optional partition( + const RowVector& input, + std::vector& partitions) override { + auto size = input.size(); + partitions.resize(size); + for (auto i = 0; i < size; ++i) { + partitions[i] = counter_ % numPartitions_; + ++counter_; + } + return std::nullopt; + } + + private: + const int numPartitions_; + uint32_t counter_{0}; +}; + +class RoundRobinRowPartitionFunctionSpec : public core::PartitionFunctionSpec { + public: + std::unique_ptr create( + int numPartitions, + bool /*localExchange*/) const override { + return std::make_unique(numPartitions); + } + + std::string toString() const override { + return "ROUND ROBIN ROW"; + } + + folly::dynamic serialize() const override { + folly::dynamic obj = folly::dynamic::object; + obj["name"] = fmt::format("RoundRobinRowPartitionFunctionSpec"); + return obj; + } + + static core::PartitionFunctionSpecPtr deserialize( + const folly::dynamic& /*obj*/, + void* /*context*/) { + return std::make_shared(); + } +}; +} // namespace + +PlanBuilder& PlanBuilder::localPartitionRoundRobinRow() { + planNode_ = std::make_shared( + nextPlanNodeId(), + core::LocalPartitionNode::Type::kRepartition, + /*scaleWriter=*/false, + std::make_shared(), + std::vector{planNode_}); + VELOX_CHECK(planNode_->supportsBarrier()); + return *this; +} + +PlanBuilder& PlanBuilder::hashJoin( + const std::vector& leftKeys, + const std::vector& rightKeys, + const core::PlanNodePtr& build, + const std::string& filter, + const std::vector& outputLayout, + core::JoinType joinType, + bool nullAware) { + VELOX_CHECK_NOT_NULL(planNode_, "HashJoin cannot be the source node"); + VELOX_CHECK_EQ(leftKeys.size(), rightKeys.size()); + + auto leftType = planNode_->outputType(); + auto rightType = build->outputType(); + auto resultType = concat(leftType, rightType); + core::TypedExprPtr filterExpr; + if (!filter.empty()) { + filterExpr = parseExpr(filter, resultType, options_, pool_); + } + + RowTypePtr outputType; + if (isLeftSemiProjectJoin(joinType) || isRightSemiProjectJoin(joinType)) { + std::vector names = outputLayout; + + // Last column in 'outputLayout' must be a boolean 'match'. + std::vector types; + types.reserve(outputLayout.size()); + for (auto i = 0; i < outputLayout.size() - 1; ++i) { + types.emplace_back(resultType->findChild(outputLayout[i])); + } + types.emplace_back(BOOLEAN()); + + outputType = ROW(std::move(names), std::move(types)); + } else { + outputType = extract(resultType, outputLayout); + } + + auto leftKeyFields = fields(leftType, leftKeys); + auto rightKeyFields = fields(rightType, rightKeys); + + planNode_ = std::make_shared( + nextPlanNodeId(), + joinType, + nullAware, + leftKeyFields, + rightKeyFields, + std::move(filterExpr), + std::move(planNode_), + build, + outputType); + VELOX_CHECK(!planNode_->supportsBarrier()); + return *this; +} + +PlanBuilder& PlanBuilder::mergeJoin( + const std::vector& leftKeys, + const std::vector& rightKeys, + const core::PlanNodePtr& build, + const std::string& filter, + const std::vector& outputLayout, + core::JoinType joinType) { + VELOX_CHECK_NOT_NULL(planNode_, "MergeJoin cannot be the source node"); + VELOX_CHECK_EQ(leftKeys.size(), rightKeys.size()); + + auto leftType = planNode_->outputType(); + auto rightType = build->outputType(); + auto resultType = concat(leftType, rightType); + core::TypedExprPtr filterExpr; + if (!filter.empty()) { + filterExpr = parseExpr(filter, resultType, options_, pool_); + } + auto outputType = extract(resultType, outputLayout); + auto leftKeyFields = fields(leftType, leftKeys); + auto rightKeyFields = fields(rightType, rightKeys); + + planNode_ = std::make_shared( + nextPlanNodeId(), + joinType, + leftKeyFields, + rightKeyFields, + std::move(filterExpr), + std::move(planNode_), + build, + outputType); + VELOX_CHECK(planNode_->supportsBarrier()); + return *this; +} + +PlanBuilder& PlanBuilder::nestedLoopJoin( + const core::PlanNodePtr& right, + const std::vector& outputLayout, + core::JoinType joinType) { + return nestedLoopJoin(right, "", outputLayout, joinType); +} + +PlanBuilder& PlanBuilder::nestedLoopJoin( + const core::PlanNodePtr& right, + const std::string& joinCondition, + const std::vector& outputLayout, + core::JoinType joinType) { + VELOX_CHECK_NOT_NULL(planNode_, "NestedLoopJoin cannot be the source node"); + auto resultType = concat(planNode_->outputType(), right->outputType()); + if (isLeftSemiProjectJoin(joinType)) { + resultType = concat(resultType, ROW({"match"}, {BOOLEAN()})); + } + + auto outputType = extract(resultType, outputLayout); + + core::TypedExprPtr joinConditionExpr{}; + if (!joinCondition.empty()) { + joinConditionExpr = parseExpr(joinCondition, resultType, options_, pool_); + } + + planNode_ = std::make_shared( + nextPlanNodeId(), + joinType, + std::move(joinConditionExpr), + std::move(planNode_), + right, + outputType); + VELOX_CHECK(!planNode_->supportsBarrier()); + return *this; +} + +namespace { +core::TypedExprPtr removeCastTypedExpr(const core::TypedExprPtr& expr) { + core::TypedExprPtr convertedTypedExpr = expr; + while (auto castTypedExpr = + std::dynamic_pointer_cast( + convertedTypedExpr)) { + VELOX_CHECK_EQ(castTypedExpr->inputs().size(), 1); + convertedTypedExpr = castTypedExpr->inputs()[0]; + } + return convertedTypedExpr; +} + +template +core::TypedExprPtr castConstantArrayConditionInput( + const core::ConstantTypedExprPtr& constantExpr) { + if (SrcKind == DstKind) { + return constantExpr; + } + + auto srcVector = constantExpr->valueVector(); + BaseVector::flattenVector(srcVector); + auto* srcArrayVector = srcVector->asChecked(); + VELOX_CHECK_EQ(srcArrayVector->size(), 1); + using SrcCppType = typename velox::TypeTraits::NativeType; + auto* srcValueVector = srcArrayVector->elements()->asFlatVector(); + + const auto dstType = createScalarType(DstKind); + auto dstValueVector = BaseVector::create( + dstType, srcValueVector->size(), srcArrayVector->pool()); + using DstCppType = typename velox::TypeTraits::NativeType; + auto* dstFlatValueVector = + dstValueVector->template asFlatVector(); + + velox::DecodedVector decodedSrcValueVector{*srcValueVector}; + velox::exec::VectorReader srcValueReader{&decodedSrcValueVector}; + for (auto row = 0; row < srcValueVector->size(); ++row) { + const auto value = srcValueReader[row]; + dstFlatValueVector->set(row, static_cast(value)); + } + auto dstArrayVector = std::make_shared( + srcArrayVector->pool(), + ARRAY(dstType), + nullptr, + 1, + srcArrayVector->offsets(), + srcArrayVector->sizes(), + dstValueVector); + return std::make_shared(dstArrayVector); +} + +template +core::TypedExprPtr castConstantConditionInput( + const core::ConstantTypedExprPtr& constantExpr) { + if (SrcKind == DstKind) { + return constantExpr; + } + const auto dstType = createScalarType(DstKind); + return std::make_shared( + dstType, + static_cast::NativeType>( + constantExpr->value().value())); +} + +template +core::TypedExprPtr castIndexConditionInputExpr(const core::TypedExprPtr& expr) { + core::TypedExprPtr convertedTypedExpr = removeCastTypedExpr(expr); + if (std::dynamic_pointer_cast( + convertedTypedExpr)) { + VELOX_CHECK( + convertedTypedExpr->type()->kind() == Kind || + std::dynamic_pointer_cast(convertedTypedExpr->type()) + ->elementType() + ->kind() == Kind); + return convertedTypedExpr; + } + + const auto constantTypedExpr = + std::dynamic_pointer_cast( + convertedTypedExpr); + VELOX_CHECK_NOT_NULL(constantTypedExpr, "{}", expr->toString()); + + if (constantTypedExpr->type()->isArray()) { + const auto arrayType = + std::dynamic_pointer_cast(constantTypedExpr->type()); + if (arrayType->elementType()->kind() == Kind) { + return constantTypedExpr; + } + switch (arrayType->elementType()->kind()) { + case TypeKind::INTEGER: + return castConstantArrayConditionInput( + constantTypedExpr); + case TypeKind::BIGINT: + return castConstantArrayConditionInput( + constantTypedExpr); + case TypeKind::SMALLINT: + return castConstantArrayConditionInput( + constantTypedExpr); + default: + VELOX_UNSUPPORTED( + "Incompatible condition input type: {}, index column kind: {}", + constantTypedExpr->type()->toString(), + Kind); + } + } + + if (constantTypedExpr->type()->kind() == Kind) { + return convertedTypedExpr; + } + + switch (constantTypedExpr->type()->kind()) { + case TypeKind::INTEGER: + return castConstantConditionInput( + constantTypedExpr); + case TypeKind::BIGINT: + return castConstantConditionInput( + constantTypedExpr); + case TypeKind::SMALLINT: + return castConstantConditionInput( + constantTypedExpr); + default: + VELOX_UNSUPPORTED( + "Incompatible condition input type: {}, index column kind: {}", + constantTypedExpr->type()->toString(), + Kind); + } +} + +core::TypedExprPtr castIndexConditionInputExpr( + const core::TypedExprPtr& expr, + const TypePtr& indexType) { + switch (indexType->kind()) { + case TypeKind::INTEGER: + return castIndexConditionInputExpr(expr); + case TypeKind::BIGINT: + return castIndexConditionInputExpr(expr); + case TypeKind::SMALLINT: + return castIndexConditionInputExpr(expr); + default: + VELOX_UNSUPPORTED("Unsupported index column kind: {}", expr->toString()); + } +} +} // namespace + +// static +core::IndexLookupConditionPtr PlanBuilder::parseIndexJoinCondition( + const std::string& joinCondition, + const RowTypePtr& rowType, + memory::MemoryPool* pool) { + const auto joinConditionExpr = + parseExpr(joinCondition, rowType, parse::ParseOptions{}, pool); + const auto typedCallExpr = + std::dynamic_pointer_cast(joinConditionExpr); + VELOX_CHECK_NOT_NULL(typedCallExpr); + if (typedCallExpr->name() == "contains") { + VELOX_CHECK_EQ(typedCallExpr->inputs().size(), 2); + const auto keyColumnExpr = + std::dynamic_pointer_cast( + removeCastTypedExpr(typedCallExpr->inputs()[1])); + VELOX_CHECK_NOT_NULL( + keyColumnExpr, "{}", typedCallExpr->inputs()[1]->toString()); + return std::make_shared( + keyColumnExpr, + castIndexConditionInputExpr( + typedCallExpr->inputs()[0], keyColumnExpr->type())); + } + + if (typedCallExpr->name() == "between") { + VELOX_CHECK_EQ(typedCallExpr->inputs().size(), 3); + const auto keyColumnExpr = + std::dynamic_pointer_cast( + removeCastTypedExpr(typedCallExpr->inputs()[0])); + VELOX_CHECK_NOT_NULL( + keyColumnExpr, "{}", typedCallExpr->inputs()[0]->toString()); + return std::make_shared( + keyColumnExpr, + castIndexConditionInputExpr( + typedCallExpr->inputs()[1], keyColumnExpr->type()), + castIndexConditionInputExpr( + typedCallExpr->inputs()[2], keyColumnExpr->type())); + } + + if (typedCallExpr->name() == "eq") { + VELOX_CHECK_EQ(typedCallExpr->inputs().size(), 2); + const auto keyColumnExpr = + std::dynamic_pointer_cast( + removeCastTypedExpr(typedCallExpr->inputs()[0])); + VELOX_CHECK_NOT_NULL( + keyColumnExpr, "{}", typedCallExpr->inputs()[0]->toString()); + return std::make_shared( + keyColumnExpr, + castIndexConditionInputExpr( + typedCallExpr->inputs()[1], keyColumnExpr->type())); + } + VELOX_USER_FAIL( + "Invalid index join condition: {}, and we only support in, between, and equal conditions", + joinCondition); +} + +PlanBuilder& PlanBuilder::indexLookupJoin( + const std::vector& leftKeys, + const std::vector& rightKeys, + const core::TableScanNodePtr& right, + const std::vector& joinConditions, + bool includeMatchColumn, + const std::vector& outputLayout, + core::JoinType joinType) { + VELOX_CHECK_NOT_NULL(planNode_, "indexLookupJoin cannot be the source node"); + auto inputType = concat(planNode_->outputType(), right->outputType()); + if (includeMatchColumn) { + auto names = inputType->names(); + names.push_back(outputLayout.back()); + auto types = inputType->children(); + types.push_back(BOOLEAN()); + inputType = ROW(std::move(names), std::move(types)); + } + auto outputType = extract(inputType, outputLayout); + auto leftKeyFields = fields(planNode_->outputType(), leftKeys); + auto rightKeyFields = fields(right->outputType(), rightKeys); + + std::vector joinConditionPtrs{}; + joinConditionPtrs.reserve(joinConditions.size()); + for (const auto& joinCondition : joinConditions) { + joinConditionPtrs.push_back( + parseIndexJoinCondition(joinCondition, inputType, pool_)); + } + + planNode_ = std::make_shared( + nextPlanNodeId(), + joinType, + std::move(leftKeyFields), + std::move(rightKeyFields), + std::move(joinConditionPtrs), + includeMatchColumn, + std::move(planNode_), + right, + std::move(outputType)); + VELOX_CHECK(planNode_->supportsBarrier()); + return *this; +} + +PlanBuilder& PlanBuilder::unnest( + const std::vector& replicateColumns, + const std::vector& unnestColumns, + const std::optional& ordinalColumn, + const std::optional& emptyUnnestValueName) { + VELOX_CHECK_NOT_NULL(planNode_, "Unnest cannot be the source node"); + std::vector> + replicateFields; + replicateFields.reserve(replicateColumns.size()); + for (const auto& name : replicateColumns) { + replicateFields.emplace_back(field(name)); + } + + std::vector> unnestFields; + unnestFields.reserve(unnestColumns.size()); + for (const auto& name : unnestColumns) { + unnestFields.emplace_back(field(name)); + } + + std::vector unnestNames; + for (const auto& name : unnestColumns) { + auto input = planNode_->outputType()->findChild(name); + if (input->isArray()) { + unnestNames.push_back(name + "_e"); + } else if (input->isMap()) { + unnestNames.push_back(name + "_k"); + unnestNames.push_back(name + "_v"); + } else { + VELOX_NYI( + "Unsupported type of unnest variable. Expected ARRAY or MAP, but got {}.", + input->toString()); + } + } + + planNode_ = std::make_shared( + nextPlanNodeId(), + replicateFields, + unnestFields, + unnestNames, + ordinalColumn, + emptyUnnestValueName, + planNode_); + VELOX_CHECK(planNode_->supportsBarrier()); + return *this; +} + +namespace { +std::string throwWindowFunctionDoesntExist(const std::string& name) { + std::stringstream error; + error << "Window function doesn't exist: " << name << "."; + if (exec::windowFunctions().empty()) { + error << " Registry of window functions is empty. " + "Make sure to register some window functions."; + } + VELOX_USER_FAIL(error.str()); +} + +std::string throwWindowFunctionSignatureNotSupported( + const std::string& name, + const std::vector& types, + const std::vector& signatures) { + std::stringstream error; + error << "Window function signature is not supported: " + << exec::toString(name, types) + << ". Supported signatures: " << toString(signatures) << "."; + VELOX_USER_FAIL(error.str()); +} + +TypePtr resolveWindowType( + const std::string& windowFunctionName, + const std::vector& inputTypes, + bool nullOnFailure) { + if (auto signatures = exec::getWindowFunctionSignatures(windowFunctionName)) { + for (const auto& signature : signatures.value()) { + exec::SignatureBinder binder(*signature, inputTypes); + if (binder.tryBind()) { + return binder.tryResolveType(signature->returnType()); + } + } + + if (nullOnFailure) { + return nullptr; + } + throwWindowFunctionSignatureNotSupported( + windowFunctionName, inputTypes, signatures.value()); + } + + if (nullOnFailure) { + return nullptr; + } + throwWindowFunctionDoesntExist(windowFunctionName); + return nullptr; +} + +class WindowTypeResolver { + public: + explicit WindowTypeResolver() + : previousHook_(core::Expressions::getResolverHook()) { + core::Expressions::setTypeResolverHook( + [&](const auto& inputs, const auto& expr, bool nullOnFailure) { + return resolveType(inputs, expr, nullOnFailure); + }); + } + + ~WindowTypeResolver() { + core::Expressions::setTypeResolverHook(previousHook_); + } + + private: + TypePtr resolveType( + const std::vector& inputs, + const std::shared_ptr& expr, + bool nullOnFailure) const { + std::vector types; + for (auto& input : inputs) { + types.push_back(input->type()); + } + + const auto& functionName = expr->name(); + + return resolveWindowType(functionName, types, nullOnFailure); + } + + const core::Expressions::TypeResolverHook previousHook_; +}; + +const core::WindowNode::Frame createWindowFrame( + const duckdb::IExprWindowFrame& windowFrame, + const TypePtr& inputRow, + memory::MemoryPool* pool) { + core::WindowNode::Frame frame; + frame.type = (windowFrame.type == duckdb::WindowType::kRows) + ? core::WindowNode::WindowType::kRows + : core::WindowNode::WindowType::kRange; + + auto boundTypeConversion = + [](duckdb::BoundType boundType) -> core::WindowNode::BoundType { + switch (boundType) { + case duckdb::BoundType::kCurrentRow: + return core::WindowNode::BoundType::kCurrentRow; + case duckdb::BoundType::kFollowing: + return core::WindowNode::BoundType::kFollowing; + case duckdb::BoundType::kPreceding: + return core::WindowNode::BoundType::kPreceding; + case duckdb::BoundType::kUnboundedFollowing: + return core::WindowNode::BoundType::kUnboundedFollowing; + case duckdb::BoundType::kUnboundedPreceding: + return core::WindowNode::BoundType::kUnboundedPreceding; + } + VELOX_UNREACHABLE(); + }; + frame.startType = boundTypeConversion(windowFrame.startType); + frame.startValue = windowFrame.startValue + ? core::Expressions::inferTypes(windowFrame.startValue, inputRow, pool) + : nullptr; + frame.endType = boundTypeConversion(windowFrame.endType); + frame.endValue = windowFrame.endValue + ? core::Expressions::inferTypes(windowFrame.endValue, inputRow, pool) + : nullptr; + return frame; +} + +std::vector parsePartitionKeys( + const duckdb::IExprWindowFunction& windowExpr, + const std::string& windowString, + const TypePtr& inputRow, + memory::MemoryPool* pool) { + std::vector partitionKeys; + for (const auto& partitionKey : windowExpr.partitionBy) { + auto typedExpr = + core::Expressions::inferTypes(partitionKey, inputRow, pool); + auto typedPartitionKey = + std::dynamic_pointer_cast(typedExpr); + VELOX_CHECK_NOT_NULL( + typedPartitionKey, + "PARTITION BY clause must use a column name, not an expression: {}", + windowString); + partitionKeys.emplace_back(typedPartitionKey); + } + return partitionKeys; +} + +std::pair< + std::vector, + std::vector> +parseOrderByKeys( + const duckdb::IExprWindowFunction& windowExpr, + const std::string& windowString, + const TypePtr& inputRow, + memory::MemoryPool* pool) { + std::vector sortingKeys; + std::vector sortingOrders; + + for (const auto& orderBy : windowExpr.orderBy) { + auto typedExpr = + core::Expressions::inferTypes(orderBy.expr, inputRow, pool); + auto sortingKey = + std::dynamic_pointer_cast(typedExpr); + VELOX_CHECK_NOT_NULL( + sortingKey, + "ORDER BY clause must use a column name, not an expression: {}", + windowString); + sortingKeys.emplace_back(sortingKey); + sortingOrders.emplace_back(orderBy.ascending, orderBy.nullsFirst); + } + return {sortingKeys, sortingOrders}; +} + +bool equalFieldAccessTypedExprPtrList( + const std::vector& lhs, + const std::vector& rhs) { + return std::equal( + lhs.begin(), + lhs.end(), + rhs.begin(), + [](const core::FieldAccessTypedExprPtr& e1, + const core::FieldAccessTypedExprPtr& e2) { + return e1->name() == e2->name(); + }); +} + +bool equalSortOrderList( + const std::vector& lhs, + const std::vector& rhs) { + return std::equal( + lhs.begin(), + lhs.end(), + rhs.begin(), + [](const core::SortOrder& s1, const core::SortOrder& s2) { + return s1.isAscending() == s2.isAscending() && + s1.isNullsFirst() == s2.isNullsFirst(); + }); +} + +} // namespace + +PlanBuilder& PlanBuilder::window( + const std::vector& windowFunctions, + bool inputSorted) { + VELOX_CHECK_NOT_NULL(planNode_, "Window cannot be the source node"); + VELOX_CHECK_GT( + windowFunctions.size(), + 0, + "Window Node requires at least one window function."); + + std::vector partitionKeys; + std::vector sortingKeys; + std::vector sortingOrders; + std::vector windowNodeFunctions; + std::vector windowNames; + + bool first = true; + auto inputType = planNode_->outputType(); + int i = 0; + + auto errorOnMismatch = [&](const std::string& windowString, + const std::string& mismatchTypeString) -> void { + std::stringstream error; + error << "Window function invocations " << windowString << " and " + << windowFunctions[0] << " do not match " << mismatchTypeString + << " clauses."; + VELOX_USER_FAIL(error.str()); + }; + + WindowTypeResolver windowResolver; + facebook::velox::duckdb::ParseOptions options; + options.parseIntegerAsBigint = options_.parseIntegerAsBigint; + for (const auto& windowString : windowFunctions) { + const auto& windowExpr = duckdb::parseWindowExpr(windowString, options); + // All window function SQL strings in the list are expected to have the same + // PARTITION BY and ORDER BY clauses. Validate this assumption. + if (first) { + partitionKeys = + parsePartitionKeys(windowExpr, windowString, inputType, pool_); + auto sortPair = + parseOrderByKeys(windowExpr, windowString, inputType, pool_); + sortingKeys = sortPair.first; + sortingOrders = sortPair.second; + first = false; + } else { + auto latestPartitionKeys = + parsePartitionKeys(windowExpr, windowString, inputType, pool_); + auto [latestSortingKeys, latestSortingOrders] = + parseOrderByKeys(windowExpr, windowString, inputType, pool_); + + if (!equalFieldAccessTypedExprPtrList( + partitionKeys, latestPartitionKeys)) { + errorOnMismatch(windowString, "PARTITION BY"); + } + + if (!equalFieldAccessTypedExprPtrList(sortingKeys, latestSortingKeys)) { + errorOnMismatch(windowString, "ORDER BY"); + } + + if (!equalSortOrderList(sortingOrders, latestSortingOrders)) { + errorOnMismatch(windowString, "ORDER BY"); + } + } + + auto windowCall = std::dynamic_pointer_cast( + core::Expressions::inferTypes( + windowExpr.functionCall, planNode_->outputType(), pool_)); + windowNodeFunctions.push_back( + {std::move(windowCall), + createWindowFrame(windowExpr.frame, planNode_->outputType(), pool_), + windowExpr.ignoreNulls}); + if (windowExpr.functionCall->alias().has_value()) { + windowNames.push_back(windowExpr.functionCall->alias().value()); + } else { + windowNames.push_back(fmt::format("w{}", i++)); + } + } + + planNode_ = std::make_shared( + nextPlanNodeId(), + partitionKeys, + sortingKeys, + sortingOrders, + windowNames, + windowNodeFunctions, + inputSorted, + planNode_); + VELOX_CHECK(!planNode_->supportsBarrier()); + return *this; +} + +PlanBuilder& PlanBuilder::window( + const std::vector& windowFunctions) { + return window(windowFunctions, false); +} + +PlanBuilder& PlanBuilder::streamingWindow( + const std::vector& windowFunctions) { + return window(windowFunctions, true); +} + +PlanBuilder& PlanBuilder::rowNumber( + const std::vector& partitionKeys, + std::optional limit, + const bool generateRowNumber) { + std::optional rowNumberColumnName; + if (generateRowNumber) { + rowNumberColumnName = "row_number"; + } + planNode_ = std::make_shared( + nextPlanNodeId(), + fields(partitionKeys), + rowNumberColumnName, + limit, + planNode_); + VELOX_CHECK(!planNode_->supportsBarrier()); + return *this; +} + +PlanBuilder& PlanBuilder::topNRank( + std::string_view function, + const std::vector& partitionKeys, + const std::vector& sortingKeys, + int32_t limit, + bool generateRowNumber) { + VELOX_CHECK_NOT_NULL(planNode_, "TopNRowNumber cannot be the source node"); + auto [sortingFields, sortingOrders] = + parseOrderByClauses(sortingKeys, planNode_->outputType(), pool_); + std::optional rowNumberColumnName; + if (generateRowNumber) { + rowNumberColumnName = "row_number"; + } + planNode_ = std::make_shared( + nextPlanNodeId(), + core::TopNRowNumberNode::rankFunctionFromName(function), + fields(partitionKeys), + sortingFields, + sortingOrders, + rowNumberColumnName, + limit, + planNode_); + VELOX_CHECK(!planNode_->supportsBarrier()); + return *this; +} + +PlanBuilder& PlanBuilder::topNRowNumber( + const std::vector& partitionKeys, + const std::vector& sortingKeys, + int32_t limit, + bool generateRowNumber) { + return topNRank( + "row_number", partitionKeys, sortingKeys, limit, generateRowNumber); +} + +PlanBuilder& PlanBuilder::markDistinct( + std::string markerKey, + const std::vector& distinctKeys) { + VELOX_CHECK_NOT_NULL(planNode_, "MarkDistinct cannot be the source node"); + planNode_ = std::make_shared( + nextPlanNodeId(), + std::move(markerKey), + fields(planNode_->outputType(), distinctKeys), + planNode_); + VELOX_CHECK(!planNode_->supportsBarrier()); + return *this; +} + +core::PlanNodeId PlanBuilder::nextPlanNodeId() { + return planNodeIdGenerator_->next(); +} + +// static +std::shared_ptr PlanBuilder::field( + const RowTypePtr& inputType, + const std::string& name) { + auto index = inputType->getChildIdx(name); + return field(inputType, index); +} + +// static +std::shared_ptr PlanBuilder::field( + const RowTypePtr& inputType, + column_index_t index) { + auto name = inputType->names()[index]; + auto type = inputType->childAt(index); + return std::make_shared(type, name); +} + +// static +std::vector> +PlanBuilder::fields( + const RowTypePtr& inputType, + const std::vector& names) { + std::vector> fields; + for (const auto& name : names) { + fields.push_back(field(inputType, name)); + } + return fields; +} + +// static +std::vector> +PlanBuilder::fields( + const RowTypePtr& inputType, + const std::vector& indices) { + std::vector> fields; + for (auto& index : indices) { + fields.push_back(field(inputType, index)); + } + return fields; +} + +std::shared_ptr PlanBuilder::field( + column_index_t index) { + VELOX_CHECK_NOT_NULL(planNode_); + return field(planNode_->outputType(), index); +} + +std::shared_ptr PlanBuilder::field( + const std::string& name) { + VELOX_CHECK_NOT_NULL(planNode_); + return field(planNode_->outputType(), name); +} + +std::vector> +PlanBuilder::fields(const std::vector& names) { + VELOX_CHECK_NOT_NULL(planNode_); + return fields(planNode_->outputType(), names); +} + +std::vector> +PlanBuilder::fields(const std::vector& indices) { + VELOX_CHECK_NOT_NULL(planNode_); + return fields(planNode_->outputType(), indices); +} + +std::vector PlanBuilder::exprs( + const std::vector& expressions, + const RowTypePtr& inputType) { + std::vector typedExpressions; + for (auto& expr : expressions) { + auto typedExpression = core::Expressions::inferTypes( + parse::parseExpr(expr, options_), inputType, pool_); + + if (dynamic_cast( + typedExpression.get())) { + typedExpressions.push_back(typedExpression); + } else if (dynamic_cast( + typedExpression.get())) { + typedExpressions.push_back(typedExpression); + } else { + VELOX_FAIL("Expected field name or constant: {}", expr); + } + } + + return typedExpressions; +} + +core::TypedExprPtr PlanBuilder::inferTypes( + const std::shared_ptr& untypedExpr) { + VELOX_CHECK_NOT_NULL(planNode_); + return core::Expressions::inferTypes( + untypedExpr, planNode_->outputType(), pool_); +} +} // namespace facebook::velox::connector::lakehouse::common::test diff --git a/velox/connectors/lakehouse/common/tests/PlanBuilder.h b/velox/connectors/lakehouse/common/tests/PlanBuilder.h new file mode 100644 index 000000000000..73b762ec7860 --- /dev/null +++ b/velox/connectors/lakehouse/common/tests/PlanBuilder.h @@ -0,0 +1,1504 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#pragma once + +#include "velox/connectors/lakehouse/common/HiveDataSink.h" +#include "velox/core/Expressions.h" +#include "velox/core/ITypedExpr.h" +#include "velox/core/PlanFragment.h" +#include "velox/core/PlanNode.h" +#include "velox/parse/ExpressionsParser.h" +#include "velox/parse/IExpr.h" +#include "velox/parse/PlanNodeIdGenerator.h" + +namespace facebook::velox::tpch { +enum class Table : uint8_t; +} + +namespace facebook::velox::connector::lakehouse::common::test { + +struct PushdownConfig { + velox::common::SubfieldFilters subfieldFiltersMap; + std::string remainingFilter; +}; + +/// A builder class with fluent API for building query plans. Plans are built +/// bottom up starting with the source node (table scan or similar). Expressions +/// and orders can be specified using SQL. See filter, project and orderBy +/// methods for details. +/// +/// For example, to build a query plan for a leaf fragment of a simple query +/// SELECT a, sum(b) FROM t GROUP BY 1 +/// +/// auto plan = PlanBuilder() +/// .tableScan(ROW({"a", "b"}, {INTEGER(), DOUBLE()})) +/// .partialAggregation({"a"}, {"sum(b)"}) +/// .planNode(); +/// +/// Here, we use default PlanNodeIdGenerator that starts from zero, hence, table +/// scan node ID will be "0". You'll need to use this ID when adding splits. +/// +/// A join query plan would be a bit more complex: +/// SELECT t.a, u.b FROM t, u WHERE t.key = u.key +/// +/// auto planNodeIdGenerator = std::make_shared(); +/// core::PlanNodeId tScanId; // ID of the table scan node for 't'. +/// core::PlanNodeId uScanId; // ID of the table scan node for 'u'. +/// auto plan = PlanBuilder(planNodeIdGenerator) +/// .tableScan(ROW({"key", "a"}, {INTEGER(), BIGINT()})) +/// .capturePlanNodeId(tScanId) +/// .hashJoin( +/// {"key"}, +/// {"key"}, +/// PlanBuilder(planNodeIdGenerator) +/// .tableScan(ROW({"key", "b"}, {INTEGER(), DOUBLE()}))) +/// .capturePlanNodeId(uScanId) +/// .planNode(), +/// "", // no extra join filter +/// {"a", "b"}) +/// .planNode(); +/// +/// We use two builders, one for the right-side and another for the left-side +/// of the join. To ensure plan node IDs are unique in the final plan, we use +/// the same instance of PlanNodeIdGenerator with both builders. We also use +/// capturePlanNodeId method to capture the IDs of the table scan nodes for +/// 't' and 'u'. We need these to add splits. +class PlanBuilder { + public: + /// Constructor taking an instance of PlanNodeIdGenerator and a memory pool. + /// + /// The memory pool is used when parsing expressions containing complex-type + /// literals, e.g. arrays, maps or structs. The memory pool can be empty if + /// such expressions are not used in the plan. + /// + /// When creating tree-shaped plans, e.g. join queries, use the same instance + /// of PlanNodeIdGenerator for all builders to ensure unique plan node IDs + /// across the plan. + explicit PlanBuilder( + std::shared_ptr planNodeIdGenerator, + memory::MemoryPool* pool = nullptr) + : planNodeIdGenerator_{std::move(planNodeIdGenerator)}, pool_{pool} {} + + /// Constructor with no required parameters suitable for creating + /// straight-line (e.g. no joins) query plans. + explicit PlanBuilder(memory::MemoryPool* pool = nullptr) + : PlanBuilder(std::make_shared(), pool) {} + + /// Constructor that allows an initial plane node to be specified for testing + /// this is useful when testing additional connectors that do not rely on the + /// table scan node supported below. + PlanBuilder( + core::PlanNodePtr initialPlanNode, + std::shared_ptr planNodeIdGenerator, + memory::MemoryPool* pool = nullptr) + : planNode_(std::move(initialPlanNode)), + planNodeIdGenerator_{std::move(planNodeIdGenerator)}, + pool_{pool} {} + + virtual ~PlanBuilder() = default; + + static constexpr const std::string_view kHiveDefaultConnectorId{"test-hive"}; + static constexpr const std::string_view kTpchDefaultConnectorId{"test-tpch"}; + + /// + /// TableScan + /// + + /// Add a TableScanNode to scan a Hive table. + /// + /// @param outputType List of column names and types to read from the table. + /// @param subfieldFilters A list of SQL expressions for the range filters to + /// apply to individual columns. Supported filters are: column <= value, + /// column < value, column >= value, column > value, column = value, column IN + /// (v1, v2,.. vN), column < v1 OR column >= v2. + /// @param remainingFilter SQL expression for the additional conjunct. May + /// include multiple columns and SQL functions. The remainingFilter is AND'ed + /// with all the subfieldFilters. + /// @param dataColumns can be different from 'outputType' for the purposes + /// of testing queries using missing columns. It is used, if specified, for + /// parseExpr call and as 'dataColumns' for the TableHandle. You supply more + /// types (for all columns) in this argument as opposed to 'outputType', where + /// you define the output types only. See 'missingColumns' test in + /// 'TableScanTest'. + /// @param assignments Optional ColumnHandles. + PlanBuilder& tableScan( + const RowTypePtr& outputType, + const std::vector& subfieldFilters = {}, + const std::string& remainingFilter = "", + const RowTypePtr& dataColumns = nullptr, + const connector::ColumnHandleMap& assignments = {}); + + /// Add a TableScanNode to scan a Hive table. + /// + /// @param tableName The name of the table to scan. + /// @param outputType List of column names and types to read from the table. + /// @param columnAliases Optional aliases for the column names. The key is the + /// alias (name in 'outputType'), value is the name in the files. + /// @param subfieldFilters A list of SQL expressions for the range filters to + /// apply to individual columns. Should use column name aliases, not column + /// names in the files. Supported filters are: column <= value, column < + /// value, column >= value, column > value, column = value, column IN (v1, + /// v2,.. vN), column < v1 OR column >= v2. + /// @param remainingFilter SQL expression for the additional conjunct. May + /// include multiple columns and SQL functions. Should use column name + /// aliases, not column names in the files. The remainingFilter is AND'ed + /// with all the subfieldFilters. + /// @param dataColumns can be different from 'outputType' for the purposes + /// of testing queries using missing columns. It is used, if specified, for + /// parseExpr call and as 'dataColumns' for the TableHandle. You supply more + /// types (for all columns) in this argument as opposed to 'outputType', where + /// you define the output types only. See 'missingColumns' test in + /// 'TableScanTest'. + PlanBuilder& tableScan( + const std::string& tableName, + const RowTypePtr& outputType, + const std::unordered_map& columnAliases = {}, + const std::vector& subfieldFilters = {}, + const std::string& remainingFilter = "", + const RowTypePtr& dataColumns = nullptr, + const connector::ColumnHandleMap& assignments = {}); + + /// Add a TableScanNode to scan a Hive table with direct SubfieldFilters. + /// + /// @param outputType List of column names and types to read from the table. + /// @param PushdownConfig Contains pushdown configs for the table scan. + /// @param dataColumns Optional data columns that may differ from outputType. + /// @param assignments Optional ColumnHandles. + + PlanBuilder& tableScanWithPushDown( + const RowTypePtr& outputType, + const PushdownConfig& pushdownConfig, + const RowTypePtr& dataColumns = nullptr, + const connector::ColumnHandleMap& assignments = {}); + + /// Add a TableScanNode to scan a TPC-H table. + /// + /// @param tpchTableHandle The handle that specifies the target TPC-H table + /// and scale factor. + /// @param columnNames The columns to be returned from that table. + /// @param scaleFactor The TPC-H scale factor. + /// @param connectorId The TPC-H connector id. + /// @param filter Optional SQL expression to filter the data at the connector + /// level. + PlanBuilder& tpchTableScan( + tpch::Table table, + std::vector columnNames, + double scaleFactor = 1, + std::string_view connectorId = kTpchDefaultConnectorId, + const std::string& filter = ""); + + /// Helper class to build a custom TableScanNode. + /// Uses a planBuilder instance to get the next plan id, memory pool, and + /// parse options. + /// + /// Uses the hive connector by default. Specify outputType, tableHandle, and + /// assignments for other connectors. If these three are specified, all other + /// builder arguments will be ignored. + class TableScanBuilder { + public: + TableScanBuilder(PlanBuilder& builder) : planBuilder_(builder) {} + + /// @param tableName The name of the table to scan. + TableScanBuilder& tableName(std::string tableName) { + tableName_ = std::move(tableName); + return *this; + } + + /// if 'idGenerator' is non-nullptr, produces filters that would be pushed + /// down into the scan as a separate FilterNode instead. 'idGenerator' + /// produces the id for the filterNode. + TableScanBuilder& filtersAsNode( + std::shared_ptr idGenerator) { + filtersAsNode_ = idGenerator != nullptr; + planNodeIdGenerator_ = idGenerator; + return *this; + } + + /// @param connectorId The id of the connector to scan. + TableScanBuilder& connectorId(std::string connectorId) { + connectorId_ = std::move(connectorId); + return *this; + } + + /// @param outputType List of column names and types to read from the table. + /// This property is required. + TableScanBuilder& outputType(RowTypePtr outputType) { + outputType_ = std::move(outputType); + return *this; + } + + /// @param subfieldFilters A list of SQL expressions to apply to individual + /// columns. These are range filters that can be efficiently applied as data + /// is read/decoded. Supported filters are: + /// + /// > column <= value + /// > column < value + /// > column >= value + /// > column > value + /// > column = value + /// > column IN (v1, v2,.. vN) + /// > column < v1 + /// > column >= v2 + TableScanBuilder& subfieldFilters(std::vector subfieldFilters); + + // @param subfieldFiltersMap A map of Subfield to Filters. + TableScanBuilder& subfieldFiltersMap( + const velox::common::SubfieldFilters& filtersMap); + + /// @param subfieldFilter A single SQL expression to be applied to an + /// individual column. + TableScanBuilder& subfieldFilter(std::string subfieldFilter) { + return subfieldFilters({std::move(subfieldFilter)}); + } + + /// @param remainingFilter SQL expression for the additional conjunct. May + /// include multiple columns and SQL functions. The remainingFilter is + /// AND'ed with all the subfieldFilters. + TableScanBuilder& remainingFilter(std::string remainingFilter); + + /// @param dataColumns can be different from 'outputType' for the purposes + /// of testing queries using missing columns. It is used, if specified, for + /// parseExpr call and as 'dataColumns' for the TableHandle. You supply more + /// types (for all columns) in this argument as opposed to 'outputType', + /// where you define the output types only. See 'missingColumns' test in + /// 'TableScanTest'. + TableScanBuilder& dataColumns(RowTypePtr dataColumns) { + dataColumns_ = std::move(dataColumns); + return *this; + } + + /// @param columnAliases Optional aliases for the column names. The key is + /// the alias (name in 'outputType'), value is the name in the files. + TableScanBuilder& columnAliases( + std::unordered_map columnAliases) { + columnAliases_ = std::move(columnAliases); + return *this; + } + + /// @param tableHandle Optional tableHandle. Other builder arguments such as + /// the `subfieldFilters` and `remainingFilter` will be ignored. + TableScanBuilder& tableHandle( + connector::ConnectorTableHandlePtr tableHandle) { + tableHandle_ = std::move(tableHandle); + return *this; + } + + /// @param assignments Optional ColumnHandles. + /// outputType names should match the keys in the 'assignments' map. The + /// 'assignments' map may contain more columns than 'outputType' if some + /// columns are only used by pushed-down filters. + TableScanBuilder& assignments(connector::ColumnHandleMap assignments) { + assignments_ = std::move(assignments); + return *this; + } + + /// Stop the TableScanBuilder. + PlanBuilder& endTableScan() { + planBuilder_.planNode_ = build(planBuilder_.nextPlanNodeId()); + return planBuilder_; + } + + private: + /// Build the plan node TableScanNode. + core::PlanNodePtr build(core::PlanNodeId id); + + PlanBuilder& planBuilder_; + std::string tableName_{"hive_table"}; + std::string connectorId_{kHiveDefaultConnectorId}; + RowTypePtr outputType_; + core::ExprPtr remainingFilter_; + RowTypePtr dataColumns_; + std::unordered_map columnAliases_; + connector::ConnectorTableHandlePtr tableHandle_; + connector::ColumnHandleMap assignments_; + + // produce filters as a FilterNode instead of pushdown. + bool filtersAsNode_{false}; + + // Generates the id of a FilterNode if 'filtersAsNode_'. + std::shared_ptr planNodeIdGenerator_; + + // SubfieldFilters object containing filters to apply. + velox::common::SubfieldFilters subfieldFiltersMap_; + }; + + /// Start a TableScanBuilder. + TableScanBuilder& startTableScan() { + tableScanBuilder_.reset(new TableScanBuilder(*this)); + return *tableScanBuilder_; + } + + /// + /// TableWriter + /// + + /// Helper class to build a custom TableWriteNode. + /// Uses a planBuilder instance to get the next plan id, memory pool, and + /// upstream node (the node that will produce the data). + /// + /// Uses the Hive connector by default. + class TableWriterBuilder { + public: + explicit TableWriterBuilder(PlanBuilder& builder) : planBuilder_(builder) {} + + /// @param outputType The schema that will be written to the output file. It + /// may reference a subset or change the order of columns from the input + /// (upstream operator output). + TableWriterBuilder& outputType(RowTypePtr outputType) { + outputType_ = std::move(outputType); + return *this; + } + + /// @param outputDirectoryPath Path in which output files will be created. + TableWriterBuilder& outputDirectoryPath(std::string outputDirectoryPath) { + outputDirectoryPath_ = std::move(outputDirectoryPath); + return *this; + } + + /// @param outputFileName File name of the output (optional). If specified + /// (non-empty), use it instead of generating the file name in Velox. Should + /// only be specified in non-bucketing write. + TableWriterBuilder& outputFileName(std::string outputFileName) { + outputFileName_ = std::move(outputFileName); + return *this; + } + + /// @param connectorId The id of the connector to write to. + TableWriterBuilder& connectorId(std::string_view connectorId) { + connectorId_ = connectorId; + return *this; + } + + /// @param insertHandle TableInsertHandle (optional). Other builder + /// arguments such as the `connectorId`, `outputDirectoryPath`, `fileFormat` + /// and so on will be ignored. + TableWriterBuilder& insertHandle( + std::shared_ptr insertHandle) { + insertHandle_ = std::move(insertHandle); + return *this; + } + + /// @param partitionBy Specifies the partition key columns. + TableWriterBuilder& partitionBy(std::vector partitionBy) { + partitionBy_ = std::move(partitionBy); + return *this; + } + + /// @param bucketCount Specifies the bucket count. + TableWriterBuilder& bucketCount(int32_t count) { + bucketCount_ = count; + return *this; + } + + /// @param bucketedBy Specifies the bucket by columns. + TableWriterBuilder& bucketedBy(std::vector bucketedBy) { + bucketedBy_ = std::move(bucketedBy); + return *this; + } + + /// @param aggregates Aggregations for column statistics collection during + /// write. + TableWriterBuilder& aggregates(std::vector aggregates) { + aggregates_ = std::move(aggregates); + return *this; + } + + /// @param sortBy Specifies the sort by columns. + TableWriterBuilder& sortBy( + std::vector> sortBy) { + sortBy_ = std::move(sortBy); + return *this; + } + + /// @param serdeParameters Additional parameters passed to the writer. + TableWriterBuilder& serdeParameters( + std::unordered_map serdeParameters) { + serdeParameters_ = std::move(serdeParameters); + return *this; + } + + /// @param Option objects passed to the writer. + TableWriterBuilder& options( + std::shared_ptr options) { + options_ = std::move(options); + return *this; + } + + /// @param fileFormat File format to use for the written data. + TableWriterBuilder& fileFormat(dwio::common::FileFormat fileFormat) { + fileFormat_ = fileFormat; + return *this; + } + + /// @param compressionKind Compression scheme to use for writing the + /// output data files. + TableWriterBuilder& compressionKind( + velox::common::CompressionKind compressionKind) { + compressionKind_ = compressionKind; + return *this; + } + + /// @param ensureFiles When set the Task will always output a file, even if + /// it's empty. + TableWriterBuilder& ensureFiles(bool ensureFiles) { + ensureFiles_ = ensureFiles; + return *this; + } + + /// Specifies commitStrategy for writing to the connector. + /// @param commitStrategy The commit strategy to use for the table write + /// operation. + TableWriterBuilder& commitStrategy( + connector::CommitStrategy commitStrategy) { + commitStrategy_ = commitStrategy; + return *this; + } + + /// Stop the TableWriterBuilder. + PlanBuilder& endTableWriter() { + planBuilder_.planNode_ = build(planBuilder_.nextPlanNodeId()); + return planBuilder_; + } + + private: + /// Build the plan node TableWriteNode. + core::PlanNodePtr build(core::PlanNodeId id); + + PlanBuilder& planBuilder_; + RowTypePtr outputType_; + std::string outputDirectoryPath_; + std::string outputFileName_; + std::string connectorId_{kHiveDefaultConnectorId}; + std::shared_ptr insertHandle_; + + std::vector partitionBy_; + int32_t bucketCount_{0}; + std::vector bucketedBy_; + std::vector aggregates_; + std::vector< + std::shared_ptr> + sortBy_; + + std::unordered_map serdeParameters_; + std::shared_ptr options_; + + dwio::common::FileFormat fileFormat_{dwio::common::FileFormat::DWRF}; + velox::common::CompressionKind compressionKind_{ + velox::common::CompressionKind_NONE}; + + bool ensureFiles_{false}; + connector::CommitStrategy commitStrategy_{ + connector::CommitStrategy::kNoCommit}; + }; + + /// Start a TableWriterBuilder. + TableWriterBuilder& startTableWriter() { + tableWriterBuilder_.reset(new TableWriterBuilder(*this)); + return *tableWriterBuilder_; + } + + /// Add a ValuesNode using specified data. + /// + /// @param values The data to use. + /// @param parallelizable If true, ValuesNode can run multi-threaded, in which + /// case it will produce duplicate data from each thread, e.g. each thread + /// will return all the data in 'values'. Useful for testing. + /// @param repeatTimes The number of times data is produced as input. If + /// greater than one, each RowVector will produce data as input `repeatTimes`. + /// For example, in case `values` has 3 vectors {v1, v2, v3} and repeatTimes + /// is 2, the input produced will be {v1, v2, v3, v1, v2, v3}. Useful for + /// testing. + PlanBuilder& values( + const std::vector& values, + bool parallelizable = false, + size_t repeatTimes = 1); + + PlanBuilder& filtersAsNode(bool filtersAsNode) { + filtersAsNode_ = filtersAsNode; + return *this; + } + + /// Adds a QueryReplayNode for query tracing. + /// + /// @param traceNodeDir The trace directory for a given plan node. + /// @param pipelineId The pipeline id for the traced operator instantiated + /// from the given plan node. + /// @param driverIds The target driver ID list for replay. The replaying + /// operator uses its driver instance id as the list index to get the traced + /// driver id for replay. + /// @param outputType The type of the tracing data. + PlanBuilder& traceScan( + const std::string& traceNodeDir, + uint32_t pipelineId, + std::vector driverIds, + const RowTypePtr& outputType); + + /// Add an ExchangeNode. + /// + /// Use capturePlanNodeId method to capture the node ID needed for adding + /// splits. + /// + /// @param outputType The type of the data coming in and out of the exchange. + /// @param serdekind The kind of seralized data format. + PlanBuilder& exchange( + const RowTypePtr& outputType, + VectorSerde::Kind serdekind); + + /// Add a MergeExchangeNode using specified ORDER BY clauses. + /// + /// For example, + /// + /// .mergeExchange(outputRowType, {"a", "b DESC", "c ASC NULLS FIRST"}) + /// + /// By default, uses ASC NULLS LAST sort order, e.g. column "a" above will use + /// ASC NULLS LAST and column "b" will use DESC NULLS LAST. + PlanBuilder& mergeExchange( + const RowTypePtr& outputType, + const std::vector& keys, + VectorSerde::Kind serdekind); + + /// Add a ProjectNode using specified SQL expressions. + /// + /// For example, + /// + /// .project({"a + b", "c * 3"}) + /// + /// The names of the projections can be specified using SQL statement AS: + /// + /// .project({"a + b AS sum_ab", "c * 3 AS triple_c"}) + /// + /// If AS statement is not used, the names of the projections will be + /// generated as p0, p1, p2, etc. Names of columns projected as is will be + /// preserved. + /// + /// For example, + /// + /// project({"a + b AS sum_ab", "c", "d * 7") + /// + /// will produce projected columns named sum_ab, c and p2. + PlanBuilder& project(const std::vector& projections); + + /// Add a ParallelProjectNode using groups of independent SQL expressions. + /// + /// @param projectionGroups One or more groups of expressions that depend on + /// disjunct sets of inputs. + /// @param noLoadColumn Optional columns to pass through as is without + /// loading. These columns must be distinct from the set of columns used in + /// 'projectionGroups'. + PlanBuilder& parallelProject( + const std::vector>& projectionGroups, + const std::vector& noLoadColumns = {}); + + /// Add a LazyDereferenceNode to the plan. + /// @param projections Same format as in `project`, but can only contain + /// field/subfield accesses. + PlanBuilder& lazyDereference(const std::vector& projections); + + /// Add a ProjectNode to keep all existing columns and append more columns + /// using specified expressions. + /// @param newColumns A list of one or more expressions to use for computing + /// additional columns. + PlanBuilder& appendColumns(const std::vector& newColumns); + + /// Variation of project that takes untyped expressions. Used for access + /// deeply nested types, in which case Duck DB often fails to parse or infer + /// the type. + PlanBuilder& projectExpressions( + const std::vector& projections); + + PlanBuilder& projectExpressions( + const std::vector& projections); + + /// Similar to project() except 'optionalProjections' could be empty and the + /// function will skip creating a ProjectNode in that case. + PlanBuilder& optionalProject( + const std::vector& optionalProjections); + + /// Add a FilterNode using specified SQL expression. + /// + /// @param filter SQL expression of type boolean. + PlanBuilder& filter(const std::string& filter); + + /// Similar to filter() except 'optionalFilter' could be empty and the + /// function will skip creating a FilterNode in that case. + PlanBuilder& optionalFilter(const std::string& optionalFilter); + + /// Adds a TableWriteNode to write all input columns into an un-partitioned + /// un-bucketed Hive table without compression. + /// + /// @param outputDirectoryPath Path to a directory to write data to. + /// @param fileFormat File format to use for the written data. + /// @param aggregates Aggregations for column statistics collection during + /// @param polymorphic options object to be passed to the writer. + /// write, supported aggregation types vary for different column types. + /// @param outputFileName Optional file name of the output. If specified + /// (non-empty), use it instead of generating the file name in Velox. Should + /// only be specified in non-bucketing write. + /// For example: + /// Boolean: count, countIf. + /// NumericType/Date/Timestamp: min, max, approx_distinct, count. + /// Varchar: count, approx_distinct, sum_data_size_for_stats, + /// max_data_size_for_stats. + PlanBuilder& tableWrite( + const std::string& outputDirectoryPath, + const dwio::common::FileFormat fileFormat = + dwio::common::FileFormat::DWRF, + const std::vector& aggregates = {}, + const std::shared_ptr& options = nullptr, + const std::string& outputFileName = ""); + + /// Adds a TableWriteNode to write all input columns into a partitioned Hive + /// table without compression. + /// + /// @param outputDirectoryPath Path to a directory to write data to. + /// @param partitionBy Specifies the partition key columns. + /// @param fileFormat File format to use for the written data. + /// @param aggregates Aggregations for column statistics collection during + /// write. + /// @param polymorphic options object to be passed to the writer. + PlanBuilder& tableWrite( + const std::string& outputDirectoryPath, + const std::vector& partitionBy, + const dwio::common::FileFormat fileFormat = + dwio::common::FileFormat::DWRF, + const std::vector& aggregates = {}, + const std::shared_ptr& options = nullptr); + + /// Adds a TableWriteNode to write all input columns into a non-sorted + /// bucketed Hive table without compression. + /// + /// @param outputDirectoryPath Path to a directory to write data to. + /// @param partitionBy Specifies the partition key columns. + /// @param bucketCount Specifies the bucket count. + /// @param bucketedBy Specifies the bucket by columns. + /// @param fileFormat File format to use for the written data. + /// @param aggregates Aggregations for column statistics collection during + /// write. + /// @param polymorphic options object to be passed to the writer. + PlanBuilder& tableWrite( + const std::string& outputDirectoryPath, + const std::vector& partitionBy, + int32_t bucketCount, + const std::vector& bucketedBy, + const dwio::common::FileFormat fileFormat = + dwio::common::FileFormat::DWRF, + const std::vector& aggregates = {}, + const std::shared_ptr& options = nullptr); + + /// Adds a TableWriteNode to write all input columns into a sorted bucket Hive + /// table without compression. + /// + /// @param outputDirectoryPath Path to a directory to write data to. + /// @param partitionBy Specifies the partition key columns. + /// @param bucketCount Specifies the bucket count. + /// @param bucketedBy Specifies the bucket by columns. + /// @param sortBy Specifies the sort by columns. + /// @param fileFormat File format to use for the written data. + /// @param aggregates Aggregations for column statistics collection during + /// write. + /// @param connectorId Name used to register the connector. + /// @param serdeParameters Additional parameters passed to the writer. + /// @param Option objects passed to the writer. + /// @param outputFileName Optional file name of the output. If specified + /// (non-empty), use it instead of generating the file name in Velox. Should + /// only be specified in non-bucketing write. + /// @param compressionKind Compression scheme to use for writing the + /// output data files. + /// @param schema Output schema to be passed to the writer. By default use the + /// output of the previous operator. + /// @param ensureFiles When this option is set the HiveDataSink will always + /// create a file even if there is no data. + PlanBuilder& tableWrite( + const std::string& outputDirectoryPath, + const std::vector& partitionBy, + int32_t bucketCount, + const std::vector& bucketedBy, + const std::vector>& sortBy, + const dwio::common::FileFormat fileFormat = + dwio::common::FileFormat::DWRF, + const std::vector& aggregates = {}, + const std::string_view& connectorId = kHiveDefaultConnectorId, + const std::unordered_map& serdeParameters = {}, + const std::shared_ptr& options = nullptr, + const std::string& outputFileName = "", + const velox::common::CompressionKind = + velox::common::CompressionKind_NONE, + const RowTypePtr& schema = nullptr, + const bool ensureFiles = false, + const connector::CommitStrategy commitStrategy = + connector::CommitStrategy::kNoCommit); + + /// Add a TableWriteMergeNode. + PlanBuilder& tableWriteMerge( + const core::AggregationNodePtr& aggregationNode = nullptr); + + /// Add an AggregationNode representing partial aggregation with the + /// specified grouping keys, aggregates and optional masks. + /// + /// Aggregates are specified as function calls over unmodified input columns, + /// e.g. sum(a), avg(b), min(c). SQL statement AS can be used to specify names + /// for the aggregation result columns. In the absence of AS statement, result + /// columns are named a0, a1, a2, etc. + /// + /// For example, + /// + /// partialAggregation({}, {"min(a) AS min_a", "max(b)"}) + /// + /// will produce output columns min_a and a1, while + /// + /// partialAggregation({"k1", "k2"}, {"min(a) AS min_a", "max(b)"}) + /// + /// will produce output columns k1, k2, min_a and a1, assuming the names of + /// the first two input columns are k1 and k2. + PlanBuilder& partialAggregation( + const std::vector& groupingKeys, + const std::vector& aggregates, + const std::vector& masks = {}) { + return aggregation( + groupingKeys, + {}, + aggregates, + masks, + core::AggregationNode::Step::kPartial, + false); + } + + /// Add final aggregation plan node to match the current partial aggregation + /// node. Should be called directly after partialAggregation() method or + /// directly after intermediateAggregation() that follows + /// partialAggregation(). Can be called also if there is a local exchange + /// after partial or intermediate aggregation. + PlanBuilder& finalAggregation(); + + /// Add final aggregation plan node using specified grouping keys, aggregate + /// expressions and their types. + /// + /// @param rawInputTypes Raw input types for the aggregate functions. + PlanBuilder& finalAggregation( + const std::vector& groupingKeys, + const std::vector& aggregates, + const std::vector>& rawInputTypes) { + return aggregation( + groupingKeys, + {}, + aggregates, + {}, + core::AggregationNode::Step::kFinal, + false, + rawInputTypes); + } + + /// Add intermediate aggregation plan node to match the current partial + /// aggregation node. Should be called directly after partialAggregation() + /// method. Can be called also if there is a local exchange after partial + /// aggregation. + PlanBuilder& intermediateAggregation(); + + /// Add intermediate aggregation plan node using specified grouping keys, + /// aggregate expressions and their types. + PlanBuilder& intermediateAggregation( + const std::vector& groupingKeys, + const std::vector& aggregates) { + return aggregation( + groupingKeys, + {}, + aggregates, + {}, + core::AggregationNode::Step::kIntermediate, + false); + } + + /// Add a single aggregation plan node using specified grouping keys and + /// aggregate expressions. See 'partialAggregation' method for the supported + /// types of aggregate expressions. + PlanBuilder& singleAggregation( + const std::vector& groupingKeys, + const std::vector& aggregates, + const std::vector& masks = {}) { + return aggregation( + groupingKeys, + {}, + aggregates, + masks, + core::AggregationNode::Step::kSingle, + false); + } + + /// Add an AggregationNode using specified grouping keys, + /// aggregate expressions and masks. See 'partialAggregation' method for the + /// supported types of aggregate expressions. + /// + /// @param groupingKeys A list of grouping keys. Can be empty for global + /// aggregations. + /// @param aggregates A list of aggregate expressions. Must contain at least + /// one expression. + /// @param masks An optional list of boolean input columns to use as masks for + /// the aggregates. Can be empty or have fewer elements than 'aggregates' or + /// have some elements being empty strings. Non-empty elements must refer to a + /// boolean input column, which will be used to mask a corresponding + /// aggregate, e.g. aggregate will skip rows where 'mask' column is false. + /// @param step Aggregation step: partial, final, intermediate or single. + /// @param ignoreNullKeys Boolean indicating whether to skip input rows where + /// one of the grouping keys is null. + PlanBuilder& aggregation( + const std::vector& groupingKeys, + const std::vector& aggregates, + const std::vector& masks, + core::AggregationNode::Step step, + bool ignoreNullKeys) { + return aggregation( + groupingKeys, {}, aggregates, masks, step, ignoreNullKeys); + } + + /// Same as above, but also allows to specify a subset of grouping keys on + /// which the input is pre-grouped or clustered. Pre-grouped keys enable + /// streaming or partially streaming aggregation algorithms which use less + /// memory and CPU then hash aggregation. The caller is responsible + /// that input data is indeed clustered on the specified keys. If that's not + /// the case, the query may return incorrect results. + PlanBuilder& aggregation( + const std::vector& groupingKeys, + const std::vector& preGroupedKeys, + const std::vector& aggregates, + const std::vector& masks, + core::AggregationNode::Step step, + bool ignoreNullKeys) { + return aggregation( + groupingKeys, + preGroupedKeys, + aggregates, + masks, + step, + ignoreNullKeys, + {}); + } + + /// A convenience method to create partial aggregation plan node for the case + /// where input is clustered on all grouping keys. + PlanBuilder& partialStreamingAggregation( + const std::vector& groupingKeys, + const std::vector& aggregates, + const std::vector& masks = {}) { + return streamingAggregation( + groupingKeys, + aggregates, + masks, + core::AggregationNode::Step::kPartial, + false); + } + + /// A convenience method to create final aggregation plan node for the case + /// where input is clustered on all grouping keys. + PlanBuilder& finalStreamingAggregation( + const std::vector& groupingKeys, + const std::vector& aggregates) { + return streamingAggregation( + groupingKeys, + aggregates, + {}, + core::AggregationNode::Step::kFinal, + false); + } + + /// Add an AggregationNode assuming input is clustered on all grouping keys. + PlanBuilder& streamingAggregation( + const std::vector& groupingKeys, + const std::vector& aggregates, + const std::vector& masks, + core::AggregationNode::Step step, + bool ignoreNullKeys); + + /// Add a GroupIdNode using the specified grouping keys, grouping sets, + /// aggregation inputs and a groupId column name. + /// The grouping keys can specify aliases if an input column is mapped + /// to an output column with a different name. + /// e.g. Grouping keys {"k1", "k1 as k2"} means there are 2 grouping keys: + /// the input column k1 and output column k2 which is an alias of column k1. + /// Grouping sets using above grouping keys use the output column aliases. + /// e.g. Grouping sets in the above case could be {{"k1"}, {"k2"}, {}} + /// The GroupIdNode output columns have grouping keys in the order specified + /// in groupingKeys variable. + PlanBuilder& groupId( + const std::vector& groupingKeys, + const std::vector>& groupingSets, + const std::vector& aggregationInputs, + std::string groupIdName = "group_id"); + + /// Add an ExpandNode using specified projections. See comments for + /// ExpandNode class for description of this plan node. + /// + /// @param projections A list of projection expressions. Each expression is + /// either a column name, null or non-null constant. + /// + /// For example, + /// + /// .expand( + /// {{"k1", "null:: bigint k2", "a", "b", "0 as gid"}, // + /// Column name will be extracted from the first projection. If the + /// column is null, it is also necessary to specify the column + /// type. + /// {"k1", "null", "a", "b", "1"}, + /// {"null", "null", "a", "b", "2"}}) + /// + /// + PlanBuilder& expand(const std::vector>& projections); + + /// Add a LocalMergeNode using specified ORDER BY clauses. + /// + /// For example, + /// + /// .localMerge({"a", "b DESC", "c ASC NULLS FIRST"}) + /// + /// By default, uses ASC NULLS LAST sort order, e.g. column "a" above will use + /// ASC NULLS LAST and column "b" will use DESC NULLS LAST. + PlanBuilder& localMerge( + const std::vector& keys, + std::vector sources); + + /// A convenience method to add a LocalMergeNode with a single source (the + /// current plan node). + PlanBuilder& localMerge(const std::vector& keys); + + /// Adds an OrderByNode using specified ORDER BY clauses. + /// + /// For example, + /// + /// .orderBy({"a", "b DESC", "c ASC NULLS FIRST"}) + /// + /// By default, uses ASC NULLS LAST sort order, e.g. column "a" above will use + /// ASC NULLS LAST and column "b" will use DESC NULLS LAST. + PlanBuilder& orderBy(const std::vector& keys, bool isPartial); + + /// Add a TopNNode using specified N and ORDER BY clauses. + /// + /// For example, + /// + /// .topN({"a", "b DESC", "c ASC NULLS FIRST"}, 10, true) + /// + /// By default, uses ASC NULLS LAST sort order, e.g. column "a" above will use + /// ASC NULLS LAST and column "b" will use DESC NULLS LAST. + PlanBuilder& + topN(const std::vector& keys, int32_t count, bool isPartial); + + /// Add a LimitNode. + /// + /// @param offset Offset, i.e. number of rows of input to skip. + /// @param count Maximum number of rows to produce after skipping 'offset' + /// rows. + /// @param isPartial Boolean indicating whether the limit node is partial or + /// final. Partial limit can run multi-threaded. Final limit must run + /// single-threaded. + PlanBuilder& limit(int64_t offset, int64_t count, bool isPartial); + + /// Add an EnforceSingleRowNode to ensure input has at most one row at + /// runtime. + PlanBuilder& enforceSingleRow(); + + /// Add an AssignUniqueIdNode to add a column with query-scoped unique value + /// per row. + /// + /// @param idName The name of output column that contains the unique ID. + /// Column type is assumed as BIGINT. + /// @param taskUniqueId ID of the Task that will be used to run the query + /// plan. The ID must be unique across all the tasks of a single query. Tasks + /// may possibly run on different machines. + PlanBuilder& assignUniqueId( + const std::string& idName = "unique", + const int32_t taskUniqueId = 1); + + /// Add a PartitionedOutputNode to hash-partition the input on the specified + /// keys using exec::HashPartitionFunction. + /// + /// @param keys Partitioning keys. May be empty, in which case all input will + /// be places in a single partition. + /// @param numPartitions Number of partitions. Must be greater than or equal + /// to 1. Keys must not be empty if greater than 1. + /// @param replicateNullsAndAny Boolean indicating whether to replicate one + /// arbitrary entry and all entries with null keys to all partitions. Used to + /// implement proper ANTI join semantics in a distributed execution + /// environment. + /// @param outputLayout Optional output layout in case it is different then + /// the input. Output columns may appear in different order from the input, + /// some input columns may be missing in the output, some columns may be + /// duplicated in the output. + PlanBuilder& partitionedOutput( + const std::vector& keys, + int numPartitions, + bool replicateNullsAndAny, + const std::vector& outputLayout = {}, + VectorSerde::Kind serdeKind = VectorSerde::Kind::kPresto); + + /// Same as above, but assumes 'replicateNullsAndAny' is false. + PlanBuilder& partitionedOutput( + const std::vector& keys, + int numPartitions, + const std::vector& outputLayout = {}, + VectorSerde::Kind serdeKind = VectorSerde::Kind::kPresto); + + /// Same as above, but allows to provide custom partition function. + PlanBuilder& partitionedOutput( + const std::vector& keys, + int numPartitions, + bool replicateNullsAndAny, + core::PartitionFunctionSpecPtr partitionFunctionSpec, + const std::vector& outputLayout = {}, + VectorSerde::Kind serdeKind = VectorSerde::Kind::kPresto); + + /// Adds a PartitionedOutputNode to broadcast the input data. + /// + /// @param outputLayout Optional output layout in case it is different then + /// the input. Output columns may appear in different order from the input, + /// some input columns may be missing in the output, some columns may be + /// duplicated in the output. + PlanBuilder& partitionedOutputBroadcast( + const std::vector& outputLayout = {}, + VectorSerde::Kind serdeKind = VectorSerde::Kind::kPresto); + + /// Adds a PartitionedOutputNode to put data into arbitrary buffer. + PlanBuilder& partitionedOutputArbitrary( + const std::vector& outputLayout = {}, + VectorSerde::Kind serdeKind = VectorSerde::Kind::kPresto); + + /// Adds a LocalPartitionNode to hash-partition the input on the specified + /// keys using exec::HashPartitionFunction. Number of partitions is determined + /// at runtime based on parallelism of the downstream pipeline. + /// + /// @param keys Partitioning keys. May be empty, in which case all input will + /// be places in a single partition. + /// @param sources One or more plan nodes that produce input data. + PlanBuilder& localPartition( + const std::vector& keys, + const std::vector& sources); + + /// A convenience method to add a LocalPartitionNode with a single source (the + /// current plan node). + PlanBuilder& localPartition(const std::vector& keys); + + /// A convenience method to add a LocalPartitionNode with hive partition + /// function. + PlanBuilder& localPartition( + int numBuckets, + const std::vector& channels, + const std::vector& constValues); + + /// A convenience method to add a LocalPartitionNode with a single source (the + /// current plan node) and hive bucket property. + PlanBuilder& localPartitionByBucket( + const std::shared_ptr& + bucketProperty); + + /// Add a LocalPartitionNode to partition the input using batch-level + /// round-robin. Number of partitions is determined at runtime based on + /// parallelism of the downstream pipeline. + /// + /// @param sources One or more plan nodes that produce input data. + PlanBuilder& localPartitionRoundRobin( + const std::vector& sources); + + /// A convenience method to add a LocalPartitionNode with a single source (the + /// current plan node). + PlanBuilder& localPartitionRoundRobin(); + + /// A convenience method to add a LocalPartitionNode for scale writer with + /// hash partitioning. + PlanBuilder& scaleWriterlocalPartition(const std::vector& keys); + + /// A convenience method to add a LocalPartitionNode for scale writer with + /// round-robin partitioning. + PlanBuilder& scaleWriterlocalPartitionRoundRobin(); + + /// Add a LocalPartitionNode to partition the input using row-wise + /// round-robin. Number of partitions is determined at runtime based on + /// parallelism of the downstream pipeline. + PlanBuilder& localPartitionRoundRobinRow(); + + /// Add a HashJoinNode to join two inputs using one or more join keys and an + /// optional filter. + /// + /// @param leftKeys Join keys from the probe side, the preceding plan node. + /// Cannot be empty. + /// @param rightKeys Join keys from the build side, the plan node specified in + /// 'build' parameter. The number and types of left and right keys must be the + /// same. + /// @param build Plan node for the build side. Typically, to reduce memory + /// usage, the smaller input is placed on the build-side. + /// @param filter Optional SQL expression for the additional join filter. Can + /// use columns from both probe and build sides of the join. + /// @param outputLayout Output layout consisting of columns from probe and + /// build sides. + /// @param joinType Type of the join: inner, left, right, full, semi, or anti. + /// @param nullAware Applies to semi and anti joins. Indicates whether the + /// join follows IN (null-aware) or EXISTS (regular) semantic. + PlanBuilder& hashJoin( + const std::vector& leftKeys, + const std::vector& rightKeys, + const core::PlanNodePtr& build, + const std::string& filter, + const std::vector& outputLayout, + core::JoinType joinType = core::JoinType::kInner, + bool nullAware = false); + + /// Add a MergeJoinNode to join two inputs using one or more join keys and an + /// optional filter. The caller is responsible to ensure that inputs are + /// sorted in ascending order on the join keys. If that's not the case, the + /// query may produce incorrect results. + /// + /// See hashJoin method for the description of the parameters. + PlanBuilder& mergeJoin( + const std::vector& leftKeys, + const std::vector& rightKeys, + const core::PlanNodePtr& build, + const std::string& filter, + const std::vector& outputLayout, + core::JoinType joinType = core::JoinType::kInner); + + /// Add a NestedLoopJoinNode to join two inputs using filter as join + /// condition to perform equal/non-equal join. Only supports inner/outer + /// joins. + /// + /// @param right Right-side input. Typically, to reduce memory usage, the + /// smaller input is placed on the right-side. + /// @param joinCondition SQL expression as the join condition. Can + /// use columns from both probe and build sides of the join. + /// @param outputLayout Output layout consisting of columns from probe and + /// build sides. + /// @param joinType Type of the join: inner, left, right, full. + PlanBuilder& nestedLoopJoin( + const core::PlanNodePtr& right, + const std::string& joinCondition, + const std::vector& outputLayout, + core::JoinType joinType = core::JoinType::kInner); + + /// Add a NestedLoopJoinNode to produce a cross product of the inputs. First + /// input comes from the preceding plan node. Second input is specified in + /// 'right' parameter. + /// + /// @param right Right-side input. Typically, to reduce memory usage, the + /// smaller input is placed on the right-side. + /// @param outputLayout Output layout consisting of columns from left and + /// right sides. + PlanBuilder& nestedLoopJoin( + const core::PlanNodePtr& right, + const std::vector& outputLayout, + core::JoinType joinType = core::JoinType::kInner); + + static core::IndexLookupConditionPtr parseIndexJoinCondition( + const std::string& joinCondition, + const RowTypePtr& rowType, + memory::MemoryPool* pool); + + /// Add an IndexLoopJoinNode to join two inputs using one or more join keys + /// plus optional join conditions. First input comes from the preceding plan + /// node. Second input is specified in 'right' parameter and must be a + /// table source with the connector table handle with index lookup support. + /// + /// @param right The right input source with index lookup support. + /// @param joinConditions SQL expressions as the join conditions. Each join + /// condition must use columns from both sides. For the right side, it can + /// only use one index column. Currently we support "in" and "between" join + /// conditions: + /// "in" condition is written as SQL expression as "contains(a, b)" where "b" + /// is the index column from right side and "a" is the condition column from + /// left side. "b" has type T and "a" has type ARRAT(T). + /// "between" condition is written as SQL expression as "a between b and c" + /// where "a" is the index column from right side and "b", "c" are either + /// condition column from left side or a constant but at least one of them + /// must not be constant. They all have the same type. + /// @param joinType Type of the join supported: inner, left. + /// @param includeMatchColumn if true, 'outputLayout' should include a boolean + /// column at the end to indicate if a join output row has a match or not. + /// This only applies for left join. + /// + /// See hashJoin method for the description of the other parameters. + PlanBuilder& indexLookupJoin( + const std::vector& leftKeys, + const std::vector& rightKeys, + const core::TableScanNodePtr& right, + const std::vector& joinConditions, + bool includeMatchColumn, + const std::vector& outputLayout, + core::JoinType joinType = core::JoinType::kInner); + + /// Add an UnnestNode to unnest one or more columns of type array or map. + /// + /// The output will contain 'replicatedColumns' followed by unnested columns, + /// followed by an optional ordinality column. + /// + /// Array columns are unnested into a single column whose name is generated by + /// appending '_e' suffix to the array column name. + /// + /// Map columns are unnested into two columns whoes names are generated by + /// appending '_k' and '_v' suffixes to the map column name. + /// + /// @param replicateColumns A subset of input columns to include in the output + /// unmodified. + /// @param unnestColumns A subset of input columns to unnest. These columns + /// must be of type array or map. + /// @param ordinalColumn An optional name for the 'ordinal' column to produce. + /// This column contains the index of the element of the unnested array or + /// map. If not specified, the output will not contain this column. + /// @param emptyUnnestValueName An optional name for the + /// 'emptyUnnestValue' column to produce. This column contains a boolean + /// indicating if the output row has empty unnest value or not. If not + /// specified, the output will not contain this column and the unnest operator + /// also skips producing output rows with empty unnest value. + PlanBuilder& unnest( + const std::vector& replicateColumns, + const std::vector& unnestColumns, + const std::optional& ordinalColumn = std::nullopt, + const std::optional& emptyUnnestValueName = std::nullopt); + + /// Add a WindowNode to compute one or more windowFunctions. + /// @param windowFunctions A list of one or more window function SQL like + /// strings to be computed by this windowNode. + /// A window function SQL string looks like : + /// "name(parameters) OVER (PARTITION BY partition_keys ORDER BY + /// sorting_keys [ROWS|RANGE BETWEEN [UNBOUNDED PRECEDING | x PRECEDING | + /// CURRENT ROW] AND [UNBOUNDED FOLLOWING | x FOLLOWING | CURRENT ROW]] AS + /// columnName" + /// The PARTITION BY and ORDER BY clauses are optional. An empty PARTITION + /// list means all the table rows are in a single partition. + /// An empty ORDER BY list means the window functions will be computed over + /// all the rows in the partition in a random order. Also, the default frame + /// if unspecified is RANGE OVER UNBOUNDED PRECEDING AND CURRENT ROW. + /// Some examples of window function strings are as follows: + /// "first_value(c) over (partition by a order by b) as d" + /// "first_value(c) over (partition by a) as d" + /// "first_value(c) over ()" + /// "row_number() over (order by b) as a" + /// "row_number() over (partition by a order by b + /// rows between a + 10 preceding and 10 following)" + PlanBuilder& window(const std::vector& windowFunctions); + + /// Adds WindowNode to compute window functions over pre-sorted inputs. + /// All functions must use same partition by and sorting keys and input must + /// be already sorted on these. + PlanBuilder& streamingWindow(const std::vector& windowFunctions); + + /// Add a RowNumberNode to compute single row_number window function with an + /// optional limit and no sorting. + PlanBuilder& rowNumber( + const std::vector& partitionKeys, + std::optional limit = std::nullopt, + bool generateRowNumber = true); + + /// Add a TopNRowNumberNode to compute row_number + /// function with a limit applied to sorted partitions. + PlanBuilder& topNRowNumber( + const std::vector& partitionKeys, + const std::vector& sortingKeys, + int32_t limit, + bool generateRowNumber); + + /// Add a TopNRowNumberNode to compute row_number, rank or dense_rank window + /// function with a limit applied to sorted partitions. + PlanBuilder& topNRank( + std::string_view function, + const std::vector& partitionKeys, + const std::vector& sortingKeys, + int32_t limit, + bool generateRowNumber); + + /// Add a MarkDistinctNode to compute aggregate mask channel + /// @param markerKey Name of output mask channel + /// @param distinctKeys List of columns to be marked distinct. + PlanBuilder& markDistinct( + std::string markerKey, + const std::vector& distinctKeys); + + /// Stores the latest plan node ID into the specified variable. Useful for + /// capturing IDs of the leaf plan nodes (table scans, exchanges, etc.) to use + /// when adding splits at runtime. + PlanBuilder& capturePlanNodeId(core::PlanNodeId& id) { + VELOX_CHECK_NOT_NULL(planNode_); + id = planNode_->id(); + return *this; + } + + /// Captures the id for the latest TableScanNode. this is useful when using + /// filtersAsNode(), where a table scan can have a filter over it. + PlanBuilder& captureScanNodeId(core::PlanNodeId& id) { + auto node = planNode_; + for (;;) { + VELOX_CHECK_NOT_NULL(node); + if (dynamic_cast(node.get())) { + id = node->id(); + return *this; + } + node = node->sources()[0]; + } + } + + /// Stores the latest plan node into the specified variable. Useful for + /// capturing intermediate plan nodes without interrupting the build flow. + template + PlanBuilder& capturePlanNode(std::shared_ptr& planNode) { + VELOX_CHECK_NOT_NULL(planNode_); + planNode = std::dynamic_pointer_cast(planNode_); + VELOX_CHECK_NOT_NULL(planNode); + return *this; + } + + /// Return the latest plan node, e.g. the root node of the plan + /// tree. The DistributedPlanBuilder override additionally moves stage + /// information to a parent PlanBuilder. + const core::PlanNodePtr& planNode() const { + return planNode_; + } + + /// Return tha latest plan node wrapped in core::PlanFragment struct. + core::PlanFragment planFragment() const { + return core::PlanFragment{planNode_}; + } + + /// Add a user-defined PlanNode as the root of the plan. 'func' takes + /// the current root of the plan and returns the new root. + PlanBuilder& addNode( + std::function + func) { + planNode_ = func(nextPlanNodeId(), planNode_); + return *this; + } + + /// Set parsing options + PlanBuilder& setParseOptions(const parse::ParseOptions& options) { + options_ = options; + return *this; + } + + /// In a DistributedPlanBuilder, introduces a shuffle boundary. The plan so + /// far is shuffled and subsequent nodes consume the shuffle. Arguments are as + /// in partitionedOutput(). + virtual PlanBuilder& shufflePartitioned( + const std::vector& keys, + int numPartitions, + bool replicateNullsAndAny, + const std::vector& outputLayout = {}) { + VELOX_UNSUPPORTED("Needs DistributedPlanBuilder"); + } + + /// In a DistributedPlanBuilder, returns an Exchange on top of the plan built + /// so far and couples it to the current stage in the enclosing builder. + /// Arguments are as in shuffle(). + virtual core::PlanNodePtr shufflePartitionedResult( + const std::vector& keys, + int numPartitions, + bool replicateNullsAndAny, + const std::vector& outputLayout = {}) { + VELOX_UNSUPPORTED("Needs DistributedPlanBuilder"); + } + + /// In a DistributedPlanBuilder, returns an Exchange on top of the plan built + /// so far that ends with a broadcast PartitionedOutput node, and couples the + /// Exchange to the current stage in the enclosing builder. + virtual core::PlanNodePtr shuffleBroadcastResult() { + VELOX_UNSUPPORTED("Needs DistributedPlanBuilder"); + } + + protected: + // Users who create custom operators might want to extend the PlanBuilder to + // customize extended plan builders. Those functions are needed in such + // extensions. + core::PlanNodeId nextPlanNodeId(); + + std::shared_ptr inferTypes( + const core::ExprPtr& untypedExpr); + + std::shared_ptr planNodeIdGenerator() const { + return planNodeIdGenerator_; + } + + memory::MemoryPool* pool() const { + return pool_; + } + + private: + std::shared_ptr field(column_index_t index); + + std::vector> fields( + const std::vector& indices); + + std::shared_ptr field( + const std::string& name); + + std::vector exprs( + const std::vector& expressions, + const RowTypePtr& inputType); + + std::vector> fields( + const std::vector& names); + + static std::vector> fields( + const RowTypePtr& inputType, + const std::vector& names); + + static std::vector> fields( + const RowTypePtr& inputType, + const std::vector& indices); + + static std::shared_ptr field( + const RowTypePtr& inputType, + column_index_t index); + + static std::shared_ptr field( + const RowTypePtr& inputType, + const std::string& name); + + core::PlanNodePtr createIntermediateOrFinalAggregation( + core::AggregationNode::Step step, + const core::AggregationNode* partialAggNode); + + struct AggregatesAndNames { + std::vector aggregates; + std::vector names; + }; + + AggregatesAndNames createAggregateExpressionsAndNames( + const std::vector& aggregates, + const std::vector& masks, + core::AggregationNode::Step step, + const std::vector>& rawInputTypes = {}); + + PlanBuilder& aggregation( + const std::vector& groupingKeys, + const std::vector& preGroupedKeys, + const std::vector& aggregates, + const std::vector& masks, + core::AggregationNode::Step step, + bool ignoreNullKeys, + const std::vector>& rawInputTypes); + + /// Create WindowNode based on whether input is sorted and then compute the + /// window functions. + PlanBuilder& window( + const std::vector& windowFunctions, + bool inputSorted); + + protected: + core::PlanNodePtr planNode_; + parse::ParseOptions options_; + std::shared_ptr tableScanBuilder_; + std::shared_ptr tableWriterBuilder_; + + private: + std::shared_ptr planNodeIdGenerator_; + memory::MemoryPool* pool_; + bool filtersAsNode_{false}; +}; +} // namespace facebook::velox::connector::lakehouse::common::test diff --git a/velox/connectors/lakehouse/iceberg/CMakeLists.txt b/velox/connectors/lakehouse/iceberg/CMakeLists.txt new file mode 100644 index 000000000000..ad0dac120466 --- /dev/null +++ b/velox/connectors/lakehouse/iceberg/CMakeLists.txt @@ -0,0 +1,30 @@ +# Copyright (c) Facebook, Inc. and its affiliates. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +velox_add_library( + velox_lakehouse_iceberg_splitreader + EqualityDeleteFileReader.cpp + FilterUtil.cpp + IcebergDeleteFile.cpp + IcebergSplitReader.cpp + IcebergSplit.cpp + PositionalDeleteFileReader.cpp + IcebergDataSink.cpp) + +velox_link_libraries(velox_lakehouse_iceberg_splitreader velox_connector + velox_dwio_common Folly::folly) + +if(${VELOX_BUILD_TESTING}) + add_subdirectory(tests) +endif() diff --git a/velox/connectors/lakehouse/iceberg/EqualityDeleteFileReader.cpp b/velox/connectors/lakehouse/iceberg/EqualityDeleteFileReader.cpp new file mode 100644 index 000000000000..31f09bcff99d --- /dev/null +++ b/velox/connectors/lakehouse/iceberg/EqualityDeleteFileReader.cpp @@ -0,0 +1,224 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "velox/connectors/lakehouse/iceberg/EqualityDeleteFileReader.h" + +#include "velox/connectors/lakehouse/common/HiveConnectorUtil.h" +#include "velox/connectors/lakehouse/iceberg/FilterUtil.h" +#include "velox/connectors/lakehouse/iceberg/IcebergDeleteFile.h" +#include "velox/core/Expressions.h" +#include "velox/dwio/common/ReaderFactory.h" + +using namespace facebook::velox::common; +using namespace facebook::velox::core; +using namespace facebook::velox::exec; + +namespace facebook::velox::connector::lakehouse::iceberg { + +static constexpr const int kMaxBatchRows = 10'000; + +EqualityDeleteFileReader::EqualityDeleteFileReader( + const IcebergDeleteFile& deleteFile, + std::shared_ptr baseFileSchema, + common::FileHandleFactory* fileHandleFactory, + folly::Executor* executor, + const ConnectorQueryCtx* connectorQueryCtx, + const std::shared_ptr& hiveConfig, + const std::shared_ptr ioStats, + const std::shared_ptr& fsStats, + const std::string& connectorId) + : deleteFile_(deleteFile), + baseFileSchema_(baseFileSchema), + fileHandleFactory_(fileHandleFactory), + pool_(connectorQueryCtx->memoryPool()), + deleteSplit_(nullptr), + deleteRowReader_(nullptr) { + VELOX_CHECK_EQ(deleteFile_.content, FileContent::kEqualityDeletes); + + if (deleteFile_.recordCount == 0) { + return; + } + + // TODO: push down filter if previous delete file contains this one. E.g. + // previous equality delete file has a=1, and this file also contains + // columns a, then a!=1 can be pushed as a filter when reading this delete + // file. + + deleteSplit_ = std::make_shared( + connectorId, + deleteFile_.filePath, + deleteFile_.fileFormat, + 0, + deleteFile_.fileSizeInBytes); + + // Create the Reader and RowReader for the equality delete file + + dwio::common::ReaderOptions deleteReaderOpts(pool_); + configureReaderOptions( + hiveConfig, + connectorQueryCtx, + nullptr, + deleteSplit_, + {}, + deleteReaderOpts); + + const common::FileHandleKey fileHandleKey{ + .filename = deleteFile_.filePath, + .tokenProvider = connectorQueryCtx->fsTokenProvider()}; + auto deleteFileHandleCachePtr = fileHandleFactory_->generate(fileHandleKey); + auto deleteFileInput = common::createBufferedInput( + *deleteFileHandleCachePtr, + deleteReaderOpts, + connectorQueryCtx, + ioStats, + fsStats, + executor); + + auto deleteReader = + dwio::common::getReaderFactory(deleteReaderOpts.fileFormat()) + ->createReader(std::move(deleteFileInput), deleteReaderOpts); + + // For now, we assume only the delete columns are written in the delete file + deleteFileRowType_ = deleteReader->rowType(); + auto scanSpec = std::make_shared(""); + scanSpec->addAllChildFields(deleteFileRowType_->asRow()); + + dwio::common::RowReaderOptions deleteRowReaderOpts; + configureRowReaderOptions( + {}, + scanSpec, + nullptr, + deleteFileRowType_, + deleteSplit_, + hiveConfig, + connectorQueryCtx->sessionProperties(), + deleteRowReaderOpts); + + deleteRowReader_.reset(); + deleteRowReader_ = deleteReader->createRowReader(deleteRowReaderOpts); +} + +void EqualityDeleteFileReader::readDeleteValues( + SubfieldFilters& subfieldFilters, + std::vector& expressionInputs) { + VELOX_CHECK(deleteRowReader_); + VELOX_CHECK(deleteSplit_); + + if (!deleteValuesOutput_) { + deleteValuesOutput_ = BaseVector::create(deleteFileRowType_, 0, pool_); + } + + // TODO: verfiy if the field is an Iceberg RowId. Velox currently doesn't + // support pushing down filters to non-RowId types, i.e. sub-fields of Array + // or Map + if (deleteFileRowType_->size() == 1) { + // Construct the IN list filter that can be pushed down to the base file + // readers, then update the baseFileScanSpec. + buildDomainFilter(subfieldFilters); + } else { + // Build the filter functions that will be evaluated after all base file + // read is done + buildFilterFunctions(expressionInputs); + } + + deleteSplit_.reset(); +} + +void EqualityDeleteFileReader::buildDomainFilter( + SubfieldFilters& subfieldFilters) { + std::unique_ptr filter = std::make_unique(); + auto name = baseFileSchema_->childByFieldId(deleteFile_.equalityFieldIds[0]) + ->fullName(); + while (deleteRowReader_->next(kMaxBatchRows, deleteValuesOutput_)) { + if (deleteValuesOutput_->size() == 0) { + continue; + } + + deleteValuesOutput_->loadedVector(); + auto vector = + std::dynamic_pointer_cast(deleteValuesOutput_)->childAt(0); + + auto typeKind = vector->type()->kind(); + VELOX_CHECK( + typeKind != TypeKind::DOUBLE && typeKind != TypeKind::REAL, + "Iceberg does not allow DOUBLE or REAL columns as the equality delete columns: {} : {}", + name, + typeKind); + + auto notExistsFilter = + createNotInFilter(vector, 0, deleteValuesOutput_->size(), typeKind); + filter = filter->mergeWith(notExistsFilter.get()); + } + + if (filter->kind() != FilterKind::kAlwaysTrue) { + if (subfieldFilters.find(velox::common::Subfield(name)) != subfieldFilters.end()) { + subfieldFilters[velox::common::Subfield(name)] = + subfieldFilters[velox::common::Subfield(name)]->mergeWith(filter.get()); + } else { + subfieldFilters[velox::common::Subfield(name)] = std::move(filter); + } + } +} + +void EqualityDeleteFileReader::buildFilterFunctions( + std::vector& expressionInputs) { + auto numDeleteFields = deleteFileRowType_->size(); + VELOX_CHECK_GT( + numDeleteFields, + 0, + "Iceberg equality delete file should have at least one field."); + + // TODO: logical expression simplifications + while (deleteRowReader_->next(kMaxBatchRows, deleteValuesOutput_)) { + if (deleteValuesOutput_->size() == 0) { + continue; + } + + deleteValuesOutput_->loadedVector(); + auto rowVector = std::dynamic_pointer_cast(deleteValuesOutput_); + auto numDeletedValues = rowVector->childAt(0)->size(); + + for (int i = 0; i < numDeletedValues; i++) { + std::vector disconjunctInputs; + + for (int j = 0; j < numDeleteFields; j++) { + auto type = deleteFileRowType_->childAt(j); + auto name = + baseFileSchema_->childByFieldId(deleteFile_.equalityFieldIds[j]) + ->fullName(); + auto value = BaseVector::wrapInConstant(1, i, rowVector->childAt(j)); + + std::vector isNotEqualInputs; + isNotEqualInputs.push_back( + std::make_shared(type, name)); + isNotEqualInputs.push_back(std::make_shared(value)); + // TODO: generalize this to support different engines. Currently, only + // Presto "neq" is supported. Spark does not register the "neq" function + // but does support "not" and "equalto" functions. + auto isNotEqualExpr = + std::make_shared(BOOLEAN(), isNotEqualInputs, "neq"); + + disconjunctInputs.push_back(isNotEqualExpr); + } + + auto disconjunctNotEqualExpr = + std::make_shared(BOOLEAN(), disconjunctInputs, "or"); + expressionInputs.push_back(disconjunctNotEqualExpr); + } + } +} + +} // namespace facebook::velox::connector::lakehouse::iceberg diff --git a/velox/connectors/lakehouse/iceberg/EqualityDeleteFileReader.h b/velox/connectors/lakehouse/iceberg/EqualityDeleteFileReader.h new file mode 100644 index 000000000000..897c9bc961ba --- /dev/null +++ b/velox/connectors/lakehouse/iceberg/EqualityDeleteFileReader.h @@ -0,0 +1,87 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#pragma once + +#include "velox/connectors/Connector.h" +#include "velox/connectors/lakehouse/common/FileHandle.h" +#include "velox/connectors/lakehouse/common/HiveConfig.h" +#include "velox/connectors/lakehouse/common/HiveConnectorSplit.h" +#include "velox/dwio/common/Reader.h" +#include "velox/expression/Expr.h" + +namespace facebook::velox::connector::lakehouse::iceberg { + +class IcebergDeleteFile; + +using SubfieldFilters = + std::unordered_map>; + +class EqualityDeleteFileReader { + public: + EqualityDeleteFileReader( + const IcebergDeleteFile& deleteFile, + std::shared_ptr baseFileSchema, + common::FileHandleFactory* fileHandleFactory, + folly::Executor* executor, + const ConnectorQueryCtx* connectorQueryCtx, + const std::shared_ptr& hiveConfig, + const std::shared_ptr ioStats, + const std::shared_ptr& fsStats, + const std::string& connectorId); + + /// Reads the delete values from the equality delete file, and interprets them + /// as filters for the base file reader. + /// + /// @subfieldFilters The built SubfieldFilter that can be pushed down to the + /// base file RowReader, when the equality delete file only contains one + /// single subfield of Iceberg RowId type. + /// @typedExpressions The built TypedExpr that will be evaluated by the + /// connector DataSource after rows are read from the base file RowReader. + void readDeleteValues( + SubfieldFilters& subfieldFilters, + std::vector& typedExpressions); + + private: + void buildDomainFilter(SubfieldFilters& subfieldFilters); + + void buildFilterFunctions(std::vector& expressionInputs); + + // The equality delete file to read + const IcebergDeleteFile& deleteFile_; + // The schema of the base file in terms of TypeWithId tree. In addition to the + // existing fields that were included in the base file ScanSpec, it also + // contains the extra fields that are in the equality delete file but not + // in the ScanSpec of the base file + const std::shared_ptr baseFileSchema_; + + // The cache factory of the file handles, which can be used to return the file + // handle of the delete file. + common::FileHandleFactory* const fileHandleFactory_; + memory::MemoryPool* const pool_; + + // The split of the equality delete file to be processed by the delete file + // RowReader. + std::shared_ptr deleteSplit_; + // The RowType of the equality delete file + RowTypePtr deleteFileRowType_; + // The RowReader to read the equality delete file + std::unique_ptr deleteRowReader_; + // The output vector to hold the delete values + VectorPtr deleteValuesOutput_; +}; + +} // namespace facebook::velox::connector::lakehouse::iceberg diff --git a/velox/connectors/lakehouse/iceberg/FilterUtil.cpp b/velox/connectors/lakehouse/iceberg/FilterUtil.cpp new file mode 100644 index 000000000000..1a32fe366aa2 --- /dev/null +++ b/velox/connectors/lakehouse/iceberg/FilterUtil.cpp @@ -0,0 +1,87 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#include "velox/connectors/lakehouse/iceberg/FilterUtil.h" + +namespace facebook::velox::connector::lakehouse::iceberg { + +using namespace facebook::velox::exec; +using namespace facebook::velox::core; + +template +std::unique_ptr createNegatedBigintValuesFilter( + const VectorPtr& valuesVector, + vector_size_t offset, + vector_size_t size) { + auto valuesPair = + common::deDuplicateValues(valuesVector, offset, size); + + const auto& values = valuesPair.first; + bool hasNull = valuesPair.second; + + return common::createNegatedBigintValues(values, !hasNull); +} + +std::unique_ptr createNotInFilter( + const VectorPtr& elements, + vector_size_t offset, + vector_size_t size, + TypeKind type) { + std::unique_ptr filter; + switch (type) { + case TypeKind::HUGEINT: + // TODO: createNegatedHugeintValuesFilter is not implemented yet. + VELOX_NYI("createNegatedHugeintValuesFilter is not implemented yet"); + case TypeKind::BIGINT: + filter = createNegatedBigintValuesFilter(elements, offset, size); + break; + case TypeKind::INTEGER: + filter = createNegatedBigintValuesFilter(elements, offset, size); + break; + case TypeKind::SMALLINT: + filter = createNegatedBigintValuesFilter(elements, offset, size); + break; + case TypeKind::TINYINT: + filter = createNegatedBigintValuesFilter(elements, offset, size); + break; + case TypeKind::BOOLEAN: + // Hack: using BIGINT filter for bool, which is essentially "int1_t". + filter = createNegatedBigintValuesFilter(elements, offset, size); + break; + case TypeKind::TIMESTAMP: + filter = + createNegatedBigintValuesFilter(elements, offset, size); + break; + case TypeKind::VARCHAR: + case TypeKind::VARBINARY: + // TODO: createNegatedBytesValuesFilter is not implemented yet. + VELOX_NYI("createNegatedBytesValuesFilter is not implemented yet"); + case TypeKind::REAL: + case TypeKind::DOUBLE: + case TypeKind::UNKNOWN: + [[fallthrough]]; + case TypeKind::ARRAY: + [[fallthrough]]; + case TypeKind::MAP: + [[fallthrough]]; + case TypeKind::ROW: + [[fallthrough]]; + default: + VELOX_USER_FAIL( + "Iceberg equality delete column cannot be of type ", type); + } + return filter; +} +} // namespace facebook::velox::connector::lakehouse::iceberg diff --git a/velox/connectors/lakehouse/iceberg/FilterUtil.h b/velox/connectors/lakehouse/iceberg/FilterUtil.h new file mode 100644 index 000000000000..f2cfaaee0312 --- /dev/null +++ b/velox/connectors/lakehouse/iceberg/FilterUtil.h @@ -0,0 +1,30 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#pragma once + +#include "velox/expression/Expr.h" +#include "velox/type/Filter.h" +#include "velox/vector/ComplexVector.h" + +namespace facebook::velox::connector::lakehouse::iceberg { +std::unique_ptr createNotInFilter( + const VectorPtr& elements, + vector_size_t offset, + vector_size_t size, + TypeKind type); + +} // namespace facebook::velox::connector::lakehouse::iceberg diff --git a/velox/connectors/lakehouse/iceberg/IcebergDataSink.cpp b/velox/connectors/lakehouse/iceberg/IcebergDataSink.cpp new file mode 100644 index 000000000000..feb1257ac838 --- /dev/null +++ b/velox/connectors/lakehouse/iceberg/IcebergDataSink.cpp @@ -0,0 +1,254 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "velox/connectors/lakehouse/iceberg/IcebergDataSink.h" +#include "velox/common/base/Fs.h" +#include "velox/connectors/lakehouse/common/HiveConnectorUtil.h" + +namespace facebook::velox::connector::lakehouse::iceberg { + +namespace { + +std::string toJson(const std::vector& partitionValues) { + folly::dynamic jsonObject = folly::dynamic::object(); + folly::dynamic valuesArray = folly::dynamic::array(); + for (const auto& value : partitionValues) { + valuesArray.push_back(value); + } + jsonObject["partitionValues"] = valuesArray; + return folly::toJson(jsonObject); +} + +template +folly::dynamic extractPartitionValue( + const DecodedVector* block, + vector_size_t row) { + using T = typename TypeTraits::NativeType; + return block->valueAt(row); +} + +template <> +folly::dynamic extractPartitionValue( + const DecodedVector* block, + vector_size_t row) { + return block->toString(row); +} + +template <> +folly::dynamic extractPartitionValue( + const DecodedVector* block, + vector_size_t row) { + VELOX_NYI("Partition on varbinary column is not supported yet."); +} + +class IcebergFileNameGenerator : public common::FileNameGenerator { + public: + IcebergFileNameGenerator() {} + + std::pair gen( + std::optional bucketId, + const std::shared_ptr insertTableHandle, + const ConnectorQueryCtx& connectorQueryCtx, + bool commitRequired) const override; + + folly::dynamic serialize() const override; + + std::string toString() const override; +}; + +std::pair IcebergFileNameGenerator::gen( + std::optional bucketId, + const std::shared_ptr insertTableHandle, + const ConnectorQueryCtx& connectorQueryCtx, + bool commitRequired) const { + auto targetFileName = insertTableHandle->locationHandle()->targetFileName(); + if (targetFileName.empty()) { + targetFileName = fmt::format("{}", common::makeUuid()); + } + + return { + fmt::format("{}{}", targetFileName, ".parquet"), + fmt::format("{}{}", targetFileName, ".parquet")}; +} + +folly::dynamic IcebergFileNameGenerator::serialize() const { + VELOX_UNREACHABLE("Unexpected code path, implement serialize() first."); +} + +std::string IcebergFileNameGenerator::toString() const { + return "IcebergFileNameGenerator"; +} + +} // namespace + +IcebergInsertTableHandle::IcebergInsertTableHandle( + std::vector> inputColumns, + std::shared_ptr locationHandle, + std::shared_ptr partitionSpec, + dwio::common::FileFormat tableStorageFormat, + std::shared_ptr bucketProperty, + std::optional compressionKind, + const std::unordered_map& serdeParameters) + : common::HiveInsertTableHandle( + std::move(inputColumns), + std::move(locationHandle), + tableStorageFormat, + std::move(bucketProperty), + compressionKind, + serdeParameters, + nullptr, + false, + std::make_shared()), + partitionSpec_(std::move(partitionSpec)) {} + +IcebergDataSink::IcebergDataSink( + facebook::velox::RowTypePtr inputType, + std::shared_ptr insertTableHandle, + const facebook::velox::connector::ConnectorQueryCtx* connectorQueryCtx, + facebook::velox::connector::CommitStrategy commitStrategy, + const std::shared_ptr& hiveConfig) + : IcebergDataSink( + std::move(inputType), + insertTableHandle, + connectorQueryCtx, + commitStrategy, + hiveConfig, + [&insertTableHandle]() { + std::vector channels( + insertTableHandle->inputColumns().size()); + std::iota(channels.begin(), channels.end(), 0); + return channels; + }()) {} + +IcebergDataSink::IcebergDataSink( + RowTypePtr inputType, + std::shared_ptr insertTableHandle, + const ConnectorQueryCtx* connectorQueryCtx, + CommitStrategy commitStrategy, + const std::shared_ptr& hiveConfig, + const std::vector& dataChannels) + : common::HiveDataSink( + inputType, + insertTableHandle, + connectorQueryCtx, + commitStrategy, + hiveConfig, + 0, + nullptr, + dataChannels) { + if (isPartitioned()) { + partitionData_.resize(maxOpenWriters_); + } +} + +std::vector IcebergDataSink::commitMessage() const { + auto icebergInsertTableHandle = + std::dynamic_pointer_cast( + insertTableHandle_); + + std::vector commitTasks; + commitTasks.reserve(writerInfo_.size()); + std::string fileFormat(toString(insertTableHandle_->storageFormat())); + std::transform( + fileFormat.begin(), fileFormat.end(), fileFormat.begin(), ::toupper); + + for (auto i = 0; i < writerInfo_.size(); ++i) { + const auto& info = writerInfo_.at(i); + VELOX_CHECK_NOT_NULL(info); + // Following metadata (json format) is consumed by Presto CommitTaskData. + // It contains the minimal subset of metadata. + // Complete metrics is missing now and this could lead to suboptimal query + // plan, will collect full iceberg metrics in following PR. + // clang-format off + folly::dynamic commitData = folly::dynamic::object( + "path", info->writerParameters.writeDirectory() + "/" + + info->writerParameters.writeFileName()) + ("fileSizeInBytes", ioStats_.at(i)->rawBytesWritten()) + ("metrics", + folly::dynamic::object("recordCount", info->numWrittenRows)) + ("partitionSpecJson", icebergInsertTableHandle->partitionSpec()->specId) + ("fileFormat", fileFormat) + ("content", "DATA"); + // clang-format on + if (!(partitionData_.empty() || partitionData_[i].empty())) { + commitData["partitionDataJson"] = toJson(partitionData_[i]); + } + auto commitDataJson = folly::toJson(commitData); + commitTasks.push_back(commitDataJson); + } + return commitTasks; +} + +void IcebergDataSink::splitInputRowsAndEnsureWriters(RowVectorPtr input) { + VELOX_CHECK(isPartitioned()); + + std::fill(partitionSizes_.begin(), partitionSizes_.end(), 0); + + const auto numRows = partitionIds_.size(); + for (auto row = 0; row < numRows; ++row) { + auto id = getIcebergWriterId(row); + uint32_t index = ensureWriter(id); + + updatePartitionRows(index, numRows, row); + + if (!partitionData_[index].empty()) { + continue; + } + + std::vector partitionValues(partitionChannels_.size()); + + for (auto i = 0; i < partitionChannels_.size(); ++i) { + auto block = input->childAt(partitionChannels_[i]); + if (block->type()->isDecimal()) { + VELOX_NYI("Partition on decimal column is not supported yet."); + } + DecodedVector decoded(*block); + partitionValues[i] = VELOX_DYNAMIC_SCALAR_TYPE_DISPATCH( + extractPartitionValue, block->typeKind(), &decoded, row); + } + + partitionData_[index] = partitionValues; + } + + for (auto i = 0; i < partitionSizes_.size(); ++i) { + if (partitionSizes_[i] != 0) { + VELOX_CHECK_NOT_NULL(partitionRows_[i]); + partitionRows_[i]->setSize(partitionSizes_[i] * sizeof(vector_size_t)); + } + } +} + +common::HiveWriterId IcebergDataSink::getIcebergWriterId(size_t row) const { + std::optional partitionId; + if (isPartitioned()) { + VELOX_CHECK_LT(partitionIds_[row], std::numeric_limits::max()); + partitionId = static_cast(partitionIds_[row]); + } + + return common::HiveWriterId{partitionId, std::nullopt}; +} + +std::optional IcebergDataSink::getPartitionName( + const common::HiveWriterId& id) const { + std::optional partitionName; + if (isPartitioned()) { + partitionName = + partitionIdGenerator_->partitionName(id.partitionId.value(), "null"); + } + return partitionName; +} + +} // namespace facebook::velox::connector::lakehouse::iceberg diff --git a/velox/connectors/lakehouse/iceberg/IcebergDataSink.h b/velox/connectors/lakehouse/iceberg/IcebergDataSink.h new file mode 100644 index 000000000000..393a3a3e6a63 --- /dev/null +++ b/velox/connectors/lakehouse/iceberg/IcebergDataSink.h @@ -0,0 +1,79 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#pragma once + +#include "velox/connectors/lakehouse/common/HiveDataSink.h" +#include "velox/connectors/lakehouse/iceberg/PartitionSpec.h" + +namespace facebook::velox::connector::lakehouse::iceberg { + +// Represents a request for Iceberg write. +class IcebergInsertTableHandle final : public common::HiveInsertTableHandle { + public: + IcebergInsertTableHandle( + std::vector> inputColumns, + std::shared_ptr locationHandle, + std::shared_ptr partitionSpec, + dwio::common::FileFormat tableStorageFormat = + dwio::common::FileFormat::PARQUET, + std::shared_ptr bucketProperty = nullptr, + std::optional compressionKind = {}, + const std::unordered_map& serdeParameters = {}); + + ~IcebergInsertTableHandle() = default; + + std::shared_ptr partitionSpec() const { + return partitionSpec_; + } + + private: + std::shared_ptr partitionSpec_; +}; + +class IcebergDataSink : public common::HiveDataSink { + public: + IcebergDataSink( + RowTypePtr inputType, + std::shared_ptr insertTableHandle, + const ConnectorQueryCtx* connectorQueryCtx, + CommitStrategy commitStrategy, + const std::shared_ptr& hiveConfig); + + private: + IcebergDataSink( + RowTypePtr inputType, + std::shared_ptr insertTableHandle, + const ConnectorQueryCtx* connectorQueryCtx, + CommitStrategy commitStrategy, + const std::shared_ptr& hiveConfig, + const std::vector& dataChannels); + + void splitInputRowsAndEnsureWriters(RowVectorPtr input) override; + + std::vector commitMessage() const override; + + common::HiveWriterId getIcebergWriterId(size_t row) const; + + std::optional getPartitionName( + const common::HiveWriterId& id) const override; + + // Below are structures for partitions from all inputs. partitionData_ + // is indexed by partitionId. + std::vector> partitionData_; +}; + +} // namespace facebook::velox::connector::lakehouse::iceberg diff --git a/velox/connectors/lakehouse/iceberg/IcebergDeleteFile.cpp b/velox/connectors/lakehouse/iceberg/IcebergDeleteFile.cpp new file mode 100644 index 000000000000..bdd927a9b85a --- /dev/null +++ b/velox/connectors/lakehouse/iceberg/IcebergDeleteFile.cpp @@ -0,0 +1,40 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "velox/connectors/lakehouse/iceberg/IcebergDeleteFile.h" + +namespace facebook::velox::connector::lakehouse::iceberg { + +fmt::underlying_t format_as(FileContent f) { + return fmt::underlying(f); +} + +} // namespace facebook::velox::connector::lakehouse::iceberg diff --git a/velox/connectors/lakehouse/iceberg/IcebergDeleteFile.h b/velox/connectors/lakehouse/iceberg/IcebergDeleteFile.h new file mode 100644 index 000000000000..ef5e33445352 --- /dev/null +++ b/velox/connectors/lakehouse/iceberg/IcebergDeleteFile.h @@ -0,0 +1,72 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#pragma once + +#include +#include +#include + +#include "velox/dwio/common/Options.h" + +namespace facebook::velox::connector::lakehouse::iceberg { + +enum class FileContent { + kData, + kPositionalDeletes, + kEqualityDeletes, +}; + +/// Formatter for fmt. +fmt::underlying_t format_as(FileContent f); + +struct IcebergDeleteFile { + FileContent content; + const std::string filePath; + dwio::common::FileFormat fileFormat; + uint64_t recordCount; + uint64_t fileSizeInBytes; + // The field ids for the delete columns for equality delete files + std::vector equalityFieldIds; + // The lower bounds of the in-file positions for the deleted rows, identified + // by each column's field id. E.g. The deleted rows for a column with field id + // 1 is in range [10, 50], where 10 and 50 are the deleted row positions in + // the data file, then lowerBounds would contain entry <1, "10"> + std::unordered_map lowerBounds; + // The upper bounds of the in-file positions for the deleted rows, identified + // by each column's field id. E.g. The deleted rows for a column with field id + // 1 is in range [10, 50], then upperBounds will contain entry <1, "50"> + std::unordered_map upperBounds; + + IcebergDeleteFile( + FileContent _content, + const std::string& _filePath, + dwio::common::FileFormat _fileFormat, + uint64_t _recordCount, + uint64_t _fileSizeInBytes, + std::vector _equalityFieldIds = {}, + std::unordered_map _lowerBounds = {}, + std::unordered_map _upperBounds = {}) + : content(_content), + filePath(_filePath), + fileFormat(_fileFormat), + recordCount(_recordCount), + fileSizeInBytes(_fileSizeInBytes), + equalityFieldIds(_equalityFieldIds), + lowerBounds(_lowerBounds), + upperBounds(_upperBounds) {} +}; + +} // namespace facebook::velox::connector::lakehouse::iceberg diff --git a/velox/connectors/lakehouse/iceberg/IcebergMetadataColumns.h b/velox/connectors/lakehouse/iceberg/IcebergMetadataColumns.h new file mode 100644 index 000000000000..2954e702b759 --- /dev/null +++ b/velox/connectors/lakehouse/iceberg/IcebergMetadataColumns.h @@ -0,0 +1,55 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#pragma once + +#include + +#include "velox/type/Type.h" + +namespace facebook::velox::connector::lakehouse::iceberg { + +struct IcebergMetadataColumn { + int id; + std::string name; + std::shared_ptr type; + std::string doc; + + IcebergMetadataColumn( + int _id, + const std::string& _name, + std::shared_ptr _type, + const std::string& _doc) + : id(_id), name(_name), type(_type), doc(_doc) {} + + static std::shared_ptr icebergDeleteFilePathColumn() { + return std::make_shared( + 2147483546, + "file_path", + VARCHAR(), + "Path of a file in which a deleted row is stored"); + } + + static std::shared_ptr icebergDeletePosColumn() { + return std::make_shared( + 2147483545, + "pos", + BIGINT(), + "Ordinal position of a deleted row in the data file"); + } +}; + +} // namespace facebook::velox::connector::lakehouse::iceberg diff --git a/velox/connectors/lakehouse/iceberg/IcebergSplit.cpp b/velox/connectors/lakehouse/iceberg/IcebergSplit.cpp new file mode 100644 index 000000000000..939bb825bc1c --- /dev/null +++ b/velox/connectors/lakehouse/iceberg/IcebergSplit.cpp @@ -0,0 +1,91 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "velox/connectors/lakehouse/iceberg/IcebergSplit.h" + +#include "velox/connectors/lakehouse/iceberg/IcebergDeleteFile.h" + +namespace facebook::velox::connector::lakehouse::iceberg { + +HiveIcebergSplit::HiveIcebergSplit( + const std::string& connectorId, + const std::string& filePath, + dwio::common::FileFormat fileFormat, + uint64_t start, + uint64_t length, + const std::unordered_map>& + partitionKeys, + std::optional tableBucketNumber, + const std::unordered_map& customSplitInfo, + const std::shared_ptr& extraFileInfo, + bool cacheable, + const std::unordered_map& infoColumns, + std::optional properties) + : common::HiveConnectorSplit( + connectorId, + filePath, + fileFormat, + start, + length, + partitionKeys, + tableBucketNumber, + customSplitInfo, + extraFileInfo, + /*serdeParameters=*/{}, + /*splitWeight=*/0, + cacheable, + infoColumns, + properties, + std::nullopt, + std::nullopt) { + // TODO: Deserialize _extraFileInfo to get deleteFiles; +} + +// For tests only +HiveIcebergSplit::HiveIcebergSplit( + const std::string& connectorId, + const std::string& filePath, + dwio::common::FileFormat fileFormat, + uint64_t start, + uint64_t length, + const std::unordered_map>& + partitionKeys, + std::optional tableBucketNumber, + const std::unordered_map& customSplitInfo, + const std::shared_ptr& extraFileInfo, + bool cacheable, + std::vector deletes, + const std::unordered_map& infoColumns, + std::optional properties) + : common::HiveConnectorSplit( + connectorId, + filePath, + fileFormat, + start, + length, + partitionKeys, + tableBucketNumber, + customSplitInfo, + extraFileInfo, + /*serdeParameters=*/{}, + 0, + cacheable, + infoColumns, + properties, + std::nullopt, + std::nullopt), + deleteFiles(std::move(deletes)) {} +} // namespace facebook::velox::connector::lakehouse::iceberg diff --git a/velox/connectors/lakehouse/iceberg/IcebergSplit.h b/velox/connectors/lakehouse/iceberg/IcebergSplit.h new file mode 100644 index 000000000000..8e350887fe80 --- /dev/null +++ b/velox/connectors/lakehouse/iceberg/IcebergSplit.h @@ -0,0 +1,61 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#pragma once + +#include + +#include "velox/connectors/lakehouse/common/HiveConnectorSplit.h" +#include "velox/connectors/lakehouse/iceberg/IcebergDeleteFile.h" + +namespace facebook::velox::connector::lakehouse::iceberg { + +struct HiveIcebergSplit : public connector::lakehouse::common::HiveConnectorSplit { + std::vector deleteFiles; + + HiveIcebergSplit( + const std::string& connectorId, + const std::string& filePath, + dwio::common::FileFormat fileFormat, + uint64_t start = 0, + uint64_t length = std::numeric_limits::max(), + const std::unordered_map>& + partitionKeys = {}, + std::optional tableBucketNumber = std::nullopt, + const std::unordered_map& customSplitInfo = {}, + const std::shared_ptr& extraFileInfo = {}, + bool cacheable = true, + const std::unordered_map& infoColumns = {}, + std::optional fileProperties = std::nullopt); + + // For tests only + HiveIcebergSplit( + const std::string& connectorId, + const std::string& filePath, + dwio::common::FileFormat fileFormat, + uint64_t start = 0, + uint64_t length = std::numeric_limits::max(), + const std::unordered_map>& + partitionKeys = {}, + std::optional tableBucketNumber = std::nullopt, + const std::unordered_map& customSplitInfo = {}, + const std::shared_ptr& extraFileInfo = {}, + bool cacheable = true, + std::vector deletes = {}, + const std::unordered_map& infoColumns = {}, + std::optional fileProperties = std::nullopt); +}; + +} // namespace facebook::velox::connector::lakehouse::iceberg diff --git a/velox/connectors/lakehouse/iceberg/IcebergSplitReader.cpp b/velox/connectors/lakehouse/iceberg/IcebergSplitReader.cpp new file mode 100644 index 000000000000..ffe60708a816 --- /dev/null +++ b/velox/connectors/lakehouse/iceberg/IcebergSplitReader.cpp @@ -0,0 +1,243 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "velox/connectors/lakehouse/iceberg/IcebergSplitReader.h" + +#include "velox/connectors/lakehouse/iceberg/EqualityDeleteFileReader.h" +#include "velox/connectors/lakehouse/iceberg/IcebergDeleteFile.h" +#include "velox/connectors/lakehouse/iceberg/IcebergSplit.h" +#include "velox/dwio/common/BufferUtil.h" + +using namespace facebook::velox::dwio::common; + +namespace facebook::velox::connector::lakehouse::iceberg { + +IcebergSplitReader::IcebergSplitReader( + const std::shared_ptr& hiveSplit, + const common::HiveTableHandlePtr& hiveTableHandle, + const std::unordered_map* partitionKeys, + const ConnectorQueryCtx* connectorQueryCtx, + const std::shared_ptr& hiveConfig, + const RowTypePtr& readerOutputType, + const std::shared_ptr& ioStats, + const std::shared_ptr& fsStats, + common::FileHandleFactory* const fileHandleFactory, + folly::Executor* executor, + const std::shared_ptr& scanSpec, + core::ExpressionEvaluator* expressionEvaluator, + std::atomic& totalRemainingFilterTime) + : SplitReader( + hiveSplit, + hiveTableHandle, + partitionKeys, + connectorQueryCtx, + hiveConfig, + readerOutputType, + ioStats, + fsStats, + fileHandleFactory, + executor, + scanSpec), + baseReadOffset_(0), + splitOffset_(0), + deleteBitmap_(nullptr), + deleteExprSet_(nullptr), + expressionEvaluator_(expressionEvaluator), + totalRemainingFilterMs_(totalRemainingFilterTime) {} + +IcebergSplitReader::~IcebergSplitReader() {} + +void IcebergSplitReader::prepareSplit( + std::shared_ptr metadataFilter, + dwio::common::RuntimeStatistics& runtimeStats) { + createReader(); + if (emptySplit_) { + return; + } + auto rowType = getAdaptedRowType(); + + std::shared_ptr icebergSplit = + std::dynamic_pointer_cast(hiveSplit_); + const auto& deleteFiles = icebergSplit->deleteFiles; + std::unordered_set equalityFieldIds; + for (const auto& deleteFile : deleteFiles) { + if (deleteFile.content == FileContent::kEqualityDeletes && + deleteFile.recordCount > 0) { + equalityFieldIds.insert( + deleteFile.equalityFieldIds.begin(), + deleteFile.equalityFieldIds.end()); + } + } + + // checkIfSplitIsEmpty needs to use the base reader's schemaWithId_. For that + // we need to update the base RowReader to include these extra fields from the + // equality delete file first, so that the schemaWithId_ of the base file is + // updated when we call baseFileSchema() later. + baseReader_->setRequiredExtraFieldIds(equalityFieldIds); + + if (checkIfSplitIsEmpty(runtimeStats)) { + VELOX_CHECK(emptySplit_); + return; + } + + // Process the equality delete files to update the scan spec and remaining + // filters. It needs to be done after creating the Reader and before creating + // the RowReader. + + SubfieldFilters subfieldFilters; + std::vector conjunctInputs; + + for (const auto& deleteFile : deleteFiles) { + if (deleteFile.content == FileContent::kEqualityDeletes && + deleteFile.recordCount > 0) { + // TODO: build cache of to avoid repeating file + // parsing across partitions. Within a single partition, the splits should + // be with the same equality delete files and only need to be parsed once. + auto equalityDeleteReader = std::make_unique( + deleteFile, + baseFileSchema(), + fileHandleFactory_, + executor_, + connectorQueryCtx_, + hiveConfig_, + ioStats_, + fsStats_, + hiveSplit_->connectorId); + equalityDeleteReader->readDeleteValues(subfieldFilters, conjunctInputs); + } + } + + if (!subfieldFilters.empty()) { + for (const auto& [key, filter] : subfieldFilters) { + auto childSpec = scanSpec_->getOrCreateChild(key, true); + childSpec->addFilter(*filter); + childSpec->setHasTempFilter(true); + childSpec->setSubscript(scanSpec_->children().size() - 1); + } + } + + if (!conjunctInputs.empty()) { + core::TypedExprPtr expression = + std::make_shared(BOOLEAN(), conjunctInputs, "and"); + deleteExprSet_ = expressionEvaluator_->compile(expression); + VELOX_CHECK_EQ(deleteExprSet_->size(), 1); + } + + createRowReader(std::move(metadataFilter), std::move(rowType)); + + baseReadOffset_ = 0; + splitOffset_ = baseRowReader_->nextRowNumber(); + + // Create the positional deletes file readers. They need to be created after + // the RowReader is created. + positionalDeleteFileReaders_.clear(); + for (const auto& deleteFile : deleteFiles) { + if (deleteFile.content == FileContent::kPositionalDeletes) { + if (deleteFile.recordCount > 0) { + positionalDeleteFileReaders_.push_back( + std::make_unique( + deleteFile, + hiveSplit_->filePath, + fileHandleFactory_, + connectorQueryCtx_, + executor_, + hiveConfig_, + ioStats_, + fsStats_, + runtimeStats, + splitOffset_, + hiveSplit_->connectorId)); + } + } + } +} + +std::shared_ptr +IcebergSplitReader::baseFileSchema() { + VELOX_CHECK_NOT_NULL(baseReader_.get()); + return baseReader_->typeWithId(); +} + +uint64_t IcebergSplitReader::next(uint64_t size, VectorPtr& output) { + Mutation mutation; + mutation.randomSkip = baseReaderOpts_.randomSkip().get(); + mutation.deletedRows = nullptr; + + if (deleteBitmap_) { + std::memset( + (void*)(deleteBitmap_->asMutable()), 0L, deleteBitmap_->size()); + } + + const auto actualSize = baseRowReader_->nextReadSize(size); + + if (actualSize == dwio::common::RowReader::kAtEnd) { + return 0; + } + + if (!positionalDeleteFileReaders_.empty()) { + auto numBytes = bits::nbytes(actualSize); + dwio::common::ensureCapacity( + deleteBitmap_, numBytes, connectorQueryCtx_->memoryPool(), false, true); + + for (auto iter = positionalDeleteFileReaders_.begin(); + iter != positionalDeleteFileReaders_.end();) { + (*iter)->readDeletePositions(baseReadOffset_, actualSize, deleteBitmap_); + + if ((*iter)->noMoreData()) { + iter = positionalDeleteFileReaders_.erase(iter); + } else { + ++iter; + } + } + } + + mutation.deletedRows = deleteBitmap_ && deleteBitmap_->size() > 0 + ? deleteBitmap_->as() + : nullptr; + + auto rowsScanned = baseRowReader_->next(actualSize, output, &mutation); + + // Evaluate the remaining filter deleteExprSet_ for every batch and update the + // output vector if it reduces any rows. + if (deleteExprSet_) { + auto filterStartMs = getCurrentTimeMs(); + + filterRows_.resize(output->size()); + auto rowVector = std::dynamic_pointer_cast(output); + expressionEvaluator_->evaluate( + deleteExprSet_.get(), filterRows_, *rowVector, filterResult_); + auto numRemainingRows = exec::processFilterResults( + filterResult_, filterRows_, filterEvalCtx_, pool_); + + if (numRemainingRows < output->size()) { + output = exec::wrap( + numRemainingRows, filterEvalCtx_.selectedIndices, rowVector); + } + + totalRemainingFilterMs_.fetch_add( + (getCurrentTimeMs() - filterStartMs), std::memory_order_relaxed); + } + + baseReadOffset_ += rowsScanned; + + if (rowsScanned == 0) { + scanSpec_->deleteTempNodes(); + } + + return rowsScanned; +} + +} // namespace facebook::velox::connector::lakehouse::iceberg diff --git a/velox/connectors/lakehouse/iceberg/IcebergSplitReader.h b/velox/connectors/lakehouse/iceberg/IcebergSplitReader.h new file mode 100644 index 000000000000..ee1308ef14e1 --- /dev/null +++ b/velox/connectors/lakehouse/iceberg/IcebergSplitReader.h @@ -0,0 +1,74 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#pragma once + +#include "velox/connectors/Connector.h" +#include "velox/connectors/lakehouse/common/SplitReader.h" +#include "velox/connectors/lakehouse/iceberg/PositionalDeleteFileReader.h" +#include "velox/exec/OperatorUtils.h" + +namespace facebook::velox::connector::lakehouse::iceberg { + +struct IcebergDeleteFile; + +class IcebergSplitReader : public common::SplitReader { + public: + IcebergSplitReader( + const std::shared_ptr& hiveSplit, + const common::HiveTableHandlePtr& hiveTableHandle, + const std::unordered_map* partitionKeys, + const ConnectorQueryCtx* connectorQueryCtx, + const std::shared_ptr& hiveConfig, + const RowTypePtr& readerOutputType, + const std::shared_ptr& ioStats, + const std::shared_ptr& fsStats, + common::FileHandleFactory* fileHandleFactory, + folly::Executor* executor, + const std::shared_ptr& scanSpec, + core::ExpressionEvaluator* expressionEvaluator, + std::atomic& totalRemainingFilterTime); + + ~IcebergSplitReader() override; + + void prepareSplit( + std::shared_ptr metadataFilter, + dwio::common::RuntimeStatistics& runtimeStats) override; + + uint64_t next(uint64_t size, VectorPtr& output) override; + + std::shared_ptr baseFileSchema(); + + private: + // The read offset to the beginning of the split in number of rows for the + // current batch for the base data file + uint64_t baseReadOffset_; + // The file position for the first row in the split + uint64_t splitOffset_; + std::list> + positionalDeleteFileReaders_; + BufferPtr deleteBitmap_; + + std::unique_ptr deleteExprSet_; + core::ExpressionEvaluator* expressionEvaluator_; + std::atomic& totalRemainingFilterMs_; + + // Reusable memory for remaining filter evaluation. + VectorPtr filterResult_; + SelectivityVector filterRows_; + exec::FilterEvalCtx filterEvalCtx_; +}; +} // namespace facebook::velox::connector::lakehouse::iceberg diff --git a/velox/connectors/lakehouse/iceberg/PartitionSpec.h b/velox/connectors/lakehouse/iceberg/PartitionSpec.h new file mode 100644 index 000000000000..c4c89030d213 --- /dev/null +++ b/velox/connectors/lakehouse/iceberg/PartitionSpec.h @@ -0,0 +1,63 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#pragma once + +#include +#include +#include + +namespace facebook::velox::connector::lakehouse::iceberg { + +enum class TransformType { + kIdentity, + kHour, + kDay, + kMonth, + kYear, + kBucket, + kTruncate +}; + +struct IcebergPartitionSpec { + struct Field { + // The column name and type of this partition field as it appears in the + // partition spec. The column can be a nested column in struct field. + std::string name; + + // The transform type applied to the source field (e.g., kIdentity, kBucket, + // kTruncate, etc.). + TransformType transformType; + + // Optional parameter for transforms that require configuration + // (e.g., bucket count or truncate width). + std::optional parameter; + + Field( + const std::string& _name, + TransformType _transform, + std::optional _parameter) + : name(_name), transformType(_transform), parameter(_parameter) {} + }; + + const int32_t specId; + const std::vector fields; + + IcebergPartitionSpec(int32_t _specId, const std::vector& _fields) + : specId(_specId), fields(_fields) {} +}; + +} // namespace facebook::velox::connector::lakehouse::iceberg diff --git a/velox/connectors/lakehouse/iceberg/PositionalDeleteFileReader.cpp b/velox/connectors/lakehouse/iceberg/PositionalDeleteFileReader.cpp new file mode 100644 index 000000000000..2f9c97e64645 --- /dev/null +++ b/velox/connectors/lakehouse/iceberg/PositionalDeleteFileReader.cpp @@ -0,0 +1,287 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "velox/connectors/lakehouse/iceberg/PositionalDeleteFileReader.h" + +#include "velox/connectors/lakehouse/common/HiveConnectorUtil.h" +#include "velox/connectors/lakehouse/common/TableHandle.h" +#include "velox/connectors/lakehouse/iceberg/IcebergDeleteFile.h" +#include "velox/connectors/lakehouse/iceberg/IcebergMetadataColumns.h" +#include "velox/dwio/common/ReaderFactory.h" + +namespace facebook::velox::connector::lakehouse::iceberg { + +PositionalDeleteFileReader::PositionalDeleteFileReader( + const IcebergDeleteFile& deleteFile, + const std::string& baseFilePath, + common::FileHandleFactory* fileHandleFactory, + const ConnectorQueryCtx* connectorQueryCtx, + folly::Executor* executor, + const std::shared_ptr& hiveConfig, + const std::shared_ptr& ioStats, + const std::shared_ptr& fsStats, + dwio::common::RuntimeStatistics& runtimeStats, + uint64_t splitOffset, + const std::string& connectorId) + : deleteFile_(deleteFile), + baseFilePath_(baseFilePath), + fileHandleFactory_(fileHandleFactory), + executor_(executor), + connectorQueryCtx_(connectorQueryCtx), + hiveConfig_(hiveConfig), + ioStats_(ioStats), + fsStats_(fsStats), + pool_(connectorQueryCtx->memoryPool()), + filePathColumn_(IcebergMetadataColumn::icebergDeleteFilePathColumn()), + posColumn_(IcebergMetadataColumn::icebergDeletePosColumn()), + splitOffset_(splitOffset), + deleteSplit_(nullptr), + deleteRowReader_(nullptr), + deletePositionsOutput_(nullptr), + deletePositionsOffset_(0), + totalNumRowsScanned_(0) { + VELOX_CHECK(deleteFile_.content == FileContent::kPositionalDeletes); + VELOX_CHECK(deleteFile_.recordCount); + + // TODO: check if the lowerbounds and upperbounds in deleteFile overlap with + // this batch. If not, no need to proceed. + + // Create the ScanSpec for this delete file + auto scanSpec = std::make_shared(""); + scanSpec->addField(posColumn_->name, 0); + auto* pathSpec = scanSpec->getOrCreateChild(filePathColumn_->name); + pathSpec->setFilter(std::make_unique( + std::vector({baseFilePath_}), false)); + + // Create the file schema (in RowType) and split that will be used by readers + std::vector deleteColumnNames( + {filePathColumn_->name, posColumn_->name}); + std::vector> deleteColumnTypes( + {filePathColumn_->type, posColumn_->type}); + RowTypePtr deleteFileSchema = + ROW(std::move(deleteColumnNames), std::move(deleteColumnTypes)); + + deleteSplit_ = std::make_shared( + connectorId, + deleteFile_.filePath, + deleteFile_.fileFormat, + 0, + deleteFile_.fileSizeInBytes); + + // Create the Reader and RowReader + + dwio::common::ReaderOptions deleteReaderOpts(pool_); + configureReaderOptions( + hiveConfig_, + connectorQueryCtx, + deleteFileSchema, + deleteSplit_, + /*tableParameters=*/{}, + deleteReaderOpts); + + const common::FileHandleKey fileHandleKey{ + .filename = deleteFile_.filePath, + .tokenProvider = connectorQueryCtx_->fsTokenProvider()}; + auto deleteFileHandleCachePtr = fileHandleFactory_->generate(fileHandleKey); + auto deleteFileInput = common::createBufferedInput( + *deleteFileHandleCachePtr, + deleteReaderOpts, + connectorQueryCtx, + ioStats_, + fsStats_, + executor_); + + auto deleteReader = + dwio::common::getReaderFactory(deleteReaderOpts.fileFormat()) + ->createReader(std::move(deleteFileInput), deleteReaderOpts); + + // Check if the whole delete file split can be skipped. This could happen when + // 1) the delete file doesn't contain the base file that is being read; 2) The + // delete file does not contain the positions in the current batch for the + // base file. + if (!common::testFilters( + scanSpec.get(), + deleteReader.get(), + deleteSplit_->filePath, + deleteSplit_->partitionKeys, + {}, + hiveConfig_->readTimestampPartitionValueAsLocalTime( + connectorQueryCtx_->sessionProperties()))) { + // We only count the number of base splits skipped as skippedSplits runtime + // statistics in Velox. Skipped delta split is only counted as skipped + // bytes. + runtimeStats.skippedSplitBytes += deleteSplit_->length; + deleteSplit_.reset(); + return; + } + + dwio::common::RowReaderOptions deleteRowReaderOpts; + configureRowReaderOptions( + {}, + scanSpec, + nullptr, + deleteFileSchema, + deleteSplit_, + nullptr, + nullptr, + deleteRowReaderOpts); + + deleteRowReader_.reset(); + deleteRowReader_ = deleteReader->createRowReader(deleteRowReaderOpts); +} + +void PositionalDeleteFileReader::readDeletePositions( + uint64_t baseReadOffset, + uint64_t size, + BufferPtr deleteBitmapBuffer) { + // We are going to read to the row number up to the end of the batch. For the + // same base file, the deleted rows are in ascending order in the same delete + // file. rowNumberUpperBound is the upperbound for the row number in this + // batch, excluding boundaries + int64_t rowNumberUpperBound = splitOffset_ + baseReadOffset + size; + + // Finish unused delete positions from last batch. + if (deletePositionsOutput_ && + deletePositionsOffset_ < deletePositionsOutput_->size()) { + updateDeleteBitmap( + std::dynamic_pointer_cast(deletePositionsOutput_) + ->childAt(0), + baseReadOffset, + rowNumberUpperBound, + deleteBitmapBuffer); + + if (readFinishedForBatch(rowNumberUpperBound)) { + return; + } + } + + if (!deleteRowReader_ || !deleteSplit_) { + return; + } + + // Read the new delete positions for this batch into deletePositionsOutput_ + // and update the delete bitmap + auto outputType = posColumn_->type; + RowTypePtr outputRowType = ROW({posColumn_->name}, {posColumn_->type}); + if (!deletePositionsOutput_) { + deletePositionsOutput_ = BaseVector::create(outputRowType, 0, pool_); + } + + do { + auto rowsScanned = deleteRowReader_->next(size, deletePositionsOutput_); + totalNumRowsScanned_ += rowsScanned; + + if (rowsScanned > 0) { + VELOX_CHECK( + !deletePositionsOutput_->mayHaveNulls(), + "Iceberg delete file pos column cannot have nulls"); + + auto numDeletedRows = deletePositionsOutput_->size(); + if (numDeletedRows > 0) { + deletePositionsOutput_->loadedVector(); + deletePositionsOffset_ = 0; + + // Convert the row numbers to set bits, up to rowNumberUpperBound. + // Beyond that the buffer of deleteBitMap is not available. + updateDeleteBitmap( + std::dynamic_pointer_cast(deletePositionsOutput_) + ->childAt(0), + baseReadOffset, + rowNumberUpperBound, + deleteBitmapBuffer); + } + } else { + // Reaching the end of the file + deleteSplit_.reset(); + break; + } + } while (!readFinishedForBatch(rowNumberUpperBound)); +} + +bool PositionalDeleteFileReader::noMoreData() { + return totalNumRowsScanned_ >= deleteFile_.recordCount && + deletePositionsOutput_ && + deletePositionsOffset_ >= deletePositionsOutput_->size(); +} + +void PositionalDeleteFileReader::updateDeleteBitmap( + VectorPtr deletePositionsVector, + uint64_t baseReadOffset, + int64_t rowNumberUpperBound, + BufferPtr deleteBitmapBuffer) { + auto deleteBitmap = deleteBitmapBuffer->asMutable(); + + // Convert the positions in file into positions relative to the start of the + // split. + const int64_t* deletePositions = + deletePositionsVector->as>()->rawValues(); + int64_t rowNumberLowerBound = baseReadOffset + splitOffset_; + + // If the rowNumberLowerBound is greater than the last position in this delete + // rows batch, nothing to delete. + if (rowNumberLowerBound > + deletePositions[deletePositionsVector->size() - 1]) { + return; + } + + // Skip the delete positions in deletePositionsVector until they are in the + // [rowNumberLowerBound, rowNumberUpperBound) range. + while (deletePositionsOffset_ < deletePositionsVector->size() && + deletePositions[deletePositionsOffset_] < rowNumberLowerBound) { + deletePositionsOffset_++; + } + while (deletePositionsOffset_ < deletePositionsVector->size() && + deletePositions[deletePositionsOffset_] < rowNumberUpperBound) { + bits::setBit( + deleteBitmap, + deletePositions[deletePositionsOffset_] - rowNumberLowerBound); + deletePositionsOffset_++; + } + + deleteBitmapBuffer->setSize(std::max( + static_cast(deleteBitmapBuffer->size()), + deletePositionsOffset_ == 0 || + (deletePositionsOffset_ < deletePositionsVector->size() && + deletePositions[deletePositionsOffset_] >= rowNumberUpperBound) + ? 0 + : bits::nbytes( + deletePositions[deletePositionsOffset_ - 1] + 1 - + rowNumberLowerBound))); +} + +bool PositionalDeleteFileReader::readFinishedForBatch( + int64_t rowNumberUpperBound) { + VELOX_CHECK_NOT_NULL(deletePositionsOutput_); + + auto deletePositionsVector = + std::dynamic_pointer_cast(deletePositionsOutput_)->childAt(0); + const int64_t* deletePositions = + deletePositionsVector->as>()->rawValues(); + + // We've read enough of the delete positions from this delete file when 1) it + // reaches the end of the file, or 2) the last read delete position is greater + // than the largest base file row number that is going to be read in this + // batch + if (totalNumRowsScanned_ >= deleteFile_.recordCount || + (deletePositionsVector->size() != 0 && + (deletePositionsOffset_ < deletePositionsVector->size() && + deletePositions[deletePositionsOffset_] >= rowNumberUpperBound))) { + return true; + } + return false; +} + +} // namespace facebook::velox::connector::lakehouse::iceberg diff --git a/velox/connectors/lakehouse/iceberg/PositionalDeleteFileReader.h b/velox/connectors/lakehouse/iceberg/PositionalDeleteFileReader.h new file mode 100644 index 000000000000..817b6c24c3a5 --- /dev/null +++ b/velox/connectors/lakehouse/iceberg/PositionalDeleteFileReader.h @@ -0,0 +1,94 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#pragma once + +#include +#include + +#include "velox/connectors/Connector.h" +#include "velox/connectors/lakehouse/common/FileHandle.h" +#include "velox/connectors/lakehouse/common/HiveConfig.h" +#include "velox/connectors/lakehouse/common/HiveConnectorSplit.h" +#include "velox/dwio/common/Reader.h" + +namespace facebook::velox::connector::lakehouse::iceberg { + +struct IcebergDeleteFile; +struct IcebergMetadataColumn; + +class PositionalDeleteFileReader { + public: + PositionalDeleteFileReader( + const IcebergDeleteFile& deleteFile, + const std::string& baseFilePath, + common::FileHandleFactory* fileHandleFactory, + const ConnectorQueryCtx* connectorQueryCtx, + folly::Executor* executor, + const std::shared_ptr& hiveConfig, + const std::shared_ptr& ioStats, + const std::shared_ptr& fsStats, + dwio::common::RuntimeStatistics& runtimeStats, + uint64_t splitOffset, + const std::string& connectorId); + + void readDeletePositions( + uint64_t baseReadOffset, + uint64_t size, + BufferPtr deleteBitmap); + + bool noMoreData(); + + private: + void updateDeleteBitmap( + VectorPtr deletePositionsVector, + uint64_t baseReadOffset, + int64_t rowNumberUpperBound, + BufferPtr deleteBitmapBuffer); + + bool readFinishedForBatch(int64_t rowNumberUpperBound); + + const IcebergDeleteFile& deleteFile_; + const std::string& baseFilePath_; + common::FileHandleFactory* const fileHandleFactory_; + folly::Executor* const executor_; + const ConnectorQueryCtx* connectorQueryCtx_; + const std::shared_ptr hiveConfig_; + const std::shared_ptr ioStats_; + const std::shared_ptr fsStats_; + const std::shared_ptr fsStats; + memory::MemoryPool* const pool_; + + std::shared_ptr filePathColumn_; + std::shared_ptr posColumn_; + uint64_t splitOffset_; + + std::shared_ptr deleteSplit_; + std::unique_ptr deleteRowReader_; + // The vector to hold the delete positions read from the positional delete + // file. These positions are relative to the start of the whole base data + // file. + VectorPtr deletePositionsOutput_; + // The index of deletePositionsOutput_ that indicates up to where the delete + // positions have been converted into the bitmap + uint64_t deletePositionsOffset_; + // Total number of rows read from this positional delete file reader, + // including the rows filtered out from filters on both filePathColumn_ and + // posColumn_. + uint64_t totalNumRowsScanned_; +}; + +} // namespace facebook::velox::connector::lakehouse::iceberg diff --git a/velox/connectors/lakehouse/iceberg/tests/CMakeLists.txt b/velox/connectors/lakehouse/iceberg/tests/CMakeLists.txt new file mode 100644 index 000000000000..3df26aaad7a1 --- /dev/null +++ b/velox/connectors/lakehouse/iceberg/tests/CMakeLists.txt @@ -0,0 +1,73 @@ +# Copyright (c) Facebook, Inc. and its affiliates. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +add_library(velox_lakehouse_iceberg_reader_benchmark_lib + IcebergSplitReaderBenchmark.cpp) +target_link_libraries( + velox_lakehouse_iceberg_reader_benchmark_lib + velox_lakehouse_common_test_lib + velox_exec_test_lib + velox_exec + velox_connector_lakehouse_common + Folly::folly + Folly::follybenchmark + ${TEST_LINK_LIBS}) + +if(VELOX_ENABLE_BENCHMARKS) + add_executable(velox_lakehouse_iceberg_reader_benchmark + IcebergSplitReaderBenchmarkMain.cpp) + target_link_libraries( + velox_lakehouse_iceberg_reader_benchmark + velox_lakehouse_iceberg_reader_benchmark_lib + velox_exec_test_lib + velox_exec + velox_connector_lakehouse_common + Folly::folly + Folly::follybenchmark + ${TEST_LINK_LIBS}) +endif() + +if(VELOX_BUILD_TESTING AND (NOT VELOX_DISABLE_GOOGLETEST)) + add_executable( + velox_lakehouse_iceberg_test + IcebergInsertTest.cpp + IcebergReadTest.cpp + IcebergSplitReaderBenchmarkTest.cpp + IcebergTestBase.cpp + Main.cpp) + add_test(velox_lakehouse_iceberg_test velox_lakehouse_iceberg_test) + + target_link_libraries( + velox_lakehouse_iceberg_test + velox_lakehouse_iceberg_reader_benchmark_lib + velox_lakehouse_common_test_lib + velox_connector_lakehouse_common + velox_lakehouse_iceberg_splitreader + velox_lakehouse_hive_partition_function + velox_dwio_common_exception + velox_dwio_common_test_utils + velox_vector_test_lib + velox_vector_fuzzer + velox_exec + velox_exec_test_lib + Folly::folly + Folly::follybenchmark + GTest::gtest + GTest::gtest_main) + + if(VELOX_ENABLE_PARQUET) + target_link_libraries(velox_lakehouse_iceberg_test velox_dwio_parquet_writer + velox_dwio_parquet_reader) + endif() + +endif() diff --git a/velox/connectors/lakehouse/iceberg/tests/IcebergInsertTest.cpp b/velox/connectors/lakehouse/iceberg/tests/IcebergInsertTest.cpp new file mode 100644 index 000000000000..c4adb78d8b70 --- /dev/null +++ b/velox/connectors/lakehouse/iceberg/tests/IcebergInsertTest.cpp @@ -0,0 +1,268 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include + +#include "velox/common/base/tests/GTestUtils.h" +#include "velox/connectors/lakehouse/common/tests/PlanBuilder.h" +#include "velox/connectors/lakehouse/iceberg/tests/IcebergTestBase.h" + +namespace facebook::velox::connector::lakehouse::iceberg::test { +class IcebergInsertTest : public IcebergTestBase { + protected: + void SetUp() override { + IcebergTestBase::SetUp(); + rowType_ = + ROW({"c1", "c2", "c3", "c4", "c5", "c6"}, + {BIGINT(), + INTEGER(), + SMALLINT(), + DECIMAL(18, 5), + BOOLEAN(), + VARCHAR()}); + } +}; + +TEST_F(IcebergInsertTest, testIcebergTableWrite) { + const auto outputDirectory = exec::test::TempDirectoryPath::create(); + const auto dataPath = fmt::format("{}", outputDirectory->getPath()); + constexpr int32_t numBatches = 10; + constexpr int32_t vectorSize = 5'000; + const auto vectors = createTestData(numBatches, vectorSize, 0.5); + auto dataSink = + createIcebergDataSink(rowType_, outputDirectory->getPath(), {}); + + for (const auto& vector : vectors) { + dataSink->appendData(vector); + } + + ASSERT_TRUE(dataSink->finish()); + const auto commitTasks = dataSink->close(); + createDuckDbTable(vectors); + auto splits = createSplitsForDirectory(dataPath); + ASSERT_EQ(splits.size(), commitTasks.size()); + auto plan = common::test::PlanBuilder().tableScan(rowType_).planNode(); + assertQuery(plan, splits, fmt::format("SELECT * FROM tmp")); +} + +TEST_F(IcebergInsertTest, testSingleColumnAsPartition) { + for (auto colIndex = 0; colIndex < rowType_->size() - 1; colIndex++) { + const auto& colName = rowType_->nameOf(colIndex); + const auto colType = rowType_->childAt(colIndex); + + const bool isDecimal = colType->isDecimal(); + const bool isVarbinary = colType->isVarbinary(); + constexpr int32_t numBatches = 2; + constexpr int32_t vectorSize = 50; + const auto outputDirectory = exec::test::TempDirectoryPath::create(); + + if (isDecimal || isVarbinary) { + const auto vectors = createTestData(numBatches, vectorSize, 0.5); + std::vector partitionTransforms = {colName}; + auto dataSink = createIcebergDataSink( + rowType_, outputDirectory->getPath(), partitionTransforms); + for (const auto& vector : vectors) { + if (isDecimal) { + VELOX_ASSERT_THROW( + dataSink->appendData(vector), + "Partition on decimal column is not supported yet."); + } else if (isVarbinary) { + VELOX_ASSERT_THROW( + dataSink->appendData(vector), + "Partition on varbinary column is not supported yet."); + } + } + continue; + } + const auto dataPath = fmt::format("{}", outputDirectory->getPath()); + const auto vectors = createTestData(numBatches, vectorSize, 0.5); + std::vector partitionTransforms = {colName}; + auto dataSink = createIcebergDataSink( + rowType_, outputDirectory->getPath(), partitionTransforms); + + for (const auto& vector : vectors) { + dataSink->appendData(vector); + } + + ASSERT_TRUE(dataSink->finish()); + const auto commitTasks = dataSink->close(); + createDuckDbTable(vectors); + auto splits = createSplitsForDirectory(dataPath); + + ASSERT_GT(commitTasks.size(), 0); + ASSERT_EQ(splits.size(), commitTasks.size()); + + for (const auto& task : commitTasks) { + auto taskJson = folly::parseJson(task); + ASSERT_TRUE(taskJson.count("partitionDataJson") > 0); + ASSERT_FALSE(taskJson["partitionDataJson"].empty()); + } + + connector::ColumnHandleMap assignments; + for (auto i = 0; i < rowType_->size(); i++) { + const auto& name = rowType_->nameOf(i); + if (i != colIndex) { + assignments.insert( + {name, + std::make_shared( + name, + common::HiveColumnHandle::ColumnType::kRegular, + rowType_->childAt(i), + rowType_->childAt(i))}); + } + } + + // Add partition column. + assignments.insert( + {colName, + std::make_shared( + colName, + common::HiveColumnHandle::ColumnType::kPartitionKey, + rowType_->childAt(colIndex), + rowType_->childAt(colIndex))}); + + auto plan = common::test::PlanBuilder(pool_.get()) + .tableScan(rowType_, {}, "", nullptr, assignments) + .planNode(); + + assertQuery(plan, splits, fmt::format("SELECT * FROM tmp")); + } +} + +TEST_F(IcebergInsertTest, testPartitionNullColumn) { + for (auto colIndex = 0; colIndex < rowType_->size() - 1; colIndex++) { + const auto& colName = rowType_->nameOf(colIndex); + const auto colType = rowType_->childAt(colIndex); + const auto outputDirectory = exec::test::TempDirectoryPath::create(); + const auto dataPath = fmt::format("{}", outputDirectory->getPath()); + constexpr int32_t numBatches = 2; + constexpr int32_t vectorSize = 100; + const bool isDecimal = colType->isDecimal(); + const bool isVarbinary = colType->isVarbinary(); + + if (isDecimal || isVarbinary) { + const auto vectors = createTestData(numBatches, vectorSize, 0.5); + std::vector partitionTransforms = {colName}; + auto dataSink = createIcebergDataSink( + rowType_, outputDirectory->getPath(), partitionTransforms); + for (const auto& vector : vectors) { + if (isDecimal) { + VELOX_ASSERT_THROW( + dataSink->appendData(vector), + "Partition on decimal column is not supported yet."); + } else if (isVarbinary) { + VELOX_ASSERT_THROW( + dataSink->appendData(vector), + "Partition on varbinary column is not supported yet."); + } + } + continue; + } + + const auto vectors = createTestData(numBatches, vectorSize, 1.0); + std::vector partitionTransforms = {colName}; + auto dataSink = createIcebergDataSink( + rowType_, outputDirectory->getPath(), partitionTransforms); + + for (const auto& vector : vectors) { + dataSink->appendData(vector); + } + + ASSERT_TRUE(dataSink->finish()); + const auto commitTasks = dataSink->close(); + + auto files = listFiles(dataPath); + ASSERT_EQ(files.size(), 1); + + for (const auto& file : files) { + std::vector pathComponents; + folly::split("/", file, pathComponents); + bool foundPartitionDir = false; + for (const auto& component : pathComponents) { + if (component.find('=') != std::string::npos) { + foundPartitionDir = true; + std::vector parts; + folly::split('=', component, parts); + ASSERT_EQ(parts.size(), 2); + ASSERT_EQ(parts[0], colName); + ASSERT_EQ(parts[1], "null"); + } + } + ASSERT_TRUE(foundPartitionDir) + << "No partition directory found in path: " << file; + } + } +} + +TEST_F(IcebergInsertTest, testColumnCombinationsAsPartition) { + std::vector> columnCombinations = { + {0, 1}, // BIGINT, INTEGER. + {2, 1}, // SMALLINT, INTEGER. + {2, 0}, // SMALLINT, BIGINT. + {0, 2, 1} // BIGINT, SMALLINT, INTEGER. + }; + + for (const auto& combination : columnCombinations) { + const auto outputDirectory = exec::test::TempDirectoryPath::create(); + const auto dataPath = fmt::format("{}", outputDirectory->getPath()); + constexpr int32_t numBatches = 2; + constexpr int32_t vectorSize = 50; + const auto vectors = createTestData(numBatches, vectorSize); + std::vector partitionTransforms; + for (auto colIndex : combination) { + partitionTransforms.push_back(rowType_->nameOf(colIndex)); + } + + auto dataSink = createIcebergDataSink( + rowType_, outputDirectory->getPath(), partitionTransforms); + + for (const auto& vector : vectors) { + dataSink->appendData(vector); + } + + ASSERT_TRUE(dataSink->finish()); + const auto commitTasks = dataSink->close(); + createDuckDbTable(vectors); + auto splits = createSplitsForDirectory(dataPath); + + ASSERT_GT(commitTasks.size(), 0); + ASSERT_EQ(splits.size(), commitTasks.size()); + + connector::ColumnHandleMap assignments; + std::unordered_set partitionColumns( + combination.begin(), combination.end()); + + for (auto i = 0; i < rowType_->size(); i++) { + const auto& name = rowType_->nameOf(i); + auto columnType = partitionColumns.count(i) > 0 + ? common::HiveColumnHandle::ColumnType::kPartitionKey + : common::HiveColumnHandle::ColumnType::kRegular; + + assignments.insert( + {name, + std::make_shared( + name, columnType, rowType_->childAt(i), rowType_->childAt(i))}); + } + + auto plan = common::test::PlanBuilder(pool_.get()) + .tableScan(rowType_, {}, "", nullptr, assignments) + .planNode(); + + assertQuery(plan, splits, fmt::format("SELECT * FROM tmp")); + } +} + +} // namespace facebook::velox::connector::lakehouse::iceberg::test diff --git a/velox/connectors/lakehouse/iceberg/tests/IcebergReadTest.cpp b/velox/connectors/lakehouse/iceberg/tests/IcebergReadTest.cpp new file mode 100644 index 000000000000..efe0cd5d80ba --- /dev/null +++ b/velox/connectors/lakehouse/iceberg/tests/IcebergReadTest.cpp @@ -0,0 +1,1277 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "velox/common/base/tests/GTestUtils.h" +#include "velox/common/file/FileSystems.h" +#include "velox/connectors/lakehouse/common/HiveConnectorSplit.h" +#include "velox/connectors/lakehouse/common/tests/HiveConnectorTestBase.h" +#include "velox/connectors/lakehouse/common/tests/PlanBuilder.h" +#include "velox/connectors/lakehouse/iceberg/IcebergDeleteFile.h" +#include "velox/connectors/lakehouse/iceberg/IcebergMetadataColumns.h" +#include "velox/connectors/lakehouse/iceberg/IcebergSplit.h" +#include "velox/exec/PlanNodeStats.h" + +#include + +using namespace facebook::velox::connector::lakehouse::common::test; +using namespace facebook::velox::exec; +using namespace facebook::velox::dwio::common; +using namespace facebook::velox::test; + +namespace facebook::velox::connector::lakehouse::iceberg::test { + +class HiveIcebergTest : public common::test::HiveConnectorTestBase { + public: + HiveIcebergTest() + : config_{std::make_shared()} { + // Make the writers flush per batch so that we can create non-aligned + // RowGroups between the base data files and delete files + flushPolicyFactory_ = []() { + return std::make_unique([]() { return true; }); + }; + } + + /// Create 1 base data file data_file_1 with 2 RowGroups of 10000 rows each. + /// Also create 1 delete file delete_file_1 which contains delete positions + /// for data_file_1. + void assertSingleBaseFileSingleDeleteFile( + const std::vector& deletePositionsVec) { + std::map> rowGroupSizesForFiles = { + {"data_file_1", {10000, 10000}}}; + std::unordered_map< + std::string, + std::multimap>> + deleteFilesForBaseDatafiles = { + {"delete_file_1", {{"data_file_1", deletePositionsVec}}}}; + + assertPositionalDeletes( + rowGroupSizesForFiles, deleteFilesForBaseDatafiles, 0); + } + + /// Create 3 base data files, where the first file data_file_0 has 500 rows, + /// the second file data_file_1 contains 2 RowGroups of 10000 rows each, and + /// the third file data_file_2 contains 500 rows. It creates 1 positional + /// delete file delete_file_1, which contains delete positions for + /// data_file_1. + void assertMultipleBaseFileSingleDeleteFile( + const std::vector& deletePositionsVec) { + int64_t previousFileRowCount = 500; + int64_t afterFileRowCount = 500; + + assertPositionalDeletes( + { + {"data_file_0", {previousFileRowCount}}, + {"data_file_1", {10000, 10000}}, + {"data_file_2", {afterFileRowCount}}, + }, + {{"delete_file_1", {{"data_file_1", deletePositionsVec}}}}, + 0); + } + + /// Create 1 base data file data_file_1 with 2 RowGroups of 10000 rows each. + /// Create multiple delete files with name data_file_1, data_file_2, and so on + void assertSingleBaseFileMultipleDeleteFiles( + const std::vector>& deletePositionsVecs) { + std::map> rowGroupSizesForFiles = { + {"data_file_1", {10000, 10000}}}; + + std::unordered_map< + std::string, + std::multimap>> + deleteFilesForBaseDatafiles; + for (int i = 0; i < deletePositionsVecs.size(); i++) { + std::string deleteFileName = fmt::format("delete_file_{}", i); + deleteFilesForBaseDatafiles[deleteFileName] = { + {"data_file_1", deletePositionsVecs[i]}}; + } + assertPositionalDeletes( + rowGroupSizesForFiles, deleteFilesForBaseDatafiles, 0); + } + + void assertMultipleSplits( + const std::vector& deletePositions, + int32_t fileCount, + int32_t numPrefetchSplits, + int rowCountPerFile = rowCount, + int32_t splitCountPerFile = 1) { + std::map> rowGroupSizesForFiles; + for (int32_t i = 0; i < fileCount; i++) { + std::string dataFileName = fmt::format("data_file_{}", i); + rowGroupSizesForFiles[dataFileName] = {rowCountPerFile}; + } + + std::unordered_map< + std::string, + std::multimap>> + deleteFilesForBaseDatafiles; + for (int i = 0; i < fileCount; i++) { + std::string deleteFileName = fmt::format("delete_file_{}", i); + deleteFilesForBaseDatafiles[deleteFileName] = { + {fmt::format("data_file_{}", i), deletePositions}}; + } + + assertPositionalDeletes( + rowGroupSizesForFiles, + deleteFilesForBaseDatafiles, + numPrefetchSplits, + splitCountPerFile); + } + + std::vector makeRandomIncreasingValues(int64_t begin, int64_t end) { + VELOX_CHECK(begin < end); + + std::mt19937 gen{0}; + std::vector values; + values.reserve(end - begin); + for (int i = begin; i < end; i++) { + if (folly::Random::rand32(0, 10, gen) > 8) { + values.push_back(i); + } + } + return values; + } + + std::vector makeContinuousIncreasingValues( + int64_t begin, + int64_t end) { + std::vector values; + values.resize(end - begin); + std::iota(values.begin(), values.end(), begin); + return values; + } + + /// @rowGroupSizesForFiles The key is the file name, and the value is a vector + /// of RowGroup sizes + /// @deleteFilesForBaseDatafiles The key is the delete file name, and the + /// value contains the information about the content of this delete file. + /// e.g. { + /// "delete_file_1", + /// { + /// {"data_file_1", {1, 2, 3}}, + /// {"data_file_1", {4, 5, 6}}, + /// {"data_file_2", {0, 2, 4}} + /// } + /// } + /// represents one delete file called delete_file_1, which contains delete + /// positions for data_file_1 and data_file_2. THere are 3 RowGroups in this + /// delete file, the first two contain positions for data_file_1, and the last + /// contain positions for data_file_2 + void assertPositionalDeletes( + const std::map>& rowGroupSizesForFiles, + const std::unordered_map< + std::string, + std::multimap>>& + deleteFilesForBaseDatafiles, + int32_t numPrefetchSplits = 0, + int32_t splitCount = 1) { + // Keep the reference to the deleteFilePath, otherwise the corresponding + // file will be deleted. + std::map> + dataFilePaths = writeDataFiles(rowGroupSizesForFiles); + std::unordered_map< + std::string, + std::pair>> + deleteFilePaths = writePositionDeleteFiles( + deleteFilesForBaseDatafiles, dataFilePaths); + + std::vector> splits; + + for (const auto& dataFile : dataFilePaths) { + std::string baseFileName = dataFile.first; + std::string baseFilePath = dataFile.second->getPath(); + + std::vector deleteFiles; + + for (auto const& deleteFile : deleteFilesForBaseDatafiles) { + std::string deleteFileName = deleteFile.first; + std::multimap> deleteFileContent = + deleteFile.second; + + if (deleteFileContent.count(baseFileName) != 0) { + // If this delete file contains rows for the target base file, then + // add it to the split + auto deleteFilePath = + deleteFilePaths[deleteFileName].second->getPath(); + IcebergDeleteFile icebergDeleteFile( + FileContent::kPositionalDeletes, + deleteFilePath, + fileFormat_, + deleteFilePaths[deleteFileName].first, + testing::internal::GetFileSize( + std::fopen(deleteFilePath.c_str(), "r"))); + deleteFiles.push_back(icebergDeleteFile); + } + } + + auto icebergSplits = + makeIcebergSplits(baseFilePath, deleteFiles, {}, splitCount); + splits.insert(splits.end(), icebergSplits.begin(), icebergSplits.end()); + } + + std::string duckdbSql = + getDuckDBQuery(rowGroupSizesForFiles, deleteFilesForBaseDatafiles); + auto plan = tableScanNode(rowType_); + auto task = HiveConnectorTestBase::assertQuery( + plan, splits, duckdbSql, numPrefetchSplits); + + auto planStats = toPlanStats(task->taskStats()); + auto scanNodeId = plan->id(); + auto it = planStats.find(scanNodeId); + ASSERT_TRUE(it != planStats.end()); + ASSERT_TRUE(it->second.peakMemoryBytes > 0); + } + + void assertEqualityDeletes( + const std::unordered_map>>& + equalityDeleteVectorMap, + const std::unordered_map>& + equalityFieldIdsMap, + std::string duckDbSql = "", + std::vector dataVectors = {}) { + VELOX_CHECK_EQ(equalityDeleteVectorMap.size(), equalityFieldIdsMap.size()); + // We will create data vectors with numColumns number of columns that is the + // max field Id in equalityFieldIds + int32_t numDataColumns = 0; + + for (auto it = equalityFieldIdsMap.begin(); it != equalityFieldIdsMap.end(); + ++it) { + auto equalityFieldIds = it->second; + auto currentMax = + *std::max_element(equalityFieldIds.begin(), equalityFieldIds.end()); + numDataColumns = std::max(numDataColumns, currentMax); + } + + VELOX_CHECK_GT(numDataColumns, 0); + VELOX_CHECK_GE(numDataColumns, equalityDeleteVectorMap.size()); + VELOX_CHECK_GT(equalityDeleteVectorMap.size(), 0); + + VELOX_CHECK_LE(equalityFieldIdsMap.size(), numDataColumns); + + std::shared_ptr dataFilePath = + writeDataFiles(rowCount, numDataColumns, 1, dataVectors)[0]; + + std::vector deleteFiles; + std::string predicates = ""; + unsigned long numDeletedValues = 0; + + std::vector> deleteFilePaths; + for (auto it = equalityFieldIdsMap.begin(); + it != equalityFieldIdsMap.end();) { + auto equalityFieldIds = it->second; + auto equalityDeleteVector = equalityDeleteVectorMap.at(it->first); + VELOX_CHECK_GT(equalityDeleteVector.size(), 0); + numDeletedValues = + std::max(numDeletedValues, equalityDeleteVector[0].size()); + deleteFilePaths.push_back(writeEqualityDeleteFile(equalityDeleteVector)); + IcebergDeleteFile deleteFile( + FileContent::kEqualityDeletes, + deleteFilePaths.back()->getPath(), + fileFormat_, + equalityDeleteVector[0].size(), + testing::internal::GetFileSize( + std::fopen(deleteFilePaths.back()->getPath().c_str(), "r")), + equalityFieldIds); + deleteFiles.push_back(deleteFile); + predicates += makePredicates(equalityDeleteVector, equalityFieldIds); + ++it; + if (it != equalityFieldIdsMap.end()) { + predicates += " AND "; + } + } + + // The default split count is 1. + auto icebergSplits = + makeIcebergSplits(dataFilePath->getPath(), deleteFiles); + + // If the caller passed in a query, use that. + if (duckDbSql == "") { + // Select all columns + duckDbSql = "SELECT * FROM tmp "; + if (numDeletedValues > 0) { + duckDbSql += fmt::format("WHERE {}", predicates); + } + } + + assertEqualityDeletes( + icebergSplits.back(), + !dataVectors.empty() ? asRowType(dataVectors[0]->type()) : rowType_, + duckDbSql); + + // Select a column that's not in the filter columns + if (numDataColumns > 1 && + equalityDeleteVectorMap.at(0).size() < numDataColumns) { + std::string duckDbQuery = "SELECT c0 FROM tmp"; + if (numDeletedValues > 0) { + duckDbQuery += fmt::format(" WHERE {}", predicates); + } + + std::vector names({"c0"}); + std::vector types(1, BIGINT()); + assertEqualityDeletes( + icebergSplits.back(), + std::make_shared(std::move(names), std::move(types)), + duckDbQuery); + } + } + + std::vector makeSequenceValues(int32_t numRows, int8_t repeat = 1) { + VELOX_CHECK_GT(repeat, 0); + + auto maxValue = std::ceil((double)numRows / repeat); + std::vector values; + values.reserve(numRows); + for (int32_t i = 0; i < maxValue; i++) { + for (int8_t j = 0; j < repeat; j++) { + values.push_back(i); + } + } + values.resize(numRows); + return values; + } + + std::vector makeRandomDeleteValues(int32_t maxRowNumber) { + std::mt19937 gen{0}; + std::vector deleteRows; + for (int i = 0; i < maxRowNumber; i++) { + if (folly::Random::rand32(0, 10, gen) > 8) { + deleteRows.push_back(i); + } + } + return deleteRows; + } + + const static int rowCount = 20000; + + protected: + std::shared_ptr config_; + std::function()> flushPolicyFactory_; + + std::vector> makeIcebergSplits( + const std::string& dataFilePath, + const std::vector& deleteFiles = {}, + const std::unordered_map>& + partitionKeys = {}, + const uint32_t splitCount = 1) { + std::unordered_map customSplitInfo; + customSplitInfo["table_format"] = "hive-iceberg"; + + auto file = filesystems::getFileSystem(dataFilePath, nullptr) + ->openFileForRead(dataFilePath); + const int64_t fileSize = file->size(); + std::vector> splits; + const uint64_t splitSize = std::floor((fileSize) / splitCount); + + for (int i = 0; i < splitCount; ++i) { + splits.emplace_back( + std::make_shared( + common::test::kHiveConnectorId, + dataFilePath, + fileFormat_, + i * splitSize, + splitSize, + partitionKeys, + std::nullopt, + customSplitInfo, + nullptr, + /*cacheable=*/true, + deleteFiles)); + } + + return splits; + } + + void assertEqualityDeletes( + std::shared_ptr split, + RowTypePtr outputRowType, + const std::string& duckDbSql) { + auto plan = tableScanNode(outputRowType); + auto task = OperatorTestBase::assertQuery(plan, {split}, duckDbSql); + + auto planStats = toPlanStats(task->taskStats()); + auto scanNodeId = plan->id(); + auto it = planStats.find(scanNodeId); + ASSERT_TRUE(it != planStats.end()); + ASSERT_TRUE(it->second.peakMemoryBytes > 0); + } + + private: + std::map> + writeDataFiles( + std::map> rowGroupSizesForFiles) { + std::map> + dataFilePaths; + + std::vector dataVectorsJoined; + dataVectorsJoined.reserve(rowGroupSizesForFiles.size()); + + int64_t startingValue = 0; + for (auto& dataFile : rowGroupSizesForFiles) { + dataFilePaths[dataFile.first] = exec::test::TempFilePath::create(); + + // We make the values are continuously increasing even across base data + // files. This is to make constructing DuckDB queries easier + std::vector dataVectors = + makeVectors(dataFile.second, startingValue); + writeToFile( + dataFilePaths[dataFile.first]->getPath(), + dataVectors, + config_, + flushPolicyFactory_); + + for (int i = 0; i < dataVectors.size(); i++) { + dataVectorsJoined.push_back(dataVectors[i]); + } + } + + createDuckDbTable(dataVectorsJoined); + return dataFilePaths; + } + + /// Input is like <"deleteFile1", <"dataFile1", {pos_RG1, pos_RG2,..}>, + /// <"dataFile2", {pos_RG1, pos_RG2,..}> + std::unordered_map< + std::string, + std::pair>> + writePositionDeleteFiles( + const std::unordered_map< + std::string, // delete file name + std::multimap< + std::string, + std::vector>>& + deleteFilesForBaseDatafiles, // + std::map> + baseFilePaths) { + std::unordered_map< + std::string, + std::pair>> + deleteFilePaths; + deleteFilePaths.reserve(deleteFilesForBaseDatafiles.size()); + + for (auto& deleteFile : deleteFilesForBaseDatafiles) { + auto deleteFileName = deleteFile.first; + auto deleteFileContent = deleteFile.second; + auto deleteFilePath = exec::test::TempFilePath::create(); + + std::vector deleteFileVectors; + int64_t totalPositionsInDeleteFile = 0; + + for (auto& deleteFileRowGroup : deleteFileContent) { + auto baseFileName = deleteFileRowGroup.first; + auto baseFilePath = baseFilePaths[baseFileName]->getPath(); + auto positionsInRowGroup = deleteFileRowGroup.second; + + auto filePathVector = makeFlatVector( + static_cast(positionsInRowGroup.size()), + [&](vector_size_t row) { return baseFilePath; }); + auto deletePosVector = makeFlatVector(positionsInRowGroup); + + RowVectorPtr deleteFileVector = makeRowVector( + {pathColumn_->name, posColumn_->name}, + {filePathVector, deletePosVector}); + + deleteFileVectors.push_back(deleteFileVector); + totalPositionsInDeleteFile += positionsInRowGroup.size(); + } + + writeToFile( + deleteFilePath->getPath(), + deleteFileVectors, + config_, + flushPolicyFactory_); + + deleteFilePaths[deleteFileName] = + std::make_pair(totalPositionsInDeleteFile, deleteFilePath); + } + + return deleteFilePaths; + } + + std::vector makeVectors( + std::vector vectorSizes, + int64_t& startingValue) { + std::vector vectors; + vectors.reserve(vectorSizes.size()); + + vectors.reserve(vectorSizes.size()); + for (int j = 0; j < vectorSizes.size(); j++) { + auto data = makeContinuousIncreasingValues( + startingValue, startingValue + vectorSizes[j]); + VectorPtr c0 = makeFlatVector(data); + vectors.push_back(makeRowVector({"c0"}, {c0})); + startingValue += vectorSizes[j]; + } + + return vectors; + } + + std::string getDuckDBQuery( + const std::map>& rowGroupSizesForFiles, + const std::unordered_map< + std::string, + std::multimap>>& + deleteFilesForBaseDatafiles) { + int64_t totalNumRowsInAllBaseFiles = 0; + std::map baseFileSizes; + for (auto rowGroupSizesInFile : rowGroupSizesForFiles) { + // Sum up the row counts in all RowGroups in each base file + baseFileSizes[rowGroupSizesInFile.first] += std::accumulate( + rowGroupSizesInFile.second.begin(), + rowGroupSizesInFile.second.end(), + 0LL); + totalNumRowsInAllBaseFiles += baseFileSizes[rowGroupSizesInFile.first]; + } + + // Group the delete vectors by baseFileName + std::map>> + deletePosVectorsForAllBaseFiles; + for (auto deleteFile : deleteFilesForBaseDatafiles) { + auto deleteFileContent = deleteFile.second; + for (auto rowGroup : deleteFileContent) { + auto baseFileName = rowGroup.first; + deletePosVectorsForAllBaseFiles[baseFileName].push_back( + rowGroup.second); + } + } + + // Flatten and deduplicate the delete position vectors in + // deletePosVectorsForAllBaseFiles from previous step, and count the total + // number of distinct delete positions for all base files + std::map> + flattenedDeletePosVectorsForAllBaseFiles; + int64_t totalNumDeletePositions = 0; + for (auto deleteVectorsForBaseFile : deletePosVectorsForAllBaseFiles) { + auto baseFileName = deleteVectorsForBaseFile.first; + auto deletePositionVectors = deleteVectorsForBaseFile.second; + std::vector deletePositionVector = + flattenAndDedup(deletePositionVectors, baseFileSizes[baseFileName]); + flattenedDeletePosVectorsForAllBaseFiles[baseFileName] = + deletePositionVector; + totalNumDeletePositions += deletePositionVector.size(); + } + + // Now build the DuckDB queries + if (totalNumDeletePositions == 0) { + return "SELECT * FROM tmp"; + } else if (totalNumDeletePositions >= totalNumRowsInAllBaseFiles) { + return "SELECT * FROM tmp WHERE 1 = 0"; + } else { + // Convert the delete positions in all base files into column values + std::vector allDeleteValues; + + int64_t numRowsInPreviousBaseFiles = 0; + for (auto baseFileSize : baseFileSizes) { + auto deletePositions = + flattenedDeletePosVectorsForAllBaseFiles[baseFileSize.first]; + + if (numRowsInPreviousBaseFiles > 0) { + for (int64_t& deleteValue : deletePositions) { + deleteValue += numRowsInPreviousBaseFiles; + } + } + + allDeleteValues.insert( + allDeleteValues.end(), + deletePositions.begin(), + deletePositions.end()); + + numRowsInPreviousBaseFiles += baseFileSize.second; + } + + return fmt::format( + "SELECT * FROM tmp WHERE c0 NOT IN ({})", + makeNotInList(allDeleteValues)); + } + } + + std::vector flattenAndDedup( + const std::vector>& deletePositionVectors, + int64_t baseFileSize) { + std::vector deletePositionVector; + for (auto vec : deletePositionVectors) { + for (auto pos : vec) { + if (pos >= 0 && pos < baseFileSize) { + deletePositionVector.push_back(pos); + } + } + } + + std::sort(deletePositionVector.begin(), deletePositionVector.end()); + auto last = + std::unique(deletePositionVector.begin(), deletePositionVector.end()); + deletePositionVector.erase(last, deletePositionVector.end()); + + return deletePositionVector; + } + + std::string makeNotInList(const std::vector& deletePositionVector) { + if (deletePositionVector.empty()) { + return ""; + } + + return std::accumulate( + deletePositionVector.begin() + 1, + deletePositionVector.end(), + std::to_string(deletePositionVector[0]), + [](const std::string& a, int64_t b) { + return a + ", " + std::to_string(b); + }); + } + + core::PlanNodePtr tableScanNode(RowTypePtr outputRowType) { + return PlanBuilder(pool_.get()).tableScan(outputRowType).planNode(); + } + + std::string makePredicates( + const std::vector>& equalityDeleteVector, + const std::vector& equalityFieldIds) { + std::string predicates(""); + int32_t numDataColumns = + *std::max_element(equalityFieldIds.begin(), equalityFieldIds.end()); + + VELOX_CHECK_GT(numDataColumns, 0); + VELOX_CHECK_GE(numDataColumns, equalityDeleteVector.size()); + VELOX_CHECK_GT(equalityDeleteVector.size(), 0); + + auto numDeletedValues = equalityDeleteVector[0].size(); + + if (numDeletedValues == 0) { + return predicates; + } + + // If all values for a column are deleted, just return an always-false + // predicate + for (auto i = 0; i < equalityDeleteVector.size(); i++) { + auto equalityFieldId = equalityFieldIds[i]; + auto deleteValues = equalityDeleteVector[i]; + + auto lastIter = std::unique(deleteValues.begin(), deleteValues.end()); + auto numDistinctValues = lastIter - deleteValues.begin(); + auto minValue = 1; + auto maxValue = *std::max_element(deleteValues.begin(), lastIter); + if (maxValue - minValue + 1 == numDistinctValues && + maxValue == (rowCount - 1) / equalityFieldId) { + return "1 = 0"; + } + } + + if (equalityDeleteVector.size() == 1) { + std::string name = fmt::format("c{}", equalityFieldIds[0] - 1); + predicates = fmt::format( + "{} NOT IN ({})", name, makeNotInList({equalityDeleteVector[0]})); + } else { + for (int i = 0; i < numDeletedValues; i++) { + std::string oneRow(""); + for (int j = 0; j < equalityFieldIds.size(); j++) { + std::string name = fmt::format("c{}", equalityFieldIds[j] - 1); + std::string predicate = + fmt::format("({} <> {})", name, equalityDeleteVector[j][i]); + + oneRow = oneRow == "" ? predicate + : fmt::format("({} OR {})", oneRow, predicate); + } + + predicates = predicates == "" + ? oneRow + : fmt::format("{} AND {}", predicates, oneRow); + } + } + return predicates; + } + + std::shared_ptr pathColumn_ = + IcebergMetadataColumn::icebergDeleteFilePathColumn(); + std::shared_ptr posColumn_ = + IcebergMetadataColumn::icebergDeletePosColumn(); + + protected: + RowTypePtr rowType_{ROW({"c0"}, {BIGINT()})}; + dwio::common::FileFormat fileFormat_{dwio::common::FileFormat::DWRF}; + + std::shared_ptr writeEqualityDeleteFile( + const std::vector>& equalityDeleteVector) { + std::vector names; + std::vector vectors; + for (int i = 0; i < equalityDeleteVector.size(); i++) { + names.push_back(fmt::format("c{}", i)); + vectors.push_back(makeFlatVector(equalityDeleteVector[i])); + } + + RowVectorPtr deleteFileVectors = makeRowVector(names, vectors); + + auto deleteFilePath = exec::test::TempFilePath::create(); + writeToFile(deleteFilePath->getPath(), deleteFileVectors); + + return deleteFilePath; + } + + std::vector> writeDataFiles( + uint64_t numRows, + int32_t numColumns = 1, + int32_t splitCount = 1, + std::vector dataVectors = {}) { + if (dataVectors.empty()) { + dataVectors = makeVectors(splitCount, numRows, numColumns); + } + VELOX_CHECK_EQ(dataVectors.size(), splitCount); + + std::vector> dataFilePaths; + dataFilePaths.reserve(splitCount); + for (auto i = 0; i < splitCount; i++) { + dataFilePaths.emplace_back(exec::test::TempFilePath::create()); + writeToFile(dataFilePaths.back()->getPath(), dataVectors[i]); + } + + createDuckDbTable(dataVectors); + return dataFilePaths; + } + + std::vector + makeVectors(int32_t count, int32_t rowsPerVector, int32_t numColumns = 1) { + std::vector types(numColumns, BIGINT()); + std::vector names; + for (int j = 0; j < numColumns; j++) { + names.push_back(fmt::format("c{}", j)); + } + + std::vector rowVectors; + for (int i = 0; i < count; i++) { + std::vector vectors; + + // Create the column values like below: + // c0 c1 c2 + // 0 0 0 + // 1 0 0 + // 2 1 0 + // 3 1 1 + // 4 2 1 + // 5 2 1 + // 6 3 2 + // ... + // In the first column c0, the values are continuously increasing and not + // repeating. In the second column c1, the values are continuously + // increasing and each value repeats once. And so on. + for (int j = 0; j < numColumns; j++) { + auto data = makeSequenceValues(rowsPerVector, j + 1); + vectors.push_back(vectorMaker_.flatVector(data)); + } + + rowVectors.push_back(makeRowVector(names, vectors)); + } + + rowType_ = std::make_shared(std::move(names), std::move(types)); + + return rowVectors; + } +}; + +/// This test creates one single data file and one delete file. The parameter +/// passed to assertSingleBaseFileSingleDeleteFile is the delete positions. +TEST_F(HiveIcebergTest, singleBaseFileSinglePositionalDeleteFile) { + folly::SingletonVault::singleton()->registrationComplete(); + + assertSingleBaseFileSingleDeleteFile({{0, 1, 2, 3}}); + // Delete the first and last row in each batch (10000 rows per batch) + assertSingleBaseFileSingleDeleteFile({{0, 9999, 10000, 19999}}); + // Delete several rows in the second batch (10000 rows per batch) + assertSingleBaseFileSingleDeleteFile({{10000, 10002, 19999}}); + // Delete random rows + assertSingleBaseFileSingleDeleteFile({makeRandomIncreasingValues(0, 20000)}); + // Delete 0 rows + assertSingleBaseFileSingleDeleteFile({}); + // Delete all rows + assertSingleBaseFileSingleDeleteFile( + {makeContinuousIncreasingValues(0, 20000)}); + // Delete rows that don't exist + assertSingleBaseFileSingleDeleteFile({{20000, 29999}}); +} + +/// This test creates 3 base data files, only the middle one has corresponding +/// delete positions. The parameter passed to +/// assertSingleBaseFileSingleDeleteFile is the delete positions.for the middle +/// base file. +TEST_F(HiveIcebergTest, MultipleBaseFilesSinglePositionalDeleteFile) { + folly::SingletonVault::singleton()->registrationComplete(); + + assertMultipleBaseFileSingleDeleteFile({0, 1, 2, 3}); + assertMultipleBaseFileSingleDeleteFile({0, 9999, 10000, 19999}); + assertMultipleBaseFileSingleDeleteFile({10000, 10002, 19999}); + assertMultipleBaseFileSingleDeleteFile({10000, 10002, 19999}); + assertMultipleBaseFileSingleDeleteFile( + makeRandomIncreasingValues(0, rowCount)); + assertMultipleBaseFileSingleDeleteFile({}); + assertMultipleBaseFileSingleDeleteFile( + makeContinuousIncreasingValues(0, rowCount)); +} + +/// This test creates one base data file/split with multiple delete files. The +/// parameter passed to assertSingleBaseFileMultipleDeleteFiles is the vector of +/// delete files. Each leaf vector represents the delete positions in that +/// delete file. +TEST_F(HiveIcebergTest, singleBaseFileMultiplePositionalDeleteFiles) { + folly::SingletonVault::singleton()->registrationComplete(); + + // Delete row 0, 1, 2, 3 from the first batch out of two. + assertSingleBaseFileMultipleDeleteFiles({{1}, {2}, {3}, {4}}); + // Delete the first and last row in each batch (10000 rows per batch). + assertSingleBaseFileMultipleDeleteFiles({{0}, {9999}, {10000}, {19999}}); + + assertSingleBaseFileMultipleDeleteFiles({{500, 21000}}); + + assertSingleBaseFileMultipleDeleteFiles( + {makeRandomIncreasingValues(0, 10000), + makeRandomIncreasingValues(10000, 20000), + makeRandomIncreasingValues(5000, 15000)}); + + assertSingleBaseFileMultipleDeleteFiles( + {makeContinuousIncreasingValues(0, 10000), + makeContinuousIncreasingValues(10000, 20000)}); + + assertSingleBaseFileMultipleDeleteFiles( + {makeContinuousIncreasingValues(0, 10000), + makeContinuousIncreasingValues(10000, 20000), + makeRandomIncreasingValues(5000, 15000)}); + + assertSingleBaseFileMultipleDeleteFiles( + {makeContinuousIncreasingValues(0, 20000), + makeContinuousIncreasingValues(0, 20000)}); + + assertSingleBaseFileMultipleDeleteFiles( + {makeRandomIncreasingValues(0, 20000), + {}, + makeRandomIncreasingValues(5000, 15000)}); + + assertSingleBaseFileMultipleDeleteFiles({{}, {}}); +} + +/// This test creates 2 base data files, and 1 or 2 delete files, with unaligned +/// RowGroup boundaries +TEST_F(HiveIcebergTest, multipleBaseFileMultiplePositionalDeleteFiles) { + folly::SingletonVault::singleton()->registrationComplete(); + + std::map> rowGroupSizesForFiles; + std::unordered_map< + std::string, + std::multimap>> + deleteFilesForBaseDatafiles; + + // Create two data files, each with two RowGroups + rowGroupSizesForFiles["data_file_1"] = {100, 85}; + rowGroupSizesForFiles["data_file_2"] = {99, 1}; + + // Delete 3 rows from the first RowGroup in data_file_1 + deleteFilesForBaseDatafiles["delete_file_1"] = {{"data_file_1", {0, 1, 99}}}; + assertPositionalDeletes(rowGroupSizesForFiles, deleteFilesForBaseDatafiles); + + // Delete 3 rows from the second RowGroup in data_file_1 + deleteFilesForBaseDatafiles["delete_file_1"] = { + {"data_file_1", {100, 101, 184}}}; + assertPositionalDeletes(rowGroupSizesForFiles, deleteFilesForBaseDatafiles); + + // Delete random rows from the both RowGroups in data_file_1 + deleteFilesForBaseDatafiles["delete_file_1"] = { + {"data_file_1", makeRandomIncreasingValues(0, 185)}}; + assertPositionalDeletes(rowGroupSizesForFiles, deleteFilesForBaseDatafiles); + + // Delete all rows in data_file_1 + deleteFilesForBaseDatafiles["delete_file_1"] = { + {"data_file_1", makeContinuousIncreasingValues(0, 185)}}; + assertPositionalDeletes(rowGroupSizesForFiles, deleteFilesForBaseDatafiles); + // + // Delete non-existent rows from data_file_1 + deleteFilesForBaseDatafiles["delete_file_1"] = { + {"data_file_1", makeRandomIncreasingValues(186, 300)}}; + assertPositionalDeletes(rowGroupSizesForFiles, deleteFilesForBaseDatafiles); + + // Delete several rows from both RowGroups in both data files + deleteFilesForBaseDatafiles.clear(); + deleteFilesForBaseDatafiles["delete_file_1"] = { + {"data_file_1", {0, 100, 102, 184}}, {"data_file_2", {1, 98, 99}}}; + assertPositionalDeletes(rowGroupSizesForFiles, deleteFilesForBaseDatafiles); + + // The delete file delete_file_1 contains 3 RowGroups itself, with the first 3 + // deleting some repeating rows in data_file_1, and the last 2 RowGroups + // deleting some repeating rows in data_file_2 + deleteFilesForBaseDatafiles.clear(); + deleteFilesForBaseDatafiles["delete_file_1"] = { + {"data_file_1", {0, 1, 2, 3}}, + {"data_file_1", {1, 2, 3, 4}}, + {"data_file_1", makeRandomIncreasingValues(0, 185)}, + {"data_file_2", {1, 3, 5, 7}}, + {"data_file_2", makeRandomIncreasingValues(0, 100)}}; + assertPositionalDeletes(rowGroupSizesForFiles, deleteFilesForBaseDatafiles); + + // delete_file_2 contains non-overlapping delete rows for each data files in + // each RowGroup + deleteFilesForBaseDatafiles.clear(); + deleteFilesForBaseDatafiles["delete_file_1"] = { + {"data_file_1", {0, 1, 2, 3}}, {"data_file_2", {1, 3, 5, 7}}}; + deleteFilesForBaseDatafiles["delete_file_2"] = { + {"data_file_1", {1, 2, 3, 4}}, + {"data_file_1", {98, 99, 100, 101, 184}}, + {"data_file_2", {3, 5, 7, 9}}, + {"data_file_2", {98, 99, 100}}}; + assertPositionalDeletes(rowGroupSizesForFiles, deleteFilesForBaseDatafiles); + + // Two delete files each containing overlapping delete rows for both data + // files + deleteFilesForBaseDatafiles.clear(); + deleteFilesForBaseDatafiles["delete_file_1"] = { + {"data_file_1", makeRandomIncreasingValues(0, 185)}, + {"data_file_2", makeRandomIncreasingValues(0, 100)}}; + deleteFilesForBaseDatafiles["delete_file_2"] = { + {"data_file_1", makeRandomIncreasingValues(10, 120)}, + {"data_file_2", makeRandomIncreasingValues(50, 100)}}; + assertPositionalDeletes(rowGroupSizesForFiles, deleteFilesForBaseDatafiles); +} + +TEST_F(HiveIcebergTest, positionalDeletesMultipleSplits) { + folly::SingletonVault::singleton()->registrationComplete(); + + assertMultipleSplits({1, 2, 3, 4}, 10, 5); + assertMultipleSplits({1, 2, 3, 4}, 10, 0); + assertMultipleSplits({1, 2, 3, 4}, 10, 10); + assertMultipleSplits({0, 9999, 10000, 19999}, 10, 3); + assertMultipleSplits(makeRandomIncreasingValues(0, 20000), 10, 3); + assertMultipleSplits(makeContinuousIncreasingValues(0, 20000), 10, 3); + assertMultipleSplits({}, 10, 3); + + assertMultipleSplits({1, 2, 3, 4}, 10, 5, 30000, 3); + assertPositionalDeletes( + { + {"data_file_0", {500}}, + {"data_file_1", {10000, 10000}}, + {"data_file_2", {500}}, + }, + {{"delete_file_1", + {{"data_file_1", makeRandomIncreasingValues(0, 20000)}}}}, + 0, + 3); + + // Include only upper bound(which is exclusive) in delete positions for the + // second 10k batch of rows. + assertMultipleSplits({1000, 9000, 20000}, 1, 0, 20000, 3); +} + +TEST_F(HiveIcebergTest, testPartitionedRead) { + RowTypePtr rowType{ROW({"c0", "ds"}, {BIGINT(), DateType::get()})}; + std::unordered_map> partitionKeys; + // Iceberg API sets partition values for dates to daysSinceEpoch, so + // in velox, we do not need to convert it to days. + // Test query on two partitions ds=17627(2018-04-06), ds=17628(2018-04-07) + std::vector> splits; + std::vector> dataFilePaths; + for (int i = 0; i <= 1; ++i) { + std::vector dataVectors; + int32_t daysSinceEpoch = 17627 + i; + VectorPtr c0 = makeFlatVector((std::vector){i}); + VectorPtr ds = + makeFlatVector((std::vector){daysSinceEpoch}); + dataVectors.push_back(makeRowVector({"c0", "ds"}, {c0, ds})); + + auto dataFilePath = exec::test::TempFilePath::create(); + dataFilePaths.push_back(dataFilePath); + writeToFile( + dataFilePath->getPath(), dataVectors, config_, flushPolicyFactory_); + partitionKeys["ds"] = std::to_string(daysSinceEpoch); + auto icebergSplits = + makeIcebergSplits(dataFilePath->getPath(), {}, partitionKeys); + splits.insert(splits.end(), icebergSplits.begin(), icebergSplits.end()); + } + + connector::ColumnHandleMap assignments; + assignments.insert( + {"c0", + std::make_shared( + "c0", + common::HiveColumnHandle::ColumnType::kRegular, + rowType->childAt(0), + rowType->childAt(0))}); + + std::vector requiredSubFields; + common::HiveColumnHandle::ColumnParseParameters columnParseParameters; + columnParseParameters.partitionDateValueFormat = + common::HiveColumnHandle::ColumnParseParameters::kDaysSinceEpoch; + assignments.insert( + {"ds", + std::make_shared( + "ds", + common::HiveColumnHandle::ColumnType::kPartitionKey, + rowType->childAt(1), + rowType->childAt(1), + std::move(requiredSubFields), + columnParseParameters)}); + + auto plan = PlanBuilder(pool_.get()) + .tableScan(rowType, {}, "", nullptr, assignments) + .planNode(); + + HiveConnectorTestBase::assertQuery( + plan, + splits, + "SELECT * FROM (VALUES (0, '2018-04-06'), (1, '2018-04-07'))", + 0); + + // Test filter on non-partitioned non-date column + std::vector nonPartitionFilters = {"c0 = 1"}; + plan = PlanBuilder(pool_.get()) + .tableScan(rowType, nonPartitionFilters, "", nullptr, assignments) + .planNode(); + + HiveConnectorTestBase::assertQuery(plan, splits, "SELECT 1, '2018-04-07'"); + + // Test filter on non-partitioned date column + std::vector filters = {"ds = date'2018-04-06'"}; + plan = PlanBuilder(pool_.get()).tableScan(rowType, filters).planNode(); + + splits.clear(); + for (auto& dataFilePath : dataFilePaths) { + auto icebergSplits = makeIcebergSplits(dataFilePath->getPath()); + splits.insert(splits.end(), icebergSplits.begin(), icebergSplits.end()); + } + + HiveConnectorTestBase::assertQuery(plan, splits, "SELECT 0, '2018-04-06'"); +} + +// Delete values from a single column file +TEST_F(HiveIcebergTest, equalityDeletesSingleFileColumn1) { + folly::SingletonVault::singleton()->registrationComplete(); + + std::unordered_map> equalityFieldIdsMap; + std::unordered_map>> + equalityDeleteVectorMap; + equalityFieldIdsMap.insert({0, {1}}); + + // Delete row 0, 1, 2, 3 from the first batch out of two. + equalityDeleteVectorMap.insert({0, {{0, 1, 2, 3}}}); + assertEqualityDeletes(equalityDeleteVectorMap, equalityFieldIdsMap); + + // Delete the first and last row in each batch (10000 rows per batch) + equalityDeleteVectorMap.clear(); + equalityDeleteVectorMap.insert({0, {{0, 9999, 10000, 19999}}}); + assertEqualityDeletes(equalityDeleteVectorMap, equalityFieldIdsMap); + + // Delete several rows in the second batch (10000 rows per batch) + equalityDeleteVectorMap.clear(); + equalityDeleteVectorMap.insert({0, {{10000, 10002, 19999}}}); + assertEqualityDeletes(equalityDeleteVectorMap, equalityFieldIdsMap); + + // Delete random rows + equalityDeleteVectorMap.clear(); + equalityDeleteVectorMap.insert({0, {makeRandomDeleteValues(rowCount)}}); + assertEqualityDeletes(equalityDeleteVectorMap, equalityFieldIdsMap); + + // Delete 0 rows + equalityDeleteVectorMap.clear(); + equalityDeleteVectorMap.insert({0, {{}}}); + assertEqualityDeletes(equalityDeleteVectorMap, equalityFieldIdsMap); + + // Delete all rows + equalityDeleteVectorMap.insert({0, {makeSequenceValues(rowCount)}}); + assertEqualityDeletes(equalityDeleteVectorMap, equalityFieldIdsMap); + + // Delete rows that don't exist + equalityDeleteVectorMap.clear(); + equalityDeleteVectorMap.insert({0, {{20000, 29999}}}); + assertEqualityDeletes(equalityDeleteVectorMap, equalityFieldIdsMap); +} + +// Delete values from the second column in a 2-column file +// +// c1 c2 +// 0 0 +// 1 0 +// 2 1 +// 3 1 +// 4 2 +// ... ... +// 19999 9999 +TEST_F(HiveIcebergTest, equalityDeletesSingleFileColumn2) { + folly::SingletonVault::singleton()->registrationComplete(); + + std::unordered_map> equalityFieldIdsMap; + std::unordered_map>> + equalityDeleteVectorMap; + equalityFieldIdsMap.insert({0, {2}}); + + // Delete values 0, 1, 2, 3 from the second column + equalityDeleteVectorMap.insert({0, {{0, 1, 2, 3}}}); + assertEqualityDeletes(equalityDeleteVectorMap, equalityFieldIdsMap); + + // Delete the smallest value 0 and the largest value 9999 from the second + // column, which has the range [0, 9999] + equalityDeleteVectorMap.clear(); + equalityDeleteVectorMap.insert({0, {{0, 9999}}}); + assertEqualityDeletes(equalityDeleteVectorMap, equalityFieldIdsMap); + + // Delete non-existent values from the second column + equalityDeleteVectorMap.clear(); + equalityDeleteVectorMap.insert({0, {{10000, 10002, 19999}}}); + assertEqualityDeletes(equalityDeleteVectorMap, equalityFieldIdsMap); + + // Delete random rows from the second column + equalityDeleteVectorMap.clear(); + equalityDeleteVectorMap.insert({0, {makeSequenceValues(rowCount)}}); + assertEqualityDeletes(equalityDeleteVectorMap, equalityFieldIdsMap); + + // Delete 0 values + equalityDeleteVectorMap.clear(); + equalityDeleteVectorMap.insert({0, {{}}}); + assertEqualityDeletes(equalityDeleteVectorMap, equalityFieldIdsMap); + + // Delete all values + equalityDeleteVectorMap.clear(); + equalityDeleteVectorMap.insert({0, {makeSequenceValues(rowCount / 2)}}); + assertEqualityDeletes(equalityDeleteVectorMap, equalityFieldIdsMap); +} + +// Delete values from 2 columns with the following data: +// +// c1 c2 +// 0 0 +// 1 0 +// 2 1 +// 3 1 +// 4 2 +// ... ... +// 19999 9999 +TEST_F(HiveIcebergTest, equalityDeletesSingleFileMultipleColumns) { + folly::SingletonVault::singleton()->registrationComplete(); + + std::unordered_map> equalityFieldIdsMap; + std::unordered_map>> + equalityDeleteVectorMap; + equalityFieldIdsMap.insert({0, {1, 2}}); + + // Delete rows 0, 1 + equalityDeleteVectorMap.insert({0, {{0, 1}, {0, 0}}}); + assertEqualityDeletes(equalityDeleteVectorMap, equalityFieldIdsMap); + + // Delete rows 0, 2, 4, 6 + equalityDeleteVectorMap.clear(); + equalityDeleteVectorMap.insert({0, {{0, 2, 4, 6}, {0, 1, 2, 3}}}); + assertEqualityDeletes(equalityDeleteVectorMap, equalityFieldIdsMap); + + // Delete the last row + equalityDeleteVectorMap.clear(); + equalityDeleteVectorMap.insert({0, {{19999}, {9999}}}); + assertEqualityDeletes(equalityDeleteVectorMap, equalityFieldIdsMap); + + // Delete non-existent values + equalityDeleteVectorMap.clear(); + equalityDeleteVectorMap.insert({0, {{20000, 30000}, {10000, 1500}}}); + assertEqualityDeletes(equalityDeleteVectorMap, equalityFieldIdsMap); + + // Delete 0 values + equalityDeleteVectorMap.clear(); + equalityDeleteVectorMap.insert({0, {{}, {}}}); + assertEqualityDeletes(equalityDeleteVectorMap, equalityFieldIdsMap); + + // Delete all values + equalityDeleteVectorMap.clear(); + equalityDeleteVectorMap.insert( + {0, {makeSequenceValues(rowCount), makeSequenceValues(rowCount, 2)}}); + assertEqualityDeletes( + equalityDeleteVectorMap, + equalityFieldIdsMap, + "SELECT * FROM tmp WHERE 1 = 0"); +} + +TEST_F(HiveIcebergTest, equalityDeletesMultipleFiles) { + folly::SingletonVault::singleton()->registrationComplete(); + + std::unordered_map> equalityFieldIdsMap; + std::unordered_map>> + equalityDeleteVectorMap; + equalityFieldIdsMap.insert({{0, {1}}, {1, {2}}}); + + // Delete rows {0, 1} from c0, {2, 3} from c1, with two equality delete files + equalityDeleteVectorMap.insert({{0, {{0, 1}}}, {1, {{2, 3}}}}); + assertEqualityDeletes(equalityDeleteVectorMap, equalityFieldIdsMap); + + // Delete using 3 equality delete files + equalityFieldIdsMap.insert({{2, {3}}}); + equalityDeleteVectorMap.insert({{0, {{0, 1}}}, {1, {{2, 3}}}, {2, {{4, 5}}}}); + assertEqualityDeletes(equalityDeleteVectorMap, equalityFieldIdsMap); + + // Delete 0 values + equalityDeleteVectorMap.clear(); + equalityDeleteVectorMap.insert({{0, {{}}}, {1, {{}}}, {2, {{}}}}); + assertEqualityDeletes(equalityDeleteVectorMap, equalityFieldIdsMap); + + // Delete all values + equalityDeleteVectorMap.clear(); + equalityDeleteVectorMap.insert( + {{0, {makeSequenceValues(rowCount)}}, + {1, {makeSequenceValues(rowCount)}}, + {2, {makeSequenceValues(rowCount)}}}); + assertEqualityDeletes( + equalityDeleteVectorMap, + equalityFieldIdsMap, + "SELECT * FROM tmp WHERE 1 = 0"); +} + +TEST_F(HiveIcebergTest, TestSubFieldEqualityDelete) { + folly::SingletonVault::singleton()->registrationComplete(); + + // Write the base file + std::shared_ptr dataFilePath = + exec::test::TempFilePath::create(); + std::vector dataVectors = {makeRowVector( + {"c_bigint", "c_row"}, + {makeFlatVector(20, [](auto row) { return row + 1; }), + makeRowVector( + {"c0", "c1", "c2"}, + {makeFlatVector(20, [](auto row) { return row + 1; }), + makeFlatVector(20, [](auto row) { return row + 1; }), + makeFlatVector(20, [](auto row) { return row + 1; })})})}; + int32_t numDataColumns = 1; + dataFilePath = writeDataFiles(rowCount, numDataColumns, 1, dataVectors)[0]; + + // Write the delete file. Equality delete field is c_row.c1 + std::vector deleteFiles; + // Delete rows {0, 1} from c_row.c1, whose schema Id is 4 + std::vector deleteDataVectors = {makeRowVector( + {"c1"}, {makeFlatVector(2, [](auto row) { return row + 1; })})}; + + std::vector> deleteFilePaths; + auto equalityFieldIds = std::vector({4}); + auto deleteFilePath = exec::test::TempFilePath::create(); + writeToFile(deleteFilePath->getPath(), deleteDataVectors.back()); + deleteFilePaths.push_back(deleteFilePath); + IcebergDeleteFile deleteFile( + FileContent::kEqualityDeletes, + deleteFilePaths.back()->getPath(), + fileFormat_, + 2, + testing::internal::GetFileSize( + std::fopen(deleteFilePaths.back()->getPath().c_str(), "r")), + equalityFieldIds); + deleteFiles.push_back(deleteFile); + + auto icebergSplits = makeIcebergSplits(dataFilePath->getPath(), deleteFiles); + + // Select both c_bigint and c_row column columns + std::string duckDbSql = "SELECT * FROM tmp WHERE c_row.c0 not in (1, 2)"; + assertEqualityDeletes( + icebergSplits.back(), asRowType(dataVectors[0]->type()), duckDbSql); + + // SELECT only c_bigint column + duckDbSql = "SELECT c_bigint FROM tmp WHERE c_row.c0 not in (1, 2)"; + assertEqualityDeletes( + icebergSplits.back(), ROW({"c_bigint"}, {BIGINT()}), duckDbSql); +} +} // namespace facebook::velox::connector::lakehouse::iceberg::test diff --git a/velox/connectors/lakehouse/iceberg/tests/IcebergSplitReaderBenchmark.cpp b/velox/connectors/lakehouse/iceberg/tests/IcebergSplitReaderBenchmark.cpp new file mode 100644 index 000000000000..831ec1b3faf8 --- /dev/null +++ b/velox/connectors/lakehouse/iceberg/tests/IcebergSplitReaderBenchmark.cpp @@ -0,0 +1,407 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "velox/connectors/lakehouse/iceberg/tests/IcebergSplitReaderBenchmark.h" + +#include + +#include + +#include "velox/vector/tests/utils/VectorMaker.h" + +using namespace facebook::velox; +using namespace facebook::velox::dwio; +using namespace facebook::velox::dwio::common; +using namespace facebook::velox::dwrf; +using namespace facebook::velox::connector::lakehouse::common; +using namespace facebook::velox::connector::lakehouse::iceberg; +using namespace facebook::velox::memory; + +namespace facebook::velox::connector::lakehouse::iceberg::test { +void IcebergSplitReaderBenchmark::writeToFile( + const std::vector& batches) { + auto path = fileFolder_->getPath() + "/" + fileName_; + auto localWriteFile = std::make_unique(path, true, false); + auto sink = std::make_unique(std::move(localWriteFile), path); + dwrf::WriterOptions options; + options.memoryPool = rootPool_.get(); + options.schema = batches[0]->type(); + dwrf::Writer dataFilewriter{std::move(sink), options}; + for (auto& batch : batches) { + dataFilewriter.write(batch); + } + dataFilewriter.flush(); + dataFilewriter.close(); +} + +void IcebergSplitReaderBenchmark::writeToPositionDeleteFile( + const std::string& filePath, + const std::vector& vectors) { + auto localPosWriteFile = + std::make_unique(filePath, true, false); + auto posDeletesink = + std::make_unique(std::move(localPosWriteFile), filePath); + dwrf::WriterOptions options; + options.memoryPool = rootPool_.get(); + options.schema = vectors[0]->type(); + dwrf::Writer posDeletewriter{std::move(posDeletesink), options}; + for (size_t i = 0; i < vectors.size(); ++i) { + posDeletewriter.write(vectors[i]); + } + posDeletewriter.close(); +} + +std::vector IcebergSplitReaderBenchmark::makeRandomDeleteRows( + int32_t deleteRowsCount) { + std::random_device rd; + std::mt19937 gen(rd()); + int64_t totalDataRows = kNumBatches * kNumRowsPerBatch; + std::uniform_int_distribution<> dis(0, totalDataRows - 1); + std::set uniqueDeleteRows; + while (uniqueDeleteRows.size() < deleteRowsCount) { + uniqueDeleteRows.insert(dis(gen)); + } + std::vector deleteRows( + uniqueDeleteRows.begin(), uniqueDeleteRows.end()); + return deleteRows; +} + +std::vector IcebergSplitReaderBenchmark::makeSequenceRows( + int32_t maxRowNumber) { + std::vector deleteRows; + deleteRows.resize(maxRowNumber); + std::iota(deleteRows.begin(), deleteRows.end(), 0); + return deleteRows; +} + +std::vector IcebergSplitReaderBenchmark::listFiles( + const std::string& dirPath) { + std::vector files; + for (auto& dirEntry : + std::filesystem::recursive_directory_iterator(dirPath)) { + if (dirEntry.is_regular_file()) { + files.push_back(dirEntry.path().string()); + } + } + return files; +} + +std::shared_ptr +IcebergSplitReaderBenchmark::makeIcebergSplit( + const std::string& dataFilePath, + const std::vector& deleteFiles) { + std::unordered_map> partitionKeys; + std::unordered_map customSplitInfo; + customSplitInfo["table_format"] = "hive-iceberg"; + + auto readFile = std::make_shared(dataFilePath); + const int64_t fileSize = readFile->size(); + + return std::make_shared( + kHiveConnectorId, + dataFilePath, + fileFomat_, + 0, + fileSize, + partitionKeys, + std::nullopt, + customSplitInfo, + nullptr, + /*cacheable=*/true, + deleteFiles); +} + +std::string IcebergSplitReaderBenchmark::writePositionDeleteFile( + const std::string& dataFilePath, + int64_t numDeleteRows) { + facebook::velox::test::VectorMaker vectorMaker{leafPool_.get()}; + auto filePathVector = + vectorMaker.flatVector(numDeleteRows, [&](auto row) { + if (row < numDeleteRows) { + return StringView(dataFilePath); + } else { + return StringView(); + } + }); + + std::vector deleteRowsVec; + deleteRowsVec.reserve(numDeleteRows); + auto deleteRows = makeRandomDeleteRows(numDeleteRows); + deleteRowsVec.insert( + deleteRowsVec.end(), deleteRows.begin(), deleteRows.end()); + + auto deletePositionsVector = vectorMaker.flatVector(deleteRowsVec); + + std::shared_ptr pathColumn = + IcebergMetadataColumn::icebergDeleteFilePathColumn(); + std::shared_ptr posColumn = + IcebergMetadataColumn::icebergDeletePosColumn(); + RowVectorPtr deleteFileVectors = vectorMaker.rowVector( + {pathColumn->name, posColumn->name}, + {filePathVector, deletePositionsVector}); + + auto deleteFilePath = deleteFileFolder_->getPath() + "/" + "posDelete.data"; + writeToPositionDeleteFile(deleteFilePath, std::vector{deleteFileVectors}); + + return deleteFilePath; +} + +std::vector> +IcebergSplitReaderBenchmark::createIcebergSplitsWithPositionalDelete( + int32_t deleteRowsPercentage, + int32_t deleteFilesCount) { + std::vector> splits; + + std::vector deleteFilePaths; + std::vector dataFilePaths = listFiles(fileFolder_->getPath()); + + for (const auto& dataFilePath : dataFilePaths) { + std::vector deleteFiles; + int64_t deleteRowsCount = + kNumBatches * kNumRowsPerBatch * deleteRowsPercentage * 0.01; + deleteFiles.reserve(deleteRowsCount); + for (int i = 0; i < deleteFilesCount; i++) { + std::string deleteFilePath = + writePositionDeleteFile(dataFilePath, deleteRowsCount); + + IcebergDeleteFile deleteFile( + FileContent::kPositionalDeletes, + deleteFilePath, + fileFomat_, + deleteRowsCount, + testing::internal::GetFileSize( + std::fopen(deleteFilePath.c_str(), "r"))); + deleteFilePaths.emplace_back(deleteFilePath); + deleteFiles.emplace_back(deleteFile); + } + splits.emplace_back(makeIcebergSplit(dataFilePath, deleteFiles)); + } + return splits; +} + +FilterSpec IcebergSplitReaderBenchmark::createFilterSpec( + const std::string& columnName, + float startPct, + float selectPct, + const TypePtr& type, + bool isForRowGroupSkip, + bool allowNulls) { + switch (type->childAt(0)->kind()) { + case TypeKind::BIGINT: + case TypeKind::INTEGER: + return FilterSpec( + columnName, + startPct, + selectPct, + FilterKind::kBigintRange, + isForRowGroupSkip, + allowNulls); + default: + VELOX_FAIL("Unsupported Data Type {}", type->childAt(0)->toString()); + } + return FilterSpec(columnName, startPct, selectPct, FilterKind(), false); +} + +std::shared_ptr IcebergSplitReaderBenchmark::createScanSpec( + const std::vector& batches, + RowTypePtr& rowType, + const std::vector& filterSpecs, + std::vector& hitRows, + SubfieldFilters& filters) { + std::unique_ptr filterGenerator = + std::make_unique(rowType, 0); + filters = filterGenerator->makeSubfieldFilters( + filterSpecs, batches, nullptr, hitRows); + auto scanSpec = filterGenerator->makeScanSpec(std::move(filters)); + return scanSpec; +} + +// This method is the place where we do the read operations using +// icebergSplitReader. scanSpec contains the setting of filters. e.g. +// filterRateX100 = 30 means it would filter out 70% of rows and 30% remain. +// deleteRateX100 = 30 means it would delete 30% of overall data rows and 70% +// remain. Return the number of rows after the filter and delete. +int IcebergSplitReaderBenchmark::read( + const RowTypePtr& rowType, + uint32_t nextSize, + std::unique_ptr icebergSplitReader) { + runtimeStats_ = RuntimeStatistics(); + icebergSplitReader->resetFilterCaches(); + int resultSize = 0; + auto result = BaseVector::create(rowType, 0, leafPool_.get()); + while (true) { + bool hasData = icebergSplitReader->next(nextSize, result); + if (!hasData) { + break; + } + auto rowsRemaining = result->size(); + resultSize += rowsRemaining; + } + icebergSplitReader->updateRuntimeStats(runtimeStats_); + return resultSize; +} + +void IcebergSplitReaderBenchmark::readSingleColumn( + const std::string& columnName, + const TypePtr& type, + float startPct, + float selectPct, + float deletePct, + uint32_t nextSize) { + folly::BenchmarkSuspender suspender; + auto rowType = ROW({columnName}, {type}); + + auto batches = + dataSetBuilder_->makeDataset(rowType, kNumBatches, kNumRowsPerBatch) + .withRowGroupSpecificData(kNumRowsPerRowGroup) + .withNullsForField(Subfield(columnName), 0) + .build(); + writeToFile(*batches); + std::vector filterSpecs; + + filterSpecs.emplace_back( + createFilterSpec(columnName, startPct, selectPct, rowType, false, false)); + + std::vector hitRows; + SubfieldFilters filters; + auto scanSpec = + createScanSpec(*batches, rowType, filterSpecs, hitRows, filters); + + std::vector> splits = + createIcebergSplitsWithPositionalDelete(deletePct, 1); + + core::TypedExprPtr remainingFilterExpr; + + std::shared_ptr hiveTableHandle = + std::make_shared( + "kHiveConnectorId", + "tableName", + false, + std::move(filters), + remainingFilterExpr, + rowType); + + std::shared_ptr hiveConfig = + std::make_shared(std::make_shared( + std::unordered_map(), true)); + const RowTypePtr readerOutputType; + const std::shared_ptr ioStats = + std::make_shared(); + const std::shared_ptr fsStats = + std::make_shared(); + + std::shared_ptr root = + memory::memoryManager()->addRootPool( + "IcebergSplitReader", kMaxMemory, MemoryReclaimer::create()); + std::shared_ptr opPool = root->addLeafChild("operator"); + std::shared_ptr connectorPool = + root->addAggregateChild(kHiveConnectorId, MemoryReclaimer::create()); + std::shared_ptr connectorSessionProperties_ = + std::make_shared( + std::unordered_map()); + + std::unique_ptr connectorQueryCtx_ = + std::make_unique( + opPool.get(), + connectorPool.get(), + connectorSessionProperties_.get(), + nullptr, + velox::common::PrefixSortConfig(), + nullptr, + nullptr, + "query.IcebergSplitReader", + "task.IcebergSplitReader", + "planNodeId.IcebergSplitReader", + 0, + ""); + + common::FileHandleFactory fileHandleFactory( + std::make_unique< + SimpleLRUCache>( + hiveConfig->numCacheFileHandles()), + std::make_unique( + connectorSessionProperties_)); + + suspender.dismiss(); + + auto ioExecutor = std::make_unique(3); + std::shared_ptr remainingFilterExprSet{nullptr}; + std::atomic totalRemainingFilterMs; + + uint64_t resultSize = 0; + for (std::shared_ptr split : splits) { + scanSpec->resetCachedValues(true); + + std::unique_ptr icebergSplitReader = + std::make_unique( + split, + hiveTableHandle, + nullptr, + connectorQueryCtx_.get(), + hiveConfig, + rowType, + ioStats, + fsStats, + &fileHandleFactory, + ioExecutor.get(), + scanSpec, + connectorQueryCtx_->expressionEvaluator(), + totalRemainingFilterMs); + + std::shared_ptr randomSkip; + icebergSplitReader->configureReaderOptions(randomSkip); + icebergSplitReader->prepareSplit(nullptr, runtimeStats_); + + // Filter range is generated from a small sample data of 4096 rows. So the + // upperBound and lowerBound are introduced to estimate the result size. + resultSize += read(rowType, nextSize, std::move(icebergSplitReader)); + } + // Calculate the expected number of rows after the filters. + // Add one to expected to avoid 0 in calculating upperBound and lowerBound. + int expected = kNumBatches * kNumRowsPerBatch * ((double)selectPct / 100) * + (1 - (double)deletePct / 100) + + 1; + + // Make the upperBound and lowerBound large enough to avoid very small + // resultSize and expected size, where the diff ratio is relatively very + // large. + int upperBound = expected * (1 + kFilterErrorMargin) + 1; + int lowerBound = expected * (1 - kFilterErrorMargin) - 1; + upperBound = std::max(16, upperBound); + lowerBound = std::max(0, lowerBound); + + VELOX_CHECK( + resultSize <= upperBound && resultSize >= lowerBound, + "Result Size {} and Expected Size {} Mismatch", + resultSize, + expected); +} + +void run( + uint32_t, + const std::string& columnName, + const TypePtr& type, + float filterRateX100, + float deleteRateX100, + uint32_t nextSize) { + RowTypePtr rowType = ROW({columnName}, {type}); + IcebergSplitReaderBenchmark benchmark; + BIGINT()->toString(); + benchmark.readSingleColumn( + columnName, type, 0, filterRateX100, deleteRateX100, nextSize); +} + +} // namespace facebook::velox::connector::lakehouse::iceberg::test diff --git a/velox/connectors/lakehouse/iceberg/tests/IcebergSplitReaderBenchmark.h b/velox/connectors/lakehouse/iceberg/tests/IcebergSplitReaderBenchmark.h new file mode 100644 index 000000000000..3cc85d511927 --- /dev/null +++ b/velox/connectors/lakehouse/iceberg/tests/IcebergSplitReaderBenchmark.h @@ -0,0 +1,136 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#pragma once + +#include "velox/common/file/FileSystems.h" +#include "velox/connectors/lakehouse/common/HiveConnectorSplit.h" +#include "velox/connectors/lakehouse/common/TableHandle.h" +#include "velox/connectors/lakehouse/iceberg/IcebergDeleteFile.h" +#include "velox/connectors/lakehouse/iceberg/IcebergMetadataColumns.h" +#include "velox/connectors/lakehouse/iceberg/IcebergSplit.h" +#include "velox/connectors/lakehouse/iceberg/IcebergSplitReader.h" +#include "velox/dwio/common/tests/utils/DataSetBuilder.h" +#include "velox/dwio/dwrf/RegisterDwrfReader.h" +#include "velox/dwio/dwrf/writer/Writer.h" +#include "velox/exec/tests/utils/TempDirectoryPath.h" +#include "velox/vector/tests/utils/VectorTestBase.h" + +#include +#include + +namespace facebook::velox::connector::lakehouse::iceberg::test { + +constexpr uint32_t kNumRowsPerBatch = 20000; +constexpr uint32_t kNumBatches = 50; +constexpr uint32_t kNumRowsPerRowGroup = 10000; +constexpr double kFilterErrorMargin = 0.2; + +class IcebergSplitReaderBenchmark { + public: + explicit IcebergSplitReaderBenchmark() { + rootPool_ = + memory::memoryManager()->addRootPool("IcebergSplitReaderBenchmark"); + leafPool_ = rootPool_->addLeafChild("IcebergSplitReaderBenchmark"); + dataSetBuilder_ = + std::make_unique(*leafPool_, 0); + filesystems::registerLocalFileSystem(); + dwrf::registerDwrfReaderFactory(); + } + + ~IcebergSplitReaderBenchmark() {} + + void writeToFile(const std::vector& batches); + + void writeToPositionDeleteFile( + const std::string& filePath, + const std::vector& vectors); + + dwio::common::FilterSpec createFilterSpec( + const std::string& columnName, + float startPct, + float selectPct, + const TypePtr& type, + bool isForRowGroupSkip, + bool allowNulls); + + std::shared_ptr createScanSpec( + const std::vector& batches, + RowTypePtr& rowType, + const std::vector& filterSpecs, + std::vector& hitRows, + velox::common::SubfieldFilters& filters); + + int read( + const RowTypePtr& rowType, + uint32_t nextSize, + std::unique_ptr + icebergSplitReader); + + void readSingleColumn( + const std::string& columnName, + const TypePtr& type, + float startPct, + float selectPct, + float deleteRate, + uint32_t nextSize); + + std::vector> + createIcebergSplitsWithPositionalDelete( + int32_t deleteRowsPercentage, + int32_t deleteFilesCount); + + std::vector listFiles(const std::string& dirPath); + + std::shared_ptr + makeIcebergSplit( + const std::string& dataFilePath, + const std::vector& + deleteFiles = {}); + + std::vector makeRandomDeleteRows(int32_t deleteRowsCount); + + std::vector makeSequenceRows(int32_t maxRowNumber); + + std::string writePositionDeleteFile( + const std::string& dataFilePath, + int64_t numDeleteRows); + + private: + const std::string fileName_ = "test.data"; + const std::shared_ptr fileFolder_ = + exec::test::TempDirectoryPath::create(); + const std::shared_ptr deleteFileFolder_ = + exec::test::TempDirectoryPath::create(); + + std::unique_ptr dataSetBuilder_; + std::shared_ptr rootPool_; + std::shared_ptr leafPool_; + std::unique_ptr writer_; + dwio::common::RuntimeStatistics runtimeStats_; + + dwio::common::FileFormat fileFomat_{dwio::common::FileFormat::DWRF}; + const std::string kHiveConnectorId = "hive-iceberg"; +}; + +void run( + uint32_t, + const std::string& columnName, + const TypePtr& type, + float filterRateX100, + float deleteRateX100, + uint32_t nextSize); + +} // namespace facebook::velox::connector::lakehouse::iceberg::test diff --git a/velox/connectors/lakehouse/iceberg/tests/IcebergSplitReaderBenchmarkMain.cpp b/velox/connectors/lakehouse/iceberg/tests/IcebergSplitReaderBenchmarkMain.cpp new file mode 100644 index 000000000000..a19093c7b81f --- /dev/null +++ b/velox/connectors/lakehouse/iceberg/tests/IcebergSplitReaderBenchmarkMain.cpp @@ -0,0 +1,67 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "velox/connectors/lakehouse/iceberg/tests/IcebergSplitReaderBenchmark.h" + +using namespace facebook::velox; +using namespace facebook::velox::dwio; +using namespace facebook::velox::dwio::common; +using namespace facebook::velox::dwrf; +using namespace facebook::velox::connector::lakehouse::iceberg::test; +using namespace facebook::velox::test; + +#define PARQUET_BENCHMARKS_FILTER_DELETES(_type_, _name_, _filter_, _deletes_) \ + BENCHMARK_NAMED_PARAM( \ + run, \ + _name_##_Filter_##_filter_##_Delete_##_deletes_##_next_5k, \ + #_name_, \ + _type_, \ + _filter_, \ + _deletes_, \ + 5000); \ + BENCHMARK_NAMED_PARAM( \ + run, \ + _name_##_Filter_##_filter_##_Delete_##_deletes_##_next_10k, \ + #_name_, \ + _type_, \ + _filter_, \ + _deletes_, \ + 10000); \ + BENCHMARK_DRAW_LINE(); + +#define PARQUET_BENCHMARKS_FILTERS(_type_, _name_, _filter_) \ + PARQUET_BENCHMARKS_FILTER_DELETES(_type_, _name_, _filter_, 0) \ + PARQUET_BENCHMARKS_FILTER_DELETES(_type_, _name_, _filter_, 20) \ + PARQUET_BENCHMARKS_FILTER_DELETES(_type_, _name_, _filter_, 50) \ + PARQUET_BENCHMARKS_FILTER_DELETES(_type_, _name_, _filter_, 70) \ + PARQUET_BENCHMARKS_FILTER_DELETES(_type_, _name_, _filter_, 100) + +#define PARQUET_BENCHMARKS(_type_, _name_) \ + PARQUET_BENCHMARKS_FILTERS(_type_, _name_, 0) \ + PARQUET_BENCHMARKS_FILTERS(_type_, _name_, 20) \ + PARQUET_BENCHMARKS_FILTERS(_type_, _name_, 50) \ + PARQUET_BENCHMARKS_FILTERS(_type_, _name_, 70) \ + PARQUET_BENCHMARKS_FILTERS(_type_, _name_, 100) \ + BENCHMARK_DRAW_LINE(); + +PARQUET_BENCHMARKS(BIGINT(), BigInt); + +int main(int argc, char** argv) { + folly::Init init{&argc, &argv}; + memory::MemoryManager::initialize(memory::MemoryManager::Options{}); + folly::runBenchmarks(); + return 0; +} diff --git a/velox/connectors/lakehouse/iceberg/tests/IcebergSplitReaderBenchmarkTest.cpp b/velox/connectors/lakehouse/iceberg/tests/IcebergSplitReaderBenchmarkTest.cpp new file mode 100644 index 000000000000..15dd0602cf16 --- /dev/null +++ b/velox/connectors/lakehouse/iceberg/tests/IcebergSplitReaderBenchmarkTest.cpp @@ -0,0 +1,30 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "velox/connectors/lakehouse/iceberg/tests/IcebergSplitReaderBenchmark.h" +#include + +namespace facebook::velox::connector::lakehouse::iceberg::test { +namespace { +TEST(IcebergSplitReaderBenchmarkTest, basic) { + memory::MemoryManager::testingSetInstance(memory::MemoryManager::Options{}); + run(1, "BigInt", BIGINT(), 20, 0, 500); + run(1, "BigInt", BIGINT(), 50, 20, 500); + run(1, "BigInt", BIGINT(), 100, 20, 500); + run(1, "BigInt", BIGINT(), 100, 100, 500); +} +} // namespace +} // namespace facebook::velox::connector::lakehouse::iceberg::test diff --git a/velox/connectors/lakehouse/iceberg/tests/IcebergTestBase.cpp b/velox/connectors/lakehouse/iceberg/tests/IcebergTestBase.cpp new file mode 100644 index 000000000000..a3a322075587 --- /dev/null +++ b/velox/connectors/lakehouse/iceberg/tests/IcebergTestBase.cpp @@ -0,0 +1,242 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "velox/connectors/lakehouse/iceberg/tests/IcebergTestBase.h" + +#include + +#include "velox/connectors/lakehouse/iceberg/PartitionSpec.h" + +namespace facebook::velox::connector::lakehouse::iceberg::test { +void IcebergTestBase::SetUp() { + HiveConnectorTestBase::SetUp(); + parquet::registerParquetReaderFactory(); + parquet::registerParquetWriterFactory(); + Type::registerSerDe(); + + // Initialize session properties and config. + connectorSessionProperties_ = std::make_shared( + std::unordered_map(), true); + + connectorConfig_ = + std::make_shared(std::make_shared( + std::unordered_map())); + + setupMemoryPools("IcebergTestBase"); + + // Initialize vector fuzzer for test data generation. + fuzzerOptions_.vectorSize = 100; + fuzzerOptions_.nullRatio = 0.1; + fuzzer_ = std::make_unique(fuzzerOptions_, opPool_.get()); + + vectorMaker_ = + std::make_unique(opPool_.get()); +} + +void IcebergTestBase::TearDown() { + vectorMaker_.reset(); + fuzzer_.reset(); + connectorQueryCtx_.reset(); + connectorPool_.reset(); + opPool_.reset(); + root_.reset(); + HiveConnectorTestBase::TearDown(); +} + +void IcebergTestBase::setupMemoryPools(const std::string& name) { + root_.reset(); + opPool_.reset(); + connectorPool_.reset(); + connectorQueryCtx_.reset(); + + root_ = memory::memoryManager()->addRootPool( + name, 1L << 30, exec::MemoryReclaimer::create()); + opPool_ = root_->addLeafChild("operator"); + connectorPool_ = + root_->addAggregateChild("connector", exec::MemoryReclaimer::create()); + + connectorQueryCtx_ = std::make_unique( + opPool_.get(), + connectorPool_.get(), + connectorSessionProperties_.get(), + nullptr, + velox::common::PrefixSortConfig(), + nullptr, + nullptr, + "query" + name, + "task" + name, + "planNodeId" + name, + 0, + ""); +} + +std::vector IcebergTestBase::createTestData( + int32_t numBatches, + vector_size_t rowsPerBatch, + double nullRatio) { + std::vector vectors; + vectors.reserve(numBatches); + + fuzzerOptions_.nullRatio = nullRatio; + fuzzerOptions_.allowDictionaryVector = false; + fuzzer_->setOptions(fuzzerOptions_); + + for (auto i = 0; i < numBatches; ++i) { + vectors.push_back(fuzzer_->fuzzRow(rowType_, rowsPerBatch, false)); + } + + return vectors; +} + +std::shared_ptr IcebergTestBase::createPartitionSpec( + const std::vector& transformSpecs) { + std::vector fields; + static const std::regex identityRegex(R"(([a-z_][a-z0-9_]*))"); + + for (const auto& spec : transformSpecs) { + TransformType transformType = TransformType::kIdentity; + std::string name; + std::smatch matches; + + if (std::regex_match(spec, matches, identityRegex)) { + transformType = TransformType::kIdentity; + name = matches[1]; + } else { + VELOX_FAIL("Unsupported transform specification: {}", spec); + } + + fields.push_back( + IcebergPartitionSpec::Field(name, transformType, std::nullopt)); + } + + return std::make_shared(1, fields); +} + +std::shared_ptr +IcebergTestBase::createIcebergInsertTableHandle( + const RowTypePtr& rowType, + const std::string& outputDirectoryPath, + const std::vector& partitionTransforms) { + std::vector> columnHandles; + for (auto i = 0; i < rowType->size(); ++i) { + auto columnName = rowType->nameOf(i); + auto columnType = common::HiveColumnHandle::ColumnType::kRegular; + for (auto transform : partitionTransforms) { + if (columnName == transform) { + columnType = common::HiveColumnHandle::ColumnType::kPartitionKey; + break; + } + } + columnHandles.push_back( + std::make_shared( + columnName, columnType, rowType->childAt(i), rowType->childAt(i))); + } + + auto locationHandle = std::make_shared( + outputDirectoryPath, + outputDirectoryPath, + common::LocationHandle::TableType::kNew); + + auto partitionSpec = createPartitionSpec(partitionTransforms); + + return std::make_shared( + columnHandles, + locationHandle, + partitionSpec, + fileFormat_, + nullptr, + velox::common::CompressionKind::CompressionKind_ZSTD); +} + +std::shared_ptr IcebergTestBase::createIcebergDataSink( + const RowTypePtr& rowType, + const std::string& outputDirectoryPath, + const std::vector& partitionTransforms) { + auto tableHandle = createIcebergInsertTableHandle( + rowType, outputDirectoryPath, partitionTransforms); + return std::make_shared( + rowType, + tableHandle, + connectorQueryCtx_.get(), + connector::CommitStrategy::kNoCommit, + connectorConfig_); +} + +std::vector IcebergTestBase::listFiles( + const std::string& dirPath) { + std::vector files; + if (!std::filesystem::exists(dirPath)) { + return files; + } + + for (auto& dirEntry : + std::filesystem::recursive_directory_iterator(dirPath)) { + if (dirEntry.is_regular_file()) { + files.push_back(dirEntry.path().string()); + } + } + return files; +} + +std::vector> +IcebergTestBase::createSplitsForDirectory(const std::string& directory) { + std::vector> splits; + std::unordered_map customSplitInfo; + customSplitInfo["table_format"] = "hive-iceberg"; + + auto files = listFiles(directory); + for (const auto& filePath : files) { + std::unordered_map> partitionKeys; + + // Extract partition keys from path if any. + std::vector pathComponents; + folly::split("/", filePath, pathComponents); + for (const auto& component : pathComponents) { + if (component.find('=') != std::string::npos) { + std::vector keys; + folly::split('=', component, keys); + if (keys.size() == 2) { + partitionKeys[keys[0]] = keys[1]; + if (keys[1] == "null") { + partitionKeys[keys[0]] = std::nullopt; + } + } + } + } + + const auto file = filesystems::getFileSystem(filePath, nullptr) + ->openFileForRead(filePath); + const auto fileSize = file->size(); + + splits.push_back( + std::make_shared( + kHiveConnectorId, + filePath, + fileFormat_, + 0, + fileSize, + partitionKeys, + std::nullopt, + customSplitInfo, + nullptr, + /*cacheable=*/true, + std::vector())); + } + + return splits; +} + +} // namespace facebook::velox::connector::lakehouse::iceberg::test diff --git a/velox/connectors/lakehouse/iceberg/tests/IcebergTestBase.h b/velox/connectors/lakehouse/iceberg/tests/IcebergTestBase.h new file mode 100644 index 000000000000..dde1c33757f1 --- /dev/null +++ b/velox/connectors/lakehouse/iceberg/tests/IcebergTestBase.h @@ -0,0 +1,88 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#pragma once + +#include + +#include "velox/connectors/lakehouse/common/tests/HiveConnectorTestBase.h" +#include "velox/connectors/lakehouse/iceberg/IcebergDataSink.h" +#include "velox/connectors/lakehouse/iceberg/IcebergSplit.h" +#include "velox/exec/tests/utils/TempDirectoryPath.h" +#include "velox/vector/fuzzer/VectorFuzzer.h" +#ifdef VELOX_ENABLE_PARQUET +#include "velox/dwio/parquet/RegisterParquetWriter.h" +#include "velox/dwio/parquet/reader/ParquetReader.h" +#endif + +namespace facebook::velox::connector::lakehouse::iceberg::test { + +class IcebergTestBase : public common::test::HiveConnectorTestBase { + protected: + void SetUp() override; + + void TearDown() override; + + std::vector createTestData( + int32_t numBatches, + vector_size_t rowsPerBatch, + double nullRatio = 0.0); + + std::shared_ptr createIcebergDataSink( + const RowTypePtr& rowType, + const std::string& outputDirectoryPath, + const std::vector& partitionTransforms = {}); + + std::vector> createSplitsForDirectory( + const std::string& directory); + + std::vector listFiles(const std::string& dirPath); + + private: + std::shared_ptr createPartitionSpec( + const std::vector& transformSpecs); + + std::shared_ptr createIcebergInsertTableHandle( + const RowTypePtr& rowType, + const std::string& outputDirectoryPath, + const std::vector& partitionTransforms = {}); + + std::vector listPartitionDirectories( + const std::string& dataPath); + + void setupMemoryPools(const std::string& name); + + protected: + RowTypePtr rowType_; + + private: + static constexpr const char* kHiveConnectorId = "test-hive"; + + // The only supported file format is PARQUET. + dwio::common::FileFormat fileFormat_ = dwio::common::FileFormat::PARQUET; + + std::shared_ptr root_; + std::shared_ptr opPool_; + std::shared_ptr connectorPool_; + std::shared_ptr connectorSessionProperties_; + std::shared_ptr connectorConfig_; + std::unique_ptr connectorQueryCtx_; + VectorFuzzer::Options fuzzerOptions_; + std::unique_ptr fuzzer_; + std::unique_ptr vectorMaker_; +}; + +} // namespace facebook::velox::connector::lakehouse::iceberg::test diff --git a/velox/connectors/lakehouse/iceberg/tests/Main.cpp b/velox/connectors/lakehouse/iceberg/tests/Main.cpp new file mode 100644 index 000000000000..3c9dd6615055 --- /dev/null +++ b/velox/connectors/lakehouse/iceberg/tests/Main.cpp @@ -0,0 +1,29 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "velox/common/process/ThreadDebugInfo.h" + +#include +#include + +// This main is needed for some tests on linux. +int main(int argc, char** argv) { + testing::InitGoogleTest(&argc, argv); + // Signal handler required for ThreadDebugInfoTest + facebook::velox::process::addDefaultFatalSignalHandler(); + folly::Init init(&argc, &argv, false); + return RUN_ALL_TESTS(); +} diff --git a/velox/dwio/common/Reader.h b/velox/dwio/common/Reader.h index 9dddfaeaca08..93f823839331 100644 --- a/velox/dwio/common/Reader.h +++ b/velox/dwio/common/Reader.h @@ -210,6 +210,18 @@ class Reader { static TypePtr updateColumnNames( const TypePtr& fileType, const TypePtr& tableType); + + const std::unordered_set& requiredExtraFieldIds() const { + return requiredExtraFieldIds_; + } + + void setRequiredExtraFieldIds( + std::unordered_set& requiredExtraFieldIds) { + requiredExtraFieldIds_ = requiredExtraFieldIds; + } + + private: + std::unordered_set requiredExtraFieldIds_; }; } // namespace facebook::velox::dwio::common diff --git a/velox/dwio/common/ScanSpec.cpp b/velox/dwio/common/ScanSpec.cpp index 1e47a4c29fc8..0285c4a5dcc5 100644 --- a/velox/dwio/common/ScanSpec.cpp +++ b/velox/dwio/common/ScanSpec.cpp @@ -36,25 +36,27 @@ std::string_view ScanSpec::columnTypeString(ScanSpec::ColumnType columnType) { } } -ScanSpec* ScanSpec::getOrCreateChild(const std::string& name) { +ScanSpec* ScanSpec::getOrCreateChild(const std::string& name, bool isTempNode) { if (auto it = this->childByFieldName_.find(name); it != this->childByFieldName_.end()) { return it->second; } - this->children_.push_back(std::make_unique(name)); + this->children_.push_back(std::make_unique(name, isTempNode)); auto* child = this->children_.back().get(); this->childByFieldName_[child->fieldName()] = child; return child; } -ScanSpec* ScanSpec::getOrCreateChild(const Subfield& subfield) { +ScanSpec* ScanSpec::getOrCreateChild( + const Subfield& subfield, + bool isTempNode) { auto* container = this; const auto& path = subfield.path(); for (size_t depth = 0; depth < path.size(); ++depth) { const auto element = path[depth].get(); VELOX_CHECK_EQ(element->kind(), kNestedField); auto* nestedField = static_cast(element); - container = container->getOrCreateChild(nestedField->name()); + container = container->getOrCreateChild(nestedField->name(), isTempNode); } return container; } @@ -70,18 +72,19 @@ bool ScanSpec::compareTimeToDropValue( } // Integer filters are before other filters if there is no // history data. - if (left->filter_ && right->filter_) { - if (left->filter_->kind() == right->filter_->kind()) { + if (!left->filters_.empty() && !right->filters_.empty()) { + if (left->filters_.back()->kind() == right->filters_.back()->kind()) { return left->fieldName_ < right->fieldName_; } - return left->filter_->kind() < right->filter_->kind(); + return left->filters_.back()->kind() < right->filters_.back()->kind(); } + // If hasFilter() is true but 'filter_' is nullptr, we have a filter // on complex type members. The simple type filter goes first. - if (left->filter_) { + if (!left->filters_.empty()) { return true; } - if (right->filter_) { + if (!right->filters_.empty()) { return false; } return left->fieldName_ < right->fieldName_; @@ -139,6 +142,20 @@ void ScanSpec::enableFilterInSubTree(bool value) { } } +void ScanSpec::deleteTempNodes() { + for (auto it = children_.begin(); it != children_.end();) { + if ((*it)->isTempNode()) { + it = children_.erase(it); + } else { + if ((*it)->hasTempFilter()) { + (*it)->popFilter(); + (*it)->setHasTempFilter(false); + } + ++it; + } + } +} + const std::vector& ScanSpec::stableChildren() { std::lock_guard l(mutex_); if (stableChildren_.empty()) { @@ -172,7 +189,7 @@ bool ScanSpec::hasFilter() const { } bool ScanSpec::hasFilterApplicableToConstant() const { - if (filter_) { + if (!filters_.empty()) { return true; } for (auto& child : children_) { @@ -211,7 +228,7 @@ void ScanSpec::moveAdaptationFrom(ScanSpec& other) { // constant will have been evaluated at split start time. If // 'child' is constant there is no adaptation that can be // received. - child->filter_ = std::move(otherChild->filter_); + child->filters_ = std::move(otherChild->filters_); child->selectivity_ = otherChild->selectivity_; } } @@ -417,8 +434,8 @@ std::string ScanSpec::toString() const { std::stringstream out; if (!fieldName_.empty()) { out << fieldName_; - if (filter_) { - out << " filter " << filter_->toString(); + if (!filters_.empty()) { + out << " filter " << filters_.back()->toString(); if (filterDisabled_) { out << " disabled"; } @@ -443,6 +460,10 @@ std::string ScanSpec::toString() const { return out.str(); } +void ScanSpec::addFilter(const Filter& filter) { + pushFilter(filter.clone()); +} + ScanSpec* ScanSpec::addField(const std::string& name, column_index_t channel) { auto child = getOrCreateChild(name); child->setProjectOut(true); @@ -578,8 +599,8 @@ void ScanSpec::applyFilter( const BaseVector& vector, vector_size_t size, uint64_t* result) const { - if (filter_) { - filterRows(vector, *filter_, size, result); + if (!filters_.empty()) { + filterRows(vector, *filters_.back(), size, result); } if (!vector.type()->isRow()) { // Filter on MAP or ARRAY children are pruning, and won't affect correctness diff --git a/velox/dwio/common/ScanSpec.h b/velox/dwio/common/ScanSpec.h index fbcac3d3a591..155d03053917 100644 --- a/velox/dwio/common/ScanSpec.h +++ b/velox/dwio/common/ScanSpec.h @@ -55,19 +55,34 @@ class ScanSpec { static constexpr const char* kMapValuesFieldName = "values"; static constexpr const char* kArrayElementsFieldName = "elements"; - explicit ScanSpec(const std::string& name) : fieldName_(name) {} + explicit ScanSpec(const std::string& name, bool isTempNode = false) + : fieldName_(name), isTempNode_(isTempNode) {} // Filter to apply. If 'this' corresponds to a struct/list/map, this // can only be isNull or isNotNull, other filtering is given by // 'children'. + const common::Filter* filter() const { - return filterDisabled_ ? nullptr : filter_.get(); + return filterDisabled_ + ? nullptr + : (filters_.empty() ? nullptr : filters_.back().get()); } // Sets 'filter_'. May be used at initialization or when adding a // pushed down filter, e.g. top k cutoff. void setFilter(std::shared_ptr filter) { - filter_ = std::move(filter); + filters_.push_back(std::move(filter)); + } + + void pushFilter(std::unique_ptr newFilter) { + if (!filters_.empty()) { + newFilter = newFilter->mergeWith(filters_.back().get()); + } + filters_.push_back(std::move(newFilter)); + } + + void popFilter() { + filters_.pop_back(); } void setMaxArrayElementsCount(vector_size_t count) { @@ -199,13 +214,15 @@ class ScanSpec { /// Returns the ScanSpec corresponding to 'name'. Creates it if needed without /// any intermediate level. - ScanSpec* getOrCreateChild(const std::string& name); + ScanSpec* getOrCreateChild(const std::string& name, bool isTempNode = false); // Returns the ScanSpec corresponding to 'subfield'. Creates it if // needed, including any intermediate levels. This is used at // TableScan initialization to create the ScanSpec tree that // corresponds to the ColumnReader tree. - ScanSpec* getOrCreateChild(const Subfield& subfield); + ScanSpec* getOrCreateChild(const Subfield& subfield, bool isTempNode = false); + + void deleteTempNodes(); ScanSpec* childByName(const std::string& name) const { auto it = childByFieldName_.find(name); @@ -296,6 +313,8 @@ class ScanSpec { std::string toString() const; + void addFilter(const Filter& filter); + // Add a field to this ScanSpec, with content projected out. ScanSpec* addField(const std::string& name, column_index_t channel); @@ -390,6 +409,18 @@ class ScanSpec { return disableStatsBasedFilterReorder_; } + bool isTempNode() const { + return isTempNode_; + } + + void setHasTempFilter(bool hasTempFilter) { + hasTempFilter_ = hasTempFilter; + } + + bool hasTempFilter() const { + return hasTempFilter_; + } + private: void reorder(); @@ -430,9 +461,9 @@ class ScanSpec { // True if a string dictionary or flat map in this field should be // returned as flat. bool makeFlat_ = false; - std::shared_ptr filter_; bool filterDisabled_ = false; dwio::common::DeltaColumnUpdater* deltaUpdate_ = nullptr; + std::vector> filters_; // Filters that will be only used for row group filtering based on metadata. // The conjunctions among these filters are tracked in MetadataFilter, with @@ -470,6 +501,11 @@ class ScanSpec { // This node represents a flat map column that need to be read as struct, // i.e. in table schema it is a MAP, but in result vector it is ROW. bool isFlatMapAsStruct_ = false; + + // This node is temporary, will be used and deleted after intermediate + // processing stages, like Iceberg equality deletes. + bool isTempNode_ = false; + bool hasTempFilter_ = false; }; template diff --git a/velox/dwio/common/TypeWithId.cpp b/velox/dwio/common/TypeWithId.cpp index 03328024ef67..7dda5055b5e7 100644 --- a/velox/dwio/common/TypeWithId.cpp +++ b/velox/dwio/common/TypeWithId.cpp @@ -74,15 +74,30 @@ int countNodes(const TypePtr& type) { std::unique_ptr TypeWithId::create( const RowTypePtr& type, - const velox::common::ScanSpec& spec) { + const velox::common::ScanSpec& spec, + const std::unordered_set& requiredExtraFieldIds) { uint32_t next = 1; std::vector> children(type->size()); + + auto extraFieldSelector = [&requiredExtraFieldIds](size_t id, size_t maxId) { + for (auto extraFieldId : requiredExtraFieldIds) { + if (extraFieldId >= id && extraFieldId < maxId) { + return true; + } + } + return false; + }; + for (int i = 0, size = type->size(); i < size; ++i) { + // Recursively create children if the corresponding field is needed and is + // not constant value, or is one of the extra fields required. auto* childSpec = spec.childByName(type->nameOf(i)); - if (childSpec && !childSpec->isConstant()) { + uint32_t nodeCount = countNodes(type->childAt(i)); + if ((childSpec && !childSpec->isConstant()) || + extraFieldSelector(next, next + nodeCount)) { children[i] = create(type->childAt(i), next, i); } else { - next += countNodes(type->childAt(i)); + next += nodeCount; } } return std::make_unique( @@ -98,6 +113,20 @@ const std::shared_ptr& TypeWithId::childAt( return children_.at(idx); } +const std::shared_ptr& TypeWithId::childByFieldId( + uint32_t fieldId) const { + std::vector childNames; + std::vector childTypes; + for (auto& child : children_) { + if (fieldId == child->id()) { + return child; + } else if (fieldId > child->id() && fieldId <= child->maxId()) { + return child->childByFieldId(fieldId); + } + } + VELOX_FAIL("FieldId {} not found", fieldId); +} + std::unique_ptr TypeWithId::create( const std::shared_ptr& type, uint32_t& next, diff --git a/velox/dwio/common/TypeWithId.h b/velox/dwio/common/TypeWithId.h index a147cfe5066f..8a9645edebfe 100644 --- a/velox/dwio/common/TypeWithId.h +++ b/velox/dwio/common/TypeWithId.h @@ -43,9 +43,14 @@ class TypeWithId : public velox::Tree> { /// Create TypeWithId node but leave all the unselected children as nullptr. /// The ids are set correctly even when some of the previous nodes are not /// selected. + /// @requiredExtraFieldIds is used to determine the ids of the fields should + /// be included. This is because some connectors may require extra fields to + /// be included in the TypeWithId and ScanSpec after the base ScanSpec is + /// created. static std::unique_ptr create( const RowTypePtr& type, - const velox::common::ScanSpec& spec); + const velox::common::ScanSpec& spec, + const std::unordered_set& requiredExtraFieldIds); uint32_t size() const override; @@ -71,6 +76,9 @@ class TypeWithId : public velox::Tree> { const std::shared_ptr& childAt(uint32_t idx) const override; + const std::shared_ptr& childByFieldId( + uint32_t fieldId) const; + const std::shared_ptr& childByName( const std::string& name) const { VELOX_CHECK_EQ(type_->kind(), velox::TypeKind::ROW); diff --git a/velox/dwio/dwrf/reader/DwrfReader.h b/velox/dwio/dwrf/reader/DwrfReader.h index dcb38dbb5a1d..784c4dcbee83 100644 --- a/velox/dwio/dwrf/reader/DwrfReader.h +++ b/velox/dwio/dwrf/reader/DwrfReader.h @@ -288,7 +288,7 @@ class DwrfReader : public dwio::common::Reader { const std::shared_ptr& typeWithId() const override { - return readerBase_->schemaWithId(); + return readerBase_->schemaWithId(requiredExtraFieldIds()); } const PostScript& getPostscript() const { diff --git a/velox/dwio/dwrf/reader/ReaderBase.h b/velox/dwio/dwrf/reader/ReaderBase.h index 561d88e56841..ac9dde9afe01 100644 --- a/velox/dwio/dwrf/reader/ReaderBase.h +++ b/velox/dwio/dwrf/reader/ReaderBase.h @@ -134,11 +134,12 @@ class ReaderBase { schema_ = std::move(newSchema); } - const std::shared_ptr& schemaWithId() const { + const std::shared_ptr& schemaWithId( + const std::unordered_set& requiredExtraFieldIds = {}) const { if (!schemaWithId_) { if (options_.scanSpec()) { - schemaWithId_ = - dwio::common::TypeWithId::create(schema_, *options_.scanSpec()); + schemaWithId_ = dwio::common::TypeWithId::create( + schema_, *options_.scanSpec(), requiredExtraFieldIds); } else { schemaWithId_ = dwio::common::TypeWithId::create(schema_); } diff --git a/velox/functions/prestosql/InPredicate.cpp b/velox/functions/prestosql/InPredicate.cpp index 5ae0c6334373..6496f7572a1f 100644 --- a/velox/functions/prestosql/InPredicate.cpp +++ b/velox/functions/prestosql/InPredicate.cpp @@ -139,40 +139,6 @@ class VectorSetInPredicate : public exec::VectorFunction { const VectorPtr originalValues_; }; -// Read 'size' values from 'valuesVector' starting at 'offset', de-duplicate -// remove nulls and sort. Return a list of unique non-null values sorted in -// ascending order and a boolean indicating whether there were any null values. -template -std::pair, bool> toValues( - const VectorPtr& valuesVector, - vector_size_t offset, - vector_size_t size) { - auto simpleValues = valuesVector->as>(); - - bool nullAllowed = false; - std::vector values; - values.reserve(size); - - for (auto i = offset; i < offset + size; i++) { - if (simpleValues->isNullAt(i)) { - nullAllowed = true; - } else { - if constexpr (std::is_same_v) { - values.emplace_back(simpleValues->valueAt(i).toMillis()); - } else { - values.emplace_back(simpleValues->valueAt(i)); - } - } - } - - // In-place sort, remove duplicates, and later std::move to save memory - std::sort(values.begin(), values.end()); - auto last = std::unique(values.begin(), values.end()); - values.resize(std::distance(values.begin(), last)); - - return {std::move(values), nullAllowed}; -} - // Creates a filter for constant values. A null filter means either // no values or only null values. The boolean is true if the list is // non-empty and consists of nulls only. @@ -181,7 +147,8 @@ std::pair, bool> createBigintValuesFilter( const VectorPtr& valuesVector, vector_size_t offset, vector_size_t size) { - auto valuesPair = toValues(valuesVector, offset, size); + auto valuesPair = + common::deDuplicateValues(valuesVector, offset, size); const auto& values = valuesPair.first; bool nullAllowed = valuesPair.second; @@ -210,7 +177,7 @@ createFloatingPointValuesFilter( const VectorPtr& valuesVector, vector_size_t offset, vector_size_t size) { - auto valuesPair = toValues(valuesVector, offset, size); + auto valuesPair = common::deDuplicateValues(valuesVector, offset, size); auto& values = valuesPair.first; bool nullAllowed = valuesPair.second; @@ -252,7 +219,8 @@ std::pair, bool> createHugeintValuesFilter( const VectorPtr& valuesVector, vector_size_t offset, vector_size_t size) { - auto valuesPair = toValues(valuesVector, offset, size); + auto valuesPair = + common::deDuplicateValues(valuesVector, offset, size); const auto& values = valuesPair.first; bool nullAllowed = valuesPair.second; @@ -278,8 +246,8 @@ std::pair, bool> createBytesValuesFilter( const VectorPtr& valuesVector, vector_size_t offset, vector_size_t size) { - auto valuesPair = - toValues(valuesVector, offset, size); + auto valuesPair = common::deDuplicateValues( + valuesVector, offset, size); const auto& values = valuesPair.first; bool nullAllowed = valuesPair.second; diff --git a/velox/type/Filter.cpp b/velox/type/Filter.cpp index dfa059a94b1f..477a29c690aa 100644 --- a/velox/type/Filter.cpp +++ b/velox/type/Filter.cpp @@ -1213,7 +1213,7 @@ std::unique_ptr createBigintValuesFilter( std::unique_ptr createBigintValues( const std::vector& values, bool nullAllowed) { - return createBigintValuesFilter(values, nullAllowed, false); + return common::createBigintValuesFilter(values, nullAllowed, false); } std::unique_ptr createHugeintValues( @@ -1229,7 +1229,7 @@ std::unique_ptr createHugeintValues( std::unique_ptr createNegatedBigintValues( const std::vector& values, bool nullAllowed) { - return createBigintValuesFilter(values, nullAllowed, true); + return common::createBigintValuesFilter(values, nullAllowed, true); } BigintMultiRange::BigintMultiRange( diff --git a/velox/type/Filter.h b/velox/type/Filter.h index 28cfd58b9230..317fa0a533c0 100644 --- a/velox/type/Filter.h +++ b/velox/type/Filter.h @@ -31,6 +31,7 @@ #include "velox/type/StringView.h" #include "velox/type/Subfield.h" #include "velox/type/Type.h" +#include "velox/vector/BaseVector.h" namespace facebook::velox::common { @@ -2259,6 +2260,40 @@ static inline bool applyFilter(TFilter& filter, StringView value) { return filter.testBytes(value.data(), value.size()); } +// Read 'size' values from 'valuesVector' starting at 'offset', de-duplicate +// remove nulls and sort. Return a list of unique non-null values sorted in +// ascending order and a boolean indicating whether there were any null values. +template +std::pair, bool> deDuplicateValues( + const VectorPtr& valuesVector, + vector_size_t offset, + vector_size_t size) { + auto simpleValues = valuesVector->as>(); + + bool hasNull = false; + std::vector values; + values.reserve(size); + + for (auto i = offset; i < offset + size; i++) { + if (simpleValues->isNullAt(i)) { + hasNull = true; + } else { + if constexpr (std::is_same_v) { + values.emplace_back(simpleValues->valueAt(i).toMillis()); + } else { + values.emplace_back(simpleValues->valueAt(i)); + } + } + } + + // In-place sort, remove duplicates, and later std::move to save memory. + std::sort(values.begin(), values.end()); + auto last = std::unique(values.begin(), values.end()); + values.resize(std::distance(values.begin(), last)); + + return {std::move(values), hasNull}; +} + // Creates a hash or bitmap based IN filter depending on value distribution. std::unique_ptr createBigintValues( const std::vector& values,