diff --git a/velox/connectors/hive/CMakeLists.txt b/velox/connectors/hive/CMakeLists.txt index e7bbb90fe04a..8406c2f941a8 100644 --- a/velox/connectors/hive/CMakeLists.txt +++ b/velox/connectors/hive/CMakeLists.txt @@ -15,8 +15,6 @@ velox_add_library(velox_hive_config OBJECT HiveConfig.cpp) velox_link_libraries(velox_hive_config velox_core velox_exception) -add_subdirectory(iceberg) - velox_add_library( velox_hive_connector OBJECT @@ -33,9 +31,11 @@ velox_add_library( TableHandle.cpp ) +add_subdirectory(iceberg) + velox_link_libraries( velox_hive_connector - PUBLIC velox_hive_iceberg_splitreader + PRIVATE velox_hive_iceberg_splitreader PRIVATE velox_common_io velox_connector velox_dwio_catalog_fbhive velox_hive_partition_function ) diff --git a/velox/connectors/hive/HiveConfig.cpp b/velox/connectors/hive/HiveConfig.cpp index 8b354e2ed7d4..3a0b2db01670 100644 --- a/velox/connectors/hive/HiveConfig.cpp +++ b/velox/connectors/hive/HiveConfig.cpp @@ -273,4 +273,9 @@ std::string HiveConfig::schema(const config::ConfigBase* session) const { kSchema, config_->get(kSchema, "")); } +bool HiveConfig::fanoutEnabled(const config::ConfigBase* session) const { + return session->get( + kFanoutEnabledSession, config_->get(kFanoutEnabled, true)); +} + } // namespace facebook::velox::connector::hive diff --git a/velox/connectors/hive/HiveConfig.h b/velox/connectors/hive/HiveConfig.h index f69da84ff0c1..ad5034f9f3a1 100644 --- a/velox/connectors/hive/HiveConfig.h +++ b/velox/connectors/hive/HiveConfig.h @@ -208,6 +208,12 @@ class HiveConfig { static constexpr const char* kSource = "source"; static constexpr const char* kSchema = "schema"; + /// Controls the writer mode, whether the fanout mode writer is enabled, + /// default value is true, setting to false means clustered mode. + /// Currently applies only to the Iceberg writer. + static constexpr const char* kFanoutEnabled = "fanout-enabled"; + static constexpr const char* kFanoutEnabledSession = "fanout_enabled"; + InsertExistingPartitionsBehavior insertExistingPartitionsBehavior( const config::ConfigBase* session) const; @@ -305,6 +311,9 @@ class HiveConfig { /// Schema of the query. Used for storage logging. std::string schema(const config::ConfigBase* session) const; + /// Return if fanout writer mode is enabled. + bool fanoutEnabled(const config::ConfigBase* session) const; + HiveConfig(std::shared_ptr config) { VELOX_CHECK_NOT_NULL( config, "Config is null for HiveConfig initialization"); diff --git a/velox/connectors/hive/HiveConnectorUtil.cpp b/velox/connectors/hive/HiveConnectorUtil.cpp index f0dc05c96e89..bf5e0c3ac1c4 100644 --- a/velox/connectors/hive/HiveConnectorUtil.cpp +++ b/velox/connectors/hive/HiveConnectorUtil.cpp @@ -24,6 +24,10 @@ #include "velox/expression/ExprConstants.h" #include "velox/expression/ExprToSubfieldFilter.h" +#include +#include +#include + namespace facebook::velox::connector::hive { namespace { @@ -812,7 +816,6 @@ std::unique_ptr createBufferedInput( } namespace { - core::CallTypedExprPtr replaceInputs( const core::CallTypedExpr* call, std::vector&& inputs) { @@ -946,6 +949,10 @@ core::TypedExprPtr extractFiltersFromRemainingFilter( } } // namespace +std::string makeUuid() { + return boost::lexical_cast(boost::uuids::random_generator()()); +} + core::TypedExprPtr extractFiltersFromRemainingFilter( const core::TypedExprPtr& expr, core::ExpressionEvaluator* evaluator, diff --git a/velox/connectors/hive/HiveConnectorUtil.h b/velox/connectors/hive/HiveConnectorUtil.h index a1e35cdcd27a..d0749e70ff21 100644 --- a/velox/connectors/hive/HiveConnectorUtil.h +++ b/velox/connectors/hive/HiveConnectorUtil.h @@ -152,4 +152,6 @@ core::TypedExprPtr extractFiltersFromRemainingFilter( common::SubfieldFilters& filters, double& sampleRate); +std::string makeUuid(); + } // namespace facebook::velox::connector::hive diff --git a/velox/connectors/hive/HiveDataSink.cpp b/velox/connectors/hive/HiveDataSink.cpp index 1ac9243f050f..94435abd38e7 100644 --- a/velox/connectors/hive/HiveDataSink.cpp +++ b/velox/connectors/hive/HiveDataSink.cpp @@ -29,10 +29,6 @@ #include "velox/exec/OperatorUtils.h" #include "velox/exec/SortBuffer.h" -#include -#include -#include - using facebook::velox::common::testutil::TestValue; namespace facebook::velox::connector::hive { @@ -94,14 +90,12 @@ std::vector getPartitionChannels( // Returns the column indices of non-partition data columns. std::vector getNonPartitionChannels( - const std::vector& partitionChannels, - const column_index_t childrenSize) { + const std::shared_ptr& insertTableHandle) { std::vector dataChannels; - dataChannels.reserve(childrenSize - partitionChannels.size()); - for (column_index_t i = 0; i < childrenSize; i++) { - if (std::find(partitionChannels.cbegin(), partitionChannels.cend(), i) == - partitionChannels.cend()) { + for (column_index_t i = 0; i < insertTableHandle->inputColumns().size(); + i++) { + if (!insertTableHandle->inputColumns()[i]->isPartitionKey()) { dataChannels.push_back(i); } } @@ -118,10 +112,6 @@ std::string makePartitionDirectory( return tableDirectory; } -std::string makeUuid() { - return boost::lexical_cast(boost::uuids::random_generator()()); -} - std::unordered_map tableTypeNames() { return { {LocationHandle::TableType::kNew, "kNew"}, @@ -382,6 +372,18 @@ HiveDataSink::HiveDataSink( ? createBucketFunction( *insertTableHandle->bucketProperty(), inputType) + : nullptr, + getPartitionChannels(insertTableHandle), + getNonPartitionChannels(insertTableHandle), + !getPartitionChannels(insertTableHandle).empty() + ? std::make_unique( + inputType, + getPartitionChannels(insertTableHandle), + hiveConfig->maxPartitionsPerWriters( + connectorQueryCtx->sessionProperties()), + connectorQueryCtx->memoryPool(), + hiveConfig->isPartitionPathAsLowerCase( + connectorQueryCtx->sessionProperties())) : nullptr) {} HiveDataSink::HiveDataSink( @@ -391,7 +393,10 @@ HiveDataSink::HiveDataSink( CommitStrategy commitStrategy, const std::shared_ptr& hiveConfig, uint32_t bucketCount, - std::unique_ptr bucketFunction) + std::unique_ptr bucketFunction, + const std::vector& partitionChannels, + const std::vector& dataChannels, + std::unique_ptr partitionIdGenerator) : inputType_(std::move(inputType)), insertTableHandle_(std::move(insertTableHandle)), connectorQueryCtx_(connectorQueryCtx), @@ -400,19 +405,9 @@ HiveDataSink::HiveDataSink( 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_( - getNonPartitionChannels(partitionChannels_, inputType_->size())), + partitionChannels_(partitionChannels), + partitionIdGenerator_(std::move(partitionIdGenerator)), + dataChannels_(dataChannels), bucketCount_(static_cast(bucketCount)), bucketFunction_(std::move(bucketFunction)), writerFactory_( @@ -488,6 +483,8 @@ void HiveDataSink::appendData(RowVectorPtr input) { // 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) { @@ -496,8 +493,6 @@ void HiveDataSink::appendData(RowVectorPtr input) { return; } - splitInputRowsAndEnsureWriters(); - for (auto index = 0; index < writers_.size(); ++index) { const vector_size_t partitionSize = partitionSizes_[index]; if (partitionSize == 0) { @@ -692,23 +687,23 @@ std::vector HiveDataSink::commitMessage() const { 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)); + 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); } @@ -755,11 +750,7 @@ uint32_t HiveDataSink::appendWriter(const HiveWriterId& id) { VELOX_CHECK_EQ(writers_.size(), writerInfo_.size()); VELOX_CHECK_EQ(writerIndexMap_.size(), writerInfo_.size()); - std::optional partitionName; - if (isPartitioned()) { - partitionName = - partitionIdGenerator_->partitionName(id.partitionId.value()); - } + std::optional partitionName = getPartitionName(id); // Without explicitly setting flush policy, the default memory based flush // policy is used. @@ -782,6 +773,42 @@ uint32_t HiveDataSink::appendWriter(const HiveWriterId& id) { setMemoryReclaimers(writerInfo_.back().get(), ioStats_.back().get()); + auto options = createWriterOptions(); + // 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(), + .fileSystemStats = fileSystemStats_.get(), + }), + options); + writer = maybeCreateBucketSortWriter(std::move(writer)); + writers_.emplace_back(std::move(writer)); + + extendBuffersForPartitionedTables(); + addThreadLocalRuntimeStat( + fmt::format( + "{}WriterCount", + dwio::common::toString(insertTableHandle_->storageFormat())), + RuntimeCounter(1)); + // Extends the buffer used for partition rows calculations. + partitionSizes_.emplace_back(0); + partitionRows_.emplace_back(nullptr); + rawPartitionRows_.emplace_back(nullptr); + + writerIndexMap_.emplace(id, writers_.size() - 1); + return writerIndexMap_[id]; +} + +std::shared_ptr HiveDataSink::createWriterOptions() + const { // Take the writer options provided by the user as a starting point, or // allocate a new one. auto options = insertTableHandle_->writerOptions(); @@ -832,36 +859,17 @@ uint32_t HiveDataSink::appendWriter(const HiveWriterId& id) { options->adjustTimestampToTimezone = connectorQueryCtx_->adjustTimestampToTimezone(); options->processConfigs(*hiveConfig_->config(), *connectorSessionProperties); + return options; +} - // 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(), - .fileSystemStats = fileSystemStats_.get(), - }), - options); - writer = maybeCreateBucketSortWriter(std::move(writer)); - writers_.emplace_back(std::move(writer)); - addThreadLocalRuntimeStat( - fmt::format( - "{}WriterCount", - dwio::common::toString(insertTableHandle_->storageFormat())), - RuntimeCounter(1)); - // Extends the buffer used for partition rows calculations. - partitionSizes_.emplace_back(0); - partitionRows_.emplace_back(nullptr); - rawPartitionRows_.emplace_back(nullptr); - - 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 @@ -891,6 +899,13 @@ HiveDataSink::maybeCreateBucketSortWriter( 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()) { @@ -905,7 +920,25 @@ HiveWriterId HiveDataSink::getWriterId(size_t row) const { return HiveWriterId{partitionId, bucketId}; } -void HiveDataSink::splitInputRowsAndEnsureWriters() { +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()); @@ -919,18 +952,7 @@ void HiveDataSink::splitInputRowsAndEnsureWriters() { const auto id = getWriterId(row); const uint32_t index = ensureWriter(id); - 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]; + updatePartitionRows(index, numRows, row); } for (uint32_t i = 0; i < partitionSizes_.size(); ++i) { @@ -941,6 +963,15 @@ void HiveDataSink::splitInputRowsAndEnsureWriters() { } } +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 { diff --git a/velox/connectors/hive/HiveDataSink.h b/velox/connectors/hive/HiveDataSink.h index 788294984967..709a3f616955 100644 --- a/velox/connectors/hive/HiveDataSink.h +++ b/velox/connectors/hive/HiveDataSink.h @@ -530,7 +530,10 @@ class HiveDataSink : public DataSink { CommitStrategy commitStrategy, const std::shared_ptr& hiveConfig, uint32_t bucketCount, - std::unique_ptr bucketFunction); + std::unique_ptr bucketFunction, + const std::vector& partitionChannels, + const std::vector& dataChannels, + std::unique_ptr partitionIdGenerator); void appendData(RowVectorPtr input) override; @@ -631,7 +634,7 @@ class HiveDataSink : public DataSink { // 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. - void splitInputRowsAndEnsureWriters(); + virtual void splitInputRowsAndEnsureWriters(RowVectorPtr input); // Makes sure to create one writer for the given writer id. The function // returns the corresponding index in 'writers_'. @@ -641,10 +644,25 @@ class HiveDataSink : public DataSink { // the newly created writer in 'writers_'. uint32_t appendWriter(const HiveWriterId& id); - std::unique_ptr + virtual std::shared_ptr createWriterOptions() + const; + + virtual std::optional getPartitionName( + const HiveWriterId& id) const; + + virtual 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; @@ -668,7 +686,7 @@ class HiveDataSink : public DataSink { // Invoked to write 'input' to the specified file writer. void write(size_t index, RowVectorPtr input); - void closeInternal(); + virtual void closeInternal(); // IMPORTANT NOTE: these are passed to writers as raw pointers. HiveDataSink // owns the lifetime of these objects, and therefore must destroy them last. diff --git a/velox/connectors/hive/HiveDataSource.cpp b/velox/connectors/hive/HiveDataSource.cpp index 267e3a4cd5c7..00689f9d2230 100644 --- a/velox/connectors/hive/HiveDataSource.cpp +++ b/velox/connectors/hive/HiveDataSource.cpp @@ -253,7 +253,9 @@ std::unique_ptr HiveDataSource::createSplitReader() { fsStats_, fileHandleFactory_, ioExecutor_, - scanSpec_); + scanSpec_, + expressionEvaluator_, + totalRemainingFilterTime_); } std::vector HiveDataSource::setupBucketConversion() { @@ -353,6 +355,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 c98d431771e6..1e8771ee9812 100644 --- a/velox/connectors/hive/HiveDataSource.h +++ b/velox/connectors/hive/HiveDataSource.h @@ -163,7 +163,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/PartitionIdGenerator.cpp b/velox/connectors/hive/PartitionIdGenerator.cpp index 6f1b416b79f4..976f50996113 100644 --- a/velox/connectors/hive/PartitionIdGenerator.cpp +++ b/velox/connectors/hive/PartitionIdGenerator.cpp @@ -29,10 +29,10 @@ PartitionIdGenerator::PartitionIdGenerator( uint32_t maxPartitions, memory::MemoryPool* pool, bool partitionPathAsLowerCase) - : pool_(pool), - partitionChannels_(std::move(partitionChannels)), + : partitionChannels_(std::move(partitionChannels)), maxPartitions_(maxPartitions), - partitionPathAsLowerCase_(partitionPathAsLowerCase) { + partitionPathAsLowerCase_(partitionPathAsLowerCase), + pool_(pool) { VELOX_USER_CHECK( !partitionChannels_.empty(), "There must be at least one partition key."); for (auto channel : partitionChannels_) { @@ -60,6 +60,19 @@ PartitionIdGenerator::PartitionIdGenerator( } } +PartitionIdGenerator::PartitionIdGenerator( + std::vector partitionChannels, + uint32_t maxPartitions, + memory::MemoryPool* pool, + bool partitionPathAsLowerCase) + : partitionChannels_(std::move(partitionChannels)), + maxPartitions_(maxPartitions), + partitionPathAsLowerCase_(partitionPathAsLowerCase), + pool_(pool) { + VELOX_USER_CHECK( + !partitionChannels_.empty(), "There must be at least one partition key."); +} + void PartitionIdGenerator::run( const RowVectorPtr& input, raw_vector& result) { @@ -170,7 +183,7 @@ void PartitionIdGenerator::updateValueToPartitionIdMapping() { } void PartitionIdGenerator::savePartitionValues( - uint64_t partitionId, + uint32_t partitionId, const RowVectorPtr& input, vector_size_t row) { for (auto i = 0; i < partitionChannels_.size(); ++i) { diff --git a/velox/connectors/hive/PartitionIdGenerator.h b/velox/connectors/hive/PartitionIdGenerator.h index c4e0320b46c1..43a77740b90d 100644 --- a/velox/connectors/hive/PartitionIdGenerator.h +++ b/velox/connectors/hive/PartitionIdGenerator.h @@ -38,10 +38,12 @@ class PartitionIdGenerator { memory::MemoryPool* pool, bool partitionPathAsLowerCase); + virtual ~PartitionIdGenerator() = default; + /// 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); + virtual void run(const RowVectorPtr& input, raw_vector& result); /// Return the total number of distinct partitions processed so far. uint64_t numPartitions() const { @@ -52,16 +54,38 @@ class PartitionIdGenerator { /// 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; + virtual std::string partitionName(uint64_t partitionId) const; - private: - static constexpr const int32_t kHasherReservePct = 20; + protected: + PartitionIdGenerator( + std::vector partitionChannels, + uint32_t maxPartitions, + memory::MemoryPool* pool, + bool partitionPathAsLowerCase); // Computes value IDs using VectorHashers for all rows in 'input'. void computeValueIds( const RowVectorPtr& input, raw_vector& valueIds); + const std::vector partitionChannels_; + + std::vector> hashers_; + + // A vector holding unique partition key values. One row per partition. Row + // numbers match partition IDs. + RowVectorPtr partitionValues_; + + const uint32_t maxPartitions_; + + // A mapping from value ID produced by VectorHashers to a partition ID. + std::unordered_map partitionIds_; + + const bool partitionPathAsLowerCase_; + + private: + static constexpr const int32_t kHasherReservePct = 20; + // 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 @@ -70,29 +94,15 @@ class PartitionIdGenerator { // Copies partition values of 'row' from 'input' into 'partitionId' row in // 'partitionValues_'. - void savePartitionValues( - uint64_t partitionId, + virtual void savePartitionValues( + uint32_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_; }; diff --git a/velox/connectors/hive/SplitReader.cpp b/velox/connectors/hive/SplitReader.cpp index 47ba102904ef..de0954a29ecd 100644 --- a/velox/connectors/hive/SplitReader.cpp +++ b/velox/connectors/hive/SplitReader.cpp @@ -98,7 +98,9 @@ std::unique_ptr SplitReader::create( const std::shared_ptr& fsStats, FileHandleFactory* fileHandleFactory, folly::Executor* ioExecutor, - 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") { @@ -113,7 +115,9 @@ std::unique_ptr SplitReader::create( fsStats, fileHandleFactory, ioExecutor, - 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 46ecccb4eca7..9f9a2926d0ea 100644 --- a/velox/connectors/hive/SplitReader.h +++ b/velox/connectors/hive/SplitReader.h @@ -97,7 +97,9 @@ class SplitReader { const std::shared_ptr& fsStats, FileHandleFactory* fileHandleFactory, folly::Executor* ioExecutor, - 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/CMakeLists.txt b/velox/connectors/hive/iceberg/CMakeLists.txt index 0c6662595097..4c70d203871c 100644 --- a/velox/connectors/hive/iceberg/CMakeLists.txt +++ b/velox/connectors/hive/iceberg/CMakeLists.txt @@ -14,15 +14,28 @@ velox_add_library( velox_hive_iceberg_splitreader + DataFileStatsCollector.cpp + EqualityDeleteFileReader.cpp + FilterUtil.cpp + IcebergColumnHandle.cpp IcebergDataSink.cpp + IcebergDeleteFile.cpp + IcebergPartitionIdGenerator.cpp IcebergSplit.cpp IcebergSplitReader.cpp + Murmur3.cpp PartitionSpec.cpp PositionalDeleteFileReader.cpp + TransformFactory.cpp + Transforms.cpp ) velox_link_libraries(velox_hive_iceberg_splitreader velox_connector Folly::folly) +if(VELOX_ENABLE_PARQUET) + velox_link_libraries(velox_hive_iceberg_splitreader velox_dwio_arrow_parquet_writer) +endif() + if(${VELOX_BUILD_TESTING}) add_subdirectory(tests) endif() diff --git a/velox/connectors/hive/iceberg/DataFileStatsCollector.cpp b/velox/connectors/hive/iceberg/DataFileStatsCollector.cpp new file mode 100644 index 000000000000..98b05f81f0e9 --- /dev/null +++ b/velox/connectors/hive/iceberg/DataFileStatsCollector.cpp @@ -0,0 +1,121 @@ +/* + * 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/DataFileStatsCollector.h" +#include "velox/common/base/Exceptions.h" +#include "velox/common/encode/Base64.h" +#include "velox/dwio/parquet/writer/arrow/Metadata.h" +#include "velox/dwio/parquet/writer/arrow/Statistics.h" + +namespace facebook::velox::connector::hive::iceberg { + +using namespace facebook::velox::parquet; + +DataFileStatsCollector::DataFileStatsCollector( + std::shared_ptr< + std::vector>> + settings) + : FileStatsCollector(std::move(settings)) {} + +void DataFileStatsCollector::collectStats( + const void* metadata, + const std::shared_ptr& dataFileStats) { + const auto& fileMetadata = + *static_cast*>( + metadata); + VELOX_CHECK_NOT_NULL(fileMetadata); + + std::unordered_set skipBoundsFields; + std::function processFields = + [&skipBoundsFields, + &processFields](IcebergDataFileStatsSettings* field) -> int32_t { + if (field->skipBounds) { + skipBoundsFields.insert(field->fieldId); + } + if (field->children.empty()) { + return 1; + } + int32_t count = 0; + for (const auto& child : field->children) { + count += processFields(child.get()); + } + return count; + }; + + // numFields is not the number of columns in Iceberg table's schema, + // e.g., schema_->size(). It also contains the sub-fields when there are + // nested types in table's schema. + int32_t numFields = 0; + for (const auto& field : *statsSetting_) { + auto* icebergField = + static_cast(field.get()); + numFields += processFields(icebergField); + } + + std::unordered_map> + globalMinStats; + std::unordered_map> + globalMaxStats; + + dataFileStats->numRecords = fileMetadata->num_rows(); + const auto numRowGroups = fileMetadata->num_row_groups(); + for (auto i = 0; i < numRowGroups; ++i) { + const auto rgm = fileMetadata->RowGroup(i); + VELOX_CHECK_EQ(numFields, rgm->num_columns()); + dataFileStats->splitOffsets.emplace_back(rgm->file_offset()); + + for (auto j = 0; j < numFields; ++j) { + const auto columnChunkMetadata = rgm->ColumnChunk(j); + const auto fieldId = columnChunkMetadata->field_id(); + const auto numValues = columnChunkMetadata->num_values(); + + dataFileStats->valueCounts[fieldId] += numValues; + dataFileStats->columnsSizes[fieldId] += + columnChunkMetadata->total_compressed_size(); + + const auto columnChunkStats = columnChunkMetadata->statistics(); + if (columnChunkStats->nan_count() > 0) { + dataFileStats->nanValueCounts[fieldId] += columnChunkStats->nan_count(); + } + dataFileStats->nullValueCounts[fieldId] += columnChunkStats->null_count(); + + if (columnChunkStats->HasMinMax() && + !skipBoundsFields.contains(fieldId)) { + if (globalMaxStats.find(fieldId) == globalMaxStats.end()) { + globalMinStats[fieldId] = columnChunkStats; + globalMaxStats[fieldId] = columnChunkStats; + } else { + globalMaxStats[fieldId] = arrow::Statistics::CompareAndGetMax( + globalMaxStats[fieldId], columnChunkStats); + globalMinStats[fieldId] = arrow::Statistics::CompareAndGetMin( + globalMinStats[fieldId], columnChunkStats); + } + } + } + } + + for (const auto& [fieldId, minStats] : globalMinStats) { + const auto lowerBound = minStats->MinValue(); + dataFileStats->lowerBounds[fieldId] = + encoding::Base64::encode(lowerBound.data(), lowerBound.size()); + } + for (const auto& [fieldId, maxStats] : globalMaxStats) { + const auto upperBound = maxStats->MaxValue(); + dataFileStats->upperBounds[fieldId] = + encoding::Base64::encode(upperBound.data(), upperBound.size()); + } +} + +} // namespace facebook::velox::connector::hive::iceberg diff --git a/velox/connectors/hive/iceberg/DataFileStatsCollector.h b/velox/connectors/hive/iceberg/DataFileStatsCollector.h new file mode 100644 index 000000000000..e379dc391799 --- /dev/null +++ b/velox/connectors/hive/iceberg/DataFileStatsCollector.h @@ -0,0 +1,48 @@ +/* + * 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/dwio/common/DataFileStatsCollector.h" + +namespace facebook::velox::connector::hive::iceberg { + +/// Settings for collecting Iceberg parquet data file statistics. +/// Holds the Iceberg source field id and whether to skip bounds +/// collection for this field. For nested field, it contains child fields. +struct IcebergDataFileStatsSettings + : public dwio::common::DataFileStatsSettings { + int32_t fieldId; + bool skipBounds; + std::vector> children; + + IcebergDataFileStatsSettings(int32_t id, bool skip) + : fieldId(id), skipBounds(skip), children() {} +}; + +class DataFileStatsCollector : public dwio::common::FileStatsCollector { + public: + explicit DataFileStatsCollector( + std::shared_ptr< + std::vector>> + settings); + + void collectStats( + const void* metadata, + const std::shared_ptr& fileStats) + override; +}; + +} // namespace facebook::velox::connector::hive::iceberg diff --git a/velox/connectors/hive/iceberg/EqualityDeleteFileReader.cpp b/velox/connectors/hive/iceberg/EqualityDeleteFileReader.cpp new file mode 100644 index 000000000000..f30acdbab043 --- /dev/null +++ b/velox/connectors/hive/iceberg/EqualityDeleteFileReader.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/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(), + nullptr, + 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/IcebergColumnHandle.cpp b/velox/connectors/hive/iceberg/IcebergColumnHandle.cpp new file mode 100644 index 000000000000..1b5028ae97be --- /dev/null +++ b/velox/connectors/hive/iceberg/IcebergColumnHandle.cpp @@ -0,0 +1,42 @@ +/* + * 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/IcebergColumnHandle.h" + +namespace facebook::velox::connector::hive::iceberg { + +IcebergColumnHandle::IcebergColumnHandle( + const std::string& name, + ColumnType columnType, + TypePtr dataType, + TypePtr hiveType, + const IcebergNestedField& nestedField, + std::vector requiredSubfields, + ColumnParseParameters columnParseParameters) + : HiveColumnHandle( + name, + columnType, + dataType, + hiveType, + std::move(requiredSubfields), + columnParseParameters), + nestedField_(nestedField) {} + +const IcebergNestedField& IcebergColumnHandle::nestedField() const { + return nestedField_; +} + +} // namespace facebook::velox::connector::hive::iceberg diff --git a/velox/connectors/hive/iceberg/IcebergColumnHandle.h b/velox/connectors/hive/iceberg/IcebergColumnHandle.h new file mode 100644 index 000000000000..5e81d9674beb --- /dev/null +++ b/velox/connectors/hive/iceberg/IcebergColumnHandle.h @@ -0,0 +1,46 @@ +/* + * 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/hive/TableHandle.h" + +namespace facebook::velox::connector::hive::iceberg { + +struct IcebergNestedField { + int32_t id; + std::vector children; +}; + +class IcebergColumnHandle : public HiveColumnHandle { + public: + IcebergColumnHandle( + const std::string& name, + ColumnType columnType, + TypePtr dataType, + TypePtr hiveType, + const IcebergNestedField& nestedField, + std::vector requiredSubfields = {}, + ColumnParseParameters columnParseParameters = {}); + + const IcebergNestedField& nestedField() const; + + private: + const IcebergNestedField nestedField_; +}; + +using IcebergColumnHandlePtr = std::shared_ptr; + +} // namespace facebook::velox::connector::hive::iceberg diff --git a/velox/connectors/hive/iceberg/IcebergDataSink.cpp b/velox/connectors/hive/iceberg/IcebergDataSink.cpp index dc88150fef87..eabc5e793e9e 100644 --- a/velox/connectors/hive/iceberg/IcebergDataSink.cpp +++ b/velox/connectors/hive/iceberg/IcebergDataSink.cpp @@ -16,18 +16,122 @@ #include "velox/connectors/hive/iceberg/IcebergDataSink.h" #include "velox/common/base/Fs.h" +#include "velox/connectors/hive/HiveConnectorUtil.h" +#include "velox/connectors/hive/iceberg/DataFileStatsCollector.h" +#include "velox/connectors/hive/iceberg/IcebergPartitionIdGenerator.h" +#include "velox/dwio/common/SortingWriter.h" +#ifdef VELOX_ENABLE_PARQUET +#include "velox/dwio/parquet/writer/Writer.h" +#endif +#include "velox/exec/OperatorUtils.h" +#include "velox/exec/SortBuffer.h" namespace facebook::velox::connector::hive::iceberg { +namespace { + +constexpr std::string_view kNotClusteredRowsErrorMsg = + "Incoming records violate the writer assumption that records are clustered by spec and \n by partition within each spec. Either cluster the incoming records or switch to fanout writers.\nEncountered records that belong to already closed files:\n"; + +#define WRITER_NON_RECLAIMABLE_SECTION_GUARD(index) \ + memory::NonReclaimableSectionGuard nonReclaimableGuard( \ + writerInfo_[(index)]->nonReclaimableSectionHolder.get()) + +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) { + auto value = block->valueAt(row); + return value.str(); +} + +template <> +folly::dynamic extractPartitionValue( + const DecodedVector* block, + vector_size_t row) { + auto value = block->valueAt(row); + return value.str(); +} + +template <> +folly::dynamic extractPartitionValue( + const DecodedVector* block, + vector_size_t row) { + auto timestamp = block->valueAt(row); + return timestamp.toMicros(); +} + +class IcebergFileNameGenerator : public 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("{}", 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::vector inputColumns, LocationHandlePtr locationHandle, + std::shared_ptr partitionSpec, + memory::MemoryPool* pool, dwio::common::FileFormat tableStorageFormat, - IcebergPartitionSpecPtr partitionSpec, + const std::vector& sortedBy, std::optional compressionKind, const std::unordered_map& serdeParameters) : HiveInsertTableHandle( - std::move(inputColumns), + std::vector>( + inputColumns.begin(), + inputColumns.end()), std::move(locationHandle), tableStorageFormat, nullptr, @@ -35,8 +139,11 @@ IcebergInsertTableHandle::IcebergInsertTableHandle( serdeParameters, nullptr, false, - std::make_shared()), - partitionSpec_(std::move(partitionSpec)) { + std::make_shared()), + partitionSpec_(std::move(partitionSpec)), + columnTransforms_( + parsePartitionTransformSpecs(partitionSpec_->fields, pool)), + sortedBy_(sortedBy) { VELOX_USER_CHECK( !inputColumns_.empty(), "Input columns cannot be empty for Iceberg tables."); @@ -50,23 +157,155 @@ IcebergDataSink::IcebergDataSink( const ConnectorQueryCtx* connectorQueryCtx, CommitStrategy commitStrategy, const std::shared_ptr& hiveConfig) - : HiveDataSink( + : IcebergDataSink( std::move(inputType), insertTableHandle, connectorQueryCtx, commitStrategy, hiveConfig, + [&insertTableHandle]() { + const auto& inputColumns = insertTableHandle->inputColumns(); + const auto& partitionSpec = insertTableHandle->partitionSpec(); + std::unordered_map partitionKeyMap; + for (auto i = 0; i < inputColumns.size(); ++i) { + if (inputColumns[i]->isPartitionKey()) { + partitionKeyMap[inputColumns[i]->name()] = i; + } + } + std::vector channels; + channels.reserve(partitionSpec->fields.size()); + for (const auto& field : partitionSpec->fields) { + if (auto it = partitionKeyMap.find(field.name); + it != partitionKeyMap.end()) { + channels.push_back(it->second); + } + } + return channels; + }(), + [&insertTableHandle]() { + std::vector channels( + insertTableHandle->inputColumns().size()); + std::iota(channels.begin(), channels.end(), 0); + return channels; + }()) {} + +IcebergDataSink::IcebergDataSink( + RowTypePtr inputType, + IcebergInsertTableHandlePtr insertTableHandle, + const ConnectorQueryCtx* connectorQueryCtx, + CommitStrategy commitStrategy, + const std::shared_ptr& hiveConfig, + const std::vector& partitionChannels, + const std::vector& dataChannels) + : HiveDataSink( + inputType, + insertTableHandle, + connectorQueryCtx, + commitStrategy, + hiveConfig, 0, - nullptr) {} + nullptr, + partitionChannels, + dataChannels, + !partitionChannels.empty() + ? std::make_unique( + partitionChannels, + hiveConfig->maxPartitionsPerWriters( + connectorQueryCtx->sessionProperties()), + connectorQueryCtx->memoryPool(), + insertTableHandle->columnTransforms(), + hiveConfig->isPartitionPathAsLowerCase( + connectorQueryCtx->sessionProperties())) + : nullptr), + fanoutEnabled_( + hiveConfig_->fanoutEnabled(connectorQueryCtx_->sessionProperties())), + currentWriterId_(0) { + if (isPartitioned()) { + partitionData_.resize(maxOpenWriters_); + } + const auto& inputColumns = insertTableHandle_->inputColumns(); -std::vector IcebergDataSink::commitMessage() const { - std::vector commitTasks; - commitTasks.reserve(writerInfo_.size()); + std::function + buildNestedField = [&](const IcebergNestedField& f, + const TypePtr& type, + bool skipBounds) -> IcebergDataFileStatsSettings { + VELOX_CHECK_NOT_NULL(type, "Input column type cannot be null."); + bool currentSkipBounds = skipBounds || type->isMap() || type->isArray(); + IcebergDataFileStatsSettings field(f.id, currentSkipBounds); + if (!f.children.empty()) { + VELOX_CHECK_EQ(f.children.size(), type->size()); + field.children.reserve(f.children.size()); + if (type->isRow()) { + auto rowType = asRowType(type); + for (size_t i = 0; i < f.children.size(); ++i) { + field.children.push_back( + std::make_unique(buildNestedField( + f.children[i], rowType->childAt(i), currentSkipBounds))); + } + } else if (type->isArray()) { + auto arrayType = type->asArray(); + field.children.push_back( + std::make_unique(buildNestedField( + f.children[0], arrayType.elementType(), currentSkipBounds))); + } else if (type->isMap()) { + auto mapType = type->asMap(); + for (size_t i = 0; i < f.children.size(); ++i) { + field.children.push_back( + std::make_unique(buildNestedField( + f.children[i], mapType.childAt(i), currentSkipBounds))); + } + } + } + return field; + }; + statsSettings_ = std::make_shared< + std::vector>>(); + for (const auto& columnHandle : inputColumns) { + auto icebergColumnHandle = + std::dynamic_pointer_cast(columnHandle); + VELOX_CHECK_NOT_NULL(icebergColumnHandle, "Invalid IcebergColumnHandle."); + statsSettings_->push_back( + std::make_unique(buildNestedField( + icebergColumnHandle->nestedField(), + icebergColumnHandle->dataType(), + false))); + } + + icebergStatsCollector_ = + std::make_unique(statsSettings_); + + const auto& sortedBy = insertTableHandle->sortedBy(); + if (!sortedBy.empty()) { + sortColumnIndices_.reserve(sortedBy.size()); + sortCompareFlags_.reserve(sortedBy.size()); + for (auto i = 0; i < sortedBy.size(); ++i) { + auto columnIndex = + inputType_->getChildIdxIfExists(sortedBy[i].sortColumn()); + if (columnIndex.has_value()) { + sortColumnIndices_.push_back(columnIndex.value()); + sortCompareFlags_.push_back( + {sortedBy[i].sortOrder().isNullsFirst(), + sortedBy[i].sortOrder().isAscending(), + false, + CompareFlags::NullHandlingMode::kNullAsValue}); + } + } + } +} + +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); @@ -75,20 +314,324 @@ std::vector IcebergDataSink::commitMessage() const { // TODO: 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", (fs::path(info->writerParameters.writeDirectory()) / - info->writerParameters.writeFileName()).string()) - ("fileSizeInBytes", ioStats_.at(i)->rawBytesWritten()) - ("metrics", - folly::dynamic::object("recordCount", info->numWrittenRows)) - ("partitionSpecJson", icebergInsertTableHandle->partitionSpec()->specId) - ("fileFormat", "PARQUET") - ("content", "DATA"); + folly::dynamic commitData = + folly::dynamic::object + ("path", + (fs::path(info->writerParameters.writeDirectory()) / + info->writerParameters.writeFileName()).string()) + ("fileSizeInBytes", ioStats_.at(i)->rawBytesWritten()) + ("metrics", dataFileStats_[i]->toJson()) + ("splitOffsets", dataFileStats_[i]->splitOffsetsAsJson()) + // Sort order evolution is not supported. Set default id to 1. + ("sortOrderId", 1) + ("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) { + 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; + } + buildPartitionData(index); + } + + 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)); + } + } +} + +void IcebergDataSink::computePartition(const RowVectorPtr& input) { + VELOX_CHECK(isPartitioned()); + partitionIdGenerator_->run(input, partitionIds_); +} + +void IcebergDataSink::appendData(RowVectorPtr input) { + checkRunning(); + if (!isPartitioned()) { + const auto index = ensureWriter(HiveWriterId::unpartitionedId()); + write(index, input); + return; + } + + computePartition(input); + + if (fanoutEnabled_) { + splitInputRowsAndEnsureWriters(input); + + for (auto index = 0; index < writers_.size(); ++index) { + const vector_size_t partitionSize = partitionSizes_[index]; + if (partitionSize == 0) { + continue; + } + + const RowVectorPtr writerInput = partitionSize == input->size() + ? input + : exec::wrap(partitionSize, partitionRows_[index], input); + write(index, writerInput); + } + } else { // Clustered mode. + std::fill(partitionSizes_.begin(), partitionSizes_.end(), 0); + const auto numRows = input->size(); + uint32_t index = 0; + for (auto row = 0; row < numRows; ++row) { + auto id = getIcebergWriterId(row); + index = ensureWriter(id); + if (currentWriterId_ != index) { + clusteredWrite(input, currentWriterId_); + closeWriter(currentWriterId_); + completedWriterIds_.insert(currentWriterId_); + VELOX_USER_CHECK_EQ( + completedWriterIds_.count(index), + 0, + "{}", + kNotClusteredRowsErrorMsg); + currentWriterId_ = index; + } + updatePartitionRows(index, numRows, row); + buildPartitionData(index); + } + clusteredWrite(input, index); + } +} + +void IcebergDataSink::buildPartitionData(int32_t index) { + std::vector partitionValues(partitionChannels_.size()); + auto icebergPartitionIdGenerator = + dynamic_cast( + partitionIdGenerator_.get()); + VELOX_CHECK_NOT_NULL(icebergPartitionIdGenerator); + const RowVectorPtr transformedValues = + icebergPartitionIdGenerator->partitionValues(); + for (auto i = 0; i < partitionChannels_.size(); ++i) { + auto block = transformedValues->childAt(i); + if (block->isNullAt(index)) { + partitionValues[i] = nullptr; + } else { + DecodedVector decoded(*block); + partitionValues[i] = VELOX_DYNAMIC_SCALAR_TYPE_DISPATCH( + extractPartitionValue, block->typeKind(), &decoded, index); + } + } + partitionData_[index] = partitionValues; +} + +void IcebergDataSink::clusteredWrite(RowVectorPtr input, int32_t writerIdx) { + if (partitionSizes_[writerIdx] != 0) { + VELOX_CHECK_NOT_NULL(partitionRows_[writerIdx]); + partitionRows_[writerIdx]->setSize( + partitionSizes_[writerIdx] * sizeof(vector_size_t)); + } + const vector_size_t partitionSize = partitionSizes_[writerIdx]; + const RowVectorPtr writerInput = partitionSize == input->size() + ? input + : exec::wrap(partitionSize, partitionRows_[writerIdx], input); + write(writerIdx, writerInput); +} + +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 HiveWriterId{partitionId, std::nullopt}; +} + +std::shared_ptr +IcebergDataSink::createWriterOptions() const { + auto options = HiveDataSink::createWriterOptions(); + options->fileStatsCollector = icebergStatsCollector_.get(); + +#ifdef VELOX_ENABLE_PARQUET + + auto parquetOptions = + std::dynamic_pointer_cast(options); + VELOX_CHECK_NOT_NULL(parquetOptions); + std::function + convertField = + [&convertField](const IcebergDataFileStatsSettings& icebergField) + -> parquet::ParquetFieldId { + parquet::ParquetFieldId parquetField; + parquetField.fieldId = icebergField.fieldId; + for (const auto& child : icebergField.children) { + parquetField.children.push_back(convertField(*child)); + } + return parquetField; + }; + + std::vector parquetFieldIds; + for (const auto& setting : *statsSettings_) { + const auto* icebergSetting = + static_cast(setting.get()); + parquetFieldIds.push_back(convertField(*icebergSetting)); + } + + parquetOptions->parquetFieldIds = + std::make_shared>(parquetFieldIds); + parquetOptions->parquetWriteTimestampTimeZone = std::nullopt; + parquetOptions->parquetWriteTimestampUnit = TimestampPrecision::kMicroseconds; + +#endif + + return options; +} + +std::optional IcebergDataSink::getPartitionName( + const HiveWriterId& id) const { + std::optional partitionName; + if (isPartitioned()) { + partitionName = + partitionIdGenerator_->partitionName(id.partitionId.value()); + } + return partitionName; +} + +void IcebergDataSink::closeInternal() { + VELOX_CHECK_NE(state_, State::kRunning); + VELOX_CHECK_NE(state_, State::kFinishing); + + common::testutil::TestValue::adjust( + "facebook::velox::connector::hive::IcebergDataSink::closeInternal", this); + + if (state_ == State::kClosed) { + for (int i = 0; i < writers_.size(); ++i) { + if (writers_[i]) { + WRITER_NON_RECLAIMABLE_SECTION_GUARD(i); + writers_[i]->close(); + dataFileStats_.push_back(writers_[i]->dataFileStats()); + } + } + } else { + for (int i = 0; i < writers_.size(); ++i) { + WRITER_NON_RECLAIMABLE_SECTION_GUARD(i); + writers_[i]->abort(); + } + } +} + +void IcebergDataSink::closeWriter(int32_t index) { + common::testutil::TestValue::adjust( + "facebook::velox::connector::hive::iceberg::IcebergDataSink::closeWriter", + this); + + if (writers_[index]) { + WRITER_NON_RECLAIMABLE_SECTION_GUARD(index); + if (sortWrite()) { + finishWriter(index); + } + writers_[index]->close(); + dataFileStats_.push_back(writers_[index]->dataFileStats()); + writers_[index] = nullptr; + } +} + +bool IcebergDataSink::finishWriter(int32_t index) { + if (!sortWrite()) { + return true; + } + + if (writers_[index]) { + const uint64_t startTimeMs = getCurrentTimeMs(); + if (!writers_[index]->finish()) { + return false; + } + if (getCurrentTimeMs() - startTimeMs > sortWriterFinishTimeSliceLimitMs_) { + return false; + } + } + return true; +} + +bool IcebergDataSink::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] && !writers_[i]->finish()) { + return false; + } + if (getCurrentTimeMs() - startTimeMs > sortWriterFinishTimeSliceLimitMs_) { + return false; + } + } + return true; +} + +std::unique_ptr +IcebergDataSink::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( + 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_); +} + +IcebergSortingColumn::IcebergSortingColumn( + const std::string& sortColumn, + const core::SortOrder& sortOrder) + : sortColumn_(sortColumn), sortOrder_(sortOrder) { + VELOX_USER_CHECK(!sortColumn_.empty(), "iceberg sort column must be set."); +} + +const std::string& IcebergSortingColumn::sortColumn() const { + return sortColumn_; +} + +const core::SortOrder& IcebergSortingColumn::sortOrder() const { + return sortOrder_; +} + +folly::dynamic IcebergSortingColumn::serialize() const { + VELOX_UNREACHABLE("Unexpected code path, implement serialize() first."); +} + } // namespace facebook::velox::connector::hive::iceberg diff --git a/velox/connectors/hive/iceberg/IcebergDataSink.h b/velox/connectors/hive/iceberg/IcebergDataSink.h index 5b6c46d2d27f..c27613d00355 100644 --- a/velox/connectors/hive/iceberg/IcebergDataSink.h +++ b/velox/connectors/hive/iceberg/IcebergDataSink.h @@ -17,11 +17,31 @@ #pragma once #include "velox/connectors/hive/HiveDataSink.h" -#include "velox/connectors/hive/iceberg/PartitionSpec.h" +#include "velox/connectors/hive/iceberg/DataFileStatsCollector.h" +#include "velox/connectors/hive/iceberg/IcebergColumnHandle.h" +#include "velox/connectors/hive/iceberg/TransformFactory.h" +#include "velox/connectors/hive/iceberg/Transforms.h" namespace facebook::velox::connector::hive::iceberg { -/// Represents a request for Iceberg write. +class IcebergSortingColumn : public ISerializable { + public: + IcebergSortingColumn( + const std::string& sortColumn, + const core::SortOrder& sortOrder); + + const std::string& sortColumn() const; + + const core::SortOrder& sortOrder() const; + + folly::dynamic serialize() const override; + + private: + const std::string sortColumn_; + const core::SortOrder sortOrder_; +}; + +// Represents a request for Iceberg write. class IcebergInsertTableHandle final : public HiveInsertTableHandle { public: /// @param inputColumns Columns from the table schema to write. @@ -33,45 +53,36 @@ class IcebergInsertTableHandle final : public HiveInsertTableHandle { /// @param locationHandle Contains the target location information including: /// - Base directory path where data files will be written. /// - File naming scheme and temporary directory paths. - /// @param tableStorageFormat File format to use for writing data files. - /// @param partitionSpec Optional partition specification defining how to - /// partition the data. If nullptr, the table is unpartitioned and all data - /// is written to a single directory. /// @param compressionKind Optional compression to apply to data files. /// @param serdeParameters Additional serialization/deserialization parameters /// for the file format. IcebergInsertTableHandle( - std::vector inputColumns, + std::vector inputColumns, LocationHandlePtr locationHandle, - dwio::common::FileFormat tableStorageFormat, - IcebergPartitionSpecPtr partitionSpec, + std::shared_ptr partitionSpec, + memory::MemoryPool* pool, + dwio::common::FileFormat tableStorageFormat = + dwio::common::FileFormat::PARQUET, + const std::vector& sortedBy = {}, std::optional compressionKind = {}, const std::unordered_map& serdeParameters = {}); -#ifdef VELOX_ENABLE_BACKWARD_COMPATIBILITY - IcebergInsertTableHandle( - std::vector inputColumns, - LocationHandlePtr locationHandle, - dwio::common::FileFormat tableStorageFormat, - std::optional compressionKind = {}, - const std::unordered_map& serdeParameters = {}) - : IcebergInsertTableHandle( - inputColumns, - locationHandle, - tableStorageFormat, - nullptr, - compressionKind, - serdeParameters) {} -#endif - - /// Returns the Iceberg partition specification that defines how the table - /// is partitioned. - const IcebergPartitionSpecPtr& partitionSpec() const { + std::shared_ptr partitionSpec() const { return partitionSpec_; } + const std::vector>& columnTransforms() const { + return columnTransforms_; + } + + const std::vector& sortedBy() const { + return sortedBy_; + } + private: - const IcebergPartitionSpecPtr partitionSpec_; + const std::shared_ptr partitionSpec_; + const std::vector> columnTransforms_; + const std::vector sortedBy_; }; using IcebergInsertTableHandlePtr = @@ -86,6 +97,12 @@ class IcebergDataSink : public HiveDataSink { CommitStrategy commitStrategy, const std::shared_ptr& hiveConfig); + void appendData(RowVectorPtr input) override; + + const std::vector>& + dataFileStats() const { + return dataFileStats_; + } /// Generates Iceberg-specific commit messages for all writers containing /// metadata about written files. Creates a JSON object for each writer /// in the format expected by Presto and Spark for Iceberg tables. @@ -107,6 +124,58 @@ class IcebergDataSink : public HiveDataSink { /// @return Vector of JSON strings, one per writer, formatted according to /// Presto and Spark Iceberg commit protocol. std::vector commitMessage() const override; + + bool finish() override; + + private: + IcebergDataSink( + RowTypePtr inputType, + IcebergInsertTableHandlePtr insertTableHandle, + const ConnectorQueryCtx* connectorQueryCtx, + CommitStrategy commitStrategy, + const std::shared_ptr& hiveConfig, + const std::vector& partitionChannels, + const std::vector& dataChannels); + + void splitInputRowsAndEnsureWriters(RowVectorPtr input) override; + + void computePartition(const RowVectorPtr& input); + + HiveWriterId getIcebergWriterId(size_t row) const; + + std::shared_ptr createWriterOptions() + const override; + + std::optional getPartitionName( + const HiveWriterId& id) const override; + + std::unique_ptr maybeCreateBucketSortWriter( + std::unique_ptr writer) override; + + void buildPartitionData(int32_t index); + + void clusteredWrite(RowVectorPtr input, int32_t writerIdx); + + void closeInternal() override; + + void closeWriter(int32_t index); + + bool finishWriter(int32_t index); + + // Below are structures for partitions from all inputs. partitionData_ + // is indexed by partitionId. + std::vector> partitionData_; + + std::vector> dataFileStats_; + std::shared_ptr< + std::vector>> + statsSettings_; + std::unique_ptr icebergStatsCollector_; + + // Below are structures for clustered mode writer. + const bool fanoutEnabled_; + uint32_t currentWriterId_; + std::unordered_set completedWriterIds_; }; } // 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/IcebergPartitionIdGenerator.cpp b/velox/connectors/hive/iceberg/IcebergPartitionIdGenerator.cpp new file mode 100644 index 000000000000..c9b22c3354bb --- /dev/null +++ b/velox/connectors/hive/iceberg/IcebergPartitionIdGenerator.cpp @@ -0,0 +1,209 @@ +/* + * 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/IcebergPartitionIdGenerator.h" + +#include "velox/connectors/hive/HivePartitionUtil.h" +#include "velox/connectors/hive/iceberg/Transforms.h" + +namespace facebook::velox::connector::hive::iceberg { + +namespace { + +template +std::pair makePartitionKeyValueString( + const BaseVector* partitionVector, + vector_size_t row, + const std::string& name, + const std::shared_ptr& columnTransform) { + using T = typename TypeTraits::NativeType; + if (partitionVector->as>()->isNullAt(row)) { + return std::make_pair(name, "null"); + } + + return std::make_pair( + name, + columnTransform->toHumanString( + partitionVector->as>()->valueAt(row))); +} + +// Iceberg spec requires URL encoding in the partition path. +// This function matches java.net.URLEncoder.encode(string, "UTF-8"). +std::string urlEncode(const StringView& data) { + std::ostringstream ret; + + for (unsigned char c : data) { + // These characters are not encoded in Java's URLEncoder. + if (std::isalnum(c) || c == '-' || c == '_' || c == '.' || c == '*') { + ret << c; + } else if (c == ' ') { + ret << '+'; + } else { + // All other characters are percent-encoded. + ret << fmt::format("%{:02X}", c); + } + } + + return ret.str(); +} + +} // namespace + +IcebergPartitionIdGenerator::IcebergPartitionIdGenerator( + std::vector partitionChannels, + uint32_t maxPartitions, + memory::MemoryPool* pool, + const std::vector>& columnTransforms, + bool partitionPathAsLowerCase) + : PartitionIdGenerator( + partitionChannels, + maxPartitions, + pool, + partitionPathAsLowerCase), + pool_(pool), + columnTransforms_(columnTransforms) { + VELOX_USER_CHECK_GT( + columnTransforms_.size(), 0, "columnTransforms_ cannot be null"); + std::vector partitionKeyTypes; + std::vector partitionKeyNames; + column_index_t i{0}; + for (const auto& columnTransform : columnTransforms_) { + hashers_.emplace_back( + exec::VectorHasher::create(columnTransform->resultType(), i++)); + VELOX_USER_CHECK( + hashers_.back()->typeSupportsValueIds(), + "Unsupported partition type: {}.", + columnTransform->resultType()->toString()); + + partitionKeyTypes.emplace_back(columnTransform->resultType()); + std::string key = + columnTransform->transformType() == TransformType::kIdentity + ? columnTransform->sourceColumnName() + : fmt::format( + "{}_{}", + columnTransform->sourceColumnName(), + columnTransform->name()); + partitionKeyNames.emplace_back(std::move(key)); + } + partitionValues_ = BaseVector::create( + ROW(std::move(partitionKeyNames), std::move(partitionKeyTypes)), + maxPartitions, + pool_); + for (auto& key : partitionValues_->children()) { + key->resize(maxPartitions); + } +} + +void IcebergPartitionIdGenerator::savePartitionValues( + uint32_t partitionId, + const RowVectorPtr& input, + vector_size_t row) { + for (auto i = 0; i < partitionChannels_.size(); ++i) { + partitionValues_->childAt(i)->copy( + input->childAt(i).get(), partitionId, row, 1); + } +} + +void IcebergPartitionIdGenerator::run( + const RowVectorPtr& input, + raw_vector& result) { + const auto numRows = input->size(); + result.resize(numRows); + std::vector columns; + std::vector names; + std::vector types; + const int32_t transformCount = columnTransforms_.size(); + columns.reserve(transformCount); + names.reserve(transformCount); + types.reserve(transformCount); + for (auto i = 0; i < transformCount; i++) { + names.emplace_back(columnTransforms_[i]->sourceColumnName()); + types.emplace_back(columnTransforms_[i]->resultType()); + columns.emplace_back( + columnTransforms_[i]->apply(input->childAt(partitionChannels_[i]))); + } + const auto rowVector = std::make_shared( + pool_, + ROW(std::move(names), std::move(types)), + nullptr, + numRows, + columns); + + // Compute value IDs using VectorHashers and store these in 'result'. + computeValueIds(rowVector, result); + + // Convert value IDs in 'result' into partition IDs using partitionIds + // mapping. Update 'result' in place. + for (auto i = 0; i < numRows; ++i) { + auto valueId = result[i]; + if (auto it = partitionIds_.find(valueId); 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, rowVector, i); + result[i] = nextPartitionId; + } + } +} + +std::vector> +IcebergPartitionIdGenerator::extractPartitionKeyValues( + const RowVectorPtr& partitionsVector, + vector_size_t row) const { + std::vector> partitionKeyValues; + VELOX_DCHECK_EQ( + partitionsVector->childrenSize(), + columnTransforms_.size(), + "Partition values and partition transform does not match."); + for (auto i = 0; i < partitionsVector->childrenSize(); i++) { + partitionKeyValues.push_back(VELOX_DYNAMIC_SCALAR_TYPE_DISPATCH( + makePartitionKeyValueString, + partitionsVector->childAt(i)->typeKind(), + partitionsVector->childAt(i)->loadedVector(), + row, + asRowType(partitionsVector->type())->nameOf(i), + columnTransforms_[i])); + } + return partitionKeyValues; +} + +std::string IcebergPartitionIdGenerator::partitionName( + uint64_t partitionId) const { + auto keyValues = extractPartitionKeyValues(partitionValues_, partitionId); + std::ostringstream ret; + + for (auto& [key, value] : keyValues) { + if (ret.tellp() > 0) { + ret << '/'; + } + + if (partitionPathAsLowerCase_) { + folly::toLowerAscii(key); + } + ret << urlEncode(key.data()) << '=' << urlEncode(value.data()); + } + + return ret.str(); +} + +} // namespace facebook::velox::connector::hive::iceberg diff --git a/velox/connectors/hive/iceberg/IcebergPartitionIdGenerator.h b/velox/connectors/hive/iceberg/IcebergPartitionIdGenerator.h new file mode 100644 index 000000000000..1f070bd6866e --- /dev/null +++ b/velox/connectors/hive/iceberg/IcebergPartitionIdGenerator.h @@ -0,0 +1,62 @@ +/* + * 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/hive/PartitionIdGenerator.h" +#include "velox/connectors/hive/iceberg/IcebergDataSink.h" + +namespace facebook::velox::connector::hive::iceberg { +class IcebergPartitionIdGenerator : public PartitionIdGenerator { + public: + IcebergPartitionIdGenerator( + std::vector partitionChannels, + uint32_t maxPartitions, + memory::MemoryPool* pool, + const std::vector>& columnTransforms, + 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) override; + + /// 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 override; + + /// Return the partition values for all partitions. + RowVectorPtr partitionValues() const { + return partitionValues_; + } + + private: + void savePartitionValues( + uint32_t partitionId, + const RowVectorPtr& input, + vector_size_t row) override; + + std::vector> extractPartitionKeyValues( + const RowVectorPtr& partitionsVector, + vector_size_t row) const; + + memory::MemoryPool* pool_; + const std::vector> columnTransforms_; +}; + +} // namespace facebook::velox::connector::hive::iceberg diff --git a/velox/connectors/hive/iceberg/IcebergSplitReader.cpp b/velox/connectors/hive/iceberg/IcebergSplitReader.cpp index 76b4c1fbacf1..71a24927872d 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, @@ -62,20 +70,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_, + ioExecutor_, + 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::nullopt); - 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) { @@ -93,12 +162,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(); @@ -138,6 +211,31 @@ uint64_t IcebergSplitReader::next(uint64_t size, VectorPtr& output) { 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); + } + + if (rowsScanned == 0) { + scanSpec_->deleteTempNodes(); + } + return rowsScanned; } diff --git a/velox/connectors/hive/iceberg/IcebergSplitReader.h b/velox/connectors/hive/iceberg/IcebergSplitReader.h index 1b56aa077433..4edad7dac4d7 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, @@ -49,6 +52,8 @@ class IcebergSplitReader : public SplitReader { uint64_t next(uint64_t size, VectorPtr& output) override; + std::shared_ptr baseFileSchema(); + private: /// Adapts the data file schema to match the table schema expected by the /// query. @@ -101,5 +106,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/Murmur3.cpp b/velox/connectors/hive/iceberg/Murmur3.cpp new file mode 100644 index 000000000000..a6558367b4fa --- /dev/null +++ b/velox/connectors/hive/iceberg/Murmur3.cpp @@ -0,0 +1,82 @@ +/* + * 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/Murmur3.h" +#include "velox/type/DecimalUtil.h" +#include "velox/type/HugeInt.h" + +namespace facebook::velox::connector::hive::iceberg { + +int32_t Murmur3Hash32::hash(const char* const input, size_t len) { + uint32_t h1{kDefaultSeed}; + uint32_t k1{0}; + const uint8_t* data = reinterpret_cast(input); + const size_t nblocks = len / 4; + + // Body. + for (size_t i = 0; i < nblocks; i++) { + uint32_t k1 = *reinterpret_cast(data + i * 4); + k1 = mixK1(k1); + h1 = mixH1(h1, k1); + } + + k1 = 0; + data = data + nblocks * 4; + + // Tail. + switch (len & 3) { + case 3: + k1 ^= (static_cast(data[2])) << 16; + [[fallthrough]]; + case 2: + k1 ^= (static_cast(data[1])) << 8; + [[fallthrough]]; + case 1: + k1 ^= data[0]; + k1 = mixK1(k1); + h1 ^= k1; + }; + + // Finalization. + return fmix32(h1, len); +} + +int32_t Murmur3Hash32::hash(const StringView& value) { + return hash(value.data(), value.size()); +} + +int32_t Murmur3Hash32::hash(uint64_t value) { + auto h1 = kDefaultSeed; + const auto low = static_cast(value & 0xFFFFFFFF); + const auto high = static_cast((value >> 32) & 0xFFFFFFFF); + + auto k1 = mixK1(low); + h1 = mixH1(h1, k1); + + k1 = mixK1(high); + h1 = mixH1(h1, k1); + + return fmix32(h1, sizeof(uint64_t)); +} + +int32_t Murmur3Hash32::hashDecimal(int128_t value) { + char bytes[16]; + const auto length = DecimalUtil::getByteArrayLength(value); + DecimalUtil::toByteArray(value, bytes); + return hash(bytes, length); +} + +} // namespace facebook::velox::connector::hive::iceberg diff --git a/velox/connectors/hive/iceberg/Murmur3.h b/velox/connectors/hive/iceberg/Murmur3.h new file mode 100644 index 000000000000..7b0d3fff81ea --- /dev/null +++ b/velox/connectors/hive/iceberg/Murmur3.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 "velox/type/HugeInt.h" +#include "velox/type/StringView.h" + +namespace facebook::velox::connector::hive::iceberg { +constexpr uint32_t kDefaultSeed = 0; +constexpr uint32_t kC1 = 0xCC9E2D51; +constexpr uint32_t kC2 = 0x1B873593; + +class Murmur3Hash32 final { + public: + static int32_t hash(uint64_t value); + + static int32_t hash(const StringView& value); + + static int32_t hash(const char* const data, size_t length); + + static int32_t hashDecimal(int128_t value); + + private: + FOLLY_ALWAYS_INLINE static uint32_t mixK1(uint32_t k1) { + k1 *= kC1; + k1 = ((k1) << (15)) | ((k1) >> (32 - (15))); + k1 *= kC2; + return k1; + } + + FOLLY_ALWAYS_INLINE static uint32_t mixH1(uint32_t h1, uint32_t k1) { + h1 ^= k1; + h1 = ((h1) << (13)) | ((h1) >> (32 - (13))); + h1 = h1 * 5 + 0xE6546B64; + return h1; + } + + FOLLY_ALWAYS_INLINE static uint32_t fmix32(uint32_t h1, size_t length) { + h1 ^= length; + h1 ^= h1 >> 16; + h1 *= 0x85EBCA6B; + h1 ^= h1 >> 13; + h1 *= 0xC2B2AE35; + h1 ^= h1 >> 16; + return h1; + } +}; + +} // namespace facebook::velox::connector::hive::iceberg diff --git a/velox/connectors/hive/iceberg/PartitionSpec.cpp b/velox/connectors/hive/iceberg/PartitionSpec.cpp index 4c9fae472ed0..6f146c2b91e0 100644 --- a/velox/connectors/hive/iceberg/PartitionSpec.cpp +++ b/velox/connectors/hive/iceberg/PartitionSpec.cpp @@ -16,61 +16,10 @@ #include "velox/connectors/hive/iceberg/PartitionSpec.h" -#include "velox/functions/prestosql/types/TimestampWithTimeZoneType.h" - namespace facebook::velox::connector::hive::iceberg { namespace { -TransformCategory getTransformCategory(TransformType transformType) { - switch (transformType) { - case TransformType::kIdentity: - return TransformCategory::kIdentity; - case TransformType::kYear: - case TransformType::kMonth: - case TransformType::kDay: - case TransformType::kHour: - return TransformCategory::kTemporal; - case TransformType::kBucket: - return TransformCategory::kBucket; - case TransformType::kTruncate: - return TransformCategory::kTruncate; - default: - VELOX_UNREACHABLE("Unknown transform type"); - } -} - -bool isValidPartitionType(const TypePtr& type) { - return !( - type->isRow() || type->isArray() || type->isMap() || type->isDouble() || - type->isReal() || isTimestampWithTimeZoneType(type)); -} - -bool canTransform(TransformType transformType, const TypePtr& type) { - switch (transformType) { - case TransformType::kIdentity: - return type->isTinyint() || type->isSmallint() || type->isInteger() || - type->isBigint() || type->isBoolean() || type->isDecimal() || - type->isDate() || type->isTimestamp() || type->isVarchar() || - type->isVarbinary(); - case TransformType::kYear: - case TransformType::kMonth: - case TransformType::kDay: - return type->isDate() || type->isTimestamp(); - case TransformType::kHour: - return type->isTimestamp(); - case TransformType::kBucket: - return type->isInteger() || type->isBigint() || type->isDecimal() || - type->isVarchar() || type->isVarbinary() || type->isDate() || - type->isTimestamp(); - case TransformType::kTruncate: - return type->isInteger() || type->isBigint() || type->isDecimal() || - type->isVarchar() || type->isVarbinary(); - default: - VELOX_UNREACHABLE("Unsupported partition transform type."); - } -} - const auto& transformTypeNames() { static const folly::F14FastMap kTransformNames = { @@ -80,78 +29,12 @@ const auto& transformTypeNames() { {TransformType::kMonth, "month"}, {TransformType::kYear, "year"}, {TransformType::kBucket, "bucket"}, - {TransformType::kTruncate, "trunc"}, - }; + {TransformType::kTruncate, "trunc"}}; return kTransformNames; } -const auto& transformCategoryNames() { - static const folly::F14FastMap - kTransformCategoryNames = { - {TransformCategory::kIdentity, "Identity"}, - {TransformCategory::kBucket, "Bucket"}, - {TransformCategory::kTruncate, "Truncate"}, - {TransformCategory::kTemporal, "Temporal"}, - }; - return kTransformCategoryNames; -} - } // namespace VELOX_DEFINE_ENUM_NAME(TransformType, transformTypeNames); -VELOX_DEFINE_ENUM_NAME(TransformCategory, transformCategoryNames); - -void IcebergPartitionSpec::checkCompatibility() const { - folly::F14FastMap> - columnTransforms; - - for (const auto& field : fields) { - const auto& type = field.type; - const auto& name = field.name; - VELOX_USER_CHECK( - isValidPartitionType(type), - "Type is not supported as a partition column: {}", - type->name()); - - VELOX_USER_CHECK( - canTransform(field.transformType, type), - "Transform is not supported for partition column. Column: '{}', Type: '{}', Transform: '{}'.", - name, - type->name(), - TransformTypeName::toName(field.transformType)); - - columnTransforms[name].emplace_back(field.transformType); - } - - // Check for duplicate transform categories per column. - std::vector errors; - for (const auto& [columnName, transforms] : columnTransforms) { - folly::F14FastSet seenCategories; - for (const auto& transform : transforms) { - auto category = getTransformCategory(transform); - if (!seenCategories.insert(category).second) { - std::vector transformNames; - for (const auto& t : transforms) { - transformNames.emplace_back( - std::string(TransformTypeName::toName(t))); - } - errors.emplace_back( - fmt::format( - "Column: '{}', Category: {}, Transforms: [{}]", - columnName, - TransformCategoryName::toName(category), - folly::join(", ", transformNames))); - break; - } - } - } - - VELOX_USER_CHECK( - errors.empty(), - "Multiple transforms of the same category on a column are not allowed. " - "Each transform category can appear at most once per column. {}", - folly::join("; ", errors)); -} - } // namespace facebook::velox::connector::hive::iceberg diff --git a/velox/connectors/hive/iceberg/PartitionSpec.h b/velox/connectors/hive/iceberg/PartitionSpec.h index 10d489fd61cc..915538c06b66 100644 --- a/velox/connectors/hive/iceberg/PartitionSpec.h +++ b/velox/connectors/hive/iceberg/PartitionSpec.h @@ -16,130 +16,48 @@ #pragma once +#include "velox/common/Enums.h" #include "velox/type/Type.h" namespace facebook::velox::connector::hive::iceberg { -/// Partition transform types. -/// Defines how source column values are converted into partition keys. -/// See https://iceberg.apache.org/spec/#partition-transforms. enum class TransformType { - /// Use the source value as-is (no transformation). kIdentity, - /// Extract a timestamp hour, as hours from 1970-01-01 00:00:00. kHour, - /// Extract a date or timestamp day, as days from 1970-01-01. kDay, - /// Extract a date or timestamp month, as months from 1970-01. kMonth, - /// Extract a date or timestamp year, as years from 1970. kYear, - /// Hash the value into N buckets for even distribution. Requires an integer - /// parameter specifying the bucket count. kBucket, - /// Truncate strings or numbers to a specified width. Requires an integer - /// parameter specifying the truncate width. kTruncate }; VELOX_DECLARE_ENUM_NAME(TransformType); -/// A single column can be used to produce multiple partition keys, but with -/// following restrictions: -/// - Transforms are organized into 4 categories: Identity, Temporal, -/// Bucket, and Truncate. -/// - Each category can appear at most once per column. -/// - Sample valid specs on same column: ['truncate(a,2)', 'bucket(a,16)', 'a'] -/// or ['year(b)', 'bucket(b, 16)', 'b'] -enum class TransformCategory { - kIdentity, - /// Year/Month/Day/Hour - kTemporal, - kBucket, - kTruncate, -}; - -VELOX_DECLARE_ENUM_NAME(TransformCategory); - -/// Represents how to produce partition data for an Iceberg table. -/// -/// This structure corresponds to the Iceberg Java PartitionSpec class but -/// contains only the necessary fields for Velox. Partition keys are computed -/// by transforming columns in a table. -/// -/// The upstream engine processes this specification through the Iceberg Java -/// library to validate column types, detect duplicates, and generate the -/// partition spec that is passed to Velox. -/// -/// IMPORTANT: Iceberg spec uses field IDs to identify source columns, but -/// Velox RowType only supports matching fields by name. Therefore, Velox uses -/// the partition field name to match against the table schema column names. -/// Callers must ensure that partition field names exactly match the column -/// names in the table schema. -/// -/// The partition spec contains: -/// - Unique ID for versioning and evolution. -/// - Which source columns in current table schema to use for partitioning -/// (identified by field name, not field ID as in the Iceberg spec). -/// - What transforms to apply (identity, bucket, truncate etc.). -/// - Transform parameters (e.g., bucket count, truncate width). struct IcebergPartitionSpec { struct Field { - /// Column name as defined in table schema. This column's value is used to - /// compute partition key by applying 'transformType' transformation. - const std::string name; - - /// Column type. - const TypePtr type; - - /// Transform to apply. Callers must ensure the transform is compatible with - /// the column type. - const TransformType transformType; - - /// Optional parameter for transforms that require configuration. - const std::optional parameter; - - /// Returns the result type after applying this transform. - TypePtr resultType() const { - switch (transformType) { - case TransformType::kBucket: - case TransformType::kYear: - case TransformType::kMonth: - case TransformType::kHour: - return INTEGER(); - case TransformType::kDay: - return DATE(); - case TransformType::kIdentity: - case TransformType::kTruncate: - return type; - } - } + // The field name of this partition field as it appears in the partition + // spec. This is the original Iceberg field name, not the transformed name + // from org.apache.iceberg.PartitionField which includes the transform as a + // suffix. + std::string name; + + // The source column type. + TypePtr type; + + // 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; }; const int32_t specId; const std::vector fields; - /// Constructor with validation that: - /// - Each field's type is supported for partitioning. - /// - Each field's transform type is compatible with its data type. - /// - No transform category appears more than once per column (Identity, - /// Temporal, Bucket, and Truncate are separate categories). - /// - /// @param _specId Partition specification ID. - /// @param _fields Vector of partition fields. When empty indicates no - /// partition. - /// @throws VeloxUserError if validation fails. - IcebergPartitionSpec(int32_t _specId, std::vector _fields) - : specId(_specId), fields(std::move(_fields)) { - checkCompatibility(); - } - - private: - // Validates partition fields for correctness. - // Checks type/transform compatibility and transform combination rules. - void checkCompatibility() const; + IcebergPartitionSpec(int32_t _specId, const std::vector& _fields) + : specId(_specId), fields(_fields) {} }; -using IcebergPartitionSpecPtr = std::shared_ptr; - } // namespace facebook::velox::connector::hive::iceberg diff --git a/velox/connectors/hive/iceberg/TransformFactory.cpp b/velox/connectors/hive/iceberg/TransformFactory.cpp new file mode 100644 index 000000000000..2c45106c01bd --- /dev/null +++ b/velox/connectors/hive/iceberg/TransformFactory.cpp @@ -0,0 +1,290 @@ +/* + * 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/TransformFactory.h" + +#include "velox/functions/lib/TimeUtils.h" +#include "velox/functions/prestosql/types/TimestampWithTimeZoneType.h" +#include "velox/vector/ComplexVector.h" + +namespace facebook::velox::connector::hive::iceberg { +namespace { +int32_t epochYear(int32_t daysSinceEpoch) { + const std::tm tm = functions::getDateTime(daysSinceEpoch); + // tm_year is the number of years since 1900. + return tm.tm_year + 1900 - 1970; +} + +int32_t epochYear(Timestamp ts) { + return functions::getYear(functions::getDateTime(ts, nullptr)) - 1970; +} + +int32_t epochMonth(int32_t daysSinceEpoch) { + const std::tm tm = functions::getDateTime(daysSinceEpoch); + return (tm.tm_year + 1900 - 1970) * 12 + tm.tm_mon; +} + +int32_t epochMonth(Timestamp ts) { + const std::tm tm = functions::getDateTime(ts, nullptr); + return (tm.tm_year + 1900 - 1970) * 12 + tm.tm_mon; +} + +int32_t epochDay(int32_t daysSinceEpoch) { + return daysSinceEpoch; +} + +int32_t epochDay(Timestamp ts) { + const auto seconds = ts.getSeconds(); + return (seconds >= 0) ? seconds / Timestamp::kSecondsInDay + : ((seconds + 1) / Timestamp::kSecondsInDay) - 1; +} + +int32_t epochHour(Timestamp ts) { + const auto seconds = ts.getSeconds(); + return (seconds >= 0) ? seconds / 3600 : ((seconds + 1) / 3600) - 1; +} + +bool isValidPartitionType(TypePtr type) { + if (type->isRow() || type->isArray() || type->isMap() || + isTimestampWithTimeZoneType(type)) { + return false; + } + return true; +} + +template +std::shared_ptr createDateTimeTransform( + TransformType transformType, + const IcebergPartitionSpec::Field& field, + std::function::NativeType)> epochFunc, + memory::MemoryPool* pool) { + using NativeType = typename TypeTraits::NativeType; + VELOX_DCHECK_EQ( + true, + field.type->isDate() || field.type->isTimestamp(), + "Unsupported column type {} for transform {}", + field.type->name(), + TransformTypeName::toName(transformType)); + return std::make_shared>( + field.type, transformType, field.name, pool, epochFunc); +} + +template +std::shared_ptr createIdentityTransform( + const IcebergPartitionSpec::Field& field, + memory::MemoryPool* pool) { + using NativeType = typename TypeTraits::NativeType; + return std::make_shared>( + field.type, field.name, pool); +} + +template +std::shared_ptr createBucketTransform( + const IcebergPartitionSpec::Field& field, + int32_t count, + memory::MemoryPool* pool) { + VELOX_USER_CHECK_GT(count, 0, "Bucket count must be positive."); + using NativeType = typename TypeTraits::NativeType; + return std::make_shared>( + count, field.type, field.name, pool); +} + +template +std::shared_ptr createTruncateTransform( + const IcebergPartitionSpec::Field& field, + int32_t width, + memory::MemoryPool* pool) { + VELOX_USER_CHECK_GT(width, 0, "Truncate width must be positive."); + using NativeType = typename TypeTraits::NativeType; + return std::make_shared>( + width, field.type, field.name, pool); +} + +std::shared_ptr buildColumnTransform( + const IcebergPartitionSpec::Field& field, + memory::MemoryPool* pool) { + if (!isValidPartitionType(field.type)) { + VELOX_USER_FAIL( + fmt::format( + "Type not supported as partition column: {}.", field.type->name())); + } + switch (field.transformType) { + // Identity transform. + case TransformType::kIdentity: { + return VELOX_DYNAMIC_SCALAR_TYPE_DISPATCH( + createIdentityTransform, field.type->kind(), field, pool); + } + // Year transform. + case TransformType::kYear: { + if (field.type->isDate()) { + return createDateTimeTransform( + TransformType::kYear, + field, + [](int32_t v) { return epochYear(v); }, + pool); + } + + if (field.type->isTimestamp()) { + return createDateTimeTransform( + TransformType::kYear, + field, + [](Timestamp v) { return epochYear(v); }, + pool); + } + + VELOX_UNREACHABLE( + fmt::format( + "Unsupported column type {} for transform year.", + field.type->name())); + } + // Month transform. + case TransformType::kMonth: { + if (field.type->isDate()) { + return createDateTimeTransform( + TransformType::kMonth, + field, + [](int32_t v) { return epochMonth(v); }, + pool); + } + + if (field.type->isTimestamp()) { + return createDateTimeTransform( + TransformType::kMonth, + field, + [](Timestamp v) { return epochMonth(v); }, + pool); + } + + VELOX_UNREACHABLE( + fmt::format( + "Unsupported column type {} for transform month.", + field.type->name())); + } + // Day transform. + case TransformType::kDay: { + if (field.type->isDate()) { + return createDateTimeTransform( + TransformType::kDay, + field, + [](int32_t v) { return epochDay(v); }, + pool); + } + + if (field.type->isTimestamp()) { + return createDateTimeTransform( + TransformType::kDay, + field, + [](Timestamp v) { return epochDay(v); }, + pool); + } + + VELOX_UNREACHABLE( + fmt::format( + "Unsupported column type {} for transform day.", + field.type->name())); + } + // Hour transform. + case TransformType::kHour: { + if (field.type->isTimestamp()) { + return createDateTimeTransform( + TransformType::kHour, + field, + [](Timestamp v) { return epochHour(v); }, + pool); + } + + VELOX_UNREACHABLE( + fmt::format( + "Unsupported column type {} for transform hour.", + field.type->name())); + } + // Bucket transform. + case TransformType::kBucket: { + VELOX_USER_CHECK( + field.parameter.has_value() && field.parameter.value() > 0, + "Bucket transform requires a positive parameter."); + auto numBuckets = field.parameter.value(); + + if (field.type->isInteger() || field.type->isDate()) { + return createBucketTransform( + field, numBuckets, pool); + } + if (field.type->isBigint() || field.type->isShortDecimal()) { + return createBucketTransform(field, numBuckets, pool); + } + if (field.type->isTimestamp()) { + return createBucketTransform( + field, numBuckets, pool); + } + if (field.type->isLongDecimal()) { + return createBucketTransform( + field, numBuckets, pool); + } + if (field.type->isVarchar()) { + return createBucketTransform( + field, numBuckets, pool); + } + if (field.type->isVarbinary()) { + return createBucketTransform( + field, numBuckets, pool); + } + VELOX_UNREACHABLE( + fmt::format( + "Unsupported column type {} for transform bucket.", + field.type->name())); + } + // Truncate transform. + case TransformType::kTruncate: { + VELOX_USER_CHECK( + field.parameter.has_value() && field.parameter.value() > 0, + "Truncate transform requires a positive parameter."); + auto width = field.parameter.value(); + if (field.type->isInteger()) { + return createTruncateTransform(field, width, pool); + } + if (field.type->isBigint() || field.type->isShortDecimal()) { + return createTruncateTransform(field, width, pool); + } + if (field.type->isVarchar()) { + return createTruncateTransform(field, width, pool); + } + if (field.type->isVarbinary()) { + return createTruncateTransform(field, width, pool); + } + VELOX_UNREACHABLE( + fmt::format( + "Unsupported column type {} for transform truncate.", + field.type->name())); + } + default: + VELOX_UNREACHABLE("Unsupported transform."); + } +} + +} // namespace + +std::vector> parsePartitionTransformSpecs( + const std::vector& fields, + memory::MemoryPool* pool) { + std::vector> transforms; + transforms.reserve(fields.size()); + for (auto& field : fields) { + transforms.emplace_back(buildColumnTransform(field, pool)); + } + return transforms; +} + +} // namespace facebook::velox::connector::hive::iceberg diff --git a/velox/connectors/hive/iceberg/TransformFactory.h b/velox/connectors/hive/iceberg/TransformFactory.h new file mode 100644 index 000000000000..ea6642db7dc1 --- /dev/null +++ b/velox/connectors/hive/iceberg/TransformFactory.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/connectors/hive/iceberg/PartitionSpec.h" +#include "velox/connectors/hive/iceberg/Transforms.h" + +namespace facebook::velox::connector::hive::iceberg { + +std::vector> parsePartitionTransformSpecs( + const std::vector& fields, + memory::MemoryPool* pool); + +} // namespace facebook::velox::connector::hive::iceberg diff --git a/velox/connectors/hive/iceberg/Transforms.cpp b/velox/connectors/hive/iceberg/Transforms.cpp new file mode 100644 index 000000000000..a400270c58d7 --- /dev/null +++ b/velox/connectors/hive/iceberg/Transforms.cpp @@ -0,0 +1,226 @@ +/* + * 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/Transforms.h" + +#include "velox/connectors/hive/iceberg/Murmur3.h" +#include "velox/functions/lib/string/StringImpl.h" +#include "velox/vector/DecodedVector.h" +#include "velox/vector/FlatVector.h" + +namespace facebook::velox::connector::hive::iceberg { + +namespace { + +template +FOLLY_ALWAYS_INLINE void transformValues( + const VectorPtr& block, + const DecodedVector* decoded, + const VectorPtr& result, + ProcessFunc&& processValue) { + if (!decoded->mayHaveNulls()) { + for (auto i = 0; i < decoded->size(); ++i) { + processValue(i); + } + } else { + block->mutableNulls(block->size()); + result->setNulls(block->nulls()); + for (auto i = 0; i < decoded->size(); ++i) { + if (!decoded->isNullAt(i)) { + processValue(i); + } + } + } +} + +} // namespace + +VectorPtr Transform::transform( + const RowVectorPtr& input, + std::optional channel) const { + VectorPtr currentVector = nullptr; + if (channel.has_value()) { + currentVector = input->childAt(channel.value()); + } + VELOX_CHECK_NOT_NULL(currentVector); + return apply(currentVector); +} + +std::string Transform::toHumanString(Timestamp value) const { + TimestampToStringOptions options; + options.precision = TimestampPrecision::kMilliseconds; + options.zeroPaddingYear = true; + options.skipTrailingZeros = true; + options.leadingPositiveSign = true; + options.skipTrailingZeroSeconds = true; + return value.toString(options); +} + +template +VectorPtr IdentityTransform::apply(const VectorPtr& block) const { + if constexpr (!std::is_same_v) { + return block; + } + if (sourceType_->isVarchar()) { + return block; + } + + auto result = + BaseVector::create>(sourceType_, block->size(), pool_); + DecodedVector decoded(*block); + + auto processValue = [&](auto i) { + if constexpr (std::is_same_v) { + T value = decoded.valueAt(i); + auto encodedValue = encoding::Base64::encode(value.data(), value.size()); + result->set(i, StringView(encodedValue)); + } + }; + + transformValues(block, &decoded, result, processValue); + return result; +} + +template +VectorPtr BucketTransform::apply(const VectorPtr& block) const { + auto result = + BaseVector::create>(INTEGER(), block->size(), pool_); + + DecodedVector decoded(*block); + + auto processValue = [&](auto i) { + T value = decoded.valueAt(i); + int32_t hashValue; + if constexpr (std::is_same_v || std::is_same_v) { + if (sourceType_->isDecimal()) { + hashValue = Murmur3Hash32::hashDecimal(value); + } else { + hashValue = Murmur3Hash32::hash(value); + } + } else if constexpr (std::is_same_v) { + hashValue = Murmur3Hash32::hash(value.toMicros()); + } else { + hashValue = Murmur3Hash32::hash(value); + } + result->set(i, (hashValue & 0x7FFFFFFF) % numBuckets_); + }; + + transformValues(block, &decoded, result, processValue); + return result; +} + +template +VectorPtr TruncateTransform::apply(const VectorPtr& block) const { + auto result = + BaseVector::create>(sourceType_, block->size(), pool_); + + auto flatResult = result->template as>(); + char* rawBuffer = nullptr; + BufferPtr buffer; + if (std::is_same_v) { + if (sourceType_->isVarchar()) { + buffer = result->getBufferWithSpace(block->size() * width_); + } else { + buffer = result->getBufferWithSpace( + block->size() * encoding::Base64::calculateEncodedSize(width_)); + } + rawBuffer = buffer->asMutable() + buffer->size(); + } + + DecodedVector decoded(*block); + auto processValue = [&](auto i) { + T value = decoded.valueAt(i); + if constexpr ( + std::is_same_v || std::is_same_v || + std::is_same_v) { + flatResult->set(i, value - ((value % width_) + width_) % width_); + } else if constexpr (std::is_same_v) { + if (sourceType_->isVarchar()) { + auto length = + functions::stringImpl::cappedByteLength(value, width_); + if (StringView::isInline(length)) { + flatResult->set(i, StringView(value.data(), length)); + } else { + memcpy(rawBuffer, value.data(), length); + flatResult->setNoCopy(i, StringView(rawBuffer, length)); + rawBuffer += length; + } + } else if (sourceType_->isVarbinary()) { + auto encoded = encoding::Base64::encode( + value.data(), width_ > value.size() ? value.size() : width_); + auto length = encoded.length(); + if (StringView::isInline(length)) { + flatResult->set(i, StringView(encoded)); + } else { + memcpy(rawBuffer, encoded.data(), length); + flatResult->setNoCopy(i, StringView(rawBuffer, length)); + rawBuffer += length; + } + } + } + }; + + transformValues(block, &decoded, result, processValue); + + if constexpr (std::is_same_v) { + buffer->setSize(rawBuffer - (buffer->asMutable() + buffer->size())); + } + return result; +} + +template +VectorPtr TemporalTransform::apply(const VectorPtr& block) const { + auto result = + BaseVector::create>(INTEGER(), block->size(), pool_); + + DecodedVector decoded(*block); + auto processValue = [&](auto i) { + T value = decoded.valueAt(i); + result->set(i, epochFunc_(value)); + }; + + transformValues(block, &decoded, result, processValue); + + return result; +} + +template class IdentityTransform; +template class IdentityTransform; +template class IdentityTransform; +template class IdentityTransform; +template class IdentityTransform; +template class IdentityTransform; +template class IdentityTransform; +template class IdentityTransform; +template class IdentityTransform; +template class IdentityTransform; + +template class BucketTransform; +template class BucketTransform; +template class BucketTransform; +template class BucketTransform; +template class BucketTransform; + +template class TruncateTransform; +template class TruncateTransform; +template class TruncateTransform; +template class TruncateTransform; + +template class TemporalTransform; +template class TemporalTransform; +template class TemporalTransform; + +} // namespace facebook::velox::connector::hive::iceberg diff --git a/velox/connectors/hive/iceberg/Transforms.h b/velox/connectors/hive/iceberg/Transforms.h new file mode 100644 index 000000000000..698c0afcbc54 --- /dev/null +++ b/velox/connectors/hive/iceberg/Transforms.h @@ -0,0 +1,241 @@ +/* + * 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/encode/Base64.h" +#include "velox/connectors/hive/iceberg/PartitionSpec.h" +#include "velox/type/DecimalUtil.h" +#include "velox/type/Type.h" +#include "velox/vector/ComplexVector.h" + +namespace facebook::velox::connector::hive::iceberg { + +static constexpr int32_t kEpochYear = 1970; + +class Transform { + public: + Transform( + TypePtr type, + TransformType transformType, + const std::string& columnName, + memory::MemoryPool* pool) + : sourceType_(type), + transformType_(transformType), + sourceColumnName_(columnName), + pool_(pool) {} + + virtual ~Transform() = default; + + virtual VectorPtr apply(const VectorPtr& block) const = 0; + + virtual const TypePtr resultType() const = 0; + + TransformType transformType() const { + return transformType_; + } + + const std::string& sourceColumnName() const { + return sourceColumnName_; + } + + // Convert the transformed value to partition name + template + std::string toHumanString(T value) const { + return folly::to(value); + } + + virtual std::string toHumanString(int32_t value) const { + return folly::to(value); + } + + std::string toHumanString(int64_t value) const { + if (sourceType_->isShortDecimal()) { + return decimalToHumanString(value); + } + return folly::to(value); + } + + std::string toHumanString(int128_t value) const { + return decimalToHumanString(value); + } + + std::string toHumanString(bool value) const { + return value ? "true" : "false"; + } + + // Match Iceberg spec Java implementation + // DateTimeFormatter.ISO_LOCAL_DATE_TIME + std::string toHumanString(Timestamp value) const; + + std::string_view name() const { + return TransformTypeName::toName(transformType_); + } + + /// Applies the transform to the specified column in the input row vector. + /// @param input The input row vector containing the column to transform. + /// @param channel Optional column index (0-based) to transform. If not + /// provided, uses sourceColumnName_ to locate the column in the input. + /// @return The transformed vector. + [[nodiscard]] VectorPtr transform( + const RowVectorPtr& input, + std::optional channel = std::nullopt) const; + + private: + template + std::string decimalToHumanString(T value) const { + const auto [p, s] = getDecimalPrecisionScale(*sourceType_); + const auto maxSize = DecimalUtil::maxStringViewSize(p, s); + std::string buffer(maxSize, '\0'); + const auto actualSize = + DecimalUtil::castToString(value, s, maxSize, buffer.data()); + buffer.resize(actualSize); + return buffer; + } + + protected: + const TypePtr sourceType_; + const TransformType transformType_; + const std::string sourceColumnName_; + memory::MemoryPool* pool_; +}; + +template +class IdentityTransform final : public Transform { + public: + IdentityTransform( + const TypePtr& type, + const std::string& columnName, + memory::MemoryPool* pool) + : Transform(type, TransformType::kIdentity, columnName, pool) {} + + VectorPtr apply(const VectorPtr& block) const override; + + const TypePtr resultType() const override { + return sourceType_; + } + + std::string toHumanString(int32_t value) const override { + if (sourceType_->isDate()) { + return DATE()->toString(value); + } + return folly::to(value); + } +}; + +template +class BucketTransform final : public Transform { + public: + BucketTransform( + int32_t count, + const TypePtr& type, + const std::string& columnName, + memory::MemoryPool* pool) + : Transform(type, TransformType::kBucket, columnName, pool), + numBuckets_(count) {} + + VectorPtr apply(const VectorPtr& block) const override; + + const TypePtr resultType() const override { + return INTEGER(); + } + + private: + const int32_t numBuckets_; +}; + +template +class TruncateTransform final : public Transform { + public: + TruncateTransform( + int32_t width, + const TypePtr& type, + const std::string& columnName, + memory::MemoryPool* pool) + : Transform(type, TransformType::kTruncate, columnName, pool), + width_(width) {} + + VectorPtr apply(const VectorPtr& block) const override; + + const TypePtr resultType() const override { + return sourceType_; + } + + private: + const int32_t width_; +}; + +template +class TemporalTransform final : public Transform { + public: + TemporalTransform( + const TypePtr& type, + TransformType transformType, + const std::string& columnName, + memory::MemoryPool* pool, + const std::function& epochFunc) + : Transform(type, transformType, columnName, pool), + epochFunc_(epochFunc) {} + + VectorPtr apply(const VectorPtr& block) const override; + + const TypePtr resultType() const override { + return INTEGER(); + } + + std::string toHumanString(int32_t value) const override { + switch (transformType_) { + case TransformType::kYear: { + return fmt::format("{:04d}", kEpochYear + value); + } + case TransformType::kMonth: { + int32_t year = kEpochYear + value / 12; + int32_t month = 1 + value % 12; + if (month <= 0) { + month += 12; + year -= 1; + } + return fmt::format("{:04d}-{:02d}", year, month); + } + case TransformType::kHour: { + int64_t seconds = static_cast(value) * 3600; + std::tm tmValue; + VELOX_USER_CHECK( + Timestamp::epochToCalendarUtc(seconds, tmValue), + "Can't convert seconds {}*3600 to time.", + seconds); + + return fmt::format( + "{:04d}-{:02d}-{:02d}-{:02d}", + tmValue.tm_year + 1900, + tmValue.tm_mon + 1, + tmValue.tm_mday, + tmValue.tm_hour); + } + case TransformType::kDay: { + return DATE()->toString(value); + } + default: { + VELOX_UNREACHABLE("Unsupported transform type."); + } + } + } + + private: + const std::function epochFunc_; +}; + +} // namespace facebook::velox::connector::hive::iceberg diff --git a/velox/connectors/hive/iceberg/tests/CMakeLists.txt b/velox/connectors/hive/iceberg/tests/CMakeLists.txt index 8abc11f7b9a1..539a8e472016 100644 --- a/velox/connectors/hive/iceberg/tests/CMakeLists.txt +++ b/velox/connectors/hive/iceberg/tests/CMakeLists.txt @@ -49,6 +49,7 @@ if(NOT VELOX_DISABLE_GOOGLETEST) velox_dwio_common_exception velox_dwio_common_test_utils velox_vector_test_lib + velox_vector_fuzzer velox_exec velox_exec_test_lib Folly::folly @@ -59,10 +60,17 @@ if(NOT VELOX_DISABLE_GOOGLETEST) add_executable( velox_hive_iceberg_insert_test + ColumnTransformTest.cpp IcebergInsertTest.cpp + IcebergPartitionIdGeneratorTest.cpp + IcebergSortOrderTest.cpp + IcebergStatsTest.cpp IcebergTestBase.cpp + IcebergTransformE2ETest.cpp + IcebergTransformUnitTest.cpp + IcebergWriterModeTest.cpp Main.cpp - PartitionSpecTest.cpp + Murmur3Test.cpp ) add_test(velox_hive_iceberg_insert_test velox_hive_iceberg_insert_test) @@ -74,11 +82,14 @@ if(NOT VELOX_DISABLE_GOOGLETEST) velox_hive_iceberg_splitreader velox_vector_fuzzer GTest::gtest - GTest::gtest_main ) if(VELOX_ENABLE_PARQUET) - target_link_libraries(velox_hive_iceberg_test velox_dwio_parquet_reader) + target_link_libraries( + velox_hive_iceberg_test + velox_dwio_parquet_writer + velox_dwio_parquet_reader + ) file(COPY ${CMAKE_CURRENT_SOURCE_DIR}/examples DESTINATION ${CMAKE_CURRENT_BINARY_DIR}) diff --git a/velox/connectors/hive/iceberg/tests/ColumnTransformTest.cpp b/velox/connectors/hive/iceberg/tests/ColumnTransformTest.cpp new file mode 100644 index 000000000000..1b83e1b15b56 --- /dev/null +++ b/velox/connectors/hive/iceberg/tests/ColumnTransformTest.cpp @@ -0,0 +1,120 @@ +/* + * 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/Transforms.h" +#include "velox/connectors/hive/iceberg/tests/IcebergTestBase.h" +#include "velox/vector/tests/utils/VectorMaker.h" + +using namespace facebook::velox; +using namespace facebook::velox::connector::hive::iceberg; +using namespace facebook::velox::test; + +namespace facebook::velox::connector::hive::iceberg::test { + +class ColumnTransformTest : public IcebergTestBase {}; + +TEST_F(ColumnTransformTest, testConstructor) { + auto transform = std::make_shared>( + INTEGER(), "test_column", opPool_.get()); + + EXPECT_EQ(transform->sourceColumnName(), "test_column"); + EXPECT_EQ(transform->name(), "identity"); + EXPECT_EQ(transform->resultType(), INTEGER()); +} + +TEST_F(ColumnTransformTest, testTransformName) { + auto identityTransform = std::make_shared>( + INTEGER(), "col1", opPool_.get()); + EXPECT_EQ(identityTransform->name(), "identity"); + + auto bucketTransform = std::make_shared>( + 16, INTEGER(), "col2", opPool_.get()); + EXPECT_EQ(bucketTransform->name(), "bucket"); + + auto truncateTransform = std::make_shared>( + 10, INTEGER(), "col3", opPool_.get()); + EXPECT_EQ(truncateTransform->name(), "trunc"); + + auto yearTransform = std::make_shared>( + INTEGER(), TransformType::kYear, "col4", opPool_.get(), [](int32_t v) { + return v; + }); + EXPECT_EQ(yearTransform->name(), "year"); + + auto monthTransform = std::make_shared>( + INTEGER(), TransformType::kMonth, "col5", opPool_.get(), [](int32_t v) { + return v; + }); + EXPECT_EQ(monthTransform->name(), "month"); + + auto dayTransform = std::make_shared>( + INTEGER(), TransformType::kDay, "col6", opPool_.get(), [](int32_t v) { + return v; + }); + EXPECT_EQ(dayTransform->name(), "day"); + + auto hourTransform = std::make_shared>( + INTEGER(), TransformType::kHour, "col7", opPool_.get(), [](Timestamp v) { + return v.getSeconds() / 3600; + }); + EXPECT_EQ(hourTransform->name(), "hour"); +} + +TEST_F(ColumnTransformTest, testResultType) { + auto intTransform = std::make_shared>( + INTEGER(), "col_int", opPool_.get()); + EXPECT_EQ(intTransform->resultType(), INTEGER()); + + auto bigintTransform = std::make_shared>( + BIGINT(), "col_bigint", opPool_.get()); + EXPECT_EQ(bigintTransform->resultType(), BIGINT()); + + auto varcharTransform = std::make_shared>( + VARCHAR(), "col_varchar", opPool_.get()); + EXPECT_EQ(varcharTransform->resultType(), VARCHAR()); + + auto bucketTransform = std::make_shared>( + 16, VARCHAR(), "col_bucket", opPool_.get()); + EXPECT_EQ(bucketTransform->resultType(), INTEGER()); + + auto yearTransform = std::make_shared>( + DATE(), TransformType::kYear, "col_year", opPool_.get(), [](int32_t v) { + return v; + }); + EXPECT_EQ(yearTransform->resultType(), INTEGER()); +} + +TEST_F(ColumnTransformTest, testTransformSimpleColumn) { + auto intVector = makeFlatVector({1, 2, 3, 4, 5}); + auto rowVector = makeRowVector({"col_int"}, {intVector}); + + auto transform = std::make_shared>( + INTEGER(), "col_int", opPool_.get()); + + auto result = transform->transform(rowVector, 0); + + ASSERT_EQ(result->size(), 5); + ASSERT_EQ(result->type(), INTEGER()); + + auto resultVector = result->as>(); + EXPECT_EQ(resultVector->valueAt(0), 1); + EXPECT_EQ(resultVector->valueAt(1), 2); + EXPECT_EQ(resultVector->valueAt(2), 3); + EXPECT_EQ(resultVector->valueAt(3), 4); + EXPECT_EQ(resultVector->valueAt(4), 5); +} + +} // namespace facebook::velox::connector::hive::iceberg::test diff --git a/velox/connectors/hive/iceberg/tests/IcebergInsertTest.cpp b/velox/connectors/hive/iceberg/tests/IcebergInsertTest.cpp index e7cf026830a3..7e0019dda6a2 100644 --- a/velox/connectors/hive/iceberg/tests/IcebergInsertTest.cpp +++ b/velox/connectors/hive/iceberg/tests/IcebergInsertTest.cpp @@ -14,8 +14,8 @@ * limitations under the License. */ +#include "velox/common/base/tests/GTestUtils.h" #include "velox/connectors/hive/iceberg/tests/IcebergTestBase.h" -#include "velox/exec/tests/utils/AssertQueryBuilder.h" #include "velox/exec/tests/utils/PlanBuilder.h" namespace facebook::velox::connector::hive::iceberg { @@ -25,19 +25,25 @@ class IcebergInsertTest : public test::IcebergTestBase { protected: void test(const RowTypePtr& rowType, double nullRatio = 0.0) { const auto outputDirectory = exec::test::TempDirectoryPath::create(); - const auto dataPath = outputDirectory->getPath(); + const auto dataPath = fmt::format("{}", outputDirectory->getPath()); constexpr int32_t numBatches = 10; constexpr int32_t vectorSize = 5'000; const auto vectors = createTestData(rowType, numBatches, vectorSize, nullRatio); - const auto& dataSink = - createDataSinkAndAppendData(rowType, vectors, dataPath); - const auto commitTasks = dataSink->close(); + 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 = exec::test::PlanBuilder().tableScan(rowType).planNode(); - exec::test::AssertQueryBuilder(plan).splits(splits).assertResults(vectors); + assertQuery(plan, splits, "SELECT * FROM tmp"); } }; @@ -72,5 +78,226 @@ TEST_F(IcebergInsertTest, bigDecimal) { } #endif +TEST_F(IcebergInsertTest, testSingleColumnAsPartition) { + auto rowType = ROW( + {"c1", "c2", "c3", "c4", "c5", "c6"}, + {BIGINT(), INTEGER(), SMALLINT(), DECIMAL(18, 5), BOOLEAN(), VARCHAR()}); + for (auto colIndex = 0; colIndex < rowType->size() - 1; colIndex++) { + const auto& colName = rowType->nameOf(colIndex); + const auto outputDirectory = exec::test::TempDirectoryPath::create(); + constexpr int32_t numBatches = 2; + constexpr int32_t vectorSize = 50; + const auto vectors = createTestData(rowType, numBatches, vectorSize, 0.5); + std::vector partitionTransforms = { + {colIndex, TransformType::kIdentity, std::nullopt}}; + 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(outputDirectory->getPath()); + + 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, + HiveColumnHandle::ColumnType::kRegular, + rowType->childAt(i), + rowType->childAt(i))}); + } + } + + // Add partition column. + assignments.insert( + {colName, + std::make_shared( + colName, + HiveColumnHandle::ColumnType::kPartitionKey, + rowType->childAt(colIndex), + rowType->childAt(colIndex))}); + + auto plan = exec::test::PlanBuilder(pool_.get()) + .tableScan(rowType, {}, "", nullptr, assignments) + .planNode(); + + assertQuery(plan, splits, fmt::format("SELECT * FROM tmp")); + } +} + +TEST_F(IcebergInsertTest, testPartitionNullColumn) { + auto rowType = ROW( + {"c1", "c2", "c3", "c4", "c5", "c6"}, + {BIGINT(), INTEGER(), SMALLINT(), DECIMAL(18, 5), BOOLEAN(), VARCHAR()}); + 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(); + constexpr int32_t numBatches = 2; + constexpr int32_t vectorSize = 100; + + const auto vectors = createTestData(rowType, numBatches, vectorSize, 1.0); + std::vector partitionTransforms = { + {colIndex, TransformType::kIdentity, std::nullopt}}; + auto dataSink = createIcebergDataSink( + rowType, outputDirectory->getPath(), partitionTransforms); + + for (const auto& vector : vectors) { + dataSink->appendData(vector); + } + + ASSERT_TRUE(dataSink->finish()); + const auto commitTasks = dataSink->close(); + ASSERT_EQ(1, commitTasks.size()); + auto taskJson = folly::parseJson(commitTasks.at(0)); + ASSERT_EQ(1, taskJson.count("partitionDataJson")); + auto partitionDataStr = taskJson["partitionDataJson"].asString(); + auto partitionData = folly::parseJson(partitionDataStr); + ASSERT_EQ(1, partitionData.count("partitionValues")); + auto partitionValues = partitionData["partitionValues"]; + ASSERT_TRUE(partitionValues.isArray()); + ASSERT_TRUE(partitionValues[0].isNull()); + + auto files = listFiles(outputDirectory->getPath()); + 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) { + auto rowType = ROW( + {"c1", "c2", "c3", "c4", "c5", "c6"}, + {BIGINT(), INTEGER(), SMALLINT(), DECIMAL(18, 5), BOOLEAN(), VARCHAR()}); + std::vector> columnCombinations = { + {0, 1}, // BIGINT, INTEGER. + {2, 1}, // SMALLINT, INTEGER. + {2, 3}, // SMALLINT, DECIMAL. + {0, 2, 1} // BIGINT, SMALLINT, INTEGER. + }; + + for (const auto& combination : columnCombinations) { + const auto outputDirectory = exec::test::TempDirectoryPath::create(); + constexpr int32_t numBatches = 2; + constexpr int32_t vectorSize = 50; + const auto vectors = createTestData(rowType, numBatches, vectorSize); + std::vector partitionTransforms; + for (auto colIndex : combination) { + partitionTransforms.push_back( + {colIndex, TransformType::kIdentity, std::nullopt}); + } + + 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(outputDirectory->getPath()); + + 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 + ? HiveColumnHandle::ColumnType::kPartitionKey + : HiveColumnHandle::ColumnType::kRegular; + + assignments.insert( + {name, + std::make_shared( + name, columnType, rowType->childAt(i), rowType->childAt(i))}); + } + + auto plan = exec::test::PlanBuilder(pool_.get()) + .tableScan(rowType, {}, "", nullptr, assignments) + .planNode(); + + assertQuery(plan, splits, fmt::format("SELECT * FROM tmp")); + } +} + +TEST_F(IcebergInsertTest, testInfinityValues) { + const auto outputDirectory = exec::test::TempDirectoryPath::create(); + auto realVector = makeFlatVector( + {std::numeric_limits::max(), + -std::numeric_limits::infinity(), + std::numeric_limits::infinity(), + std::numeric_limits::min(), + std::numeric_limits::lowest(), + std::numeric_limits::quiet_NaN()}); + + auto doubleVector = makeFlatVector( + {std::numeric_limits::max(), + -std::numeric_limits::infinity(), + std::numeric_limits::infinity(), + std::numeric_limits::min(), + std::numeric_limits::lowest(), + std::numeric_limits::quiet_NaN()}); + + auto idVector = makeFlatVector({0, 1, 2, 3, 4, 5}); + + auto rowType = + ROW({"id", "real_col", "double_col"}, {BIGINT(), REAL(), DOUBLE()}); + auto vector = makeRowVector( + {"id", "real_col", "double_col"}, {idVector, realVector, doubleVector}); + + auto dataSink = + createIcebergDataSink(rowType, outputDirectory->getPath(), {}); + dataSink->appendData(vector); + ASSERT_TRUE(dataSink->finish()); + dataSink->close(); + + createDuckDbTable({vector}); + auto splits = createSplitsForDirectory(outputDirectory->getPath()); + + auto plan = + exec::test::PlanBuilder(pool_.get()).tableScan(rowType).planNode(); + + assertQuery(plan, splits, "SELECT * FROM tmp ORDER BY id"); +} + } // namespace } // namespace facebook::velox::connector::hive::iceberg diff --git a/velox/connectors/hive/iceberg/tests/IcebergPartitionIdGeneratorTest.cpp b/velox/connectors/hive/iceberg/tests/IcebergPartitionIdGeneratorTest.cpp new file mode 100644 index 000000000000..f6aafa4be221 --- /dev/null +++ b/velox/connectors/hive/iceberg/tests/IcebergPartitionIdGeneratorTest.cpp @@ -0,0 +1,364 @@ +/* + * 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/IcebergPartitionIdGenerator.h" +#include "velox/connectors/hive/iceberg/Transforms.h" +#include "velox/connectors/hive/iceberg/tests/IcebergTestBase.h" + +using namespace facebook::velox; + +namespace facebook::velox::connector::hive::iceberg::test { + +class IcebergPartitionIdGeneratorTest : public IcebergTestBase { + protected: + std::vector> createColumnTransforms( + const std::vector& columnNames, + const std::vector& types, + const std::vector& transformTypes, + const std::vector>& parameters = {}) { + std::vector fields; + fields.reserve(columnNames.size()); + + for (size_t i = 0; i < columnNames.size(); ++i) { + std::optional parameter = + parameters.size() > i ? parameters[i] : std::nullopt; + + fields.emplace_back( + IcebergPartitionSpec::Field{ + columnNames[i], types[i], transformTypes[i], parameter}); + } + + return parsePartitionTransformSpecs(fields, pool_.get()); + } + + std::unique_ptr createGenerator( + const std::vector>& transforms, + bool partitionPathAsLowerCase = false) { + std::vector partitionChannels; + for (size_t i = 0; i < transforms.size(); ++i) { + partitionChannels.push_back(i); + } + + return std::make_unique( + partitionChannels, + 128, + pool_.get(), + transforms, + partitionPathAsLowerCase); + } + + void verifyPartitionComponents( + const std::string& partitionName, + const std::vector& expectedComponents) { + std::vector actualComponents; + folly::split('/', partitionName, actualComponents); + ASSERT_EQ(actualComponents.size(), expectedComponents.size()); + for (size_t i = 0; i < expectedComponents.size(); ++i) { + ASSERT_EQ(actualComponents[i], expectedComponents[i]); + } + } +}; + +TEST_F(IcebergPartitionIdGeneratorTest, partitionNameWithIdentityTransforms) { + std::vector columnNames = { + "c_int", "c_bigint", "c_varchar", "c_decimal", "c_bool", "c_date"}; + + std::vector columns = { + makeConstant(42, 1), + makeConstant(9'876'543'210, 1), + makeConstant("test string", 1), + makeConstant(12'345'678'901'234, 1, DECIMAL(18, 4)), + makeConstant(true, 1), + makeConstant(18'262, 1, DATE())}; + + std::vector types = { + INTEGER(), BIGINT(), VARCHAR(), DECIMAL(18, 4), BOOLEAN(), DATE()}; + auto rowVector = makeRowVector(columnNames, columns); + std::vector transformTypes( + columnNames.size(), TransformType::kIdentity); + auto transforms = createColumnTransforms(columnNames, types, transformTypes); + auto generator = createGenerator(transforms); + raw_vector partitionIds(1); + generator->run(rowVector, partitionIds); + + std::string partitionName = generator->partitionName(partitionIds[0]); + std::vector expectedComponents = { + "c_int=42", + "c_bigint=9876543210", + "c_varchar=test+string", + "c_decimal=1234567890.1234", + "c_bool=true", + "c_date=2020-01-01"}; + verifyPartitionComponents(partitionName, expectedComponents); +} + +TEST_F( + IcebergPartitionIdGeneratorTest, + partitionNameWithTimestampIdentitySpecialValues) { + std::vector timestamps = { + Timestamp(253402300800, 100000000), // +10000-01-01T00:00:00.1. + Timestamp(-62170000000, 0), // -0001-11-29T19:33:20. + Timestamp(-62135577748, 999000000), // 0001-01-01T05:17:32.999. + Timestamp(0, 0), // 1970-01-01T00:00. + Timestamp(1609459200, 999000000), // 2021-01-01T00:00. + Timestamp(1640995200, 500000000), // 2022-01-01T00:00:00.5. + Timestamp(1672531200, 123000000), // 2023-01-01T00:00:00.123. + Timestamp(-1, 999000000), // 1969-12-31T23:59:59.999. + Timestamp(1, 1000000), // 1970-01-01T00:00:01.001. + Timestamp(-62167219199, 0), // 0000-01-01T00:00:01. + Timestamp(-377716279140, 321000000), // -10000-01-01T01:01:00.321. + Timestamp(253402304660, 321000000), // +10000-01-01T01:01:00.321. + Timestamp(951782400, 0), // 2000-02-29T00:00:00 (leap year). + Timestamp(4107456000, 0), // 2100-02-28T00:00:00 (not leap year). + Timestamp(86400, 0), // 1970-01-02T00:00:00. + Timestamp(-86400, 0), // 1969-12-31T00:00:00. + Timestamp(1672531200, 456000000), // 2023-01-01T00:00:00.456. + Timestamp(1672531200, 789000000), // 2023-01-01T00:00:00.789. + }; + + std::vector expectedPartitionNames = { + "c_timestamp=%2B10000-01-01T00%3A00%3A00.1", + "c_timestamp=-0001-11-29T19%3A33%3A20", + "c_timestamp=0001-01-01T05%3A17%3A32.999", + "c_timestamp=1970-01-01T00%3A00", + "c_timestamp=2021-01-01T00%3A00%3A00.999", + "c_timestamp=2022-01-01T00%3A00%3A00.5", + "c_timestamp=2023-01-01T00%3A00%3A00.123", + "c_timestamp=1969-12-31T23%3A59%3A59.999", + "c_timestamp=1970-01-01T00%3A00%3A01.001", + "c_timestamp=0000-01-01T00%3A00%3A01", + "c_timestamp=-10000-08-24T19%3A21%3A00.321", + "c_timestamp=%2B10000-01-01T01%3A04%3A20.321", + "c_timestamp=2000-02-29T00%3A00", + "c_timestamp=2100-02-28T00%3A00", + "c_timestamp=1970-01-02T00%3A00", + "c_timestamp=1969-12-31T00%3A00", + "c_timestamp=2023-01-01T00%3A00%3A00.456", + "c_timestamp=2023-01-01T00%3A00%3A00.789", + }; + + auto timestampVector = makeFlatVector(timestamps); + std::vector columnNames = {"c_timestamp"}; + std::vector columns = {timestampVector}; + std::vector types = {TIMESTAMP()}; + auto rowVector = makeRowVector(columnNames, columns); + + std::vector transformTypes = {TransformType::kIdentity}; + auto transforms = createColumnTransforms(columnNames, types, transformTypes); + auto generator = createGenerator(transforms); + raw_vector partitionIds(timestamps.size()); + generator->run(rowVector, partitionIds); + + for (size_t i = 0; i < timestamps.size(); ++i) { + std::string partitionName = generator->partitionName(partitionIds[i]); + ASSERT_EQ(partitionName, expectedPartitionNames[i]); + } +} + +TEST_F(IcebergPartitionIdGeneratorTest, partitionNameWithMixedTransforms) { + std::vector columnNames = { + "c_int", + "c_bigint", + "c_varchar", + "c_year", + "c_month", + "c_day", + "c_hour", + "c_bool"}; + + std::vector columns = { + makeConstant(42, 1), + makeConstant(9'876'543'210, 1), + makeConstant("test string", 1), + makeConstant(Timestamp(1'577'836'800, 0), 1), + makeConstant(Timestamp(1'578'836'800, 0), 1), + makeConstant(Timestamp(1'579'836'800, 0), 1), + makeConstant(Timestamp(1'57'936'800, 0), 1), + makeConstant(true, 1)}; + + std::vector types = { + INTEGER(), + BIGINT(), + VARCHAR(), + TIMESTAMP(), + TIMESTAMP(), + TIMESTAMP(), + TIMESTAMP(), + BOOLEAN()}; + + auto rowVector = makeRowVector(columnNames, columns); + + std::vector transformTypes = { + TransformType::kBucket, + TransformType::kTruncate, + TransformType::kTruncate, + TransformType::kYear, + TransformType::kMonth, + TransformType::kDay, + TransformType::kHour, + TransformType::kIdentity}; + + std::vector> parameters = {4, 1'000, 5, std::nullopt}; + auto transforms = + createColumnTransforms(columnNames, types, transformTypes, parameters); + + auto generator = createGenerator(transforms); + raw_vector partitionIds(1); + generator->run(rowVector, partitionIds); + + std::string partitionName = generator->partitionName(partitionIds[0]); + std::vector expectedComponents = { + "c_int_bucket=2", + "c_bigint_trunc=9876543000", + "c_varchar_trunc=test+", + "c_year_year=2020", + "c_month_month=2020-01", + "c_day_day=2020-01-24", + "c_hour_hour=1975-01-02-23", + "c_bool=true"}; + verifyPartitionComponents(partitionName, expectedComponents); +} + +TEST_F(IcebergPartitionIdGeneratorTest, partitionNameWithNullValues) { + std::vector columnNames = {"c_int", "c_varchar", "c_decimal"}; + std::vector columns = { + makeConstant(std::nullopt, 1), + makeConstant(std::nullopt, 1), + makeConstant(std::nullopt, 1, DECIMAL(18, 4))}; + std::vector types = {INTEGER(), VARCHAR(), DECIMAL(18, 3)}; + auto rowVector = makeRowVector(columnNames, columns); + + std::vector transformTypes = { + TransformType::kBucket, + TransformType::kTruncate, + TransformType::kIdentity}; + std::vector> parameters = {4, 1'000, std::nullopt}; + auto transforms = + createColumnTransforms(columnNames, types, transformTypes, parameters); + auto generator = createGenerator(transforms); + raw_vector partitionIds(1); + generator->run(rowVector, partitionIds); + + std::string partitionName = generator->partitionName(partitionIds[0]); + std::vector expectedComponents = { + "c_int_bucket=null", "c_varchar_trunc=null", "c_decimal=null"}; + verifyPartitionComponents(partitionName, expectedComponents); +} + +TEST_F(IcebergPartitionIdGeneratorTest, partitionNameWithLowerCase) { + auto varcharVector = makeConstant("MiXeD_CaSe", 1); + std::vector columnNames = {"MiXeD_CoLuMn"}; + std::vector columns = {varcharVector}; + std::vector types = {VARCHAR()}; + auto rowVector = makeRowVector(columnNames, columns); + std::vector transformTypes = {TransformType::kIdentity}; + auto transforms = createColumnTransforms(columnNames, types, transformTypes); + auto generator = createGenerator(transforms, true); + raw_vector partitionIds(1); + generator->run(rowVector, partitionIds); + std::string partitionName = generator->partitionName(partitionIds[0]); + std::vector expectedPartitionName = {"mixed_column=MiXeD_CaSe"}; + verifyPartitionComponents(partitionName, expectedPartitionName); + + generator = createGenerator(transforms); + generator->run(rowVector, partitionIds); + partitionName = generator->partitionName(partitionIds[0]); + expectedPartitionName = {"MiXeD_CoLuMn=MiXeD_CaSe"}; + verifyPartitionComponents(partitionName, expectedPartitionName); +} + +TEST_F(IcebergPartitionIdGeneratorTest, urlEncodingForSpecialChars) { + std::vector> testCases = { + {"space test", "space+test"}, + {"slash/test", "slash%2Ftest"}, + {"question?test", "question%3Ftest"}, + {"percent%test", "percent%25test"}, + {"hash#test", "hash%23test"}, + {"ampersand&test", "ampersand%26test"}, + {"equals=test", "equals%3Dtest"}, + {"plus+test", "plus%2Btest"}, + {"comma,test", "comma%2Ctest"}, + {"semicolon;test", "semicolon%3Btest"}, + {"at@test", "at%40test"}, + {"dollar$test", "dollar%24test"}, + {"backslash\\test", "backslash%5Ctest"}, + {"quote\"test", "quote%22test"}, + {"apostrophe'test", "apostrophe%27test"}, + {"lessthan", "greater%3Ethan"}, + {"colon:test", "colon%3Atest"}, + {"pipe|test", "pipe%7Ctest"}, + {"bracket[test", "bracket%5Btest"}, + {"bracket]test", "bracket%5Dtest"}, + {"brace{test", "brace%7Btest"}, + {"brace}test", "brace%7Dtest"}, + {"caret^test", "caret%5Etest"}, + {"tilde~test", "tilde%7Etest"}, + {"backtick`test", "backtick%60test"}, + {"unicode\u00A9test", "unicode%C2%A9test"}, + {"email@example.com", "email%40example.com"}, + {"user:password@host:port/path", "user%3Apassword%40host%3Aport%2Fpath"}, + {"https://github.ibm.com/IBM/velox", + "https%3A%2F%2Fgithub.ibm.com%2FIBM%2Fvelox"}, + {"a+b=c&d=e+f", "a%2Bb%3Dc%26d%3De%2Bf"}, + {"special!@#$%^&*()_+", "special%21%40%23%24%25%5E%26*%28%29_%2B"}, + }; + + std::vector transformTypes = {TransformType::kIdentity}; + std::vector types = {VARCHAR()}; + std::vector columnNames = {"ColumnWithSpecialChars"}; + auto transforms = createColumnTransforms(columnNames, types, transformTypes); + raw_vector partitionIds(1); + auto generator = createGenerator(transforms); + + for (const auto& [input, expectedEncoded] : testCases) { + auto varcharVector = makeConstant(StringView(input), 1); + auto rowVector = makeRowVector(columnNames, {varcharVector}); + generator->run(rowVector, partitionIds); + std::string partitionName = generator->partitionName(partitionIds[0]); + std::string expectedPartitionName = + fmt::format("{}={}", columnNames[0], expectedEncoded); + ASSERT_EQ(partitionName, expectedPartitionName); + } +} + +TEST_F(IcebergPartitionIdGeneratorTest, multipleRows) { + std::vector columnNames = {"c_int", "c_varchar"}; + auto rowVector = makeRowVector( + columnNames, + {makeFlatVector({10, 20, 30}), + makeFlatVector({"value1", "value2", "value3"})}); + + std::vector types = {INTEGER(), VARCHAR()}; + std::vector transformTypes( + columnNames.size(), TransformType::kIdentity); + auto transforms = createColumnTransforms(columnNames, types, transformTypes); + auto generator = createGenerator(transforms); + raw_vector partitionIds(3); + generator->run(rowVector, partitionIds); + + std::vector expectedNames = { + "c_int=10/c_varchar=value1", + "c_int=20/c_varchar=value2", + "c_int=30/c_varchar=value3"}; + + for (size_t i = 0; i < 3; ++i) { + std::string partitionName = generator->partitionName(partitionIds[i]); + ASSERT_EQ(partitionName, expectedNames[i]); + } +} + +} // namespace facebook::velox::connector::hive::iceberg::test diff --git a/velox/connectors/hive/iceberg/tests/IcebergReadTest.cpp b/velox/connectors/hive/iceberg/tests/IcebergReadTest.cpp index 34e87fc25257..a345c7515a09 100644 --- a/velox/connectors/hive/iceberg/tests/IcebergReadTest.cpp +++ b/velox/connectors/hive/iceberg/tests/IcebergReadTest.cpp @@ -220,7 +220,7 @@ class HiveIcebergTest : public HiveConnectorTestBase { IcebergDeleteFile icebergDeleteFile( FileContent::kPositionalDeletes, deleteFilePath, - fileFomat_, + fileFormat_, deleteFilePaths[deleteFileName].first, testing::internal::GetFileSize( std::fopen(deleteFilePath.c_str(), "r"))); @@ -246,6 +246,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: @@ -274,7 +393,7 @@ class HiveIcebergTest : public HiveConnectorTestBase { std::make_shared( kHiveConnectorId, dataFilePath, - fileFomat_, + fileFormat_, i * splitSize, splitSize, partitionKeys, @@ -332,7 +451,7 @@ class HiveIcebergTest : public HiveConnectorTestBase { IcebergDeleteFile icebergDeleteFile( FileContent::kPositionalDeletes, deleteFilePath->getPath(), - fileFomat_, + fileFormat_, deletedPositionSize, testing::internal::GetFileSize( std::fopen(deleteFilePath->getPath().c_str(), "r"))); @@ -358,6 +477,20 @@ class HiveIcebergTest : public HiveConnectorTestBase { } #endif + 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) { @@ -560,13 +693,166 @@ class HiveIcebergTest : public HiveConnectorTestBase { return deletePositionVector; } - dwio::common::FileFormat fileFomat_{dwio::common::FileFormat::DWRF}; + 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 = 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 diff --git a/velox/connectors/hive/iceberg/tests/IcebergSortOrderTest.cpp b/velox/connectors/hive/iceberg/tests/IcebergSortOrderTest.cpp new file mode 100644 index 000000000000..7fc881210c31 --- /dev/null +++ b/velox/connectors/hive/iceberg/tests/IcebergSortOrderTest.cpp @@ -0,0 +1,592 @@ +/* + * 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/connectors/hive/iceberg/IcebergSplit.h" +#include "velox/connectors/hive/iceberg/tests/IcebergTestBase.h" +#include "velox/exec/tests/utils/PlanBuilder.h" + +using namespace facebook::velox::exec::test; + +namespace facebook::velox::connector::hive::iceberg::test { + +class IcebergSortOrderTest : public IcebergTestBase { + protected: + void SetUp() override { + IcebergTestBase::SetUp(); + rowType_ = ROW( + {"c_int", + "c_bigint", + "c_varchar", + "c_date", + "c_decimal", + "c_varbinary"}, + {INTEGER(), BIGINT(), VARCHAR(), DATE(), DECIMAL(18, 3), VARBINARY()}); + } + + // Verify data in the file is sorted according to the specified sort columns. + void verifySortOrder( + const std::string& dataPath, + const std::vector& sortColumns) { + auto splits = createSplitsForDirectory(dataPath); + ASSERT_FALSE(splits.empty()) << "No data files found in " << dataPath; + + // Create a projection that selects all columns. + std::vector allColumns; + for (auto i = 0; i < rowType_->size(); ++i) { + allColumns.push_back(rowType_->nameOf(i)); + } + + auto plan = + PlanBuilder().tableScan(rowType_).project(allColumns).planNode(); + auto result = + AssertQueryBuilder(plan).splits(splits).copyResults(opPool_.get()); + + ASSERT_GT(result->size(), 0) << "No rows found in the data file"; + + // For each sort column, verify the data is sorted. + for (const auto& sortExpr : sortColumns) { + std::string columnName; + bool isAscending = true; + bool isNullsFirst = true; + + std::istringstream iss(sortExpr); + iss >> columnName; + std::string token; + if (iss >> token) { + if (token == "DESC") { + isAscending = false; + } else if (token != "ASC") { + iss.seekg(-(int32_t)token.length(), std::ios_base::cur); + } + + if (iss >> token && token == "NULLS") { + if (iss >> token && token == "LAST") { + isNullsFirst = false; + } else if (token != "FIRST") { + ASSERT_TRUE(token == "FIRST") + << "Invalid NULLS ordering: " << token; + } + } + } + + int32_t columnIndex = -1; + for (auto i = 0; i < rowType_->size(); ++i) { + if (rowType_->nameOf(i) == columnName) { + columnIndex = i; + break; + } + } + ASSERT_NE(columnIndex, -1) + << "Column " << columnName << " not found in row type"; + + auto columnVector = result->childAt(columnIndex); + bool hasNulls = false; + bool hasNonNulls = false; + vector_size_t firstNonNullIndex = 0; + vector_size_t lastNullIndex = 0; + + for (auto i = 0; i < columnVector->size(); ++i) { + if (columnVector->isNullAt(i)) { + hasNulls = true; + lastNullIndex = i; + } else { + if (!hasNonNulls) { + firstNonNullIndex = i; + hasNonNulls = true; + } + } + } + + if (hasNulls && hasNonNulls) { + if (isNullsFirst) { + ASSERT_LT(lastNullIndex, firstNonNullIndex) + << "NULL values should come before non-NULL values when NULLS FIRST is specified"; + } else { + ASSERT_GT(lastNullIndex, firstNonNullIndex) + << "NULL values should come after non-NULL values when NULLS LAST is specified"; + } + } + + DecodedVector decoded; + SelectivityVector rows(columnVector->size()); + decoded.decode(*columnVector, rows); + + for (auto i = 1; i < columnVector->size(); ++i) { + // Skip if either current or previous is null. + if (columnVector->isNullAt(i) || columnVector->isNullAt(i - 1)) { + continue; + } + + // Compare values based on type. + int32_t comparison = 0; + switch (auto kind = rowType_->childAt(columnIndex)->kind()) { + case TypeKind::INTEGER: { + auto prev = decoded.valueAt(i - 1); + auto curr = decoded.valueAt(i); + comparison = prev < curr ? -1 : (prev > curr ? 1 : 0); + break; + } + case TypeKind::BIGINT: { + auto prev = decoded.valueAt(i - 1); + auto curr = decoded.valueAt(i); + comparison = prev < curr ? -1 : (prev > curr ? 1 : 0); + break; + } + case TypeKind::VARCHAR: { + auto prev = decoded.valueAt(i - 1); + auto curr = decoded.valueAt(i); + comparison = prev < curr ? -1 : (prev > curr ? 1 : 0); + break; + } + case TypeKind::VARBINARY: { + auto prev = decoded.valueAt(i - 1); + auto curr = decoded.valueAt(i); + comparison = prev < curr ? -1 : (prev > curr ? 1 : 0); + break; + } + case TypeKind::HUGEINT: { + if (rowType_->childAt(columnIndex)->isLongDecimal()) { + auto prev = decoded.valueAt(i - 1); + auto curr = decoded.valueAt(i); + comparison = prev < curr ? -1 : (prev > curr ? 1 : 0); + } + break; + } + default: + ASSERT_TRUE(false) + << "Unsupported column type for sorting verification: " << kind; + } + + if (isAscending) { + ASSERT_LE(comparison, 0) + << "Data not sorted in ascending order at row " << i + << " for column " << columnName; + } else { + ASSERT_GE(comparison, 0) + << "Data not sorted in descending order at row " << i + << " for column " << columnName; + } + + // If values are equal, continue to next row. + if (comparison == 0) { + continue; + } + break; + } + } + } + + // Verify that data is sorted according to multiple sort columns. + void verifyMultiColumnSortOrder( + const std::string& dataPath, + const std::vector& sortColumns) { + auto splits = createSplitsForDirectory(dataPath); + ASSERT_FALSE(splits.empty()) << "No data files found in " << dataPath; + std::vector allColumns; + for (auto i = 0; i < rowType_->size(); ++i) { + allColumns.push_back(rowType_->nameOf(i)); + } + + auto plan = + PlanBuilder().tableScan(rowType_).project(allColumns).planNode(); + auto result = + AssertQueryBuilder(plan).splits(splits).copyResults(opPool_.get()); + + ASSERT_GT(result->size(), 0) << "No rows found in the data file"; + + std::vector columnNames; + std::vector isAscending; + std::vector isNullsFirst; + std::vector columnIndices; + + for (const auto& sortExpr : sortColumns) { + std::string columnName; + bool ascending = true; + bool nullsFirst = true; + + std::istringstream iss(sortExpr); + iss >> columnName; + std::string token; + if (iss >> token) { + if (token == "DESC") { + ascending = false; + } else if (token != "ASC") { + iss.seekg(-(int32_t)token.length(), std::ios_base::cur); + } + + if (iss >> token && token == "NULLS") { + if (iss >> token && token == "LAST") { + nullsFirst = false; + } else if (token != "FIRST") { + ASSERT_TRUE(token == "FIRST") + << "Invalid NULLS ordering: " << token; + } + } + } + + int32_t columnIndex = -1; + for (auto i = 0; i < rowType_->size(); ++i) { + if (rowType_->nameOf(i) == columnName) { + columnIndex = i; + break; + } + } + ASSERT_NE(columnIndex, -1) + << "Column " << columnName << " not found in row type"; + + columnNames.push_back(columnName); + isAscending.push_back(ascending); + isNullsFirst.push_back(nullsFirst); + columnIndices.push_back(columnIndex); + } + + // Verify the sort order row by row. + for (auto i = 1; i < result->size(); ++i) { + // Compare row i-1 with row i using all sort columns in order. + for (size_t colIdx = 0; colIdx < columnIndices.size(); ++colIdx) { + int32_t columnIndex = columnIndices[colIdx]; + auto columnVector = result->childAt(columnIndex); + bool ascending = isAscending[colIdx]; + bool nullsFirst = isNullsFirst[colIdx]; + bool prevIsNull = columnVector->isNullAt(i - 1); + bool currIsNull = columnVector->isNullAt(i); + + if (prevIsNull && currIsNull) { + // Both null, continue to next column. + continue; + } else if (prevIsNull) { + // Previous is null, current is not. + ASSERT_TRUE(nullsFirst) + << "NULL values should come last at row " << (i - 1) + << " for column " << columnNames[colIdx] << " in " << dataPath; + break; + } else if (currIsNull) { + // Current is null, previous is not. + ASSERT_FALSE(nullsFirst) + << "NULL values should come first at row " << i << " for column " + << columnNames[colIdx] << " in " << dataPath; + break; + } + + // Both values are non-null, compare them. + DecodedVector decoded; + SelectivityVector rows(columnVector->size()); + decoded.decode(*columnVector, rows); + + int32_t comparison = 0; + switch (auto kind = rowType_->childAt(columnIndex)->kind()) { + case TypeKind::INTEGER: { + auto prev = decoded.valueAt(i - 1); + auto curr = decoded.valueAt(i); + comparison = prev < curr ? -1 : (prev > curr ? 1 : 0); + break; + } + case TypeKind::BIGINT: { + auto prev = decoded.valueAt(i - 1); + auto curr = decoded.valueAt(i); + comparison = prev < curr ? -1 : (prev > curr ? 1 : 0); + break; + } + case TypeKind::VARCHAR: { + auto prev = decoded.valueAt(i - 1); + auto curr = decoded.valueAt(i); + comparison = prev < curr ? -1 : (prev > curr ? 1 : 0); + break; + } + case TypeKind::VARBINARY: { + auto prev = decoded.valueAt(i - 1); + auto curr = decoded.valueAt(i); + comparison = prev < curr ? -1 : (prev > curr ? 1 : 0); + break; + } + case TypeKind::HUGEINT: { + if (rowType_->childAt(columnIndex)->isLongDecimal()) { + auto prev = decoded.valueAt(i - 1); + auto curr = decoded.valueAt(i); + comparison = prev < curr ? -1 : (prev > curr ? 1 : 0); + } + break; + } + default: + ASSERT_TRUE(false) + << "Unsupported column type for sorting verification: " << kind; + } + + if (comparison != 0) { + if (ascending) { + ASSERT_LE(comparison, 0) + << "Data not sorted in ascending order at row " << i + << " for column " << columnNames[colIdx] << " in " << dataPath + << ". Previous value: " << columnVector->toString(i - 1) + << ", Current value: " << columnVector->toString(i); + } else { + ASSERT_GE(comparison, 0) + << "Data not sorted in descending order at row " << i + << " for column " << columnNames[colIdx] << " in " << dataPath + << ". Previous value: " << columnVector->toString(i - 1) + << ", Current value: " << columnVector->toString(i); + } + // Found definitive ordering, no need to check further columns. + break; + } + // If values are equal, continue to next column. + } + // Rows can be equal across all sort columns. + } + } + + std::vector stringBuffer_; + VectorFuzzer::Options fuzzerOptions_; + std::unique_ptr fuzzer_; + static constexpr auto numBatches = 10; + static constexpr auto rowsPerBatch = 1'000; + + void testSorting( + const std::vector& sortExpressions, + double nullRatio = 0.0) { + std::vector vectors = + createTestData(rowType_, numBatches, rowsPerBatch, nullRatio); + auto outputDirectory = TempDirectoryPath::create(); + + auto dataSink = createIcebergDataSink( + rowType_, outputDirectory->getPath(), {}, sortExpressions); + + for (const auto& vector : vectors) { + dataSink->appendData(vector); + } + + ASSERT_TRUE(dataSink->finish()); + dataSink->close(); + if (sortExpressions.size() == 1) { + verifySortOrder(outputDirectory->getPath(), sortExpressions); + } else { + verifyMultiColumnSortOrder(outputDirectory->getPath(), sortExpressions); + } + } + + void testSortingWithPartitioning( + const std::vector& partitionTransforms, + const std::vector& sortExpressions, + const double nullRatio = 0.0) { + std::vector vectors = + createTestData(rowType_, numBatches, rowsPerBatch, nullRatio); + const auto outputDirectory = TempDirectoryPath::create(); + + auto dataSink = createIcebergDataSink( + rowType_, + outputDirectory->getPath(), + partitionTransforms, + sortExpressions); + + for (const auto& vector : vectors) { + dataSink->appendData(vector); + } + + ASSERT_TRUE(dataSink->finish()); + dataSink->close(); + + // For partitioned data, we need to find all partition directories. + std::vector partitionDirs; + std::function findLeafDataDirs = + [&partitionDirs, &findLeafDataDirs](const std::string& dir) { + bool hasSubDirs = false; + + for (const auto& entry : std::filesystem::directory_iterator(dir)) { + if (entry.is_directory()) { + hasSubDirs = true; + findLeafDataDirs(entry.path().string()); + } + } + if (!hasSubDirs) { + partitionDirs.push_back(dir); + } + }; + + // Start the recursive search from the data directory. + if (std::filesystem::exists(outputDirectory->getPath())) { + findLeafDataDirs(outputDirectory->getPath()); + } + if (partitionDirs.empty()) { + partitionDirs.push_back(outputDirectory->getPath()); + } + + // Verify each partition directory has properly sorted data. + ASSERT_FALSE(partitionDirs.empty()) << "No partition directories found"; + for (const auto& partitionDir : partitionDirs) { + if (sortExpressions.size() == 1) { + verifySortOrder(partitionDir, sortExpressions); + } else { + verifyMultiColumnSortOrder(partitionDir, sortExpressions); + } + } + } +}; + +TEST_F(IcebergSortOrderTest, singleColumnSortDefault) { + testSorting({"c_int"}); + testSorting({"c_bigint"}); + testSorting({"c_varchar"}); + testSorting({"c_date"}); + testSorting({"c_decimal"}); + testSorting({"c_varbinary"}); +} + +TEST_F(IcebergSortOrderTest, singleColumnSortDesc) { + testSorting({"c_int DESC"}); + testSorting({"c_bigint DESC"}); + testSorting({"c_varchar DESC"}); + testSorting({"c_date DESC"}); + testSorting({"c_decimal DESC"}); + testSorting({"c_varbinary DESC"}); +} + +TEST_F(IcebergSortOrderTest, nullOrderingFirst) { + testSorting({"c_int ASC NULLS FIRST"}, 0.2); + testSorting({"c_bigint ASC NULLS FIRST"}, 0.2); + testSorting({"c_varchar ASC NULLS FIRST"}, 0.3); + testSorting({"c_date ASC NULLS FIRST"}, 0.3); + testSorting({"c_decimal ASC NULLS FIRST"}, 0.2); + testSorting({"c_varbinary ASC NULLS FIRST"}, 0.2); + + testSorting({"c_varbinary DESC NULLS FIRST"}, 0.2); + testSorting({"c_int DESC NULLS FIRST"}, 0.2); + testSorting({"c_bigint DESC NULLS FIRST"}, 0.2); + testSorting({"c_varchar DESC NULLS FIRST"}, 0.3); + testSorting({"c_date DESC NULLS FIRST"}, 0.3); + testSorting({"c_decimal DESC NULLS FIRST"}, 0.2); +} + +TEST_F(IcebergSortOrderTest, nullOrderingLast) { + testSorting({"c_int ASC NULLS LAST"}, 0.2); + testSorting({"c_bigint ASC NULLS LAST"}, 0.2); + testSorting({"c_varchar ASC NULLS LAST"}, 0.2); + testSorting({"c_date ASC NULLS LAST"}, 0.2); + testSorting({"c_decimal ASC NULLS LAST"}, 0.2); + testSorting({"c_varbinary ASC NULLS LAST"}, 0.2); + + testSorting({"c_varbinary DESC NULLS LAST"}, 0.2); + testSorting({"c_int DESC NULLS LAST"}, 0.2); + testSorting({"c_bigint DESC NULLS LAST"}, 0.2); + testSorting({"c_varchar DESC NULLS LAST"}, 0.2); + testSorting({"c_date DESC NULLS LAST"}, 0.2); + testSorting({"c_decimal DESC NULLS LAST"}, 0.2); +} + +TEST_F(IcebergSortOrderTest, multiColumnSort) { + testSorting({"c_int ASC", "c_bigint DESC"}); + testSorting({"c_int ASC", "c_bigint ASC"}); + testSorting({"c_int DESC", "c_bigint DESC"}); + testSorting({"c_int DESC", "c_bigint ASC"}); + + testSorting({"c_int ASC", "c_varchar DESC"}); + testSorting({"c_int ASC", "c_varchar ASC"}); + testSorting({"c_int DESC", "c_varchar DESC"}); + testSorting({"c_int DESC", "c_varchar ASC"}); + + testSorting({"c_varchar ASC", "c_date DESC"}); + testSorting({"c_varchar ASC", "c_date ASC"}); + testSorting({"c_varchar DESC", "c_date DESC"}); + testSorting({"c_varchar DESC", "c_date ASC"}); + + testSorting({"c_int ASC", "c_decimal DESC"}); + testSorting({"c_decimal ASC", "c_varbinary DESC"}); + testSorting({"c_varbinary ASC", "c_decimal DESC"}); +} + +TEST_F(IcebergSortOrderTest, multiColumnSortWithNull) { + testSorting({"c_int", "c_bigint", "c_varchar"}, 0.2); + testSorting({"c_int", "c_bigint DESC NULLS LAST"}, 0.4); + testSorting( + {"c_int ASC NULLS FIRST", + "c_bigint DESC NULLS LAST", + "c_varchar ASC NULLS FIRST"}, + 0.2); + testSorting( + {"c_int ASC NULLS FIRST", "c_bigint", "c_varchar ASC NULLS FIRST"}, 0.2); + testSorting( + {"c_int ASC NULLS LAST", "c_bigint", "c_varchar ASC NULLS FIRST"}, 0.2); + testSorting( + {"c_int DESC NULLS LAST", "c_bigint", "c_varchar ASC NULLS LAST"}, 0.2); + testSorting( + {"c_int ASC NULLS FIRST", "c_bigint DESC", "c_varchar ASC NULLS FIRST"}, + 0.2); + testSorting( + {"c_int ASC NULLS FIRST", + "c_bigint DESC NULLS LAST", + "c_varchar ASC NULLS FIRST"}, + 0.2); + testSorting( + {"c_int ASC NULLS FIRST", "c_bigint DESC", "c_varchar ASC NULLS LAST"}, + 0.2); + testSorting( + {"c_int ASC NULLS FIRST", + "c_bigint DESC NULLS LAST", + "c_varchar DESC NULLS FIRST"}, + 0.2); + + testSorting( + {"c_int ASC NULLS FIRST", + "c_decimal DESC NULLS LAST", + "c_varbinary ASC NULLS FIRST"}, + 0.2); +} + +TEST_F(IcebergSortOrderTest, sortWithSinglePartitioning) { + testSortingWithPartitioning({{3, TransformType::kBucket, 5}}, {"c_int ASC"}); + testSortingWithPartitioning( + {{0, TransformType::kBucket, 7}}, {"c_varchar ASC"}); +} + +TEST_F(IcebergSortOrderTest, sortWithPartitioningOnSameColumn) { + testSortingWithPartitioning({{3, TransformType::kBucket, 5}}, {"c_date ASC"}); + testSortingWithPartitioning({{0, TransformType::kBucket, 7}}, {"c_int ASC"}); + testSortingWithPartitioning( + {{2, TransformType::kBucket, 4}}, {"c_varchar DESC"}); +} + +TEST_F(IcebergSortOrderTest, sortWithMultiPartitioning) { + testSortingWithPartitioning( + {{3, TransformType::kBucket, 3}, {2, TransformType::kBucket, 4}}, + {"c_int ASC", "c_bigint DESC"}); + + testSortingWithPartitioning( + {{3, TransformType::kTruncate, 10000}}, {"c_int ASC", "c_bigint DESC"}); +} + +TEST_F(IcebergSortOrderTest, sortWithPartitioningAndNulls) { + testSortingWithPartitioning( + {{0, TransformType::kBucket, 8}}, + {"c_int ASC NULLS FIRST", "c_bigint DESC NULLS LAST"}, + 0.2); + + testSortingWithPartitioning( + {{2, TransformType::kBucket, 8}}, + {"c_varchar ASC NULLS FIRST", "c_int DESC NULLS LAST"}, + 0.2); + + testSortingWithPartitioning( + {{4, TransformType::kBucket, 8}}, + {"c_decimal ASC NULLS FIRST", "c_int DESC NULLS LAST"}, + 0.3); + + testSortingWithPartitioning( + {{5, TransformType::kBucket, 8}}, + {"c_varbinary ASC NULLS FIRST", "c_int DESC NULLS LAST"}, + 0.3); +} + +} // namespace facebook::velox::connector::hive::iceberg::test 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/hive/iceberg/tests/IcebergStatsTest.cpp b/velox/connectors/hive/iceberg/tests/IcebergStatsTest.cpp new file mode 100644 index 000000000000..52c4f5fd4dfe --- /dev/null +++ b/velox/connectors/hive/iceberg/tests/IcebergStatsTest.cpp @@ -0,0 +1,1395 @@ +/* + * 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/connectors/hive/iceberg/tests/IcebergTestBase.h" + +namespace facebook::velox::connector::hive::iceberg::test { +class IcebergStatsTest : public IcebergTestBase { + protected: + void SetUp() override { + IcebergTestBase::SetUp(); + rowType_ = + ROW({"c_int", "c_bigint", "c_varchar", "c_date", "c_decimal"}, + {INTEGER(), BIGINT(), VARCHAR(), DATE(), DECIMAL(18, 3)}); + } + + void TearDown() override { + IcebergTestBase::TearDown(); + } +}; + +TEST_F(IcebergStatsTest, mixedNullTest) { + auto rowType = ROW({"int_col"}, {INTEGER()}); + auto outputDir = exec::test::TempDirectoryPath::create(); + auto dataSink = createIcebergDataSink(rowType, outputDir->getPath()); + constexpr vector_size_t size = 100; + auto expectedIntNulls = 34; + + auto rowVector = makeRowVector({makeFlatVector( + size, [](vector_size_t row) { return row * 10; }, nullEvery(3))}); + dataSink->appendData(rowVector); + + ASSERT_TRUE(dataSink->finish()); + auto commitTasks = dataSink->close(); + ASSERT_FALSE(commitTasks.empty()); + + const auto& fileStats = dataSink->dataFileStats(); + ASSERT_EQ(fileStats.size(), 1) << "Expected exactly one file with stats"; + const auto& stats = fileStats[0]; + EXPECT_EQ(stats->numRecords, size) << "Record count should match input size"; + ASSERT_FALSE(stats->valueCounts.empty()) + << "Should have value counts for columns"; + constexpr int32_t intColId = 1; + EXPECT_EQ(stats->valueCounts.at(intColId), size) + << "Int column value count incorrect"; + ASSERT_FALSE(stats->nullValueCounts.empty()) + << "Should have null counts for columns"; + + EXPECT_EQ(stats->nullValueCounts.at(intColId), expectedIntNulls) + << "Int column null count incorrect"; + ASSERT_FALSE(stats->lowerBounds.empty()) + << "Should have lower bounds for columns"; + ASSERT_FALSE(stats->upperBounds.empty()) + << "Should have upper bounds for columns"; + + std::string lowerBounds = + encoding::Base64::decode(stats->lowerBounds.at(intColId)); + auto lb = *reinterpret_cast(lowerBounds.data()); + EXPECT_EQ(lb, 10); + EXPECT_FALSE(stats->lowerBounds.at(intColId).empty()) + << "Int column should have non-empty lower bound"; + std::string upperBounds = + encoding::Base64::decode(stats->upperBounds.at(intColId)); + auto ub = *reinterpret_cast(upperBounds.data()); + EXPECT_EQ(ub, 980); + EXPECT_FALSE(stats->upperBounds.at(intColId).empty()) + << "Int column should have non-empty upper bound"; +} + +TEST_F(IcebergStatsTest, bigintStatsTest) { + auto rowType = ROW({"bigint_col"}, {BIGINT()}); + + auto outputDir = exec::test::TempDirectoryPath::create(); + auto dataSink = createIcebergDataSink(rowType, outputDir->getPath()); + constexpr vector_size_t size = 100; + auto expectedNulls = 25; + + auto rowVector = makeRowVector({makeFlatVector( + size, + [](vector_size_t row) { return row * 1'000'000'000LL; }, + nullEvery(4))}); + + dataSink->appendData(rowVector); + + ASSERT_TRUE(dataSink->finish()); + auto commitTasks = dataSink->close(); + ASSERT_FALSE(commitTasks.empty()); + + const auto& fileStats = dataSink->dataFileStats(); + ASSERT_EQ(fileStats.size(), 1) << "Expected exactly one file with stats"; + const auto& stats = fileStats[0]; + + EXPECT_EQ(stats->numRecords, size) << "Record count should match input size"; + ASSERT_FALSE(stats->valueCounts.empty()) + << "Should have value counts for columns"; + constexpr int32_t bigintColId = 1; + EXPECT_EQ(stats->valueCounts.at(bigintColId), size) + << "Bigint column value count incorrect"; + + ASSERT_FALSE(stats->nullValueCounts.empty()) + << "Should have null counts for columns"; + EXPECT_EQ(stats->nullValueCounts.at(bigintColId), expectedNulls) + << "Bigint column null count incorrect"; + + ASSERT_FALSE(stats->lowerBounds.empty()) + << "Should have lower bounds for columns"; + ASSERT_FALSE(stats->upperBounds.empty()) + << "Should have upper bounds for columns"; + + std::string lowerBounds = + encoding::Base64::decode(stats->lowerBounds.at(bigintColId)); + auto lb = *reinterpret_cast(lowerBounds.data()); + EXPECT_EQ(lb, 1'000'000'000LL); + + std::string upperBounds = + encoding::Base64::decode(stats->upperBounds.at(bigintColId)); + auto ub = *reinterpret_cast(upperBounds.data()); + EXPECT_EQ(ub, 99'000'000'000LL); + folly::dynamic json = stats->toJson(); + std::string jsonstring = folly::toJson(json); +} + +TEST_F(IcebergStatsTest, decimalStatsTest) { + auto rowType = ROW({"decimal_col"}, {DECIMAL(38, 3)}); + + auto outputDir = exec::test::TempDirectoryPath::create(); + auto dataSink = createIcebergDataSink(rowType, outputDir->getPath()); + constexpr vector_size_t size = 100; + auto expectedNulls = 20; + + auto rowVector = makeRowVector({makeFlatVector( + size, + [](vector_size_t row) { return HugeInt::build(row, row * 123); }, + nullEvery(5), + DECIMAL(38, 3))}); + + dataSink->appendData(rowVector); + + ASSERT_TRUE(dataSink->finish()); + auto commitTasks = dataSink->close(); + ASSERT_FALSE(commitTasks.empty()); + + const auto& fileStats = dataSink->dataFileStats(); + ASSERT_EQ(fileStats.size(), 1) << "Expected exactly one file with stats"; + const auto& stats = fileStats[0]; + + EXPECT_EQ(stats->numRecords, size) << "Record count should match input size"; + ASSERT_FALSE(stats->valueCounts.empty()) + << "Should have value counts for columns"; + constexpr int32_t decimalColId = 1; + EXPECT_EQ(stats->valueCounts.at(decimalColId), size) + << "Decimal column value count incorrect"; + ASSERT_FALSE(stats->nullValueCounts.empty()) + << "Should have null counts for columns"; + EXPECT_EQ(stats->nullValueCounts.at(decimalColId), expectedNulls) + << "Decimal column null count incorrect"; + + ASSERT_FALSE(stats->lowerBounds.empty()) + << "Should have lower bounds for columns"; + ASSERT_FALSE(stats->upperBounds.empty()) + << "Should have upper bounds for columns"; + EXPECT_FALSE(stats->lowerBounds.at(decimalColId).empty()) + << "Decimal column should have non-empty lower bound"; + EXPECT_FALSE(stats->upperBounds.at(decimalColId).empty()) + << "Decimal column should have non-empty upper bound"; +} + +TEST_F(IcebergStatsTest, varcharStatsTest) { + auto rowType = ROW({"varchar_col"}, {VARCHAR()}); + + auto outputDir = exec::test::TempDirectoryPath::create(); + auto dataSink = createIcebergDataSink(rowType, outputDir->getPath()); + constexpr vector_size_t size = 100; + auto expectedNulls = 0; + + auto varcharVector = BaseVector::create(VARCHAR(), size, opPool_.get()); + auto flatVarcharVector = varcharVector->asFlatVector(); + for (auto i = 0; i < size; ++i) { + if (i % 6 == 0) { + flatVarcharVector->setNull(i, true); + expectedNulls++; + } else { + std::string value = + "Customer#00000" + std::to_string(i) + "_" + std::string(i % 10, 'a'); + flatVarcharVector->set(i, StringView(value)); + } + } + + auto rowVector = makeRowVector({varcharVector}); + + dataSink->appendData(rowVector); + + ASSERT_TRUE(dataSink->finish()); + auto commitTasks = dataSink->close(); + ASSERT_FALSE(commitTasks.empty()); + + const auto& fileStats = dataSink->dataFileStats(); + ASSERT_EQ(fileStats.size(), 1) << "Expected exactly one file with stats"; + const auto& stats = fileStats[0]; + EXPECT_EQ(stats->numRecords, size) << "Record count should match input size"; + + ASSERT_FALSE(stats->valueCounts.empty()) + << "Should have value counts for columns"; + constexpr int32_t varcharColId = 1; + EXPECT_EQ(stats->valueCounts.at(varcharColId), size) + << "Varchar column value count incorrect"; + + ASSERT_FALSE(stats->nullValueCounts.empty()) + << "Should have null counts for columns"; + EXPECT_EQ(stats->nullValueCounts.at(varcharColId), expectedNulls) + << "Varchar column null count incorrect"; + + ASSERT_FALSE(stats->lowerBounds.empty()) + << "Should have lower bounds for columns"; + ASSERT_FALSE(stats->upperBounds.empty()) + << "Should have upper bounds for columns"; + EXPECT_FALSE(stats->lowerBounds.at(varcharColId).empty()) + << "Varchar column should have non-empty lower bound"; + EXPECT_FALSE(stats->upperBounds.at(varcharColId).empty()) + << "Varchar column should have non-empty upper bound"; + + // Decode and verify string bounds. + std::string lowerBound = + encoding::Base64::decode(stats->lowerBounds.at(varcharColId)); + std::string upperBound = + encoding::Base64::decode(stats->upperBounds.at(varcharColId)); + EXPECT_TRUE(lowerBound.find("Customer#00000") != std::string::npos) + << "Lower bound should contain 'Customer#00000'"; + EXPECT_TRUE(upperBound.find("Customer#000009") != std::string::npos) + << "Upper bound should contain 'Customer#000009'"; +} + +TEST_F(IcebergStatsTest, varbinaryStatsTest) { + auto rowType = ROW({"varbinary_col"}, {VARBINARY()}); + + auto outputDir = exec::test::TempDirectoryPath::create(); + auto dataSink = createIcebergDataSink(rowType, outputDir->getPath()); + constexpr vector_size_t size = 100; + auto expectedNulls = 0; + + auto varbinaryVector = BaseVector::create(VARBINARY(), size, opPool_.get()); + auto flatVarbinaryVector = varbinaryVector->asFlatVector(); + for (auto i = 0; i < size; ++i) { + if (i % 5 == 0) { + flatVarbinaryVector->setNull(i, true); + expectedNulls++; + } else { + // Create binary values with varying content. + std::string value(17, 11); + value[0] = static_cast(i % 256); + value[1] = static_cast((i * 3) % 256); + value[2] = static_cast((i * 7) % 256); + value[3] = static_cast((i * 11) % 256); + flatVarbinaryVector->set(i, StringView(value)); + } + } + + auto rowVector = makeRowVector({varbinaryVector}); + + dataSink->appendData(rowVector); + + ASSERT_TRUE(dataSink->finish()); + auto commitTasks = dataSink->close(); + ASSERT_FALSE(commitTasks.empty()); + + const auto& fileStats = dataSink->dataFileStats(); + ASSERT_EQ(fileStats.size(), 1) << "Expected exactly one file with stats"; + const auto& stats = fileStats[0]; + + EXPECT_EQ(stats->numRecords, size) << "Record count should match input size"; + + ASSERT_FALSE(stats->valueCounts.empty()) + << "Should have value counts for columns"; + constexpr int32_t varbinaryColId = 1; + EXPECT_EQ(stats->valueCounts.at(varbinaryColId), size) + << "Varbinary column value count incorrect"; + + ASSERT_FALSE(stats->nullValueCounts.empty()) + << "Should have null counts for columns"; + EXPECT_EQ(stats->nullValueCounts.at(varbinaryColId), expectedNulls) + << "Varbinary column null count incorrect"; + + ASSERT_FALSE(stats->lowerBounds.empty()) + << "Should have lower bounds for columns"; + ASSERT_FALSE(stats->upperBounds.empty()) + << "Should have upper bounds for columns"; + EXPECT_FALSE(stats->lowerBounds.at(varbinaryColId).empty()) + << "Varbinary column should have non-empty lower bound"; + EXPECT_FALSE(stats->upperBounds.at(varbinaryColId).empty()) + << "Varbinary column should have non-empty upper bound"; +} + +TEST_F(IcebergStatsTest, varbinaryStatsTest2) { + auto rowType = ROW({"varbinary_col"}, {VARBINARY()}); + + auto outputDir = exec::test::TempDirectoryPath::create(); + std::vector partitionTransforms = { + {0, TransformType::kBucket, 4}}; + auto dataSink = + createIcebergDataSink(rowType, outputDir->getPath(), partitionTransforms); + constexpr vector_size_t size = 10; + + auto varbinaryVector = BaseVector::create(VARBINARY(), size, opPool_.get()); + auto flatVarbinaryVector = varbinaryVector->asFlatVector(); + std::string values[] = { + "01020304", + "05060708", + "090A0B0C", + "0D0E0F10", + "11121314", + "15161718", + "191A1B1C", + "1D1E1F20", + "21222324", + "25262728"}; + for (auto i = 0; i < size; ++i) { + flatVarbinaryVector->set(i, StringView(values[i])); + } + + auto rowVector = makeRowVector({"varbinary_col"}, {varbinaryVector}); + + dataSink->appendData(rowVector); + + ASSERT_TRUE(dataSink->finish()); + auto commitTasks = dataSink->close(); + ASSERT_FALSE(commitTasks.empty()); + + const auto& fileStats = dataSink->dataFileStats(); + ASSERT_EQ(fileStats.size(), 3) << "Expected exactly one file with stats"; + const auto& stats = fileStats[0]; + + EXPECT_EQ(stats->numRecords, 5) << "Record count should match input size"; + + constexpr int32_t varbinaryColId = 1; + EXPECT_EQ(stats->valueCounts.at(varbinaryColId), 5); +} + +TEST_F(IcebergStatsTest, multipleDataTypesTest) { + auto rowType = ROW( + {"int_col", "bigint_col", "decimal_col", "varchar_col", "varbinary_col"}, + {INTEGER(), BIGINT(), DECIMAL(38, 3), VARCHAR(), VARBINARY()}); + + auto outputDir = exec::test::TempDirectoryPath::create(); + auto dataSink = createIcebergDataSink(rowType, outputDir->getPath()); + constexpr vector_size_t size = 100; + + int32_t expectedIntNulls = 34; + int32_t expectedBigintNulls = 25; + int32_t expectedDecimalNulls = 20; + int32_t expectedVarcharNulls = 0; + int32_t expectedVarbinaryNulls = 0; + // Create columns with different null patterns + auto intVector = makeFlatVector( + size, [](vector_size_t row) { return row * 10; }, nullEvery(3)); + + auto bigintVector = makeFlatVector( + size, + [](vector_size_t row) { return row * 1'000'000'000LL; }, + nullEvery(4)); + + auto decimalVector = makeFlatVector( + size, + [](vector_size_t row) { return HugeInt::build(row, row * 12'345); }, + nullEvery(5), + DECIMAL(38, 3)); + + auto varcharVector = BaseVector::create(VARCHAR(), size, opPool_.get()); + auto flatVarcharVector = varcharVector->asFlatVector(); + for (auto i = 0; i < size; ++i) { + if (i % 6 == 0) { + flatVarcharVector->setNull(i, true); + expectedVarcharNulls++; + } else { + std::string value = "str_" + std::to_string(i); + flatVarcharVector->set(i, StringView(value)); + } + } + + auto varbinaryVector = BaseVector::create(VARBINARY(), size, opPool_.get()); + auto flatVarbinaryVector = varbinaryVector->asFlatVector(); + for (auto i = 0; i < size; ++i) { + if (i % 7 == 0) { + flatVarbinaryVector->setNull(i, true); + expectedVarbinaryNulls++; + } else { + std::string value(4, 0); + value[0] = static_cast(i % 256); + value[1] = static_cast((i * 3) % 256); + value[2] = static_cast((i * 7) % 256); + value[3] = static_cast((i * 11) % 256); + flatVarbinaryVector->set(i, StringView(value)); + } + } + + auto rowVector = makeRowVector( + {intVector, bigintVector, decimalVector, varcharVector, varbinaryVector}); + + dataSink->appendData(rowVector); + + ASSERT_TRUE(dataSink->finish()); + auto commitTasks = dataSink->close(); + ASSERT_FALSE(commitTasks.empty()); + + const auto& fileStats = dataSink->dataFileStats(); + ASSERT_EQ(fileStats.size(), 1) << "Expected exactly one file with stats"; + const auto& stats = fileStats[0]; + + EXPECT_EQ(stats->numRecords, size) << "Record count should match input size"; + ASSERT_FALSE(stats->valueCounts.empty()) + << "Should have value counts for columns"; + + constexpr int32_t intColId = 1; + constexpr int32_t bigintColId = 2; + constexpr int32_t decimalColId = 3; + constexpr int32_t varcharColId = 4; + constexpr int32_t varbinaryColId = 5; + + ASSERT_FALSE(stats->nullValueCounts.empty()) + << "Should have null counts for columns"; + EXPECT_EQ(stats->nullValueCounts.at(intColId), expectedIntNulls) + << "Int column null count incorrect"; + EXPECT_EQ(stats->nullValueCounts.at(bigintColId), expectedBigintNulls) + << "Bigint column null count incorrect"; + EXPECT_EQ(stats->nullValueCounts.at(decimalColId), expectedDecimalNulls) + << "Decimal column null count incorrect"; + EXPECT_EQ(stats->nullValueCounts.at(varcharColId), expectedVarcharNulls) + << "Varchar column null count incorrect"; + EXPECT_EQ(stats->nullValueCounts.at(varbinaryColId), expectedVarbinaryNulls) + << "Varbinary column null count incorrect"; + + ASSERT_FALSE(stats->lowerBounds.empty()) + << "Should have lower bounds for columns"; + ASSERT_FALSE(stats->upperBounds.empty()) + << "Should have upper bounds for columns"; + + // Verify all columns have non-empty bounds. + EXPECT_FALSE(stats->lowerBounds.at(intColId).empty()) + << "Int column should have non-empty lower bound"; + EXPECT_FALSE(stats->upperBounds.at(intColId).empty()) + << "Int column should have non-empty upper bound"; + + EXPECT_FALSE(stats->lowerBounds.at(bigintColId).empty()) + << "Bigint column should have non-empty lower bound"; + EXPECT_FALSE(stats->upperBounds.at(bigintColId).empty()) + << "Bigint column should have non-empty upper bound"; + + EXPECT_FALSE(stats->lowerBounds.at(decimalColId).empty()) + << "Decimal column should have non-empty lower bound"; + EXPECT_FALSE(stats->upperBounds.at(decimalColId).empty()) + << "Decimal column should have non-empty upper bound"; + + EXPECT_FALSE(stats->lowerBounds.at(varcharColId).empty()) + << "Varchar column should have non-empty lower bound"; + EXPECT_FALSE(stats->upperBounds.at(varcharColId).empty()) + << "Varchar column should have non-empty upper bound"; + + EXPECT_FALSE(stats->lowerBounds.at(varbinaryColId).empty()) + << "Varbinary column should have non-empty lower bound"; + EXPECT_FALSE(stats->upperBounds.at(varbinaryColId).empty()) + << "Varbinary column should have non-empty upper bound"; +} + +TEST_F(IcebergStatsTest, dateStatsTest) { + auto rowType = ROW({"date_col"}, {DATE()}); + + auto outputDir = exec::test::TempDirectoryPath::create(); + auto dataSink = createIcebergDataSink(rowType, outputDir->getPath()); + constexpr vector_size_t size = 100; + auto expectedNulls = 20; + + auto rowVector = makeRowVector({makeFlatVector( + size, + [](vector_size_t row) { return 18262 + row; }, + nullEvery(5), + DATE())}); + + dataSink->appendData(rowVector); + + ASSERT_TRUE(dataSink->finish()); + auto commitTasks = dataSink->close(); + ASSERT_FALSE(commitTasks.empty()); + + const auto& fileStats = dataSink->dataFileStats(); + ASSERT_EQ(fileStats.size(), 1) << "Expected exactly one file with stats"; + const auto& stats = fileStats[0]; + + EXPECT_EQ(stats->numRecords, size) << "Record count should match input size"; + + ASSERT_FALSE(stats->valueCounts.empty()) + << "Should have value counts for columns"; + constexpr int32_t dateColId = 1; + EXPECT_EQ(stats->valueCounts.at(dateColId), size) + << "Date column value count incorrect"; + + ASSERT_FALSE(stats->nullValueCounts.empty()) + << "Should have null counts for columns"; + EXPECT_EQ(stats->nullValueCounts.at(dateColId), expectedNulls) + << "Date column null count incorrect"; + + ASSERT_FALSE(stats->lowerBounds.empty()) + << "Should have lower bounds for columns"; + ASSERT_FALSE(stats->upperBounds.empty()) + << "Should have upper bounds for columns"; + EXPECT_FALSE(stats->lowerBounds.at(dateColId).empty()) + << "Date column should have non-empty lower bound"; + EXPECT_FALSE(stats->upperBounds.at(dateColId).empty()) + << "Date column should have non-empty upper bound"; + + std::string lowerBounds = + encoding::Base64::decode(stats->lowerBounds.at(dateColId)); + auto lb = *reinterpret_cast(lowerBounds.data()); + EXPECT_EQ(lb, 18263) << "Lower bound should be 2020-01-02"; + + std::string upperBounds = + encoding::Base64::decode(stats->upperBounds.at(dateColId)); + auto ub = *reinterpret_cast(upperBounds.data()); + EXPECT_EQ(ub, 18262 + 99) << "Upper bound should be 2020-04-09"; +} + +TEST_F(IcebergStatsTest, booleanStatsTest) { + auto rowType = ROW({"boolean_col"}, {BOOLEAN()}); + + auto outputDir = exec::test::TempDirectoryPath::create(); + auto dataSink = createIcebergDataSink(rowType, outputDir->getPath()); + constexpr vector_size_t size = 100; + auto expectedNulls = 10; + + auto rowVector = makeRowVector({makeFlatVector( + size, + [](vector_size_t row) { return row % 2 == 1; }, + nullEvery(10), + BOOLEAN())}); + + dataSink->appendData(rowVector); + + ASSERT_TRUE(dataSink->finish()); + auto commitTasks = dataSink->close(); + ASSERT_FALSE(commitTasks.empty()); + + const auto& fileStats = dataSink->dataFileStats(); + ASSERT_EQ(fileStats.size(), 1) << "Expected exactly one file with stats"; + const auto& stats = fileStats[0]; + + EXPECT_EQ(stats->numRecords, size) << "Record count should match input size"; + + ASSERT_FALSE(stats->valueCounts.empty()) + << "Should have value counts for columns"; + constexpr int32_t boolColId = 1; + EXPECT_EQ(stats->valueCounts.at(boolColId), size) + << "Boolean column value count incorrect"; + + ASSERT_FALSE(stats->nullValueCounts.empty()) + << "Should have null counts for columns"; + EXPECT_EQ(stats->nullValueCounts.at(boolColId), expectedNulls) + << "Boolean column null count incorrect"; + + ASSERT_FALSE(stats->lowerBounds.empty()) + << "Should have lower bounds for columns"; + ASSERT_FALSE(stats->upperBounds.empty()) + << "Should have upper bounds for columns"; + EXPECT_FALSE(stats->lowerBounds.at(boolColId).empty()) + << "Boolean column should have non-empty lower bound"; + EXPECT_FALSE(stats->upperBounds.at(boolColId).empty()) + << "Boolean column should have non-empty upper bound"; + + // For boolean, the lower bound should be false (0) and upper bound should be + // true (1) if both values are present. + std::string lowerBounds = + encoding::Base64::decode(stats->lowerBounds.at(boolColId)); + auto lb = *reinterpret_cast(lowerBounds.data()); + EXPECT_FALSE(lb) << "Lower bound should be false"; + + std::string upperBounds = + encoding::Base64::decode(stats->upperBounds.at(boolColId)); + auto ub = *reinterpret_cast(upperBounds.data()); + EXPECT_TRUE(ub) << "Upper bound should be true"; +} + +TEST_F(IcebergStatsTest, emptyStatsTest) { + auto rowType = ROW({"int_col", "varchar_col"}, {INTEGER(), VARCHAR()}); + auto outputDir = exec::test::TempDirectoryPath::create(); + auto dataSink = createIcebergDataSink(rowType, outputDir->getPath()); + // Create an empty row vector (0 rows) + constexpr vector_size_t size = 0; + auto rowVector = makeRowVector( + {makeFlatVector(0), makeFlatVector(0)}); + + dataSink->appendData(rowVector); + + ASSERT_TRUE(dataSink->finish()); + auto commitTasks = dataSink->close(); + ASSERT_FALSE(commitTasks.empty()); + const auto& fileStats = dataSink->dataFileStats(); + ASSERT_EQ(fileStats.size(), 1) << "Expected exactly one file with stats"; + const auto& stats = fileStats[0]; + EXPECT_EQ(stats->numRecords, size) << "Record count should be 0"; + ASSERT_TRUE(stats->valueCounts.empty()) + << "Should no value counts for columns"; +} + +TEST_F(IcebergStatsTest, nullValuesTest) { + auto rowType = ROW({"int_col", "varchar_col"}, {INTEGER(), VARCHAR()}); + auto outputDir = exec::test::TempDirectoryPath::create(); + auto dataSink = createIcebergDataSink(rowType, outputDir->getPath()); + // Create an empty row vector (0 rows) + constexpr vector_size_t size = 100; + auto rowVector = makeRowVector( + {makeNullConstant(TypeKind::INTEGER, size), + makeNullConstant(TypeKind::VARCHAR, size)}); + + dataSink->appendData(rowVector); + + ASSERT_TRUE(dataSink->finish()); + auto commitTasks = dataSink->close(); + ASSERT_FALSE(commitTasks.empty()); + const auto& fileStats = dataSink->dataFileStats(); + ASSERT_EQ(fileStats.size(), 1) << "Expected exactly one file with stats"; + const auto& stats = fileStats[0]; + EXPECT_EQ(stats->numRecords, size) << "Record count should be 0"; + ASSERT_EQ(stats->nullValueCounts.at(1), size) << "All values is NULL."; + // Do not collect lower and upper bounds for NULLs. + ASSERT_EQ(stats->lowerBounds.size(), 0); + ASSERT_EQ(stats->upperBounds.size(), 0); +} + +TEST_F(IcebergStatsTest, realStatsTest) { + auto rowType = ROW({"real_col"}, {REAL()}); + auto outputDir = exec::test::TempDirectoryPath::create(); + auto dataSink = createIcebergDataSink(rowType, outputDir->getPath()); + constexpr vector_size_t size = 100; + auto expectedNulls = 15; + auto expectedNaNs = 0; + + auto rowVector = makeRowVector({makeFlatVector( + size, + [&](vector_size_t row) { + if (row % 3 == 0) { + expectedNaNs++; + return std::numeric_limits::quiet_NaN(); + } + if (row % 4 == 0) { + return std::numeric_limits::infinity(); + } + if (row % 5 == 0) { + return -std::numeric_limits::infinity(); + } + return row * 1.5f; + }, + nullEvery(7), + REAL())}); + + dataSink->appendData(rowVector); + + ASSERT_TRUE(dataSink->finish()); + auto commitTasks = dataSink->close(); + ASSERT_FALSE(commitTasks.empty()); + + const auto& fileStats = dataSink->dataFileStats(); + ASSERT_EQ(fileStats.size(), 1) << "Expected exactly one file with stats"; + const auto& stats = fileStats[0]; + + EXPECT_EQ(stats->numRecords, size) << "Record count should match input size"; + ASSERT_FALSE(stats->valueCounts.empty()) + << "Should have value counts for columns"; + constexpr int32_t realColId = 1; + EXPECT_EQ(stats->valueCounts.at(realColId), size) + << "Real column value count incorrect"; + + EXPECT_EQ(stats->nullValueCounts.at(realColId), expectedNulls) + << "Real column null count incorrect"; + EXPECT_EQ(stats->nanValueCounts.at(realColId), expectedNaNs) + << "Real column NaN count incorrect"; + + ASSERT_FALSE(stats->lowerBounds.empty()) + << "Should have lower bounds for columns"; + ASSERT_FALSE(stats->upperBounds.empty()) + << "Should have upper bounds for columns"; + + // Verify bounds are set correctly and NaN/infinity values don't affect + // min/max incorrectly. + std::string lowerBounds = + encoding::Base64::decode(stats->lowerBounds.at(realColId)); + auto lb = *reinterpret_cast(lowerBounds.data()); + EXPECT_FLOAT_EQ(lb, -std::numeric_limits::infinity()) + << "Lower bound should be -infinity"; + + std::string upperBounds = + encoding::Base64::decode(stats->upperBounds.at(realColId)); + auto ub = *reinterpret_cast(upperBounds.data()); + EXPECT_FLOAT_EQ(ub, std::numeric_limits::infinity()) + << "Upper bound should be infinity"; +} + +TEST_F(IcebergStatsTest, doubleStatsTest) { + auto rowType = ROW({"double_col"}, {DOUBLE()}); + + auto outputDir = exec::test::TempDirectoryPath::create(); + auto dataSink = createIcebergDataSink(rowType, outputDir->getPath()); + constexpr vector_size_t size = 100; + auto expectedNulls = 15; + auto expectedNaNs = 0; + + auto rowVector = makeRowVector({makeFlatVector( + size, + [&](vector_size_t row) { + if (row % 3 == 0) { + expectedNaNs++; + return std::numeric_limits::quiet_NaN(); + } + if (row % 4 == 0) { + return std::numeric_limits::infinity(); + } + if (row % 5 == 0) { + return -std::numeric_limits::infinity(); + } + return row * 2.5; + }, + nullEvery(7), + DOUBLE())}); + + dataSink->appendData(rowVector); + ASSERT_TRUE(dataSink->finish()); + auto commitTasks = dataSink->close(); + ASSERT_FALSE(commitTasks.empty()); + + const auto& fileStats = dataSink->dataFileStats(); + ASSERT_EQ(fileStats.size(), 1) << "Expected exactly one file with stats"; + const auto& stats = fileStats[0]; + + EXPECT_EQ(stats->numRecords, size) << "Record count should match input size"; + ASSERT_FALSE(stats->valueCounts.empty()) + << "Should have value counts for columns"; + constexpr int32_t doubleColId = 1; + EXPECT_EQ(stats->valueCounts.at(doubleColId), size) + << "Double column value count incorrect"; + ASSERT_FALSE(stats->nullValueCounts.empty()) + << "Should have null counts for columns"; + EXPECT_EQ(stats->nullValueCounts.at(doubleColId), expectedNulls) + << "Double column null count incorrect"; + EXPECT_EQ(stats->nanValueCounts.at(doubleColId), expectedNaNs) + << "Double column null count incorrect"; + ASSERT_FALSE(stats->lowerBounds.empty()) + << "Should have lower bounds for columns"; + ASSERT_FALSE(stats->upperBounds.empty()) + << "Should have upper bounds for columns"; + + // Verify bounds are set correctly and NaN/infinity values don't affect + // min/max incorrectly. + std::string lowerBounds = + encoding::Base64::decode(stats->lowerBounds.at(doubleColId)); + auto lb = *reinterpret_cast(lowerBounds.data()); + EXPECT_DOUBLE_EQ(lb, -std::numeric_limits::infinity()) + << "Lower bound should be -infinity"; + + std::string upperBounds = + encoding::Base64::decode(stats->upperBounds.at(doubleColId)); + auto ub = *reinterpret_cast(upperBounds.data()); + EXPECT_DOUBLE_EQ(ub, std::numeric_limits::infinity()) + << "Upper bound should be infinity"; +} + +TEST_F(IcebergStatsTest, MixedDoubleFloatStatsTest) { + std::vector names = {"id", "data1", "data2", "data3"}; + auto rowType = ROW(names, {INTEGER(), REAL(), DOUBLE(), DOUBLE()}); + + auto outputDir = exec::test::TempDirectoryPath::create(); + std::vector partitionTransforms = { + {0, TransformType::kIdentity, std::nullopt}}; + + auto dataSink = createIcebergDataSink(rowType, outputDir->getPath()); + constexpr vector_size_t size = 6; + + auto intVector = + makeFlatVector(size, [](vector_size_t row) { return 1; }); + auto floatVector = makeFlatVector(size, [](vector_size_t row) { + return -std::numeric_limits::infinity(); + }); + auto doubleVector1 = makeFlatVector(size, [](vector_size_t row) { + return std::numeric_limits::infinity(); + }); + auto doubleVector2 = makeFlatVector(size, [](vector_size_t row) { + switch (row) { + case 0: + return 1.23; + case 1: + return -1.23; + case 2: + return std::numeric_limits::infinity(); + case 3: + return 2.23; + case 4: + return -std::numeric_limits::infinity(); + default: + return -2.23; + } + }); + + ASSERT_TRUE( + -std::numeric_limits::infinity() < + std::numeric_limits::min() && + std::numeric_limits::min() < + std::numeric_limits::infinity()); + auto rowVector = makeRowVector( + names, {intVector, floatVector, doubleVector1, doubleVector2}); + dataSink->appendData(rowVector); + ASSERT_TRUE(dataSink->finish()); + auto commitTasks = dataSink->close(); + ASSERT_FALSE(commitTasks.empty()); + + const auto& fileStats = dataSink->dataFileStats(); + ASSERT_EQ(fileStats.size(), 1) << "Expected exactly one file with stats"; + const auto& stats = fileStats[0]; + + EXPECT_EQ(stats->numRecords, size) << "Record count should match input size"; + ASSERT_FALSE(stats->valueCounts.empty()) + << "Should have value counts for columns"; + constexpr int32_t doubleColId = 4; + EXPECT_EQ(stats->valueCounts.at(doubleColId), size) + << "Double column value count incorrect"; + ASSERT_FALSE(stats->nullValueCounts.empty()) + << "Should have null counts for columns"; + + std::string lowerBounds = + encoding::Base64::decode(stats->lowerBounds.at(doubleColId)); + auto lb = *reinterpret_cast(lowerBounds.data()); + EXPECT_DOUBLE_EQ(lb, -std::numeric_limits::infinity()) + << "Lower bound should be -infinity"; + + std::string upperBounds = + encoding::Base64::decode(stats->upperBounds.at(doubleColId)); + auto ub = *reinterpret_cast(upperBounds.data()); + EXPECT_DOUBLE_EQ(ub, std::numeric_limits::infinity()) + << "Upper bound should be infinity"; + + lowerBounds = encoding::Base64::decode(stats->lowerBounds.at(2)); + auto flb = *reinterpret_cast(lowerBounds.data()); + EXPECT_DOUBLE_EQ(flb, -std::numeric_limits::infinity()) + << "Lower bound should be -infinity"; + + upperBounds = encoding::Base64::decode(stats->upperBounds.at(2)); + auto fub = *reinterpret_cast(upperBounds.data()); + EXPECT_DOUBLE_EQ(fub, -std::numeric_limits::infinity()) + << "Upper bound should be -infinity too"; +} + +TEST_F(IcebergStatsTest, NaNStatsTest) { + auto rowType = ROW({"double_col"}, {DOUBLE()}); + + auto outputDir = exec::test::TempDirectoryPath::create(); + auto dataSink = createIcebergDataSink(rowType, outputDir->getPath()); + constexpr vector_size_t size = 1'000; + auto expectedNulls = 500; + auto expectedNaNs = 0; + + auto rowVector = makeRowVector({makeFlatVector( + size, + [&](vector_size_t row) { + expectedNaNs++; + return std::numeric_limits::quiet_NaN(); + }, + nullEvery(2), + DOUBLE())}); + + dataSink->appendData(rowVector); + ASSERT_TRUE(dataSink->finish()); + auto commitTasks = dataSink->close(); + ASSERT_FALSE(commitTasks.empty()); + + const auto& fileStats = dataSink->dataFileStats(); + ASSERT_EQ(fileStats.size(), 1) << "Expected exactly one file with stats"; + const auto& stats = fileStats[0]; + + EXPECT_EQ(stats->numRecords, size) << "Record count should match input size"; + ASSERT_FALSE(stats->valueCounts.empty()) + << "Should have value counts for columns"; + constexpr int32_t doubleColId = 1; + EXPECT_EQ(stats->valueCounts.at(doubleColId), size) + << "Double column value count incorrect"; + ASSERT_FALSE(stats->nullValueCounts.empty()) + << "Should have null counts for columns"; + EXPECT_EQ(stats->nullValueCounts.at(doubleColId), expectedNulls) + << "Double column null count incorrect"; + EXPECT_EQ(stats->nanValueCounts.at(doubleColId), expectedNaNs) + << "Double column null count incorrect"; + + // Do not collect bounds for NULLs and NaNs. + ASSERT_TRUE(stats->lowerBounds.empty()) + << "Should not have lower bounds for columns"; + ASSERT_TRUE(stats->upperBounds.empty()) + << "Should not have upper bounds for columns"; +} + +TEST_F(IcebergStatsTest, partitionedTableStatsTest) { + auto rowType = ROW( + {"int_col", "date_col", "varchar_col"}, {INTEGER(), DATE(), VARCHAR()}); + auto outputDir = exec::test::TempDirectoryPath::create(); + std::vector partitionTransforms = { + {0, TransformType::kBucket, 4}, + {1, TransformType::kDay, std::nullopt}, + {2, TransformType::kTruncate, 2}}; + + auto dataSink = + createIcebergDataSink(rowType, outputDir->getPath(), partitionTransforms); + + constexpr vector_size_t size = 100; + + auto intVector = + makeFlatVector(size, [](vector_size_t row) { return row; }); + + auto dateVector = makeFlatVector( + size, + [](vector_size_t row) { return 18262 + (row % 5); }, + nullptr, + DATE()); + + auto varcharVector = BaseVector::create(VARCHAR(), size, opPool_.get()); + auto flatVarcharVector = varcharVector->asFlatVector(); + + for (auto i = 0; i < size; ++i) { + std::string str = fmt::format("str{}", i % 10); + flatVarcharVector->set(i, StringView(str.c_str(), str.size())); + } + + auto rowVector = + makeRowVector(rowType->names(), {intVector, dateVector, varcharVector}); + + dataSink->appendData(rowVector); + + ASSERT_TRUE(dataSink->finish()); + auto commitTasks = dataSink->close(); + ASSERT_FALSE(commitTasks.empty()); + + const auto& fileStats = dataSink->dataFileStats(); + // We should have multiple files due to partitioning. + ASSERT_FALSE(fileStats.empty()) << "Should have statistics for files"; + EXPECT_GT(fileStats.size(), 1) + << "Expected multiple files due to partitioning"; + + for (const auto& stats : fileStats) { + EXPECT_GE(stats->numRecords, 0) + << "Each partition file should have records"; + ASSERT_FALSE(stats->valueCounts.empty()) + << "Should have value counts for columns"; + + constexpr int32_t intColId = 1; + constexpr int32_t dateColId = 2; + constexpr int32_t varcharColId = 3; + EXPECT_EQ(stats->valueCounts.at(intColId), stats->numRecords) + << "Integer column value count should match record count"; + EXPECT_EQ(stats->valueCounts.at(dateColId), stats->numRecords) + << "Date column value count should match record count"; + EXPECT_EQ(stats->valueCounts.at(varcharColId), stats->numRecords) + << "Varchar column value count should match record count"; + + ASSERT_FALSE(stats->lowerBounds.empty()) + << "Should have lower bounds for columns"; + ASSERT_FALSE(stats->upperBounds.empty()) + << "Should have upper bounds for columns"; + + EXPECT_FALSE(stats->lowerBounds.at(intColId).empty()) + << "Int column should have non-empty lower bound"; + EXPECT_FALSE(stats->upperBounds.at(intColId).empty()) + << "Int column should have non-empty upper bound"; + + EXPECT_FALSE(stats->lowerBounds.at(dateColId).empty()) + << "Date column should have non-empty lower bound"; + EXPECT_FALSE(stats->upperBounds.at(dateColId).empty()) + << "Date column should have non-empty upper bound"; + + EXPECT_FALSE(stats->lowerBounds.at(varcharColId).empty()) + << "Varchar column should have non-empty lower bound"; + EXPECT_FALSE(stats->upperBounds.at(varcharColId).empty()) + << "Varchar column should have non-empty upper bound"; + } + + // Verify total record count across all partitions. + auto totalRecords = 0; + for (const auto& stats : fileStats) { + totalRecords += stats->numRecords; + } + EXPECT_EQ(totalRecords, size) + << "Total records across all partitions should match input size"; +} + +TEST_F(IcebergStatsTest, multiplePartitionTransformsStatsTest) { + auto rowType = + ROW({"int_col", "date_col", "varchar_col", "bigint_col"}, + {INTEGER(), DATE(), VARCHAR(), BIGINT()}); + auto outputDir = exec::test::TempDirectoryPath::create(); + + std::vector partitionTransforms = { + {0, TransformType::kBucket, 2}, + {1, TransformType::kYear, std::nullopt}, + {2, TransformType::kTruncate, 3}, + {3, TransformType::kIdentity, std::nullopt}}; + + auto dataSink = + createIcebergDataSink(rowType, outputDir->getPath(), partitionTransforms); + + constexpr vector_size_t size = 100; + auto intVector = + makeFlatVector(size, [](vector_size_t row) { return row * 10; }); + auto flatIntVector = intVector->asFlatVector(); + + auto dateVector = makeFlatVector( + size, + [](vector_size_t row) { return 18262 + (row * 100); }, + nullptr, + DATE()); + + auto varcharVector = BaseVector::create(VARCHAR(), size, opPool_.get()); + auto flatVarcharVector = varcharVector->asFlatVector(); + for (auto i = 0; i < size; ++i) { + std::string str = fmt::format("prefix{}_value", i % 5); + flatVarcharVector->set(i, StringView(str.c_str(), str.size())); + } + auto bigintVector = makeFlatVector( + size, [](vector_size_t row) { return (row % 3) * 1'000; }); + + auto rowVector = makeRowVector( + rowType->names(), {intVector, dateVector, varcharVector, bigintVector}); + + dataSink->appendData(rowVector); + + ASSERT_TRUE(dataSink->finish()); + auto commitTasks = dataSink->close(); + ASSERT_FALSE(commitTasks.empty()); + const auto& fileStats = dataSink->dataFileStats(); + ASSERT_FALSE(fileStats.empty()) << "Should have statistics for files"; + EXPECT_GT(fileStats.size(), 1) + << "Expected multiple files due to partitioning"; + // Check each file's stats + for (const auto& stats : fileStats) { + EXPECT_GT(stats->numRecords, 0) + << "Each partition file should have records"; + constexpr int32_t intColId = 1; + constexpr int32_t dateColId = 2; + constexpr int32_t bigintColId = 4; + + if (stats->lowerBounds.find(intColId) != stats->lowerBounds.end()) { + std::string lowerBounds = + encoding::Base64::decode(stats->lowerBounds.at(intColId)); + std::string upperBounds = + encoding::Base64::decode(stats->upperBounds.at(intColId)); + + auto lb = *reinterpret_cast(lowerBounds.data()); + auto ub = *reinterpret_cast(upperBounds.data()); + + EXPECT_LE(lb, ub) + << "Lower bound should be <= upper bound for int column"; + } + + if (stats->lowerBounds.find(dateColId) != stats->lowerBounds.end()) { + std::string lowerBounds = + encoding::Base64::decode(stats->lowerBounds.at(dateColId)); + std::string upperBounds = + encoding::Base64::decode(stats->upperBounds.at(dateColId)); + + auto lb = *reinterpret_cast(lowerBounds.data()); + auto ub = *reinterpret_cast(upperBounds.data()); + + EXPECT_LE(lb, ub) + << "Lower bound should be <= upper bound for date column"; + } + + if (stats->lowerBounds.find(bigintColId) != stats->lowerBounds.end()) { + std::string lowerBounds = + encoding::Base64::decode(stats->lowerBounds.at(bigintColId)); + std::string upperBounds = + encoding::Base64::decode(stats->upperBounds.at(bigintColId)); + + auto lb = *reinterpret_cast(lowerBounds.data()); + auto ub = *reinterpret_cast(upperBounds.data()); + + EXPECT_LE(lb, ub) + << "Lower bound should be <= upper bound for bigint column"; + } + } + auto totalRecords = 0; + for (const auto& stats : fileStats) { + totalRecords += stats->numRecords; + } + EXPECT_EQ(totalRecords, size) + << "Total records across all partitions should match input size"; +} + +TEST_F(IcebergStatsTest, partitionedTableWithNullsStatsTest) { + auto rowType = ROW( + {"int_col", "date_col", "varchar_col"}, {INTEGER(), DATE(), VARCHAR()}); + auto outputDir = exec::test::TempDirectoryPath::create(); + std::vector partitionTransforms = { + {0, TransformType::kIdentity, std::nullopt}, + {1, TransformType::kMonth, std::nullopt}, + {2, TransformType::kTruncate, 2}}; + auto dataSink = + createIcebergDataSink(rowType, outputDir->getPath(), partitionTransforms); + + constexpr vector_size_t size = 100; + auto expectedNulls = 20; + auto dateNulls = 15; + auto intVector = makeFlatVector( + size, + [](vector_size_t row) { return row % 10; }, + nullEvery(5), + INTEGER()); + auto dateVector = makeFlatVector( + size, + [](vector_size_t row) { return 18262 + (row % 3) * 30; }, + nullEvery(7), + DATE()); + auto varcharVector = BaseVector::create(VARCHAR(), size, opPool_.get()); + auto flatVarcharVector = varcharVector->asFlatVector(); + auto varcharNulls = 0; + for (auto i = 0; i < size; ++i) { + if (i % 11 == 0) { + flatVarcharVector->setNull(i, true); + varcharNulls++; + } else { + std::string str = fmt::format("val{}", i % 5); + flatVarcharVector->set(i, StringView(str.c_str(), str.size())); + } + } + + auto rowVector = + makeRowVector(rowType->names(), {intVector, dateVector, varcharVector}); + + dataSink->appendData(rowVector); + ASSERT_TRUE(dataSink->finish()); + auto commitTasks = dataSink->close(); + ASSERT_FALSE(commitTasks.empty()); + const auto& fileStats = dataSink->dataFileStats(); + ASSERT_FALSE(fileStats.empty()) << "Should have statistics for files"; + auto totalIntNulls = 0; + auto totalDateNulls = 0; + auto totalVarcharNulls = 0; + auto totalRecords = 0; + + constexpr int32_t intColId = 1; + constexpr int32_t dateColId = 2; + constexpr int32_t varcharColId = 3; + + for (const auto& stats : fileStats) { + totalRecords += stats->numRecords; + // Add null counts if present. + if (stats->nullValueCounts.find(intColId) != stats->nullValueCounts.end()) { + totalIntNulls += stats->nullValueCounts.at(intColId); + } + + if (stats->nullValueCounts.find(dateColId) != + stats->nullValueCounts.end()) { + totalDateNulls += stats->nullValueCounts.at(dateColId); + } + + if (stats->nullValueCounts.find(varcharColId) != + stats->nullValueCounts.end()) { + totalVarcharNulls += stats->nullValueCounts.at(varcharColId); + } + + // Check that null count is less than or equal to value count for each + // column. + if (stats->nullValueCounts.find(intColId) != stats->nullValueCounts.end() && + stats->valueCounts.find(intColId) != stats->valueCounts.end()) { + EXPECT_LE( + stats->nullValueCounts.at(intColId), stats->valueCounts.at(intColId)) + << "Null count should be <= value count for int column"; + } + + if (stats->nullValueCounts.find(dateColId) != + stats->nullValueCounts.end() && + stats->valueCounts.find(dateColId) != stats->valueCounts.end()) { + EXPECT_LE( + stats->nullValueCounts.at(dateColId), + stats->valueCounts.at(dateColId)) + << "Null count should be <= value count for date column"; + } + + if (stats->nullValueCounts.find(varcharColId) != + stats->nullValueCounts.end() && + stats->valueCounts.find(varcharColId) != stats->valueCounts.end()) { + EXPECT_LE( + stats->nullValueCounts.at(varcharColId), + stats->valueCounts.at(varcharColId)) + << "Null count should be <= value count for varchar column"; + } + } + + // Verify total counts match expected. + EXPECT_EQ(totalRecords, size) + << "Total records across all partitions should match input size"; + EXPECT_EQ(totalIntNulls, expectedNulls) + << "Total int nulls should match expected"; + EXPECT_EQ(totalDateNulls, dateNulls) + << "Total date nulls should match expected"; + EXPECT_EQ(totalVarcharNulls, varcharNulls) + << "Total varchar nulls should match expected"; +} + +TEST_F(IcebergStatsTest, mapTypeTest) { + auto rowType = + ROW({"int_col", "map_col"}, {INTEGER(), MAP(INTEGER(), VARCHAR())}); + auto outputDir = exec::test::TempDirectoryPath::create(); + auto dataSink = createIcebergDataSink(rowType, outputDir->getPath()); + constexpr vector_size_t size = 100; + auto expectedNulls = 0; + + std::vector< + std::optional>>>> + mapData; + for (auto i = 0; i < size; ++i) { + std::vector>> mapRow; + for (auto j = 0; j < 5; ++j) { + mapRow.emplace_back(j, StringView("test_value")); + } + mapData.push_back(mapRow); + } + + auto rowVector = makeRowVector( + {makeFlatVector(size, [&](auto row) { return row * 10; }), + makeNullableMapVector(mapData)}); + + dataSink->appendData(rowVector); + ASSERT_TRUE(dataSink->finish()); + auto commitTasks = dataSink->close(); + ASSERT_FALSE(commitTasks.empty()); + + const auto& fileStats = dataSink->dataFileStats(); + ASSERT_EQ(fileStats.size(), 1) << "Expected exactly one file with stats"; + const auto& stats = fileStats[0]; + + EXPECT_EQ(stats->numRecords, size) << "Record count should match input size"; + + constexpr int32_t intColId = 1; + constexpr int32_t mapColId = 3; + + EXPECT_EQ(stats->valueCounts.at(intColId), size) + << "Int column value count incorrect"; + EXPECT_EQ(stats->nullValueCounts.at(intColId), expectedNulls) + << "Int column null count incorrect"; + EXPECT_EQ(stats->valueCounts.at(mapColId), size * 5) + << "Map column value count incorrect"; + EXPECT_TRUE(stats->lowerBounds.find(mapColId) == stats->lowerBounds.end()) + << "Map column should not have lower bounds"; + EXPECT_TRUE(stats->upperBounds.find(mapColId) == stats->upperBounds.end()) + << "Map column should not have upper bounds"; +} + +TEST_F(IcebergStatsTest, arrayTypeTest) { + auto rowType = ROW({"int_col", "array_col"}, {INTEGER(), ARRAY(VARCHAR())}); + auto outputDir = exec::test::TempDirectoryPath::create(); + auto dataSink = createIcebergDataSink(rowType, outputDir->getPath()); + constexpr vector_size_t size = 100; + auto expectedNulls = 0; + + std::vector>> arrayData; + for (auto i = 0; i < size; ++i) { + std::vector> arrayRow; + for (auto j = 0; j < 3; ++j) { + auto v = fmt::format("item_{}", i * 3 + j); + arrayRow.emplace_back(StringView(v)); + } + arrayData.push_back(arrayRow); + } + + auto rowVector = makeRowVector( + {makeFlatVector(size, [](auto row) { return row * 10; }), + makeNullableArrayVector(arrayData)}); + + dataSink->appendData(rowVector); + ASSERT_TRUE(dataSink->finish()); + auto commitTasks = dataSink->close(); + ASSERT_FALSE(commitTasks.empty()); + + const auto& fileStats = dataSink->dataFileStats(); + ASSERT_EQ(fileStats.size(), 1) << "Expected exactly one file with stats"; + const auto& stats = fileStats[0]; + + EXPECT_EQ(stats->numRecords, size) << "Record count should match input size"; + + constexpr int32_t intColId = 1; + constexpr int32_t arrayColId = 3; + + EXPECT_EQ(stats->valueCounts.at(intColId), size) + << "Int column value count incorrect"; + EXPECT_EQ(stats->nullValueCounts.at(intColId), expectedNulls) + << "Int column null count incorrect"; + EXPECT_EQ(stats->valueCounts.at(arrayColId), size * 3) + << "Array column value count incorrect"; + EXPECT_TRUE(stats->lowerBounds.find(arrayColId) == stats->lowerBounds.end()) + << "Array column should not have lower bounds"; + EXPECT_TRUE(stats->upperBounds.find(arrayColId) == stats->upperBounds.end()) + << "Array column should not have upper bounds"; +} + +// Test statistics collection for nested struct fields. +// Assume int_col's ID start with 1. +// Struct definition with field IDs: +// struct { +// int_col: INTEGER (id: 1) +// struct_col (id: 2) { +// first_level_id: INTEGER (id: 3) +// first_level_name: VARCHAR (id: 4) +// nested_struct (id: 5) { +// second_level_id: INTEGER (id: 6) +// second_level_name: VARCHAR (id: 7) +// } +// } +// } +// Need to collect statistics for field IDs [1, 3, 4, 6, 7] +TEST_F(IcebergStatsTest, structTypeTest) { + auto rowType = + ROW({"int_col", "struct_col"}, + {INTEGER(), + ROW({"first_level_id", "first_level_name", "nested_struct"}, + {INTEGER(), + VARCHAR(), + ROW({"second_level_id", "second_level_name"}, + {INTEGER(), VARCHAR()})})}); + auto outputDir = exec::test::TempDirectoryPath::create(); + auto dataSink = createIcebergDataSink(rowType, outputDir->getPath()); + constexpr vector_size_t size = 100; + auto expectedNulls = 0; + + auto intVector = + makeFlatVector(size, [](auto row) { return row * 10; }); + auto firstLevelId = makeFlatVector( + size, [](vector_size_t row) { return row % size; }, nullEvery(5)); + auto firstLevelName = makeFlatVector( + size, + [](vector_size_t row) { + auto v = fmt::format("name_{}", row * 10); + return StringView(v); + }, + nullEvery(7)); + + auto secondLevelId = makeFlatVector( + size, [](vector_size_t row) { return row * size; }, nullEvery(6)); + auto secondLevelName = makeFlatVector( + size, + [](vector_size_t row) { + auto v = fmt::format("nested_{}", row * 100); + return StringView(v); + }, + nullEvery(8)); + + auto nestedStruct = makeRowVector({secondLevelId, secondLevelName}); + auto structVector = + makeRowVector({firstLevelId, firstLevelName, nestedStruct}); + + auto rowVector = makeRowVector({intVector, structVector}); + + dataSink->appendData(rowVector); + ASSERT_TRUE(dataSink->finish()); + auto commitTasks = dataSink->close(); + ASSERT_FALSE(commitTasks.empty()); + + const auto& fileStats = dataSink->dataFileStats(); + ASSERT_EQ(fileStats.size(), 1) << "Expected exactly one file with stats"; + const auto& stats = fileStats[0]; + + EXPECT_EQ(stats->numRecords, size) << "Record count should match input size"; + + constexpr int32_t intColId = 1; + constexpr int32_t tier1ColId = 3; + constexpr int32_t tier2ColId = 6; + constexpr int32_t tier2ColId2 = 7; + + EXPECT_EQ(stats->valueCounts.size(), 5); + EXPECT_EQ(stats->lowerBounds.size(), 5); + EXPECT_EQ(stats->valueCounts.at(intColId), size); + EXPECT_EQ(stats->nullValueCounts.at(intColId), expectedNulls); + EXPECT_EQ(stats->valueCounts.at(tier1ColId), size); + EXPECT_EQ(stats->valueCounts.at(tier2ColId), size); + EXPECT_EQ(stats->nullValueCounts.at(tier1ColId), 20); + EXPECT_EQ( + encoding::Base64::decode(stats->lowerBounds.at(tier2ColId2)), + "nested_100"); + EXPECT_EQ( + encoding::Base64::decode(stats->upperBounds.at(tier2ColId2)), + "nested_9900"); +} + +} // namespace facebook::velox::connector::hive::iceberg::test diff --git a/velox/connectors/hive/iceberg/tests/IcebergTestBase.cpp b/velox/connectors/hive/iceberg/tests/IcebergTestBase.cpp index c7c7a581c46f..0b900ee8c27a 100644 --- a/velox/connectors/hive/iceberg/tests/IcebergTestBase.cpp +++ b/velox/connectors/hive/iceberg/tests/IcebergTestBase.cpp @@ -121,32 +121,66 @@ std::shared_ptr IcebergTestBase::createPartitionSpec( void addColumnHandles( const RowTypePtr& rowType, const std::vector& partitionFields, - std::vector& columnHandles) { + std::vector>& columnHandles) { std::unordered_set partitionColumnIds; for (const auto& field : partitionFields) { partitionColumnIds.insert(field.id); } + HiveColumnHandle::ColumnParseParameters columnParseParameters; + + std::function + collectNestedField = [&](const TypePtr& type, + int32_t& columnOrdinal) -> IcebergNestedField { + int32_t currentId = columnOrdinal++; + std::vector children; + if (type->isRow()) { + auto rowType = asRowType(type); + for (auto i = 0; i < rowType->size(); ++i) { + children.push_back( + collectNestedField(rowType->childAt(i), columnOrdinal)); + } + } else if (type->isArray()) { + auto arrayType = std::dynamic_pointer_cast(type); + for (auto i = 0; i < arrayType->size(); ++i) { + children.push_back( + collectNestedField(arrayType->childAt(i), columnOrdinal)); + } + } else if (type->isMap()) { + auto mapType = std::dynamic_pointer_cast(type); + for (auto i = 0; i < mapType->size(); ++i) { + children.push_back( + collectNestedField(mapType->childAt(i), columnOrdinal)); + } + } - columnHandles.reserve(rowType->size()); - for (auto i = 0; i < rowType->size(); ++i) { - const auto columnType = partitionColumnIds.contains(i) - ? HiveColumnHandle::ColumnType::kPartitionKey - : HiveColumnHandle::ColumnType::kRegular; + return IcebergNestedField{currentId, children}; + }; + int32_t startIndex = 1; + for (auto i = 0; i < rowType->size(); ++i) { + auto columnName = rowType->nameOf(i); + auto type = rowType->childAt(i); + auto field = collectNestedField(type, startIndex); columnHandles.push_back( - std::make_shared( - rowType->nameOf(i), - columnType, - rowType->childAt(i), - rowType->childAt(i))); + std::make_shared( + columnName, + partitionColumnIds.count(i) > 0 + ? HiveColumnHandle::ColumnType::kPartitionKey + : HiveColumnHandle::ColumnType::kRegular, + type, + type, + field, + std::vector{}, + columnParseParameters)); } } -IcebergInsertTableHandlePtr IcebergTestBase::createInsertTableHandle( +IcebergInsertTableHandlePtr IcebergTestBase::createIcebergInsertTableHandle( const RowTypePtr& rowType, const std::string& outputDirectoryPath, - const std::vector& partitionFields) { - std::vector columnHandles; + const std::vector& partitionFields, + const std::vector& sortedBy) { + std::vector> columnHandles; addColumnHandles(rowType, partitionFields, columnHandles); auto locationHandle = std::make_shared( @@ -156,42 +190,62 @@ IcebergInsertTableHandlePtr IcebergTestBase::createInsertTableHandle( auto partitionSpec = createPartitionSpec(partitionFields, rowType); - return std::make_shared( - /*inputColumns=*/columnHandles, + // Create sorting columns if specified + std::vector sortingColumns; + for (const auto& sortExpr : sortedBy) { + std::string columnName; + bool isAscending = true; + bool isNullsFirst = true; + + // Parse sort expression + std::istringstream iss(sortExpr); + iss >> columnName; + + std::string token; + if (iss >> token) { + if (token == "DESC") { + isAscending = false; + } else if (token != "ASC") { + // If not ASC, put it back (might be NULLS) + iss.seekg(-(int)token.length(), std::ios_base::cur); + } + + if (iss >> token && token == "NULLS") { + if (iss >> token && token == "LAST") { + isNullsFirst = false; + } + } + } + + core::SortOrder sortOrder(isAscending, isNullsFirst); + sortingColumns.push_back(IcebergSortingColumn(columnName, sortOrder)); + } + + return std::make_shared( + columnHandles, locationHandle, - /*tableStorageFormat=*/fileFormat_, partitionSpec, - /*compressionKind=*/common::CompressionKind::CompressionKind_ZSTD); + opPool_.get(), + fileFormat_, + sortingColumns, + common::CompressionKind::CompressionKind_ZSTD); } -std::shared_ptr IcebergTestBase::createDataSink( +std::shared_ptr IcebergTestBase::createIcebergDataSink( const RowTypePtr& rowType, const std::string& outputDirectoryPath, - const std::vector& partitionFields) { - auto tableHandle = - createInsertTableHandle(rowType, outputDirectoryPath, partitionFields); + const std::vector& partitionFields, + const std::vector& sortedBy) { + auto tableHandle = createIcebergInsertTableHandle( + rowType, outputDirectoryPath, partitionFields, sortedBy); return std::make_shared( rowType, tableHandle, connectorQueryCtx_.get(), - CommitStrategy::kNoCommit, + connector::CommitStrategy::kNoCommit, connectorConfig_); } -std::shared_ptr IcebergTestBase::createDataSinkAndAppendData( - const RowTypePtr& rowType, - const std::vector& vectors, - const std::string& dataPath, - const std::vector& partitionFields) { - auto dataSink = createDataSink(rowType, dataPath, partitionFields); - - for (const auto& vector : vectors) { - dataSink->appendData(vector); - } - EXPECT_TRUE(dataSink->finish()); - return dataSink; -} - std::vector IcebergTestBase::listFiles( const std::string& dirPath) { std::vector files; @@ -216,6 +270,24 @@ IcebergTestBase::createSplitsForDirectory(const std::string& directory) { 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); splits.push_back( @@ -225,11 +297,11 @@ IcebergTestBase::createSplitsForDirectory(const std::string& directory) { fileFormat_, 0, file->size(), - std::unordered_map>{}, + partitionKeys, std::nullopt, customSplitInfo, nullptr, - /*cacheable=*/true, + true, std::vector())); } diff --git a/velox/connectors/hive/iceberg/tests/IcebergTestBase.h b/velox/connectors/hive/iceberg/tests/IcebergTestBase.h index ffe0a59c09af..7fdda4618c08 100644 --- a/velox/connectors/hive/iceberg/tests/IcebergTestBase.h +++ b/velox/connectors/hive/iceberg/tests/IcebergTestBase.h @@ -19,6 +19,7 @@ #include #include "velox/connectors/hive/iceberg/IcebergDataSink.h" +#include "velox/connectors/hive/iceberg/IcebergSplit.h" #include "velox/exec/tests/utils/HiveConnectorTestBase.h" #include "velox/exec/tests/utils/TempDirectoryPath.h" #include "velox/vector/fuzzer/VectorFuzzer.h" @@ -30,10 +31,9 @@ namespace facebook::velox::connector::hive::iceberg::test { struct PartitionField { - // 0-based column index. - int32_t id; + int32_t id; // Index of column in RowType, start from 0. TransformType type; - std::optional parameter; + std::optional parameter; // Optional parameter of transform. }; class IcebergTestBase : public exec::test::HiveConnectorTestBase { @@ -48,16 +48,11 @@ class IcebergTestBase : public exec::test::HiveConnectorTestBase { vector_size_t rowsPerBatch, double nullRatio = 0.0); - std::shared_ptr createDataSink( + std::shared_ptr createIcebergDataSink( const RowTypePtr& rowType, const std::string& outputDirectoryPath, - const std::vector& partitionFields = {}); - - std::shared_ptr createDataSinkAndAppendData( - const RowTypePtr& rowType, - const std::vector& vectors, - const std::string& dataPath, - const std::vector& partitionFields = {}); + const std::vector& partitionTransforms = {}, + const std::vector& sortedBy = {}); std::vector> createSplitsForDirectory( const std::string& directory); @@ -65,26 +60,28 @@ class IcebergTestBase : public exec::test::HiveConnectorTestBase { std::vector listFiles(const std::string& dirPath); std::shared_ptr createPartitionSpec( - const std::vector& partitionFields, + const std::vector& transformSpecs, const RowTypePtr& rowType); - dwio::common::FileFormat fileFormat_{dwio::common::FileFormat::DWRF}; + void setupMemoryPools(); + + dwio::common::FileFormat fileFormat_{dwio::common::FileFormat::PARQUET}; + RowTypePtr rowType_; + std::shared_ptr opPool_; + std::shared_ptr connectorSessionProperties_; private: - IcebergInsertTableHandlePtr createInsertTableHandle( + IcebergInsertTableHandlePtr createIcebergInsertTableHandle( const RowTypePtr& rowType, const std::string& outputDirectoryPath, - const std::vector& partitionFields = {}); + const std::vector& partitionTransforms = {}, + const std::vector& sortedBy = {}); std::vector listPartitionDirectories( const std::string& dataPath); - void setupMemoryPools(); - 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_; diff --git a/velox/connectors/hive/iceberg/tests/IcebergTransformE2ETest.cpp b/velox/connectors/hive/iceberg/tests/IcebergTransformE2ETest.cpp new file mode 100644 index 000000000000..d61f6012f764 --- /dev/null +++ b/velox/connectors/hive/iceberg/tests/IcebergTransformE2ETest.cpp @@ -0,0 +1,943 @@ +/* + * 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/connectors/hive/iceberg/tests/IcebergTestBase.h" +#include "velox/exec/tests/utils/PlanBuilder.h" +#include "velox/exec/tests/utils/TempDirectoryPath.h" + +using namespace facebook::velox::exec::test; + +namespace facebook::velox::connector::hive::iceberg::test { +class IcebergTransformE2ETest : public IcebergTestBase { + protected: + void SetUp() override { + IcebergTestBase::SetUp(); + rowType_ = + ROW({"c_int", + "c_bigint", + "c_varchar", + "c_date", + "c_decimal", + "c_varbinary", + "c_timestamp"}, + {INTEGER(), + BIGINT(), + VARCHAR(), + DATE(), + DECIMAL(18, 3), + VARBINARY(), + TIMESTAMP()}); + rng_.seed(1); + } + + std::pair buildFilter( + const std::string& partitionDir) { + const auto eq = partitionDir.find('='); + const auto us = partitionDir.rfind('_', eq - 1); + const auto column = partitionDir.substr(0, us); + const auto value = partitionDir.substr(eq + 1); + return {column, value}; + } + + std::vector createTestData( + int32_t numBatches, + int32_t rowsPerBatch) { + std::vector batches; + for (auto batchIdx = 0; batchIdx < numBatches; ++batchIdx) { + std::vector columns; + columns.push_back(makeFlatVector(rowsPerBatch, [](auto row) { + return row % 100; + })); + columns.push_back(makeFlatVector(rowsPerBatch, [](auto row) { + return row * 1'000; + })); + auto varcharVector = BaseVector::create>( + VARCHAR(), rowsPerBatch, opPool_.get()); + for (auto i = 0; i < rowsPerBatch; i++) { + std::string s = + fmt::format("string_long_data_test__{}__{}", i % 10, i % 100); + varcharVector->set(i, StringView(s)); + } + columns.push_back(varcharVector); + + auto dateVector = BaseVector::create>( + DATE(), rowsPerBatch, opPool_.get()); + for (auto i = 0; i < rowsPerBatch; i++) { + static const std::vector dates = { + 18'262, 18'628, 18'993, 19'358, 19'723, 20'181}; + dateVector->set(i, dates[i % dates.size()]); + } + columns.push_back(dateVector); + + auto decimalVector = BaseVector::create>( + DECIMAL(18, 3), rowsPerBatch, opPool_.get()); + for (auto i = 0; i < rowsPerBatch; i++) { + decimalVector->set(i, (i % 10 + 1) * 123'456); + } + columns.push_back(decimalVector); + + auto varbinaryVector = BaseVector::create>( + VARBINARY(), rowsPerBatch, opPool_.get()); + std::vector binaryData; + for (auto i = 0; i < rowsPerBatch; i++) { + if (i % 5 == 0) { + for (int32_t j = 0; j < 40; j++) { + binaryData.push_back(static_cast(j + (i % 10))); + } + } else if (i % 5 == 1) { + for (int32_t j = 0; j < 40; j++) { + binaryData.push_back(static_cast(i % 256)); + } + } else if (i % 5 == 2) { + for (int32_t j = 0; j < 40; j++) { + binaryData.push_back( + static_cast(j % 2 == 0 ? 0xAA : 0x55)); + } + } else { + for (int32_t j = 0; j < 34; j++) { + binaryData.push_back(static_cast(255 - j - (i % 10))); + } + } + + varbinaryVector->set( + i, + StringView( + reinterpret_cast(binaryData.data()), + binaryData.size())); + } + columns.push_back(varbinaryVector); + + // Add timestamp column with different time values. + auto timestampVector = BaseVector::create>( + TIMESTAMP(), rowsPerBatch, opPool_.get()); + for (auto i = 0; i < rowsPerBatch; i++) { + // Create timestamps for different years, months, days, and hours + // to test various transforms. + static const std::vector timestamps = { + Timestamp(0, 0), // 1970-01-01 00:00:00 + Timestamp(3600, 0), // 1970-01-01 01:00:00 + Timestamp(86400, 0), // 1970-01-02 00:00:00 + Timestamp(2592000, 0), // 1970-01-31 00:00:00 + Timestamp(31536000, 0), // 1971-01-01 00:00:00 + Timestamp(1609459200, 0), // 2021-01-01 00:00:00 + Timestamp(1609545600, 0), // 2021-01-02 00:00:00 + Timestamp(1612224000, 0), // 2021-02-01 00:00:00 + Timestamp(1640995200, 0), // 2022-01-01 00:00:00 + Timestamp(1672531200, 0) // 2023-01-01 00:00:00 + }; + timestampVector->set(i, timestamps[i % timestamps.size()]); + } + columns.push_back(timestampVector); + + batches.push_back(makeRowVector(rowType_->names(), columns)); + } + return batches; + } + + std::vector listFirstLevelDirectories( + const std::string& basePath) { + std::vector partitionDirs; + for (const auto& entry : std::filesystem::directory_iterator(basePath)) { + if (entry.is_directory()) { + partitionDirs.push_back(entry.path().string()); + } + } + return partitionDirs; + } + + std::vector listDirectoriesRecursively(const std::string& path) { + std::vector allDirs; + auto firstLevelDirs = listFirstLevelDirectories(path); + allDirs.insert(allDirs.end(), firstLevelDirs.begin(), firstLevelDirs.end()); + + for (const auto& dir : firstLevelDirs) { + if (std::filesystem::is_directory(dir)) { + auto subDirs = listDirectoriesRecursively(dir); + allDirs.insert(allDirs.end(), subDirs.begin(), subDirs.end()); + } + } + + return allDirs; + } + + // Verify the number of partitions and their naming convention. + void verifyPartitionCount( + const std::string& outputPath, + const std::vector& partitionTransforms, + const int32_t expectedPartitionCount) { + const auto partitionDirs = listFirstLevelDirectories(outputPath); + + if (partitionTransforms.empty()) { + ASSERT_EQ(partitionDirs.size(), 1) + << "Expected 1 directory for no partitioning, got " + << partitionDirs.size(); + } else { + ASSERT_EQ(partitionDirs.size(), expectedPartitionCount) + << "Expected " << expectedPartitionCount << " partitions, got " + << partitionDirs.size(); + + for (const auto& dir : partitionDirs) { + const auto dirName = std::filesystem::path(dir).filename().string(); + ASSERT_TRUE(dirName.find('=') != std::string::npos) + << "Partition directory " << dirName + << " does not follow Iceberg naming convention"; + } + } + } + + // Verify the total row count across all partitions. + void verifyTotalRowCount( + RowTypePtr rowType, + const std::string& outputPath, + int32_t expectedRowCount) { + auto splits = createSplitsForDirectory(outputPath); + + const auto plan = PlanBuilder() + .tableScan(rowType) + .singleAggregation({}, {"count(1)"}) + .planNode(); + + const auto result = + AssertQueryBuilder(plan).splits(splits).copyResults(opPool_.get()); + + ASSERT_EQ(result->size(), 1); + ASSERT_EQ( + result->childAt(0)->asFlatVector()->valueAt(0), + expectedRowCount); + } + + // Verify data in a specific partition. + void verifyPartitionData( + RowTypePtr rowType, + const std::string& partitionPath, + const std::string& partitionFilter, + const int32_t expectedRowCount, + bool skipRowCountCheck = false) { + const auto splits = createSplitsForDirectory(partitionPath); + + const auto countPlan = PlanBuilder() + .tableScan(rowType) + .singleAggregation({}, {"count(1)"}) + .planNode(); + + const auto countResult = + AssertQueryBuilder(countPlan).splits(splits).copyResults(opPool_.get()); + + ASSERT_EQ(countResult->size(), 1); + const auto actualRowCount = + countResult->childAt(0)->asFlatVector()->valueAt(0); + + if (!skipRowCountCheck) { + ASSERT_EQ(actualRowCount, expectedRowCount); + } else { + // Just verify that we have some data. + ASSERT_GT(actualRowCount, 0); + } + + const auto dataPlan = PlanBuilder() + .tableScan(rowType) + .filter(partitionFilter) + .singleAggregation({}, {"count(1)"}) + .planNode(); + const auto dataResult = + AssertQueryBuilder(dataPlan).splits(splits).copyResults(opPool_.get()); + ASSERT_EQ(dataResult->size(), 1); + const auto filteredRowCount = + dataResult->childAt(0)->asFlatVector()->valueAt(0); + if (!skipRowCountCheck) { + ASSERT_EQ(filteredRowCount, expectedRowCount); + } else { + // Just verify that the filter matches all rows in the partition. + ASSERT_EQ(filteredRowCount, actualRowCount); + } + } + + folly::Random::DefaultGenerator rng_; +}; + +TEST_F(IcebergTransformE2ETest, identityPartitioning) { + constexpr auto numBatches = 2; + constexpr auto rowsPerBatch = 100; + auto vectors = createTestData(numBatches, rowsPerBatch); + auto outputDirectory = TempDirectoryPath::create(); + auto dataSink = createIcebergDataSink( + rowType_, + outputDirectory->getPath(), + {{0, TransformType::kIdentity, std::nullopt}}); + for (const auto& vector : vectors) { + dataSink->appendData(vector); + } + + ASSERT_TRUE(dataSink->finish()); + dataSink->close(); + verifyPartitionCount(outputDirectory->getPath(), {"c_int"}, rowsPerBatch); + verifyTotalRowCount( + rowType_, outputDirectory->getPath(), numBatches * rowsPerBatch); + + auto partitionDirs = listFirstLevelDirectories(outputDirectory->getPath()); + for (const auto& dir : partitionDirs) { + const auto dirName = std::filesystem::path(dir).filename().string(); + verifyPartitionData(rowType_, dir, dirName, numBatches); + } +} + +TEST_F(IcebergTransformE2ETest, truncatePartitioning) { + constexpr auto numBatches = 2; + constexpr auto rowsPerBatch = 100; + auto vectors = createTestData(numBatches, rowsPerBatch); + auto outputDirectory = TempDirectoryPath::create(); + auto dataSink = createIcebergDataSink( + rowType_, + outputDirectory->getPath(), + {{0, TransformType::kTruncate, 10}}); + for (const auto& vector : vectors) { + dataSink->appendData(vector); + } + + ASSERT_TRUE(dataSink->finish()); + dataSink->close(); + + verifyPartitionCount(outputDirectory->getPath(), {"truncate(c_int, 10)"}, 10); + verifyTotalRowCount( + rowType_, outputDirectory->getPath(), numBatches * rowsPerBatch); + const auto partitionDirs = + listFirstLevelDirectories(outputDirectory->getPath()); + + for (const auto& dir : partitionDirs) { + const std::string dirName = std::filesystem::path(dir).filename().string(); + auto [c, v] = buildFilter(dirName); + const std::string filter = + c + ">=" + v + " AND " + c + "<" + std::to_string(std::stoi(v) + 10); + verifyPartitionData( + rowType_, dir, filter, 20); // 10 values per batch * 2 batches. + } +} + +TEST_F(IcebergTransformE2ETest, bucketPartitioning) { + constexpr auto numBatches = 2; + constexpr auto rowsPerBatch = 100; + auto vectors = createTestData(numBatches, rowsPerBatch); + auto outputDirectory = TempDirectoryPath::create(); + auto dataSink = createIcebergDataSink( + rowType_, outputDirectory->getPath(), {{2, TransformType::kBucket, 4}}); + for (const auto& vector : vectors) { + dataSink->appendData(vector); + } + + ASSERT_TRUE(dataSink->finish()); + dataSink->close(); + + // Verify the number of partitions (should be at most 4 buckets). + auto partitionDirs = listFirstLevelDirectories(outputDirectory->getPath()); + ASSERT_EQ(partitionDirs.size(), 4); + + verifyTotalRowCount( + rowType_, outputDirectory->getPath(), numBatches * rowsPerBatch); + + int32_t totalRowsInPartitions = 0; + for (const auto& dir : partitionDirs) { + auto splits = createSplitsForDirectory(dir); + auto countPlan = PlanBuilder() + .tableScan(rowType_) + .singleAggregation({}, {"count(1)"}) + .planNode(); + auto countResult = + AssertQueryBuilder(countPlan).splits(splits).copyResults(opPool_.get()); + + auto partitionRowCount = + countResult->childAt(0)->asFlatVector()->valueAt(0); + totalRowsInPartitions += partitionRowCount; + ASSERT_GE(partitionRowCount, 0); + } + + ASSERT_EQ(totalRowsInPartitions, numBatches * rowsPerBatch); + + // Verify that each partition contains only rows with the same bucket value. + for (const auto& dir : partitionDirs) { + const auto dirName = std::filesystem::path(dir).filename().string(); + const auto equalsPos = dirName.find('='); + ASSERT_NE(equalsPos, std::string::npos); + auto dataPlan = + PlanBuilder().tableScan(rowType_).project({"c_varchar"}).planNode(); + auto dataResult = AssertQueryBuilder(dataPlan) + .splits(createSplitsForDirectory(dir)) + .copyResults(opPool_.get()); + // Verify that all rows in this partition have the same bucket hash value. + auto varcharColumn = dataResult->childAt(0)->asFlatVector(); + for (auto i = 0; i < dataResult->size(); i++) { + StringView value = varcharColumn->valueAt(i); + auto valuePlan = PlanBuilder() + .tableScan(rowType_) + .filter(fmt::format("c_varchar = '{}'", value.str())) + .project({"c_varchar"}) + .planNode(); + + auto valueResult = + AssertQueryBuilder(valuePlan) + .splits(createSplitsForDirectory(outputDirectory->getPath())) + .copyResults(opPool_.get()); + auto valueCount = valueResult->size(); + auto partitionValuePlan = + PlanBuilder() + .tableScan(rowType_) + .filter(fmt::format("c_varchar = '{}'", value.str())) + .project({"c_varchar"}) + .planNode(); + auto partitionValueResult = AssertQueryBuilder(partitionValuePlan) + .splits(createSplitsForDirectory(dir)) + .copyResults(opPool_.get()); + + ASSERT_EQ(partitionValueResult->size(), valueCount); + } + } +} + +TEST_F(IcebergTransformE2ETest, yearPartitioning) { + constexpr auto numBatches = 2; + constexpr auto rowsPerBatch = 100; + auto vectors = createTestData(numBatches, rowsPerBatch); + auto outputDirectory = TempDirectoryPath::create(); + auto dataSink = createIcebergDataSink( + rowType_, + outputDirectory->getPath(), + {{3, TransformType::kYear, std::nullopt}}); + for (const auto& vector : vectors) { + dataSink->appendData(vector); + } + ASSERT_TRUE(dataSink->finish()); + dataSink->close(); + + // Verify the number of partitions (should be 6 for years 2020-2025). + verifyPartitionCount(outputDirectory->getPath(), {"year(c_date)"}, 6); + verifyTotalRowCount( + rowType_, outputDirectory->getPath(), numBatches * rowsPerBatch); + + auto partitionDirs = listFirstLevelDirectories(outputDirectory->getPath()); + + for (int32_t year = 2020; year <= 2025; year++) { + const auto expectedDirName = fmt::format("c_date_year={}", year); + bool foundPartition = false; + auto yearFilter = [](const int32_t year) -> std::string { + return fmt::format( + "YEAR(DATE '{}-01-01')={}", + std::to_string(year), + std::to_string(year)); + }; + + for (const auto& dir : partitionDirs) { + const auto dirName = std::filesystem::path(dir).filename().string(); + if (dirName == expectedDirName) { + foundPartition = true; + auto datePlan = PlanBuilder() + .tableScan(rowType_) + .filter(yearFilter(year)) + .singleAggregation({}, {"count(1)"}) + .planNode(); + + auto dateResult = AssertQueryBuilder(datePlan) + .splits(createSplitsForDirectory(dir)) + .copyResults(opPool_.get()); + + auto partitionRowCount = + dateResult->childAt(0)->asFlatVector()->valueAt(0); + auto countPlan = PlanBuilder() + .tableScan(rowType_) + .singleAggregation({}, {"count(1)"}) + .planNode(); + auto countResult = AssertQueryBuilder(countPlan) + .splits(createSplitsForDirectory(dir)) + .copyResults(opPool_.get()); + auto totalPartitionCount = + countResult->childAt(0)->asFlatVector()->valueAt(0); + ASSERT_EQ(partitionRowCount, totalPartitionCount); + break; + } + } + ASSERT_TRUE(foundPartition) + << "Partition for year " << year << " not found"; + } +} + +TEST_F(IcebergTransformE2ETest, varbinaryTruncatePartitioning) { + constexpr auto numBatches = 2; + constexpr auto rowsPerBatch = 100; + auto vectors = createTestData(numBatches, rowsPerBatch); + auto outputDirectory = TempDirectoryPath::create(); + + std::vector partitionTransforms = { + {5, TransformType::kTruncate, 36}}; + auto dataSink = createIcebergDataSink( + rowType_, outputDirectory->getPath(), partitionTransforms); + + for (const auto& vector : vectors) { + dataSink->appendData(vector); + } + + ASSERT_TRUE(dataSink->finish()); + dataSink->close(); + verifyTotalRowCount( + rowType_, outputDirectory->getPath(), numBatches * rowsPerBatch); + + auto partitionDirs = listFirstLevelDirectories(outputDirectory->getPath()); + + for (const auto& dir : partitionDirs) { + const auto dirName = std::filesystem::path(dir).filename().string(); + auto [c, v] = buildFilter(dirName); + // For binary data, we need to use a different approach for filtering. + const auto filter = c + " IS NOT NULL"; + // Verify the partition has data. + verifyPartitionData(rowType_, dir, filter, 0, true); + } +} + +TEST_F(IcebergTransformE2ETest, multipleTransformsOnSameColumn) { + constexpr auto numBatches = 2; + constexpr auto rowsPerBatch = 100; + auto vectors = createTestData(numBatches, rowsPerBatch); + auto outputDirectory = TempDirectoryPath::create(); + std::vector partitionTransforms = { + {0, TransformType::kIdentity, std::nullopt}, // c_int. + {0, TransformType::kTruncate, 10}, // truncate(c_int, 10). + {0, TransformType::kBucket, 4}}; // bucket(c_int, 4). + auto dataSink = createIcebergDataSink( + rowType_, outputDirectory->getPath(), partitionTransforms); + for (const auto& vector : vectors) { + dataSink->appendData(vector); + } + + ASSERT_TRUE(dataSink->finish()); + dataSink->close(); + + verifyTotalRowCount( + rowType_, outputDirectory->getPath(), numBatches * rowsPerBatch); + + auto firstLevelDirs = listFirstLevelDirectories(outputDirectory->getPath()); + ASSERT_GT(firstLevelDirs.size(), 0); + for (const auto& dir : firstLevelDirs) { + const auto dirName = std::filesystem::path(dir).filename().string(); + ASSERT_TRUE(dirName.find("c_int=") != std::string::npos) + << "First level directory " << dirName + << " should use identity transform"; + + auto secondLevelDirs = listFirstLevelDirectories(dir); + ASSERT_GT(secondLevelDirs.size(), 0) + << "No second level directories found in " << dir; + + for (const auto& secondDir : secondLevelDirs) { + const auto secondDirName = + std::filesystem::path(secondDir).filename().string(); + ASSERT_TRUE(secondDirName.find("c_int_trunc=") != std::string::npos) + << "Second level directory " << secondDirName + << " should use truncate transform"; + + auto thirdLevelDirs = listFirstLevelDirectories(secondDir); + ASSERT_GT(thirdLevelDirs.size(), 0) + << "No third level directories found in " << secondDir; + + for (const auto& thirdDir : thirdLevelDirs) { + const auto thirdDirName = + std::filesystem::path(thirdDir).filename().string(); + ASSERT_TRUE(thirdDirName.find("c_int_bucket=") != std::string::npos) + << "Third level directory " << thirdDirName + << " should use bucket transform"; + + auto leafDir = thirdDir; + auto intValue = std::stoi( + std::filesystem::path(dir).filename().string().substr( + 6)); // c_int=X. + auto truncValue = std::stoi( + std::filesystem::path(secondDir).filename().string().substr( + 12)); // c_int_trunc=X. + std::string filter = fmt::format( + "c_int = {} AND c_int >= {} AND c_int < {}", + intValue, + truncValue, + truncValue + 10); + + // Verify the partition has data. + auto splits = createSplitsForDirectory(leafDir); + auto countPlan = PlanBuilder() + .tableScan(rowType_) + .singleAggregation({}, {"count(1)"}) + .planNode(); + auto countResult = + AssertQueryBuilder(countPlan).splits(splits).copyResults( + opPool_.get()); + ASSERT_GT( + countResult->childAt(0)->asFlatVector()->valueAt(0), 0) + << "Leaf partition directory " << leafDir << " has no data"; + } + } + } +} + +TEST_F(IcebergTransformE2ETest, timestampYearPartitioning) { + constexpr auto numBatches = 2; + constexpr auto rowsPerBatch = 100; + auto batches = createTestData(numBatches, rowsPerBatch); + + auto outputDirectory = TempDirectoryPath::create(); + std::vector partitionTransforms = { + {6, TransformType::kYear, std::nullopt}}; // c_timestamp column. + auto dataSink = createIcebergDataSink( + rowType_, outputDirectory->getPath(), partitionTransforms); + + for (const auto& vector : batches) { + dataSink->appendData(vector); + } + + ASSERT_TRUE(dataSink->finish()); + dataSink->close(); + + verifyTotalRowCount( + rowType_, outputDirectory->getPath(), numBatches * rowsPerBatch); + + auto partitionDirs = listFirstLevelDirectories(outputDirectory->getPath()); + std::unordered_map yearToExpectedCount; + + for (const auto& batch : batches) { + auto timestampVector = batch->childAt(6)->as>(); + for (auto i = 0; i < batch->size(); i++) { + if (!timestampVector->isNullAt(i)) { + Timestamp ts = timestampVector->valueAt(i); + std::tm tm; + if (Timestamp::epochToCalendarUtc(ts.getSeconds(), tm)) { + int32_t year = tm.tm_year + 1900; + yearToExpectedCount[year]++; + } + } + } + } + + for (const auto& dir : partitionDirs) { + const auto dirName = std::filesystem::path(dir).filename().string(); + auto [c, v] = buildFilter(dirName); + auto year = std::stoi(v); + std::string filter = fmt::format("YEAR(c_timestamp) = {}", year); + auto expectedRowCount = yearToExpectedCount.at(year); + verifyPartitionData(rowType_, dir, filter, expectedRowCount); + } +} + +TEST_F(IcebergTransformE2ETest, timestampMonthPartitioning) { + constexpr auto numBatches = 2; + constexpr auto rowsPerBatch = 100; + + auto batches = createTestData(numBatches, rowsPerBatch); + + auto outputDirectory = TempDirectoryPath::create(); + std::vector partitionTransforms = { + {6, TransformType::kMonth, std::nullopt}}; // c_timestamp column. + auto dataSink = createIcebergDataSink( + rowType_, outputDirectory->getPath(), partitionTransforms); + + for (const auto& vector : batches) { + dataSink->appendData(vector); + } + + ASSERT_TRUE(dataSink->finish()); + dataSink->close(); + verifyTotalRowCount( + rowType_, outputDirectory->getPath(), numBatches * rowsPerBatch); + + auto partitionDirs = listFirstLevelDirectories(outputDirectory->getPath()); + std::unordered_map monthToExpectedCount; + + for (const auto& batch : batches) { + auto timestampVector = batch->childAt(6)->as>(); + for (auto i = 0; i < batch->size(); i++) { + if (!timestampVector->isNullAt(i)) { + Timestamp ts = timestampVector->valueAt(i); + std::tm tm; + if (Timestamp::epochToCalendarUtc(ts.getSeconds(), tm)) { + int32_t year = tm.tm_year + 1900; + int32_t month = tm.tm_mon + 1; + std::string monthKey = fmt::format("{:04d}-{:02d}", year, month); + monthToExpectedCount[monthKey]++; + } + } + } + } + + for (const auto& dir : partitionDirs) { + const auto dirName = std::filesystem::path(dir).filename().string(); + auto [c, v] = buildFilter(dirName); + size_t dashPos = v.find('-'); + ASSERT_NE(dashPos, std::string::npos) << "Invalid month format: " << v; + + int32_t year = std::stoi(v.substr(0, dashPos)); + int32_t month = std::stoi(v.substr(dashPos + 1)); + std::string filter = fmt::format( + "YEAR(c_timestamp) = {} AND MONTH(c_timestamp) = {}", year, month); + std::string monthKey = fmt::format("{:04d}-{:02d}", year, month); + auto expectedCount = monthToExpectedCount[monthKey]; + verifyPartitionData(rowType_, dir, filter, expectedCount); + } +} + +TEST_F(IcebergTransformE2ETest, timestampDayPartitioning) { + constexpr auto numBatches = 2; + constexpr auto rowsPerBatch = 100; + auto batches = createTestData(numBatches, rowsPerBatch); + auto outputDirectory = TempDirectoryPath::create(); + std::vector partitionTransforms = { + {6, TransformType::kDay, std::nullopt}}; // c_timestamp column + auto dataSink = createIcebergDataSink( + rowType_, outputDirectory->getPath(), partitionTransforms); + + for (const auto& vector : batches) { + dataSink->appendData(vector); + } + + ASSERT_TRUE(dataSink->finish()); + dataSink->close(); + + verifyTotalRowCount( + rowType_, outputDirectory->getPath(), numBatches * rowsPerBatch); + + auto partitionDirs = listFirstLevelDirectories(outputDirectory->getPath()); + std::unordered_map dayToExpectedCount; + for (const auto& batch : batches) { + auto timestampVector = batch->childAt(6)->as>(); + for (auto i = 0; i < batch->size(); i++) { + if (!timestampVector->isNullAt(i)) { + Timestamp ts = timestampVector->valueAt(i); + std::tm tm; + if (Timestamp::epochToCalendarUtc(ts.getSeconds(), tm)) { + int32_t year = tm.tm_year + 1900; + int32_t month = tm.tm_mon + 1; + int32_t day = tm.tm_mday; + std::string dayKey = + fmt::format("{:04d}-{:02d}-{:02d}", year, month, day); + dayToExpectedCount[dayKey]++; + } + } + } + } + + for (const auto& dir : partitionDirs) { + const auto dirName = std::filesystem::path(dir).filename().string(); + auto [c, v] = buildFilter(dirName); + std::vector dateParts; + folly::split('-', v, dateParts); + ASSERT_EQ(dateParts.size(), 3) << "Invalid day format: " << v; + + int32_t year = std::stoi(dateParts[0]); + int32_t month = std::stoi(dateParts[1]); + int32_t day = std::stoi(dateParts[2]); + + std::string filter = fmt::format( + "YEAR(c_timestamp) = {} AND MONTH(c_timestamp) = {} AND DAY(c_timestamp) = {}", + year, + month, + day); + + // Get expected count for this day. + std::string dayKey = fmt::format("{:04d}-{:02d}-{:02d}", year, month, day); + auto expectedCount = dayToExpectedCount[dayKey]; + verifyPartitionData(rowType_, dir, filter, expectedCount); + } +} + +TEST_F(IcebergTransformE2ETest, timestampHourPartitioning) { + constexpr auto numBatches = 2; + constexpr auto rowsPerBatch = 100; + auto batches = createTestData(numBatches, rowsPerBatch); + + auto outputDirectory = TempDirectoryPath::create(); + std::vector partitionTransforms = { + {6, TransformType::kHour, std::nullopt}}; // c_timestamp column. + auto dataSink = createIcebergDataSink( + rowType_, outputDirectory->getPath(), partitionTransforms); + + for (const auto& vector : batches) { + dataSink->appendData(vector); + } + + ASSERT_TRUE(dataSink->finish()); + dataSink->close(); + + verifyTotalRowCount( + rowType_, outputDirectory->getPath(), numBatches * rowsPerBatch); + + auto partitionDirs = listFirstLevelDirectories(outputDirectory->getPath()); + std::unordered_map hourToExpectedCount; + + for (const auto& batch : batches) { + auto timestampVector = batch->childAt(6)->as>(); + for (auto i = 0; i < batch->size(); i++) { + if (!timestampVector->isNullAt(i)) { + Timestamp ts = timestampVector->valueAt(i); + std::tm tm; + if (Timestamp::epochToCalendarUtc(ts.getSeconds(), tm)) { + int32_t year = tm.tm_year + 1900; + int32_t month = tm.tm_mon + 1; + int32_t day = tm.tm_mday; + int32_t hour = tm.tm_hour; + std::string hourKey = fmt::format( + "{:04d}-{:02d}-{:02d}-{:02d}", year, month, day, hour); + hourToExpectedCount[hourKey]++; + } + } + } + } + + for (const auto& dir : partitionDirs) { + const auto dirName = std::filesystem::path(dir).filename().string(); + auto [c, v] = buildFilter(dirName); + std::vector dateParts; + folly::split('-', v, dateParts); + ASSERT_EQ(dateParts.size(), 4) << "Invalid hour format: " << v; + + int32_t year = std::stoi(dateParts[0]); + int32_t month = std::stoi(dateParts[1]); + int32_t day = std::stoi(dateParts[2]); + int32_t hour = std::stoi(dateParts[3]); + + std::string filter = fmt::format( + "YEAR(c_timestamp) = {} AND MONTH(c_timestamp) = {} AND " + "DAY(c_timestamp) = {} AND HOUR(c_timestamp) = {}", + year, + month, + day, + hour); + std::string hourKey = + fmt::format("{:04d}-{:02d}-{:02d}-{:02d}", year, month, day, hour); + auto expectedCount = hourToExpectedCount[hourKey]; + verifyPartitionData(rowType_, dir, filter, expectedCount); + } +} + +TEST_F(IcebergTransformE2ETest, partitionFolderNamingConventions) { + auto intVector = makeFlatVector(1, [](auto) { return 42; }); + auto bigintVector = + makeFlatVector(1, [](auto) { return 9'876'543'210; }); + auto varcharVector = + BaseVector::create>(VARCHAR(), 1, opPool_.get()); + varcharVector->set(0, StringView("test string")); + + auto varcharVector2 = + BaseVector::create>(VARCHAR(), 1, opPool_.get()); + varcharVector2->setNull(0, true); + + auto decimalVector = + BaseVector::create>(DECIMAL(18, 3), 1, opPool_.get()); + decimalVector->set(0, 1'234'567'890); + + auto varbinaryVector = + BaseVector::create>(VARBINARY(), 1, opPool_.get()); + std::string binaryData = "binary\0data\1\2\3"; + varbinaryVector->set(0, StringView(binaryData)); + + auto rowVector = makeRowVector( + {"c_int", + "c_bigint", + "c_varchar", + "c_varchar2", + "c_decimal", + "c_varbinary"}, + {intVector, + bigintVector, + varcharVector, + varcharVector2, + decimalVector, + varbinaryVector}); + auto outputDirectory = TempDirectoryPath::create(); + std::vector partitionTransforms = { + {0, TransformType::kIdentity, std::nullopt}, // c_int. + {1, TransformType::kIdentity, std::nullopt}, // c_bigint. + {2, TransformType::kIdentity, std::nullopt}, // c_varchar. + {4, TransformType::kIdentity, std::nullopt}, // c_decimal. + {5, TransformType::kIdentity, std::nullopt}, // c_varbinary. + {3, TransformType::kIdentity, std::nullopt} // c_varchar2. + }; + auto dataSink = createIcebergDataSink( + asRowType(rowVector->type()), + outputDirectory->getPath(), + partitionTransforms); + + dataSink->appendData(rowVector); + ASSERT_TRUE(dataSink->finish()); + dataSink->close(); + + verifyTotalRowCount( + asRowType(rowVector->type()), outputDirectory->getPath(), 1); + auto partitionDirs = listDirectoriesRecursively(outputDirectory->getPath()); + + const std::string expectedIntFolder = "c_int=42"; + const std::string expectedBigintFolder = "c_bigint=9876543210"; + const std::string expectedVarcharFolder = "c_varchar=test+string"; + const std::string expectedVarcharFolder2 = "c_varchar2=null"; + const std::string expectedDecimalFolder = "c_decimal=1234567.890"; + const std::string expectedVarbinary = "c_varbinary=" + + encoding::Base64::encode(binaryData.data(), binaryData.size()); + + bool foundIntPartition = false; + bool foundBigintPartition = false; + bool foundVarcharPartition = false; + bool foundVarcharPartition2 = false; + bool foundDecimalPartition = false; + bool foundVarbinaryPartition = false; + + for (const auto& dir : partitionDirs) { + const auto dirName = std::filesystem::path(dir).filename().string(); + + if (dirName == expectedIntFolder) { + foundIntPartition = true; + verifyPartitionData(asRowType(rowVector->type()), dir, "c_int = 42", 1); + } else if (dirName == expectedBigintFolder) { + foundBigintPartition = true; + verifyPartitionData( + asRowType(rowVector->type()), dir, "c_bigint = 9876543210", 1); + } else if (dirName == expectedVarcharFolder) { + foundVarcharPartition = true; + verifyPartitionData( + asRowType(rowVector->type()), dir, "c_varchar = 'test string'", 1); + } else if (dirName == expectedVarcharFolder2) { + foundVarcharPartition2 = true; + verifyPartitionData( + asRowType(rowVector->type()), dir, "c_varchar2 IS NULL", 1); + } else if (dirName == expectedDecimalFolder) { + foundDecimalPartition = true; + verifyPartitionData( + asRowType(rowVector->type()), + dir, + "c_decimal = DECIMAL '1234567.890'", + 1); + } else if (dirName.find(expectedVarbinary) == 0) { + foundVarbinaryPartition = true; + verifyPartitionData( + asRowType(rowVector->type()), dir, "c_varbinary IS NOT NULL", 1); + } + } + + ASSERT_TRUE(foundIntPartition) + << "Integer partition folder not found: " << expectedIntFolder; + ASSERT_TRUE(foundBigintPartition) + << "Bigint partition folder not found: " << expectedBigintFolder; + ASSERT_TRUE(foundVarcharPartition) + << "Varchar partition folder not found: " << expectedVarcharFolder; + ASSERT_TRUE(foundVarcharPartition2) + << "Varchar2 partition folder not found: " << expectedVarcharFolder2; + ASSERT_TRUE(foundDecimalPartition) + << "Decimal partition folder not found: " << expectedDecimalFolder; + ASSERT_TRUE(foundVarbinaryPartition) + << "Varbinary partition folder not found with prefix: " + << expectedVarbinary; +} + +} // namespace facebook::velox::connector::hive::iceberg::test diff --git a/velox/connectors/hive/iceberg/tests/IcebergTransformUnitTest.cpp b/velox/connectors/hive/iceberg/tests/IcebergTransformUnitTest.cpp new file mode 100644 index 000000000000..da0116c1c333 --- /dev/null +++ b/velox/connectors/hive/iceberg/tests/IcebergTransformUnitTest.cpp @@ -0,0 +1,801 @@ +/* + * 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/encode/Base64.h" +#include "velox/connectors/hive/iceberg/tests/IcebergTestBase.h" + +namespace facebook::velox::connector::hive::iceberg::test { + +class IcebergTransformUnitTest : public IcebergTestBase { + protected: + template + void testTransform( + const IcebergPartitionSpec::Field& field, + const std::vector& inputValues, + const std::vector>& expectedValues, + const TypePtr& type = nullptr) { + VectorPtr inputVector; + std::vector> transforms = + parsePartitionTransformSpecs({field}, opPool_.get()); + auto transform = transforms[0]; + if constexpr (std::is_same_v) { + auto size = inputValues.size(); + auto vectorType = type ? type : VARCHAR(); + inputVector = BaseVector::create>( + vectorType, size, opPool_.get()); + const auto flatVector = inputVector->asFlatVector(); + for (vector_size_t i = 0; i < size; i++) { + if (i < inputValues.size()) { + flatVector->set(i, inputValues[i]); + } else { + flatVector->setNull(i, true); + } + } + } else { + auto size = inputValues.size(); + inputVector = BaseVector::create>( + type ? type : CppToType::create(), size, opPool_.get()); + const auto flatVector = inputVector->asFlatVector(); + for (vector_size_t i = 0; i < size; i++) { + if (i < inputValues.size()) { + flatVector->set(i, inputValues[i]); + } else { + flatVector->setNull(i, true); + } + } + } + + std::vector children = {inputVector}; + std::vector names = {field.name}; + auto rowVector = makeRowVector(names, children); + const auto resultVector = transform->transform(rowVector, 0); + + ASSERT_EQ(resultVector->size(), expectedValues.size()); + for (vector_size_t i = 0; i < resultVector->size(); i++) { + if (expectedValues[i].has_value()) { + if constexpr ( + std::is_same_v && + std::is_same_v) { + if (type && type->isVarbinary()) { + EXPECT_EQ( + resultVector->as>()->valueAt(i).str(), + encoding::Base64::encode(expectedValues[i].value().str())); + } else { + EXPECT_EQ( + resultVector->as>()->valueAt(i).str(), + expectedValues[i].value().str()); + } + } else { + EXPECT_EQ( + resultVector->as>()->valueAt(i), + expectedValues[i].value()); + } + } else { + EXPECT_TRUE(resultVector->isNullAt(i)); + } + } + } +}; + +TEST_F(IcebergTransformUnitTest, testIdentityTransform) { + rowType_ = + ROW({"c_int", + "c_bigint", + "c_varchar", + "c_date", + "c_varbinary", + "c_decimal", + "c_timestamp"}, + {INTEGER(), + BIGINT(), + VARCHAR(), + DATE(), + VARBINARY(), + DECIMAL(18, 3), + TIMESTAMP()}); + + // Create partition spec with identity transforms. + const auto partitionSpec = createPartitionSpec( + {{0, TransformType::kIdentity, std::nullopt}, // c_int. + {1, TransformType::kIdentity, std::nullopt}, // c_bigint. + {2, TransformType::kIdentity, std::nullopt}, // c_varchar. + {4, TransformType::kIdentity, std::nullopt}, // c_varbinary. + {5, TransformType::kIdentity, std::nullopt}, // c_decimal. + {6, TransformType::kIdentity, std::nullopt}}, // c_timestamp. + rowType_); + + auto& intTransform = partitionSpec->fields[0]; + EXPECT_EQ(intTransform.transformType, TransformType::kIdentity); + testTransform( + intTransform, + {1, + 0, + -1, + std::numeric_limits::min(), + std::numeric_limits::max()}, + {1, + 0, + -1, + std::numeric_limits::min(), + std::numeric_limits::max()}); + + auto& bigintTransform = partitionSpec->fields[1]; + EXPECT_EQ(bigintTransform.transformType, TransformType::kIdentity); + EXPECT_EQ(bigintTransform.type->kind(), TypeKind::BIGINT); + testTransform( + bigintTransform, + {1L, + 0L, + -1L, + std::numeric_limits::min(), + std::numeric_limits::max()}, + {1, + 0, + -1, + std::numeric_limits::min(), + std::numeric_limits::max()}); + + auto& varcharTransform = partitionSpec->fields[2]; + EXPECT_EQ(varcharTransform.transformType, TransformType::kIdentity); + EXPECT_EQ(varcharTransform.type->kind(), TypeKind::VARCHAR); + testTransform( + varcharTransform, + {StringView("a"), + StringView(""), + StringView("velox"), + StringView( + "Velox is a composable execution engine distributed as an open source C++ library. It provides reusable, extensible, and high-performance data processing components that can be (re-)used to build data management systems focused on different analytical workloads, including batch, interactive, stream processing, and AI/ML. Velox was created by Meta and it is currently developed in partnership with IBM/Ahana, Intel, Voltron Data, Microsoft, ByteDance and many other companies.")}, + {StringView("a"), + StringView(""), + StringView("velox"), + StringView( + "Velox is a composable execution engine distributed as an open source C++ library. It provides reusable, extensible, and high-performance data processing components that can be (re-)used to build data management systems focused on different analytical workloads, including batch, interactive, stream processing, and AI/ML. Velox was created by Meta and it is currently developed in partnership with IBM/Ahana, Intel, Voltron Data, Microsoft, ByteDance and many other companies.")}); + + auto& varbinaryTransform = partitionSpec->fields[3]; + EXPECT_EQ(varbinaryTransform.transformType, TransformType::kIdentity); + EXPECT_EQ(varbinaryTransform.type->kind(), TypeKind::VARBINARY); + testTransform( + varbinaryTransform, + { + StringView("\x01\x02\x03", 3), + StringView("\x04\x05\x06\x07", 4), + StringView("\x08\x09", 2), + StringView("", 0), + StringView("\xFF\xFE\xFD\xFC", 4), + }, + { + StringView("\x01\x02\x03", 3), + StringView("\x04\x05\x06\x07", 4), + StringView("\x08\x09", 2), + StringView("", 0), + StringView("\xFF\xFE\xFD\xFC", 4), + }, + VARBINARY()); + + auto& timestampTransform = partitionSpec->fields[5]; + EXPECT_EQ(timestampTransform.transformType, TransformType::kIdentity); + EXPECT_EQ(timestampTransform.type->kind(), TypeKind::TIMESTAMP); + testTransform( + timestampTransform, + { + Timestamp(0, 0), + Timestamp(1609459200, 0), + Timestamp(1640995200, 0), + Timestamp(1672531200, 0), + Timestamp(9223372036854775, 999999999), + }, + { + Timestamp(0, 0), + Timestamp(1609459200, 0), + Timestamp(1640995200, 0), + Timestamp(1672531200, 0), + Timestamp(9223372036854775, 999999999), + }); +} + +TEST_F(IcebergTransformUnitTest, testTruncateTransform) { + rowType_ = + ROW({"c_int", "c_decimal", "c_varchar", "c_varbinary"}, + {INTEGER(), DECIMAL(18, 3), VARCHAR(), VARBINARY()}); + + const auto partitionSpec = createPartitionSpec( + {{0, TransformType::kTruncate, 10}, + {1, TransformType::kTruncate, 10}, + {2, TransformType::kTruncate, 2}, + {3, TransformType::kTruncate, 3}}, + rowType_); + + auto& intTruncateTransform = partitionSpec->fields[0]; + testTransform( + intTruncateTransform, + { + std::numeric_limits::min(), + std::numeric_limits::min() + 1, + std::numeric_limits::min() + 9, + std::numeric_limits::min() + 10, + -1, + 0, + 1, + 9, + std::numeric_limits::max() - 10, + std::numeric_limits::max() - 9, + std::numeric_limits::max() - 1, + std::numeric_limits::max(), + }, + { + 2'147'483'646, + 2'147'483'646, + -2'147'483'640, + -2'147'483'640, + -10, + 0, + 0, + 0, + 2'147'483'630, + 2'147'483'630, + 2'147'483'640, + 2'147'483'640, + }); + + auto& decimalTruncateTransform = partitionSpec->fields[1]; + testTransform( + decimalTruncateTransform, + { + 5000, + 5010, + 5011, + 5019, + 5020, + 5021, + -5000, + -5010, + -5011, + -5019, + -5020, + -5021, + 1234, + 1230, + 1229, + 5, + -5, + -10, + -9, + -1, + 0, + 1, + 9, + 10, + 995, + 1000, + 1005, + 1010, + + // Large values. + 999'999'999'999'999'990L, + 999'999'999'999'999'995L, + 999'999'999'999'999'999L, + // Small values. + -999'999'999'999'999'990L, + -999'999'999'999'999'995L, + -999'999'999'999'999'999L, + }, + { + 5000, + 5010, + 5010, + 5010, + 5020, + 5020, + -5000, + -5010, + -5020, + -5020, + -5020, + -5030, + 1230, + 1230, + 1220, + 0, + -10, + -10, + -10, + -10, + 0, + 0, + 0, + 10, + 990, + 1000, + 1000, + 1010, + // Expected results for large values. + 999'999'999'999'999'990L, + 999'999'999'999'999'990L, + 999'999'999'999'999'990L, + // Expected results for small values. + -999'999'999'999'999'990L, + -1'000'000'000'000'000'000L, + -1'000'000'000'000'000'000L, + }); + + auto& varcharTruncateTransform = partitionSpec->fields[2]; + testTransform( + varcharTruncateTransform, + { + StringView(""), + StringView("a"), + StringView("ab"), + StringView("abc"), + StringView("abcd"), + StringView("测"), // 1 code point, 3 bytes. + StringView("测试"), // 2 code points, 6 bytes. + StringView("测试abc"), // 5 code points. + StringView("a测b试c"), // 5 code points. + StringView("🚀"), // 1 code point, 4 bytes. + StringView("🚀🔥"), // 2 code points, 8 bytes. + StringView("abc🚀🔥"), // 5 code points. + StringView("é"), // 1 code point (e + combining acute accent). + StringView("éfac"), // 4 code points. + StringView("a\u0300"), // 'a' + combining grave accent = 1 code point. + }, + { + StringView(""), + StringView("a"), + StringView("ab"), + StringView("ab"), + StringView("ab"), + StringView("测"), + StringView("测试"), + StringView("测试"), + StringView("a测"), + StringView("🚀"), + StringView("🚀🔥"), + StringView("ab"), + StringView("é"), + StringView("éf"), + StringView("a\u0300"), + }); + + auto& varbinaryTransform = partitionSpec->fields[3]; + testTransform( + varbinaryTransform, + { + StringView("\x01\x02\x03", 3), + StringView("\x04\x05\x06\x07", 4), + StringView("\x08\x09", 2), + StringView("", 0), + StringView( + "\xFF\xFE\xFD\xFC\xFA\xFB\xFC\xF1\xF2\xF3\xF4\xF5\xF6\xF7", 14), + }, + { + StringView("\x01\x02\x03", 3), + StringView("\x04\x05\x06", 3), + StringView("\x08\x09", 2), + StringView("", 0), + StringView("\xFF\xFE\xFD", 3), + }, + VARBINARY()); +} + +TEST_F(IcebergTransformUnitTest, testBucketTransform) { + rowType_ = + ROW({"c_int", "c_bigint", "c_varchar", "c_varbinary", "c_date"}, + {INTEGER(), BIGINT(), VARCHAR(), VARBINARY(), DATE()}); + + const auto partitionSpec = createPartitionSpec( + {{0, TransformType::kBucket, 4}, + {1, TransformType::kBucket, 8}, + {2, TransformType::kBucket, 16}, + {3, TransformType::kBucket, 32}, + {4, TransformType::kBucket, 10}}, + rowType_); + + auto& intBucketTransform = partitionSpec->fields[0]; + EXPECT_EQ(intBucketTransform.transformType, TransformType::kBucket); + + testTransform( + intBucketTransform, + {8, + 34, + 0, + 1, + -1, + 42, + 100, + 1000, + std::numeric_limits::min(), + std::numeric_limits::max()}, + {3, 3, 0, 0, 0, 2, 0, 0, 0, 2}); + + auto& bigintBucketTransform = partitionSpec->fields[1]; + EXPECT_EQ(bigintBucketTransform.transformType, TransformType::kBucket); + + testTransform( + bigintBucketTransform, + {34L, + 0L, + -34L, + -1L, + 1L, + 42L, + 123'456'789L, + -123'456'789L, + std::numeric_limits::min(), + std::numeric_limits::max()}, + {3, 4, 5, 0, 4, 6, 1, 4, 5, 7}); + + auto& varcharBucketTransform = partitionSpec->fields[2]; + EXPECT_EQ(varcharBucketTransform.transformType, TransformType::kBucket); + + testTransform( + varcharBucketTransform, + {StringView("abcdefg"), + StringView("测试"), + StringView("测试ping试测"), + StringView(""), + StringView("🚀🔥"), + StringView("a\u0300\u0301"), // Combining characters. + StringView("To be or not to be, that is the question.")}, + {6, 8, 11, 0, 14, 11, 9}); + + auto& varbinaryBucketTransform = partitionSpec->fields[3]; + EXPECT_EQ(varbinaryBucketTransform.transformType, TransformType::kBucket); + + testTransform( + varbinaryBucketTransform, + {StringView("abc\0\0", 5), + StringView("\x01\x02\x03\x04", 4), + StringView("\xFF\xFE\xFD\xFC", 4), + StringView("\x00\x00\x00\x00", 4), + StringView("\xDE\xAD\xBE\xEF", 4), + StringView(std::string(100, 'x').c_str(), 100)}, + {11, 5, 15, 30, 10, 18}, + VARBINARY()); + + auto& dateBucketTransform = partitionSpec->fields[4]; + EXPECT_EQ(dateBucketTransform.transformType, TransformType::kBucket); + + testTransform( + dateBucketTransform, + { + 0, // 1970-01-01. + 365, // 1971-01-01. + 18'262, // 2020-01-01. + -365, // 1969-01-01. + -1, // 1969-12-31. + 20'181, // 2025-04-03. + -36889, // 1869-01-01. + 18'628 // 2021-01-01. + }, + {6, 1, 3, 6, 2, 5, 9, 0}); +} + +TEST_F(IcebergTransformUnitTest, testTemporalTransforms) { + rowType_ = ROW({"c_date"}, {DATE()}); + + const auto partitionSpec = createPartitionSpec( + {{0, TransformType::kYear, std::nullopt}, + {0, TransformType::kMonth, std::nullopt}, + {0, TransformType::kDay, std::nullopt}, + {0, TransformType::kHour, std::nullopt}, + {0, TransformType::kBucket, 8}, + {0, TransformType::kIdentity, std::nullopt}}, + rowType_); + + auto& yearTransform = partitionSpec->fields[0]; + EXPECT_EQ(yearTransform.transformType, TransformType::kYear); + // Create test dates (days since epoch). + testTransform( + yearTransform, + { + -36889, // 1869-01-01. + -18628, // 1919-01-01. + -365, // 1969-01-01. + -1, // 1969-12-31. + 0, // 1970-01-01 (epoch). + 31, // 1970-02-01. + 365, // 1971-01-01. + 18'262, // 2020-01-01. + 20'181 // 2025-04-03. + }, + { + -101, // 1869 - 1970 = -101. + -51, // 1919 - 1970 = -51. + -1, // 1969 - 1970 = -1. + -1, // 1969 - 1970 = -1. + 0, // 1970 - 1970 = 0. + 0, // 1970 - 1970 = 0. + 1, // 1971 - 1970 = 1. + 50, // 2020 - 1970 = 50. + 55 // 2025 - 1970 = 55. + }); + // Test month transform. + auto& monthTransform = partitionSpec->fields[1]; + EXPECT_EQ(monthTransform.transformType, TransformType::kMonth); + + testTransform( + monthTransform, + {-36525, -18263, -365, -1, 0, 31, 365, 18'262, 20'181}, + {-1201, -600, -12, -1, 0, 1, 12, 600, 663}); + // Test day transform. + auto& dayTransform = partitionSpec->fields[2]; + EXPECT_EQ(dayTransform.transformType, TransformType::kDay); + testTransform( + dayTransform, + {-36525, -18263, -365, -1, 0, 31, 365, 18'262, 20'181}, + {-36525, -18263, -365, -1, 0, 31, 365, 18'262, 20'181}); +} + +TEST_F(IcebergTransformUnitTest, testTransformOnTimestamp) { + rowType_ = ROW({"c_timestamp"}, {TIMESTAMP()}); + + const auto partitionSpec = createPartitionSpec( + {{0, TransformType::kYear, std::nullopt}, + {0, TransformType::kMonth, std::nullopt}, + {0, TransformType::kDay, std::nullopt}, + {0, TransformType::kHour, std::nullopt}, + {0, TransformType::kBucket, 8}, + {0, TransformType::kIdentity, std::nullopt}}, + rowType_); + + auto& yearTransform = partitionSpec->fields[0]; + EXPECT_EQ(yearTransform.transformType, TransformType::kYear); + testTransform( + yearTransform, + { + Timestamp(0, 0), + Timestamp(31536000, 0), // 1971-01-01 00:00:00. + Timestamp(1609459200, 0), // 2021-01-01 00:00:00. + Timestamp(1612224000, 0), // 2021-02-01 00:00:00. + }, + { + 0, // 1970 - 1970 = 0. + 1, // 1971 - 1970 = 1. + 51, // 2021 - 1970 = 51. + 51 // 2021 - 1970 = 51. + }); + + auto& monthTransform = partitionSpec->fields[1]; + EXPECT_EQ(monthTransform.transformType, TransformType::kMonth); + + testTransform( + monthTransform, + {Timestamp(0, 0), + Timestamp(31536000, 0), + Timestamp(1609459200, 0), + Timestamp(1612224000, 0)}, + {0, 12, 612, 613}); + + auto& dayTransform = partitionSpec->fields[2]; + EXPECT_EQ(dayTransform.transformType, TransformType::kDay); + testTransform( + dayTransform, + {Timestamp(0, 0), + Timestamp(31536000, 0), + Timestamp(1609459200, 0), + Timestamp(1612224000, 0)}, + {0, 365, 18628, 18660}); + + auto& hourTransform = partitionSpec->fields[3]; + EXPECT_EQ(hourTransform.transformType, TransformType::kHour); + testTransform( + hourTransform, + {Timestamp(0, 0), + Timestamp(31536000, 0), + Timestamp(1609459200, 0), + Timestamp(1612224000, 0)}, + {0, 8760, 447072, 447840}); + + auto& bucketTransform = partitionSpec->fields[4]; + EXPECT_EQ(bucketTransform.transformType, TransformType::kBucket); + testTransform( + bucketTransform, + { + Timestamp(0, 0), + Timestamp(31536000, 0), + Timestamp(1609459200, 0), + Timestamp(1612224000, 0), + Timestamp(-31536000, 0), + }, + {4, 4, 6, 5, 3}); + + auto& identityTransform = partitionSpec->fields[5]; + EXPECT_EQ(identityTransform.transformType, TransformType::kIdentity); + testTransform( + identityTransform, + {Timestamp(0, 0), + Timestamp(31536000, 0), + Timestamp(1609459200, 0), + Timestamp(1612224000, 0)}, + {Timestamp(0, 0), + Timestamp(31536000, 0), + Timestamp(1609459200, 0), + Timestamp(1612224000, 0)}); +} + +TEST_F(IcebergTransformUnitTest, testTransformsWithNulls) { + rowType_ = ROW( + {"c_int", "c_bigint", "c_decimal", "c_varchar", "c_varbinary", "c_date"}, + {INTEGER(), BIGINT(), DECIMAL(18, 3), VARCHAR(), VARBINARY(), DATE()}); + + const auto partitionSpec = createPartitionSpec( + {{0, TransformType::kIdentity, std::nullopt}, + {2, TransformType::kTruncate, 100}, + {1, TransformType::kBucket, 16}, + {5, TransformType::kYear, std::nullopt}, + {5, TransformType::kMonth, std::nullopt}, + {5, TransformType::kDay, std::nullopt}}, + rowType_); + + auto& identityTransform = partitionSpec->fields[0]; + EXPECT_EQ(identityTransform.transformType, TransformType::kIdentity); + + auto intInput = + makeNullableFlatVector({5, std::nullopt, 15, std::nullopt, 25}); + std::vector children = {intInput}; + std::vector names = {identityTransform.name}; + auto rowVector = makeRowVector(names, children); + + std::vector> transforms = + parsePartitionTransformSpecs({identityTransform}, opPool_.get()); + auto transform = transforms[0]; + auto identityResult = transform->transform(rowVector, 0); + ASSERT_EQ(identityResult->size(), 5); + EXPECT_EQ(identityResult->as>()->valueAt(0), 5); + EXPECT_TRUE(identityResult->isNullAt(1)); + EXPECT_EQ(identityResult->as>()->valueAt(2), 15); + EXPECT_TRUE(identityResult->isNullAt(3)); + EXPECT_EQ(identityResult->as>()->valueAt(4), 25); + + auto& truncateTransform = partitionSpec->fields[1]; + EXPECT_EQ(truncateTransform.transformType, TransformType::kTruncate); + + auto decimalInput = makeNullableFlatVector( + {5'000, std::nullopt, 15'000, std::nullopt, 25'000}); + children = {decimalInput}; + names = {truncateTransform.name}; + rowVector = makeRowVector(names, children); + transforms = parsePartitionTransformSpecs({truncateTransform}, opPool_.get()); + transform = transforms[0]; + auto truncateResult = transform->transform(rowVector, 0); + ASSERT_EQ(truncateResult->size(), 5); + EXPECT_EQ(truncateResult->as>()->valueAt(0), 5000); + EXPECT_TRUE(truncateResult->isNullAt(1)); + EXPECT_EQ(truncateResult->as>()->valueAt(2), 15'000); + EXPECT_TRUE(truncateResult->isNullAt(3)); + EXPECT_EQ(truncateResult->as>()->valueAt(4), 25'000); + + auto& bucketTransform = partitionSpec->fields[2]; + EXPECT_EQ(bucketTransform.transformType, TransformType::kBucket); + + auto bigintInput = makeNullableFlatVector( + {50L, std::nullopt, 150L, std::nullopt, 250L}); + children = {bigintInput}; + names = {bucketTransform.name}; + rowVector = makeRowVector(names, children); + transforms = parsePartitionTransformSpecs({bucketTransform}, opPool_.get()); + transform = transforms[0]; + auto bucketResult = transform->transform(rowVector, 0); + ASSERT_EQ(bucketResult->size(), 5); + EXPECT_TRUE(bucketResult->isNullAt(1)); + EXPECT_TRUE(bucketResult->isNullAt(3)); + + auto& yearTransform = partitionSpec->fields[3]; + EXPECT_EQ(yearTransform.transformType, TransformType::kYear); + + auto dateInput = makeNullableFlatVector( + {0, std::nullopt, 365, std::nullopt, 20'175}); + children = {dateInput}; + names = {yearTransform.name}; + rowVector = makeRowVector(names, children); + transforms = parsePartitionTransformSpecs({yearTransform}, opPool_.get()); + transform = transforms[0]; + auto yearResult = transform->transform(rowVector, 0); + ASSERT_EQ(yearResult->size(), 5); + EXPECT_EQ(yearResult->as>()->valueAt(0), 0); + EXPECT_TRUE(yearResult->isNullAt(1)); + EXPECT_EQ(yearResult->as>()->valueAt(2), 1); + EXPECT_TRUE(yearResult->isNullAt(3)); + EXPECT_EQ(yearResult->as>()->valueAt(4), 55); + + auto& monthTransform = partitionSpec->fields[4]; + EXPECT_EQ(monthTransform.transformType, TransformType::kMonth); + children = {dateInput}; + names = {monthTransform.name}; + rowVector = makeRowVector(names, children); + transforms = parsePartitionTransformSpecs({monthTransform}, opPool_.get()); + transform = transforms[0]; + auto monthResult = transform->transform(rowVector, 0); + ASSERT_EQ(monthResult->size(), 5); + EXPECT_EQ(monthResult->as>()->valueAt(0), 0); + EXPECT_TRUE(monthResult->isNullAt(1)); + EXPECT_EQ(monthResult->as>()->valueAt(2), 12); + EXPECT_TRUE(monthResult->isNullAt(3)); + EXPECT_EQ(monthResult->as>()->valueAt(4), 662); + + auto& dayTransform = partitionSpec->fields[5]; + EXPECT_EQ(dayTransform.transformType, TransformType::kDay); + names = {dayTransform.name}; + rowVector = makeRowVector(names, children); + transforms = parsePartitionTransformSpecs({dayTransform}, opPool_.get()); + transform = transforms[0]; + auto dayResult = transform->transform(rowVector, 0); + ASSERT_EQ(dayResult->size(), 5); + EXPECT_EQ(dayResult->as>()->valueAt(0), 0); + EXPECT_TRUE(dayResult->isNullAt(1)); + EXPECT_EQ(dayResult->as>()->valueAt(2), 365); + EXPECT_TRUE(dayResult->isNullAt(3)); + EXPECT_EQ(dayResult->as>()->valueAt(4), 20'175); + + auto varcharInput = makeNullableFlatVector( + {StringView("abc"), + std::nullopt, + StringView("def"), + std::nullopt, + StringView("ghi")}); + + rowType_ = ROW({"c_varchar"}, {VARCHAR()}); + auto varcharIdentityTransform = + createPartitionSpec( + {{0, TransformType::kIdentity, std::nullopt}}, rowType_) + ->fields[0]; + children = {varcharInput}; + names = {varcharIdentityTransform.name}; + rowVector = makeRowVector(names, children); + + transforms = + parsePartitionTransformSpecs({varcharIdentityTransform}, opPool_.get()); + transform = transforms[0]; + auto varcharIdentityResult = transform->transform(rowVector, 0); + ASSERT_EQ(varcharIdentityResult->size(), 5); + EXPECT_EQ( + varcharIdentityResult->as>()->valueAt(0).str(), + "abc"); + EXPECT_TRUE(varcharIdentityResult->isNullAt(1)); + EXPECT_EQ( + varcharIdentityResult->as>()->valueAt(2).str(), + "def"); + EXPECT_TRUE(varcharIdentityResult->isNullAt(3)); + EXPECT_EQ( + varcharIdentityResult->as>()->valueAt(4).str(), + "ghi"); + + auto varbinaryInput = makeNullableFlatVector( + {StringView("\x01\x02\x03", 3), + std::nullopt, + StringView("\x04\x05\x06", 3), + std::nullopt, + StringView("\x07\x08\x09", 3)}, + VARBINARY()); + + rowType_ = ROW({"c_varbinary"}, {VARBINARY()}); + auto varbinaryIdentityTransform = + createPartitionSpec( + {{0, TransformType::kIdentity, std::nullopt}}, rowType_) + ->fields[0]; + children = {varbinaryInput}; + names = {varbinaryIdentityTransform.name}; + rowVector = makeRowVector(names, children); + transforms = + parsePartitionTransformSpecs({varbinaryIdentityTransform}, opPool_.get()); + transform = transforms[0]; + auto varbinaryIdentityResult = transform->transform(rowVector, 0); + ASSERT_EQ(varbinaryIdentityResult->size(), 5); + EXPECT_TRUE(varbinaryIdentityResult->isNullAt(1)); + EXPECT_TRUE(varbinaryIdentityResult->isNullAt(3)); +} + +} // namespace facebook::velox::connector::hive::iceberg::test diff --git a/velox/connectors/hive/iceberg/tests/IcebergWriterModeTest.cpp b/velox/connectors/hive/iceberg/tests/IcebergWriterModeTest.cpp new file mode 100644 index 000000000000..d2df0c380e90 --- /dev/null +++ b/velox/connectors/hive/iceberg/tests/IcebergWriterModeTest.cpp @@ -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. + */ + +#include "velox/common/base/tests/GTestUtils.h" +#include "velox/connectors/hive/HiveConfig.h" +#include "velox/connectors/hive/iceberg/tests/IcebergTestBase.h" +#include "velox/exec/tests/utils/AssertQueryBuilder.h" +#include "velox/exec/tests/utils/PlanBuilder.h" +#include "velox/exec/tests/utils/TempDirectoryPath.h" + +using namespace facebook::velox::exec::test; + +namespace facebook::velox::connector::hive::iceberg::test { + +class IcebergWriterModeTest : public IcebergTestBase, + public ::testing::WithParamInterface { + protected: + void SetUp() override { + IcebergTestBase::SetUp(); + + std::unordered_map sessionProps = { + {HiveConfig::kFanoutEnabledSession, GetParam() ? "true" : "false"}, + }; + + connectorSessionProperties_ = + std::make_shared(std::move(sessionProps), true); + + setupMemoryPools(); + } +}; + +INSTANTIATE_TEST_SUITE_P( + FanoutModes, + IcebergWriterModeTest, + ::testing::Values(true, false), + [](const testing::TestParamInfo& info) { + return info.param ? "FanoutEnabled" : "FanoutDisabled"; + }); + +TEST_P(IcebergWriterModeTest, identityPartitioning) { + constexpr auto size = 10; + std::vector names = {"c_int"}; + std::vector types = {INTEGER()}; + rowType_ = ROW(names, types); + + auto outputDirectory = TempDirectoryPath::create(); + auto dataSink = createIcebergDataSink( + rowType_, + outputDirectory->getPath(), + {{0, TransformType::kIdentity, std::nullopt}}); + + auto intVector1 = + makeFlatVector(size, [](vector_size_t row) { return row; }); + auto vector1 = makeRowVector(names, {intVector1}); + auto intVector2 = + makeFlatVector(size, [](vector_size_t row) { return row + 10; }); + auto vector2 = makeRowVector(names, {intVector2}); + dataSink->appendData(vector1); + dataSink->appendData(vector2); + ASSERT_TRUE(dataSink->finish()); + dataSink->close(); + createDuckDbTable({vector1, vector2}); + auto splits = createSplitsForDirectory(outputDirectory->getPath()); + auto plan = exec::test::PlanBuilder().tableScan(rowType_).planNode(); + assertQuery(plan, splits, fmt::format("SELECT * FROM tmp")); +} + +TEST_P(IcebergWriterModeTest, clusteredInput) { + constexpr auto size = 100; + std::vector names = {"c_int"}; + std::vector types = {INTEGER()}; + rowType_ = ROW(names, types); + + auto outputDirectory = TempDirectoryPath::create(); + auto dataSink = createIcebergDataSink( + rowType_, + outputDirectory->getPath(), + {{0, TransformType::kIdentity, std::nullopt}}); + + auto intVector1 = makeConstant(100, size, INTEGER()); + auto vector1 = makeRowVector(names, {intVector1}); + auto intVector2 = makeConstant(100, size, INTEGER()); + auto vector2 = makeRowVector(names, {intVector2}); + dataSink->appendData(vector1); + dataSink->appendData(vector2); + ASSERT_TRUE(dataSink->finish()); + dataSink->close(); + auto stats = dataSink->dataFileStats(); + ASSERT_EQ(stats.at(0)->numRecords, size * 2); + ASSERT_FALSE(stats.at(0)->lowerBounds.empty()); + ASSERT_FALSE(stats.at(0)->upperBounds.empty()); + createDuckDbTable({vector1, vector2}); + auto splits = createSplitsForDirectory(outputDirectory->getPath()); + ASSERT_EQ(splits.size(), 1); + auto plan = exec::test::PlanBuilder().tableScan(rowType_).planNode(); + assertQuery(plan, splits, fmt::format("SELECT * FROM tmp")); +} + +TEST_P(IcebergWriterModeTest, clusteredNullInput) { + constexpr auto size = 100; + std::vector names = {"c_int"}; + std::vector types = {INTEGER()}; + rowType_ = ROW(names, types); + + auto outputDirectory = TempDirectoryPath::create(); + auto dataSink = createIcebergDataSink( + rowType_, + outputDirectory->getPath(), + {{0, TransformType::kIdentity, std::nullopt}}); + + auto intVector1 = makeNullConstant(TypeKind::INTEGER, size); + auto vector1 = makeRowVector(names, {intVector1}); + auto intVector2 = makeNullConstant(TypeKind::INTEGER, size); + auto vector2 = makeRowVector(names, {intVector2}); + dataSink->appendData(vector1); + dataSink->appendData(vector2); + ASSERT_TRUE(dataSink->finish()); + dataSink->close(); + auto stats = dataSink->dataFileStats(); + ASSERT_TRUE(stats.at(0)->upperBounds.empty()); + ASSERT_EQ(stats.at(0)->nullValueCounts.at(1), size * 2); + createDuckDbTable({vector1, vector2}); + auto splits = createSplitsForDirectory(outputDirectory->getPath()); + ASSERT_EQ(splits.size(), 1); + auto plan = exec::test::PlanBuilder().tableScan(rowType_).planNode(); + assertQuery(plan, splits, fmt::format("SELECT * FROM tmp")); +} + +TEST_P(IcebergWriterModeTest, sortedByAndIdentityPartittioning) { + constexpr auto size = 10; + std::vector names = {"c_int"}; + std::vector types = {INTEGER()}; + rowType_ = ROW(names, types); + + auto outputDirectory = TempDirectoryPath::create(); + auto dataSink = createIcebergDataSink( + rowType_, + outputDirectory->getPath(), + {{0, TransformType::kIdentity, std::nullopt}}, + {"c_int DESC"}); + + auto intVector1 = + makeFlatVector(size, [](vector_size_t row) { return row; }); + auto vector1 = makeRowVector(names, {intVector1}); + auto intVector2 = + makeFlatVector(size, [](vector_size_t row) { return row + 10; }); + auto vector2 = makeRowVector(names, {intVector2}); + dataSink->appendData(vector1); + dataSink->appendData(vector2); + ASSERT_TRUE(dataSink->finish()); + dataSink->close(); + createDuckDbTable({vector1, vector2}); + auto splits = createSplitsForDirectory(outputDirectory->getPath()); + ASSERT_EQ(splits.size(), size * 2); + auto plan = exec::test::PlanBuilder().tableScan(rowType_).planNode(); + assertQuery(plan, splits, fmt::format("SELECT * FROM tmp")); +} + +TEST_P(IcebergWriterModeTest, nonClusteredInput) { + constexpr auto size = 10; + std::vector names = {"c_int"}; + std::vector types = {INTEGER()}; + rowType_ = ROW(names, types); + + auto outputDirectory = TempDirectoryPath::create(); + auto dataSink = createIcebergDataSink( + rowType_, + outputDirectory->getPath(), + {{0, TransformType::kIdentity, std::nullopt}}); + + auto intVector1 = + makeFlatVector(size, [](vector_size_t row) { return row; }); + auto vector1 = makeRowVector(names, {intVector1}); + auto intVector2 = + makeFlatVector(size, [](vector_size_t row) { return row + 5; }); + auto vector2 = makeRowVector(names, {intVector2}); + dataSink->appendData(vector1); + if (!GetParam()) { + VELOX_ASSERT_THROW( + dataSink->appendData(vector2), + "Incoming records violate the writer assumption that records are clustered by spec and \n by partition within each spec. Either cluster the incoming records or switch to fanout writers.\nEncountered records that belong to already closed files:\n"); + } else { + dataSink->appendData(vector2); + ASSERT_TRUE(dataSink->finish()); + dataSink->close(); + createDuckDbTable({vector1, vector2}); + auto splits = createSplitsForDirectory(outputDirectory->getPath()); + auto plan = exec::test::PlanBuilder().tableScan(rowType_).planNode(); + assertQuery(plan, splits, fmt::format("SELECT * FROM tmp")); + } +} + +} // namespace facebook::velox::connector::hive::iceberg::test diff --git a/velox/connectors/hive/iceberg/tests/Murmur3Test.cpp b/velox/connectors/hive/iceberg/tests/Murmur3Test.cpp new file mode 100644 index 000000000000..c8444ecee5be --- /dev/null +++ b/velox/connectors/hive/iceberg/tests/Murmur3Test.cpp @@ -0,0 +1,181 @@ +/* + * 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/Murmur3.h" +#include +#include "folly/Random.h" +#include "velox/type/TimestampConversion.h" + +namespace facebook::velox::connector::hive::iceberg { +class Murmur3HashTest : public ::testing::Test { + public: + void SetUp() override { + rng_.seed(1); + } + + void TearDown() override {} + + // Little-endian. + static std::vector toBytes(uint64_t value) { + std::vector bytes; + bytes.reserve(sizeof(uint64_t)); + for (int32_t i = 0; i < sizeof(uint64_t); ++i) { + bytes[i] = static_cast((value >> (8 * i)) & 0xFF); + } + return bytes; + } + + template + void + verifyHashBucket(T input, uint32_t bucketCount, uint32_t expectedBucket) { + const auto hash = Murmur3Hash32::hash(input); + uint32_t actualBucket = (hash & 0X7FFFFFFF) % bucketCount; + EXPECT_EQ(actualBucket, expectedBucket) + << "Input: " << input << ", Bucket Count: " << bucketCount + << ", Hash: " << hash << ", Expected Bucket: " << expectedBucket + << ", Actual Bucket: " << actualBucket; + } + + protected: + folly::Random::DefaultGenerator rng_; +}; + +TEST_F(Murmur3HashTest, testSpecValues) { + auto hash = Murmur3Hash32::hash(34); + EXPECT_EQ(hash, 2'017'239'379); + + const auto days = + util::fromDateString("2017-11-16", util::ParseMode::kIso8601); + EXPECT_EQ(days.value(), 17'486); + hash = Murmur3Hash32::hash(days.value()); + EXPECT_EQ(hash, -653'330'422); + + auto timestampResult = util::fromTimestampString( + "2017-11-16T22:31:08", util::TimestampParseMode::kIso8601); + hash = Murmur3Hash32::hash(timestampResult.value().toMicros()); + EXPECT_EQ(hash, -2'047'944'441); + + timestampResult = util::fromTimestampString( + "2017-11-16T22:31:08.000001", util::TimestampParseMode::kIso8601); + hash = Murmur3Hash32::hash(timestampResult.value().toMicros()); + EXPECT_EQ(hash, -1'207'196'810); + + timestampResult = util::fromTimestampString( + "2017-11-16T22:31:08.000001001", util::TimestampParseMode::kIso8601); + hash = Murmur3Hash32::hash(timestampResult.value().toMicros()); + EXPECT_EQ(hash, -1'207'196'810); + + const auto bytes = new char[4]{0x00, 0x01, 0x02, 0x03}; + hash = Murmur3Hash32::hash(bytes, 4); + EXPECT_EQ(hash, -188'683'207); + + hash = Murmur3Hash32::hash("iceberg"); + EXPECT_EQ(hash, 1'210'000'089); +} + +TEST_F(Murmur3HashTest, hashString) { + const std::vector> testCases = { + {"abcdefg", 5, 4}, + {"abc", 128, 122}, + {"abcde", 64, 54}, + {"测试", 12, 8}, + {"测试raul试测", 16, 1}, + {"", 16, 0}}; + + for (const auto& [input, bucketCount, expectedBucket] : testCases) { + verifyHashBucket(input, bucketCount, expectedBucket); + } +} + +TEST_F(Murmur3HashTest, hashInteger) { + const std::vector> testCases = { + {8, 10, 3}, {34, 100, 79}}; + + for (const auto& [input, bucketCount, expectedBucket] : testCases) { + verifyHashBucket(input, bucketCount, expectedBucket); + } +} + +TEST_F(Murmur3HashTest, hashTrue) { + const auto hash = Murmur3Hash32::hash(1); + EXPECT_EQ(hash, 1'392'991'556U); +} + +TEST_F(Murmur3HashTest, hashDate) { + const std::vector> testCases = { + {util::fromDateString("1970-01-09", util::ParseMode::kIso8601).value(), + 10, + 3}, + {util::fromDateString("1970-02-04", util::ParseMode::kIso8601).value(), + 100, + 79}}; + + for (const auto& [input, bucketCount, expectedBucket] : testCases) { + verifyHashBucket(input, bucketCount, expectedBucket); + } +} + +TEST_F(Murmur3HashTest, hashLong) { + const std::vector> testCases = { + {34L, 100, 79}, {0L, 100, 76}, {-34L, 100, 97}, {-1L, 2, 0}}; + + for (const auto& [input, bucketCount, expectedBucket] : testCases) { + verifyHashBucket(input, bucketCount, expectedBucket); + } +} + +TEST_F(Murmur3HashTest, hashDecimal) { + const std::vector> testCases = { + {1234L, 64, 56}, + {1230L, 18, 13}, + {12999L, 16, 2}, + {5L, 32, 21}, + {5L, 18, 3}}; + + for (const auto& [input, bucketCount, expectedBucket] : testCases) { + const auto hash = Murmur3Hash32::hashDecimal(input); + auto actualBucket = (hash & 0X7FFFFFFF) % bucketCount; + EXPECT_EQ(actualBucket, expectedBucket); + } +} + +TEST_F(Murmur3HashTest, hashBinary) { + const std::string s("abc\0\0", 5); + const std::vector> testCases = { + {StringView("abcdefg"), 12, 10}, + {StringView(s), 18, 13}, + {StringView("abc"), 48, 42}, + {StringView("测试_"), 16, 3}}; + for (const auto& [input, bucketCount, expectedBucket] : testCases) { + verifyHashBucket(input, bucketCount, expectedBucket); + } +} + +TEST_F(Murmur3HashTest, hashIntegerAndBytes) { + const auto number = folly::Random::rand32(rng_); + const auto hashOfInteger = Murmur3Hash32::hash(number); + const auto hashOfBytes = Murmur3Hash32::hash(toBytes(number).data(), 8); + EXPECT_EQ(hashOfInteger, hashOfBytes); +} + +TEST_F(Murmur3HashTest, hashLongAndBytes) { + const auto number = folly::Random::rand64(rng_); + const auto hashOfLong = Murmur3Hash32::hash(number); + const auto hashOfBytes = Murmur3Hash32::hash(toBytes(number).data(), 8); + EXPECT_EQ(hashOfLong, hashOfBytes); +} + +} // namespace facebook::velox::connector::hive::iceberg diff --git a/velox/connectors/hive/iceberg/tests/PartitionSpecTest.cpp b/velox/connectors/hive/iceberg/tests/PartitionSpecTest.cpp deleted file mode 100644 index 23a06340aad3..000000000000 --- a/velox/connectors/hive/iceberg/tests/PartitionSpecTest.cpp +++ /dev/null @@ -1,134 +0,0 @@ -/* - * 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/PartitionSpec.h" - -#include -#include "velox/common/base/tests/GTestUtils.h" -#include "velox/functions/prestosql/types/TimestampWithTimeZoneType.h" -#include "velox/type/Type.h" - -namespace facebook::velox::connector::hive::iceberg { - -namespace { - -TEST(PartitionSpecTest, invalidColumnType) { - auto makeSpec = [](const TypePtr& type) { - std::vector fields = { - {"c0", type, TransformType::kIdentity, std::nullopt}, - }; - return std::make_shared(1, fields); - }; - - VELOX_ASSERT_USER_THROW( - makeSpec(ROW({{"a", INTEGER()}})), - "Type is not supported as a partition column: ROW"); - VELOX_ASSERT_USER_THROW( - makeSpec(ARRAY(INTEGER())), - "Type is not supported as a partition column: ARRAY"); - VELOX_ASSERT_USER_THROW( - makeSpec(MAP(VARCHAR(), INTEGER())), - "Type is not supported as a partition column: MAP"); - VELOX_ASSERT_USER_THROW( - makeSpec(TIMESTAMP_WITH_TIME_ZONE()), - "Type is not supported as a partition column: TIMESTAMP WITH TIME ZONE"); -} - -TEST(PartitionSpecTest, invalidMultipleTransforms) { - { - std::vector fields = { - {"c0", VARCHAR(), TransformType::kIdentity, std::nullopt}, - {"c0", VARCHAR(), TransformType::kIdentity, std::nullopt}, - }; - VELOX_ASSERT_USER_THROW( - std::make_shared(1, fields), - "Column: 'c0', Category: Identity, Transforms: [identity, identity]"); - } - - { - std::vector fields = { - {"c0", VARCHAR(), TransformType::kBucket, 16}, - {"c0", VARCHAR(), TransformType::kBucket, 32}, - }; - VELOX_ASSERT_USER_THROW( - std::make_shared(1, fields), - "Column: 'c0', Category: Bucket, Transforms: [bucket, bucket]"); - } - - { - std::vector fields = { - {"c0", VARCHAR(), TransformType::kTruncate, 2}, - {"c0", VARCHAR(), TransformType::kTruncate, 5}, - }; - VELOX_ASSERT_USER_THROW( - std::make_shared(1, fields), - "Column: 'c0', Category: Truncate, Transforms: [trunc, trunc]"); - } - - { - std::vector fields4 = { - {"c0", TIMESTAMP(), TransformType::kYear, std::nullopt}, - {"c0", TIMESTAMP(), TransformType::kMonth, std::nullopt}, - {"c0", TIMESTAMP(), TransformType::kDay, std::nullopt}, - {"c0", TIMESTAMP(), TransformType::kHour, std::nullopt}, - }; - VELOX_ASSERT_USER_THROW( - std::make_shared(1, fields4), - "Column: 'c0', Category: Temporal, Transforms: [year, month, day, hour]"); - } -} - -TEST(PartitionSpecTest, invalidMultipleTransformsMultipleColumns) { - std::vector fields = { - {"c0", DATE(), TransformType::kYear, std::nullopt}, - {"c0", DATE(), TransformType::kMonth, std::nullopt}, - {"c1", VARCHAR(), TransformType::kBucket, 16}, - {"c1", VARCHAR(), TransformType::kBucket, 32}, - }; - // order may vary due to map iteration. - VELOX_ASSERT_USER_THROW( - std::make_shared(1, fields), - "Column: 'c0', Category: Temporal, Transforms: [year, month]"); - VELOX_ASSERT_USER_THROW( - std::make_shared(1, fields), - "Column: 'c1', Category: Bucket, Transforms: [bucket, bucket]"); -} - -TEST(PartitionSpecTest, validMultipleTransforms) { - { - std::vector fields = { - {"c0", VARCHAR(), TransformType::kIdentity, std::nullopt}, - {"c0", VARCHAR(), TransformType::kBucket, 16}, - {"c0", VARCHAR(), TransformType::kTruncate, 10}, - }; - auto spec = std::make_shared(1, fields); - EXPECT_EQ(spec->fields.size(), 3); - } - - { - std::vector fields = { - {"c0", DATE(), TransformType::kYear, std::nullopt}, - {"c0", DATE(), TransformType::kBucket, 16}, - {"c0", DATE(), TransformType::kIdentity, std::nullopt}, - }; - auto spec = std::make_shared(1, fields); - EXPECT_EQ(spec->fields.size(), 3); - } -} - -} // namespace - -} // namespace facebook::velox::connector::hive::iceberg diff --git a/velox/docs/configs.rst b/velox/docs/configs.rst index a6df4d863f13..215ed42c6a0d 100644 --- a/velox/docs/configs.rst +++ b/velox/docs/configs.rst @@ -715,6 +715,11 @@ Each query can override the config by setting corresponding query session proper - bool - false - Whether to preserve flat maps in memory as FlatMapVectors instead of converting them to MapVectors. This is only applied during data reading inside the DWRF and Nimble readers, not during downstream processing like expression evaluation etc. + * - fanout-enabled + - fanout_enabled + - bool + - true + - Controls the writer mode, whether the fanout mode writer is enabled, default value is true, setting to false means clustered mode. Currently applies only to the Iceberg writer. ``ORC File Format Configuration`` ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ diff --git a/velox/dwio/common/CMakeLists.txt b/velox/dwio/common/CMakeLists.txt index 2023728686bc..348763bb9fbf 100644 --- a/velox/dwio/common/CMakeLists.txt +++ b/velox/dwio/common/CMakeLists.txt @@ -43,6 +43,7 @@ velox_add_library( OnDemandUnitLoader.cpp InputStream.cpp IntDecoder.cpp + DataFileStatistics.cpp MetadataFilter.cpp Options.cpp OutputStream.cpp diff --git a/velox/dwio/common/DataFileStatistics.cpp b/velox/dwio/common/DataFileStatistics.cpp new file mode 100644 index 000000000000..6968f8ee5287 --- /dev/null +++ b/velox/dwio/common/DataFileStatistics.cpp @@ -0,0 +1,51 @@ +/* + * 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/dwio/common/DataFileStatistics.h" + +namespace facebook::velox::dwio::common { + +folly::dynamic DataFileStatistics::toJson() const { + folly::dynamic json = folly::dynamic::object; + json["recordCount"] = numRecords; + + auto mapToJson = [](const auto& map) { + folly::dynamic result = folly::dynamic::object; + for (const auto& pair : map) { + result[folly::to(pair.first)] = pair.second; + } + return result; + }; + + json["columnSizes"] = mapToJson(columnsSizes); + json["valueCounts"] = mapToJson(valueCounts); + json["nullValueCounts"] = mapToJson(nullValueCounts); + json["nanValueCounts"] = mapToJson(nanValueCounts); + json["lowerBounds"] = mapToJson(lowerBounds); + json["upperBounds"] = mapToJson(upperBounds); + + return json; +} + +folly::dynamic DataFileStatistics::splitOffsetsAsJson() const { + folly::dynamic arr = folly::dynamic::array; + for (const auto& offset : splitOffsets) { + arr.push_back(offset); + } + return arr; +} + +} // namespace facebook::velox::dwio::common diff --git a/velox/dwio/common/DataFileStatistics.h b/velox/dwio/common/DataFileStatistics.h new file mode 100644 index 000000000000..d8eb1491d8e5 --- /dev/null +++ b/velox/dwio/common/DataFileStatistics.h @@ -0,0 +1,44 @@ +/* + * 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 + +namespace facebook::velox::dwio::common { + +// Iceberg data_file struct fields. +struct DataFileStatistics { + int64_t numRecords; + std::unordered_map columnsSizes; + std::unordered_map valueCounts; + std::unordered_map nullValueCounts; + std::unordered_map nanValueCounts; + std::unordered_map lowerBounds; + std::unordered_map upperBounds; + + // Split offsets for the data file. For example, all row + // group offsets in a Parquet file. Must be sorted ascending. + std::vector splitOffsets; + + DataFileStatistics() : numRecords(0) {} + + folly::dynamic toJson() const; + + folly::dynamic splitOffsetsAsJson() const; +}; + +} // namespace facebook::velox::dwio::common diff --git a/velox/dwio/common/DataFileStatsCollector.h b/velox/dwio/common/DataFileStatsCollector.h new file mode 100644 index 000000000000..0f400d43c5b8 --- /dev/null +++ b/velox/dwio/common/DataFileStatsCollector.h @@ -0,0 +1,48 @@ +/* + * 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/dwio/common/DataFileStatistics.h" + +namespace facebook::velox::dwio::common { + +/// Base settings for collecting data file statistics. Can be extended +/// by specific table formats to add format-specific fields. +struct DataFileStatsSettings { + virtual ~DataFileStatsSettings() = default; +}; + +class FileStatsCollector { + public: + explicit FileStatsCollector( + std::shared_ptr< + std::vector>> + settings) + : statsSetting_(std::move(settings)) {} + + virtual ~FileStatsCollector() = default; + + virtual void collectStats( + const void* metadata, + const std::shared_ptr& fileStats) = 0; + + protected: + std::shared_ptr< + std::vector>> + statsSetting_; +}; + +} // namespace facebook::velox::dwio::common diff --git a/velox/dwio/common/Options.h b/velox/dwio/common/Options.h index b61f91fb8e2a..9d4175833e0d 100644 --- a/velox/dwio/common/Options.h +++ b/velox/dwio/common/Options.h @@ -27,6 +27,7 @@ #include "velox/common/io/Options.h" #include "velox/common/memory/Memory.h" #include "velox/dwio/common/ColumnSelector.h" +#include "velox/dwio/common/DataFileStatsCollector.h" #include "velox/dwio/common/ErrorTolerance.h" #include "velox/dwio/common/FlatMapHelper.h" #include "velox/dwio/common/FlushPolicy.h" @@ -733,6 +734,11 @@ struct WriterOptions { std::string sessionTimezoneName; bool adjustTimestampToTimezone{false}; + /// Data file statistics collector for format-specific statistics collection + /// during write operations. Each table format (e.g., Iceberg, Hudi) can + /// provide its own implementation to collect connector-specific metadata. + FileStatsCollector* fileStatsCollector{nullptr}; + // WriterOption implementations can implement this function to specify how to // process format-specific session and connector configs. virtual void processConfigs( 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 696691222152..0fd01fe981ab 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(), SubfieldKind::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 bee9ef5b89e1..7db6e8d16730 100644 --- a/velox/dwio/common/ScanSpec.h +++ b/velox/dwio/common/ScanSpec.h @@ -54,19 +54,33 @@ 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) { @@ -200,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); @@ -297,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); @@ -391,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(); @@ -431,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 @@ -471,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/SortingWriter.cpp b/velox/dwio/common/SortingWriter.cpp index d67efd6ec223..0ac8f02d7729 100644 --- a/velox/dwio/common/SortingWriter.cpp +++ b/velox/dwio/common/SortingWriter.cpp @@ -91,6 +91,7 @@ void SortingWriter::close() { setState(State::kClosed); VELOX_CHECK_NULL(sortBuffer_); outputWriter_->close(); + dataFileStats_ = outputWriter_->dataFileStats(); } void SortingWriter::abort() { 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/common/Writer.h b/velox/dwio/common/Writer.h index 774aafe4c9a8..8420432e70c4 100644 --- a/velox/dwio/common/Writer.h +++ b/velox/dwio/common/Writer.h @@ -21,6 +21,8 @@ #include #include +#include "velox/dwio/common/DataFileStatistics.h" +#include "velox/dwio/common/DataFileStatsCollector.h" #include "velox/vector/ComplexVector.h" namespace facebook::velox::dwio::common { @@ -79,6 +81,11 @@ class Writer { /// Data can no longer be written. virtual void abort() = 0; + /// Return statistics based on each Iceberg data file + std::shared_ptr dataFileStats() const { + return dataFileStats_; + }; + protected: bool isRunning() const; bool isFinishing() const; @@ -92,6 +99,8 @@ class Writer { static void checkStateTransition(State oldState, State newState); State state_{State::kInit}; + std::shared_ptr dataFileStats_{nullptr}; + dwio::common::FileStatsCollector* statsCollector_; }; FOLLY_ALWAYS_INLINE std::ostream& operator<<( diff --git a/velox/dwio/dwrf/reader/DwrfReader.h b/velox/dwio/dwrf/reader/DwrfReader.h index ce4b8dd6b6b7..dc9ec704d74e 100644 --- a/velox/dwio/dwrf/reader/DwrfReader.h +++ b/velox/dwio/dwrf/reader/DwrfReader.h @@ -294,7 +294,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 ba560165068d..268cdab8f513 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/dwio/parquet/common/CMakeLists.txt b/velox/dwio/parquet/common/CMakeLists.txt index 159067602e9e..68e2a0dce493 100644 --- a/velox/dwio/parquet/common/CMakeLists.txt +++ b/velox/dwio/parquet/common/CMakeLists.txt @@ -18,6 +18,7 @@ velox_add_library( XxHasher.cpp LevelComparison.cpp LevelConversion.cpp + UnicodeUtil.cpp ) velox_link_libraries( diff --git a/velox/dwio/parquet/common/UnicodeUtil.cpp b/velox/dwio/parquet/common/UnicodeUtil.cpp new file mode 100644 index 000000000000..780fa8163183 --- /dev/null +++ b/velox/dwio/parquet/common/UnicodeUtil.cpp @@ -0,0 +1,81 @@ +/* + * 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/dwio/parquet/common/UnicodeUtil.h" + +#include "velox/external/utf8proc/utf8proc.h" +#include "velox/functions/lib/string/StringImpl.h" + +namespace facebook::velox::parquet { + +std::string_view UnicodeUtil::truncateStringMin( + const char* input, + int32_t inputLength, + int32_t numCodePoints) { + auto length = functions::stringImpl::cappedByteLength( + StringView(input, inputLength), numCodePoints); + return std::string_view(input, length); +} + +std::string UnicodeUtil::truncateStringMax( + const char* input, + int32_t inputLength, + int32_t numCodePoints) { + auto truncatedLength = functions::stringImpl::cappedByteLength( + StringView(input, inputLength), numCodePoints); + + if (truncatedLength == inputLength) { + return std::string(input, inputLength); + } + + // Try to increment the last code point. + for (auto i = numCodePoints - 1; i >= 0; --i) { + const char* current = input; + int32_t currentCodePoint = 0; + + // Find the i-th code point position. + while (current < input + truncatedLength && currentCodePoint < i) { + int32_t charLength; + utf8proc_codepoint(current, input + truncatedLength, charLength); + current += charLength; + currentCodePoint++; + } + + if (current >= input + truncatedLength) + continue; + + int32_t charLength; + auto codePoint = + utf8proc_codepoint(current, input + truncatedLength, charLength); + auto nextCodePoint = codePoint + 1; + + // Check if the incremented code point is valid. + if (nextCodePoint != 0 && utf8proc_codepoint_valid(nextCodePoint)) { + std::string result; + result.reserve(truncatedLength + 4); + result.assign(input, current - input); + char buffer[4]; + auto bytesWritten = utf8proc_encode_char( + nextCodePoint, reinterpret_cast(buffer)); + result.append(buffer, bytesWritten); + return result; + } + } + + return std::string(input, truncatedLength); +} + +} // namespace facebook::velox::parquet diff --git a/velox/dwio/parquet/common/UnicodeUtil.h b/velox/dwio/parquet/common/UnicodeUtil.h new file mode 100644 index 000000000000..4d0952b9f9f0 --- /dev/null +++ b/velox/dwio/parquet/common/UnicodeUtil.h @@ -0,0 +1,41 @@ +/* + * 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::parquet { + +class UnicodeUtil { + public: + static std::string_view truncateStringMin( + const char* input, + int32_t inputLength, + int32_t numCodePoints); + + static std::string truncateStringMax( + const char* input, + int32_t inputLength, + int32_t numCodePoints); + + private: + UnicodeUtil() = delete; +}; + +} // namespace facebook::velox::parquet diff --git a/velox/dwio/parquet/tests/common/CMakeLists.txt b/velox/dwio/parquet/tests/common/CMakeLists.txt index 19cf296e68fa..7960e34bb486 100644 --- a/velox/dwio/parquet/tests/common/CMakeLists.txt +++ b/velox/dwio/parquet/tests/common/CMakeLists.txt @@ -12,7 +12,7 @@ # See the License for the specific language governing permissions and # limitations under the License. -add_executable(velox_dwio_parquet_common_test LevelConversionTest.cpp) +add_executable(velox_dwio_parquet_common_test LevelConversionTest.cpp UnicodeUtilTest.cpp) add_test(velox_dwio_parquet_common_test velox_dwio_parquet_common_test) target_link_libraries( diff --git a/velox/dwio/parquet/tests/common/UnicodeUtilTest.cpp b/velox/dwio/parquet/tests/common/UnicodeUtilTest.cpp new file mode 100644 index 000000000000..9bc529ad2576 --- /dev/null +++ b/velox/dwio/parquet/tests/common/UnicodeUtilTest.cpp @@ -0,0 +1,152 @@ +/* + * 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/dwio/parquet/common/UnicodeUtil.h" +#include + +using namespace facebook::velox::parquet; + +class UnicodeUtilTest : public testing::Test {}; + +TEST_F(UnicodeUtilTest, truncateStringMin) { + // ASCII string. + std::string ascii = "Hello, world!"; + EXPECT_EQ( + UnicodeUtil::truncateStringMin(ascii.c_str(), ascii.length(), 0), ""); + EXPECT_EQ( + UnicodeUtil::truncateStringMin(ascii.c_str(), ascii.length(), 1), "H"); + EXPECT_EQ( + UnicodeUtil::truncateStringMin(ascii.c_str(), ascii.length(), 5), + "Hello"); + EXPECT_EQ( + UnicodeUtil::truncateStringMin(ascii.c_str(), ascii.length(), 13), ascii); + EXPECT_EQ( + UnicodeUtil::truncateStringMin(ascii.c_str(), ascii.length(), 20), ascii); + + // String with multi-bytes characters. + std::string unicode = "Hello, 世界!"; + EXPECT_EQ( + UnicodeUtil::truncateStringMin(unicode.c_str(), unicode.length(), 7), + "Hello, "); + EXPECT_EQ( + UnicodeUtil::truncateStringMin(unicode.c_str(), unicode.length(), 8), + "Hello, 世"); + EXPECT_EQ( + UnicodeUtil::truncateStringMin(unicode.c_str(), unicode.length(), 9), + "Hello, 世界"); + EXPECT_EQ( + UnicodeUtil::truncateStringMin(unicode.c_str(), unicode.length(), 10), + unicode); + EXPECT_EQ( + UnicodeUtil::truncateStringMin(unicode.c_str(), unicode.length(), 20), + unicode); + + // String with emoji (surrogate pairs). + std::string emoji = "Hello 🌍!"; + EXPECT_EQ( + UnicodeUtil::truncateStringMin(emoji.c_str(), emoji.length(), 6), + "Hello "); + EXPECT_EQ( + UnicodeUtil::truncateStringMin(emoji.c_str(), emoji.length(), 7), + "Hello 🌍"); + EXPECT_EQ( + UnicodeUtil::truncateStringMin(emoji.c_str(), emoji.length(), 8), emoji); + EXPECT_EQ( + UnicodeUtil::truncateStringMin(emoji.c_str(), emoji.length(), 10), emoji); + + std::string empty = ""; + EXPECT_EQ( + UnicodeUtil::truncateStringMin(empty.c_str(), empty.length(), 0), ""); + EXPECT_EQ( + UnicodeUtil::truncateStringMin(empty.c_str(), empty.length(), 5), ""); + + std::string mixed = "café世界🌍"; + EXPECT_EQ( + UnicodeUtil::truncateStringMin(mixed.c_str(), mixed.length(), 3), "caf"); + EXPECT_EQ( + UnicodeUtil::truncateStringMin(mixed.c_str(), mixed.length(), 4), "café"); + EXPECT_EQ( + UnicodeUtil::truncateStringMin(mixed.c_str(), mixed.length(), 5), + "café世"); + EXPECT_EQ( + UnicodeUtil::truncateStringMin(mixed.c_str(), mixed.length(), 6), + "café世界"); + EXPECT_EQ( + UnicodeUtil::truncateStringMin(mixed.c_str(), mixed.length(), 7), mixed); +} + +TEST_F(UnicodeUtilTest, truncateStringMax) { + std::string ascii = "Hello, world!"; + auto result = + UnicodeUtil::truncateStringMax(ascii.c_str(), ascii.length(), 0); + EXPECT_EQ(result, ""); + result = UnicodeUtil::truncateStringMax(ascii.c_str(), ascii.length(), 5); + EXPECT_EQ(result, "Hellp"); // 'o' -> 'p'. + result = UnicodeUtil::truncateStringMax( + ascii.c_str(), ascii.length(), ascii.length()); + EXPECT_EQ(result, ascii); + + ascii = "Customer#000001500"; + result = UnicodeUtil::truncateStringMax(ascii.c_str(), ascii.length(), 16); + EXPECT_EQ(result, "Customer#0000016"); //. '5' -> '6'. + + std::string unicode = "Hello, 世界!"; + result = UnicodeUtil::truncateStringMax(unicode.c_str(), unicode.length(), 8); + EXPECT_EQ(result, "Hello, 丗"); + + // No truncation needed. + std::string shortString = "Hi"; + result = UnicodeUtil::truncateStringMax( + shortString.c_str(), shortString.length(), 2); + EXPECT_EQ(result, shortString); + result = UnicodeUtil::truncateStringMax( + shortString.c_str(), shortString.length(), 20); + EXPECT_EQ(result, shortString); + + // Last character is already at maximum value. + std::string maxChar = + "Hello\U0010FFFF"; // U0010FFFF is maximum Unicode code point. + result = UnicodeUtil::truncateStringMax(maxChar.c_str(), maxChar.length(), 6); + EXPECT_EQ(result, maxChar); + + std::string empty = ""; + result = UnicodeUtil::truncateStringMax(empty.c_str(), empty.length(), 0); + EXPECT_EQ(result, ""); + result = UnicodeUtil::truncateStringMax(empty.c_str(), empty.length(), 5); + EXPECT_EQ(result, ""); + + std::string single = "a"; + result = UnicodeUtil::truncateStringMax(single.c_str(), single.length(), 1); + EXPECT_EQ(result, "a"); + + std::string zChar = "zz"; + result = UnicodeUtil::truncateStringMax(zChar.c_str(), zChar.length(), 1); + EXPECT_EQ(result, "{"); // 'z' -> '{'. + + // Emoji increment test + std::string emojiTest = "🌍!!"; + result = + UnicodeUtil::truncateStringMax(emojiTest.c_str(), emojiTest.length(), 1); + EXPECT_EQ(result, "\U0001F30E"); // U1F30D (🌍) -> U1F30E. + + std::string multiByteTest = "café+"; + result = UnicodeUtil::truncateStringMax( + multiByteTest.c_str(), multiByteTest.length(), 3); + EXPECT_EQ(result, "cag"); // 'f' -> 'g'. + result = UnicodeUtil::truncateStringMax( + multiByteTest.c_str(), multiByteTest.length(), 4); + EXPECT_EQ(result, "cafê"); // 'é' -> 'ê'. +} diff --git a/velox/dwio/parquet/tests/writer/CMakeLists.txt b/velox/dwio/parquet/tests/writer/CMakeLists.txt index 2a84d03fa032..6a42a7870969 100644 --- a/velox/dwio/parquet/tests/writer/CMakeLists.txt +++ b/velox/dwio/parquet/tests/writer/CMakeLists.txt @@ -43,6 +43,7 @@ add_test( target_link_libraries( velox_parquet_writer_test + velox_dwio_arrow_parquet_writer_test_lib velox_dwio_parquet_writer velox_dwio_parquet_reader velox_dwio_common_test_utils diff --git a/velox/dwio/parquet/tests/writer/ParquetWriterTest.cpp b/velox/dwio/parquet/tests/writer/ParquetWriterTest.cpp index 61e260ba8d72..818b3cc96e71 100644 --- a/velox/dwio/parquet/tests/writer/ParquetWriterTest.cpp +++ b/velox/dwio/parquet/tests/writer/ParquetWriterTest.cpp @@ -26,6 +26,10 @@ #include "velox/dwio/parquet/RegisterParquetWriter.h" // @manual #include "velox/dwio/parquet/reader/PageReader.h" #include "velox/dwio/parquet/tests/ParquetTestBase.h" +#include "velox/dwio/parquet/writer/Writer.h" +#include "velox/dwio/parquet/writer/arrow/Metadata.h" +#include "velox/dwio/parquet/writer/arrow/Schema.h" +#include "velox/dwio/parquet/writer/arrow/tests/FileReader.h" #include "velox/exec/Cursor.h" #include "velox/exec/tests/utils/AssertQueryBuilder.h" #include "velox/exec/tests/utils/PlanBuilder.h" @@ -991,6 +995,155 @@ TEST_F(ParquetWriterTest, allNulls) { assertReadWithReaderAndExpected(schema, *rowReader, data, *leafPool_); }; +TEST_F(ParquetWriterTest, withoutFieldIds) { + auto schema = + ROW({"a", "b", "c", "m"}, + {BIGINT(), + ROW({"x", "y"}, {INTEGER(), VARCHAR()}), + ARRAY(INTEGER()), + MAP(VARCHAR(), INTEGER())}); + + auto arrVec = makeArrayVector({{3}}); + auto mapVec = makeMapVector({{{StringView("k"), 4}}}); + auto data = makeRowVector({ + makeFlatVector(1, [](auto) { return 1; }), + makeRowVector( + {makeFlatVector(1, [](auto) { return 2; }), + makeFlatVector( + 1, [](auto) { return StringView("z"); })}), + arrVec, + mapVec, + }); + + auto sink = std::make_unique( + 4 * 1024 * 1024, + dwio::common::FileSink::Options{.pool = leafPool_.get()}); + auto* sinkPtr = sink.get(); + + parquet::WriterOptions writerOptions; + writerOptions.memoryPool = leafPool_.get(); + + auto writer = std::make_unique( + std::move(sink), writerOptions, rootPool_, schema); + writer->write(data); + writer->close(); + + std::string_view sinkData(sinkPtr->data(), sinkPtr->size()); + auto readFile = std::make_shared(sinkData); + auto input = std::make_unique(readFile, *leafPool_.get()); + dwio::common::ReaderOptions readerOptions{leafPool_.get()}; + auto veloxReader = + std::make_unique(std::move(input), readerOptions); + EXPECT_EQ(veloxReader->numberOfRows(), 1); + auto veloxRowType = veloxReader->rowType(); + EXPECT_EQ(*veloxRowType, *schema); + + auto arrowBufferReader = std::make_shared<::arrow::io::BufferReader>( + std::make_shared<::arrow::Buffer>( + reinterpret_cast(sinkData.data()), sinkData.size())); + + auto fileReader = parquet::arrow::ParquetFileReader::Open(arrowBufferReader); + auto metadata = fileReader->metadata(); + auto* descr = metadata->schema(); + auto* root = descr->group_node(); + + ASSERT_EQ(root->field_count(), 4); + + // All field IDs should be -1 (not set). + EXPECT_EQ(root->field(0)->field_id(), -1); + EXPECT_EQ(root->field(1)->field_id(), -1); + EXPECT_EQ(root->field(2)->field_id(), -1); + EXPECT_EQ(root->field(3)->field_id(), -1); +} + +TEST_F(ParquetWriterTest, withFieldIds) { + auto schema = + ROW({"a", "b", "c", "m"}, + {BIGINT(), + ROW({"x", "y"}, {INTEGER(), VARCHAR()}), + ARRAY(INTEGER()), + MAP(VARCHAR(), INTEGER())}); + + auto arrVec = makeArrayVector({{3}}); + auto mapVec = makeMapVector({{{StringView("k"), 4}}}); + auto data = makeRowVector({ + makeFlatVector(1, [](auto) { return 1; }), + makeRowVector( + {makeFlatVector(1, [](auto) { return 2; }), + makeFlatVector( + 1, [](auto) { return StringView("z"); })}), + arrVec, + mapVec, + }); + + auto sink = std::make_unique( + 4 * 1024 * 1024, + dwio::common::FileSink::Options{.pool = leafPool_.get()}); + auto* sinkPtr = sink.get(); + + parquet::WriterOptions writerOptions; + writerOptions.memoryPool = leafPool_.get(); + + // Provide Parquet field IDs aligned with the Velox schema tree. + std::vector fieldIds{ + ParquetFieldId{10, {}}, + ParquetFieldId{20, {ParquetFieldId{21, {}}, ParquetFieldId{22, {}}}}, + ParquetFieldId{30, {ParquetFieldId{31, {}}}}, + ParquetFieldId{40, {ParquetFieldId{41, {}}, ParquetFieldId{42, {}}}}, + }; + writerOptions.parquetFieldIds = + std::make_shared>(fieldIds); + + auto writer = std::make_unique( + std::move(sink), writerOptions, rootPool_, schema); + writer->write(data); + writer->close(); + + std::string_view sinkData(sinkPtr->data(), sinkPtr->size()); + auto readFile = std::make_shared(sinkData); + auto input = std::make_unique(readFile, *leafPool_.get()); + dwio::common::ReaderOptions readerOptions{leafPool_.get()}; + auto veloxReader = + std::make_unique(std::move(input), readerOptions); + EXPECT_EQ(veloxReader->numberOfRows(), 1); + auto veloxRowType = veloxReader->rowType(); + EXPECT_EQ(*veloxRowType, *schema); + + auto arrowBufferReader = std::make_shared<::arrow::io::BufferReader>( + std::make_shared<::arrow::Buffer>( + reinterpret_cast(sinkData.data()), sinkData.size())); + + auto fileReader = parquet::arrow::ParquetFileReader::Open(arrowBufferReader); + auto metadata = fileReader->metadata(); + auto* descr = metadata->schema(); + auto* root = descr->group_node(); + + ASSERT_EQ(root->field_count(), 4); + + // Top-level field IDs. + EXPECT_EQ(root->field(0)->field_id(), 10); + EXPECT_EQ(root->field(1)->field_id(), 20); + EXPECT_EQ(root->field(2)->field_id(), 30); + EXPECT_EQ(root->field(3)->field_id(), 40); + + using GroupNode = parquet::arrow::schema::GroupNode; + auto* b = static_cast(root->field(1).get()); + EXPECT_EQ(b->field(0)->field_id(), 21); + EXPECT_EQ(b->field(1)->field_id(), 22); + + auto* c = static_cast(root->field(2).get()); + auto* listEntries = c->field(0).get(); + auto* listGroup = static_cast(listEntries); + auto* element = listGroup->field(0).get(); + EXPECT_EQ(element->field_id(), 31); + + auto* m = static_cast(root->field(3).get()); + auto* keyValue = m->field(0).get(); + auto* keyValueGroup = static_cast(keyValue); + EXPECT_EQ(keyValueGroup->field(0)->field_id(), 41); + EXPECT_EQ(keyValueGroup->field(1)->field_id(), 42); +} + } // namespace int main(int argc, char** argv) { diff --git a/velox/dwio/parquet/thrift/ParquetThriftTypes.cpp b/velox/dwio/parquet/thrift/ParquetThriftTypes.cpp index 674c99300f76..757472a58ee8 100644 --- a/velox/dwio/parquet/thrift/ParquetThriftTypes.cpp +++ b/velox/dwio/parquet/thrift/ParquetThriftTypes.cpp @@ -677,6 +677,11 @@ void Statistics::__set_distinct_count(const int64_t val) { __isset.distinct_count = true; } +void Statistics::__set_nan_count(const int64_t val) { + this->nan_count = val; + __isset.nan_count = true; +} + void Statistics::__set_max_value(const std::string& val) { this->max_value = val; __isset.max_value = true; @@ -820,6 +825,7 @@ void swap(Statistics& a, Statistics& b) { swap(a.min, b.min); swap(a.null_count, b.null_count); swap(a.distinct_count, b.distinct_count); + swap(a.nan_count, b.nan_count); swap(a.max_value, b.max_value); swap(a.min_value, b.min_value); swap(a.__isset, b.__isset); @@ -830,6 +836,7 @@ Statistics::Statistics(const Statistics& other0) { min = other0.min; null_count = other0.null_count; distinct_count = other0.distinct_count; + nan_count = other0.nan_count; max_value = other0.max_value; min_value = other0.min_value; __isset = other0.__isset; @@ -839,6 +846,7 @@ Statistics& Statistics::operator=(const Statistics& other1) { min = other1.min; null_count = other1.null_count; distinct_count = other1.distinct_count; + nan_count = other1.nan_count; max_value = other1.max_value; min_value = other1.min_value; __isset = other1.__isset; diff --git a/velox/dwio/parquet/thrift/ParquetThriftTypes.h b/velox/dwio/parquet/thrift/ParquetThriftTypes.h index 0fd63e83cc0e..1e5bd41b98b8 100644 --- a/velox/dwio/parquet/thrift/ParquetThriftTypes.h +++ b/velox/dwio/parquet/thrift/ParquetThriftTypes.h @@ -466,12 +466,14 @@ typedef struct _Statistics__isset { min(false), null_count(false), distinct_count(false), + nan_count(false), max_value(false), min_value(false) {} bool max : 1; bool min : 1; bool null_count : 1; bool distinct_count : 1; + bool nan_count : 1; bool max_value : 1; bool min_value : 1; } _Statistics__isset; @@ -489,6 +491,7 @@ class Statistics : public virtual apache::thrift::TBase { min(), null_count(0), distinct_count(0), + nan_count(0), max_value(), min_value() {} @@ -516,6 +519,15 @@ class Statistics : public virtual apache::thrift::TBase { * count of distinct values occurring */ int64_t distinct_count; + + /** + * count of NaN values occurring. + * Note: This is a Velox extension to the Parquet format. The upstream + * Parquet community is considering adding official support for this field. + * See: https://github.com/apache/parquet-format/pull/514 + */ + int64_t nan_count; + /** * Min and max values for the column, determined by its ColumnOrder. * @@ -535,6 +547,8 @@ class Statistics : public virtual apache::thrift::TBase { void __set_distinct_count(const int64_t val); + void __set_nan_count(const int64_t val); + void __set_max_value(const std::string& val); void __set_min_value(const std::string& val); @@ -556,6 +570,10 @@ class Statistics : public virtual apache::thrift::TBase { return false; else if (__isset.distinct_count && !(distinct_count == rhs.distinct_count)) return false; + if (__isset.nan_count != rhs.__isset.nan_count) + return false; + else if (__isset.nan_count && !(nan_count == rhs.nan_count)) + return false; if (__isset.max_value != rhs.__isset.max_value) return false; else if (__isset.max_value && !(max_value == rhs.max_value)) diff --git a/velox/dwio/parquet/writer/Writer.cpp b/velox/dwio/parquet/writer/Writer.cpp index 23dc04b5f6a9..6fa12f4dbaeb 100644 --- a/velox/dwio/parquet/writer/Writer.cpp +++ b/velox/dwio/parquet/writer/Writer.cpp @@ -22,6 +22,7 @@ #include "velox/common/config/Config.h" #include "velox/common/testutil/TestValue.h" #include "velox/core/QueryConfig.h" +#include "velox/dwio/parquet/writer/arrow/ArrowSchema.h" #include "velox/dwio/parquet/writer/arrow/Properties.h" #include "velox/dwio/parquet/writer/arrow/Writer.h" #include "velox/exec/MemoryReclaimer.h" @@ -193,47 +194,70 @@ void validateSchemaRecursive(const RowTypePtr& schema) { } } -std::shared_ptr<::arrow::Field> updateFieldNameRecursive( +std::shared_ptr<::arrow::Field> updateFieldNameAndIdRecursive( const std::shared_ptr<::arrow::Field>& field, const Type& type, - const std::string& name = "") { + const std::string& name = "", + const ParquetFieldId* fieldId = nullptr) { + std::shared_ptr<::arrow::Field> newField{nullptr}; if (type.isRow()) { auto& rowType = type.asRow(); - auto newField = field->WithName(name); + newField = field->WithName(name); auto structType = std::dynamic_pointer_cast<::arrow::StructType>(newField->type()); auto childrenSize = rowType.size(); std::vector> newFields; newFields.reserve(childrenSize); - for (auto i = 0; i < childrenSize; i++) { - newFields.push_back(updateFieldNameRecursive( - structType->fields()[i], *rowType.childAt(i), rowType.nameOf(i))); + for (auto i = 0; i < childrenSize; ++i) { + const auto* childSetting = fieldId ? &fieldId->children.at(i) : nullptr; + newFields.push_back(updateFieldNameAndIdRecursive( + structType->fields()[i], + *rowType.childAt(i), + rowType.nameOf(i), + childSetting)); } - return newField->WithType(::arrow::struct_(newFields)); + newField = newField->WithType(::arrow::struct_(newFields)); } else if (type.isArray()) { - auto newField = field->WithName(name); + newField = field->WithName(name); auto listType = std::dynamic_pointer_cast<::arrow::BaseListType>(newField->type()); auto elementType = type.asArray().elementType(); auto elementField = listType->value_field(); - return newField->WithType( - ::arrow::list(updateFieldNameRecursive(elementField, *elementType))); + const auto* childSetting = fieldId ? &fieldId->children.at(0) : nullptr; + auto updatedElementField = updateFieldNameAndIdRecursive( + elementField, *elementType, name, childSetting); + newField = newField->WithType(::arrow::list(updatedElementField)); } else if (type.isMap()) { auto mapType = type.asMap(); - auto newField = field->WithName(name); + newField = field->WithName(name); auto arrowMapType = std::dynamic_pointer_cast<::arrow::MapType>(newField->type()); - auto newKeyField = - updateFieldNameRecursive(arrowMapType->key_field(), *mapType.keyType()); - auto newValueField = updateFieldNameRecursive( - arrowMapType->item_field(), *mapType.valueType()); - return newField->WithType( - ::arrow::map(newKeyField->type(), newValueField->type())); + const auto* keySetting = fieldId ? &fieldId->children.at(0) : nullptr; + const auto* valueSetting = fieldId ? &fieldId->children.at(1) : nullptr; + auto newKeyField = updateFieldNameAndIdRecursive( + arrowMapType->key_field(), + *mapType.keyType(), + mapType.nameOf(0), + keySetting); + auto newValueField = updateFieldNameAndIdRecursive( + arrowMapType->item_field(), + *mapType.valueType(), + mapType.nameOf(1), + valueSetting); + newField = newField->WithType( + std::make_shared<::arrow::MapType>(newKeyField, newValueField)); } else if (name != "") { - return field->WithName(name); + newField = field->WithName(name); } else { - return field; + newField = field; } + + if (fieldId) { + newField = + newField->WithMetadata(arrow::arrow::FieldIdMetadata(fieldId->fieldId)); + } + + return newField; } std::optional getTimestampUnit( @@ -352,6 +376,9 @@ Writer::Writer( setMemoryReclaimers(); writeInt96AsTimestamp_ = options.writeInt96AsTimestamp; arrowMemoryPool_ = options.arrowMemoryPool; + parquetFieldIds_ = options.parquetFieldIds; + dataFileStats_ = std::make_shared(); + statsCollector_ = options.fileStatsCollector; } Writer::Writer( @@ -448,8 +475,11 @@ void Writer::write(const VectorPtr& data) { std::vector> newFields; auto childSize = schema_->size(); for (auto i = 0; i < childSize; i++) { - newFields.push_back(updateFieldNameRecursive( - arrowSchema->fields()[i], *schema_->childAt(i), schema_->nameOf(i))); + newFields.push_back(updateFieldNameAndIdRecursive( + arrowSchema->fields()[i], + *schema_->childAt(i), + schema_->nameOf(i), + parquetFieldIds_ ? &parquetFieldIds_->at(i) : nullptr)); } PARQUET_ASSIGN_OR_THROW( @@ -493,6 +523,11 @@ void Writer::close() { if (arrowContext_->writer) { PARQUET_THROW_NOT_OK(arrowContext_->writer->Close()); + if (statsCollector_) { + auto fileMetadata = arrowContext_->writer->metadata(); + statsCollector_->collectStats( + static_cast(&fileMetadata), dataFileStats_); + } arrowContext_->writer.reset(); } PARQUET_THROW_NOT_OK(stream_->Close()); diff --git a/velox/dwio/parquet/writer/Writer.h b/velox/dwio/parquet/writer/Writer.h index 7deca7ee670b..5df2238478f1 100644 --- a/velox/dwio/parquet/writer/Writer.h +++ b/velox/dwio/parquet/writer/Writer.h @@ -88,6 +88,13 @@ class LambdaFlushPolicy : public DefaultFlushPolicy { std::function lambda_; }; +// Settings for specifying Parquet field IDs during write operations. +// Used to explicitly control field ID assignment in the Parquet schema. +struct ParquetFieldId { + int32_t fieldId; + std::vector children; +}; + struct WriterOptions : public dwio::common::WriterOptions { // Growth ratio passed to ArrowDataBufferSink. The default value is a // heuristic borrowed from @@ -116,6 +123,8 @@ struct WriterOptions : public dwio::common::WriterOptions { std::shared_ptr arrowMemoryPool; + std::shared_ptr> parquetFieldIds; + // Parsing session and hive configs. // This isn't a typo; session and hive connector config names are different @@ -214,6 +223,8 @@ class Writer : public dwio::common::Writer { std::shared_ptr arrowContext_; + std::shared_ptr> parquetFieldIds_; + std::unique_ptr flushPolicy_; const RowTypePtr schema_; diff --git a/velox/dwio/parquet/writer/arrow/ArrowSchema.cpp b/velox/dwio/parquet/writer/arrow/ArrowSchema.cpp index 0e6a0db87e91..3dd6c247bac5 100644 --- a/velox/dwio/parquet/writer/arrow/ArrowSchema.cpp +++ b/velox/dwio/parquet/writer/arrow/ArrowSchema.cpp @@ -346,14 +346,6 @@ static Status GetTimestampMetadata( static constexpr char FIELD_ID_KEY[] = "PARQUET:field_id"; -std::shared_ptr<::arrow::KeyValueMetadata> FieldIdMetadata(int field_id) { - if (field_id >= 0) { - return ::arrow::key_value_metadata({FIELD_ID_KEY}, {ToChars(field_id)}); - } else { - return nullptr; - } -} - int FieldIdFromMetadata( const std::shared_ptr& metadata) { if (!metadata) { @@ -1223,6 +1215,14 @@ Result ApplyOriginalMetadata( } // namespace +std::shared_ptr<::arrow::KeyValueMetadata> FieldIdMetadata(int field_id) { + if (field_id >= 0) { + return ::arrow::key_value_metadata({FIELD_ID_KEY}, {ToChars(field_id)}); + } else { + return nullptr; + } +} + Status FieldToNode( const std::shared_ptr& field, const WriterProperties& properties, diff --git a/velox/dwio/parquet/writer/arrow/ArrowSchema.h b/velox/dwio/parquet/writer/arrow/ArrowSchema.h index 8302bc1cdb19..d60d2090f48b 100644 --- a/velox/dwio/parquet/writer/arrow/ArrowSchema.h +++ b/velox/dwio/parquet/writer/arrow/ArrowSchema.h @@ -195,5 +195,7 @@ struct PARQUET_EXPORT SchemaManifest { } }; +std::shared_ptr<::arrow::KeyValueMetadata> FieldIdMetadata(int32_t field_id); + } // namespace arrow } // namespace facebook::velox::parquet::arrow diff --git a/velox/dwio/parquet/writer/arrow/Metadata.cpp b/velox/dwio/parquet/writer/arrow/Metadata.cpp index 0c84f2750cb6..0854b4a5e6e7 100644 --- a/velox/dwio/parquet/writer/arrow/Metadata.cpp +++ b/velox/dwio/parquet/writer/arrow/Metadata.cpp @@ -101,10 +101,12 @@ static std::shared_ptr MakeTypedColumnStats( metadata.num_values - metadata.statistics.null_count, metadata.statistics.null_count, metadata.statistics.distinct_count, + metadata.statistics.nan_count, metadata.statistics.__isset.max_value || metadata.statistics.__isset.min_value, metadata.statistics.__isset.null_count, - metadata.statistics.__isset.distinct_count); + metadata.statistics.__isset.distinct_count, + metadata.statistics.__isset.nan_count); } // Default behavior return MakeStatistics( @@ -114,9 +116,11 @@ static std::shared_ptr MakeTypedColumnStats( metadata.num_values - metadata.statistics.null_count, metadata.statistics.null_count, metadata.statistics.distinct_count, + metadata.statistics.nan_count, metadata.statistics.__isset.max || metadata.statistics.__isset.min, metadata.statistics.__isset.null_count, - metadata.statistics.__isset.distinct_count); + metadata.statistics.__isset.distinct_count, + metadata.statistics.__isset.nan_count); } std::shared_ptr MakeColumnStats( @@ -389,6 +393,10 @@ class ColumnChunkMetaData::ColumnChunkMetaDataImpl { return std::nullopt; } + inline int32_t field_id() const { + return descr_->schema_node()->field_id(); + } + private: mutable std::shared_ptr possible_stats_; std::vector encodings_; @@ -535,6 +543,10 @@ int64_t ColumnChunkMetaData::total_compressed_size() const { return impl_->total_compressed_size(); } +int32_t ColumnChunkMetaData::field_id() const { + return impl_->field_id(); +} + std::unique_ptr ColumnChunkMetaData::crypto_metadata() const { return impl_->crypto_metadata(); diff --git a/velox/dwio/parquet/writer/arrow/Metadata.h b/velox/dwio/parquet/writer/arrow/Metadata.h index c69ee5a03d41..7cb7670a0387 100644 --- a/velox/dwio/parquet/writer/arrow/Metadata.h +++ b/velox/dwio/parquet/writer/arrow/Metadata.h @@ -187,6 +187,7 @@ class PARQUET_EXPORT ColumnChunkMetaData { int64_t index_page_offset() const; int64_t total_compressed_size() const; int64_t total_uncompressed_size() const; + int32_t field_id() const; std::unique_ptr crypto_metadata() const; std::optional GetColumnIndexLocation() const; std::optional GetOffsetIndexLocation() const; diff --git a/velox/dwio/parquet/writer/arrow/Statistics.cpp b/velox/dwio/parquet/writer/arrow/Statistics.cpp index 757ac36b62e8..803c88a770bf 100644 --- a/velox/dwio/parquet/writer/arrow/Statistics.cpp +++ b/velox/dwio/parquet/writer/arrow/Statistics.cpp @@ -35,10 +35,13 @@ #include "arrow/visit_data_inline.h" #include "velox/common/base/Exceptions.h" +#include "velox/dwio/parquet/common/UnicodeUtil.h" #include "velox/dwio/parquet/writer/arrow/Encoding.h" #include "velox/dwio/parquet/writer/arrow/Exception.h" #include "velox/dwio/parquet/writer/arrow/Platform.h" #include "velox/dwio/parquet/writer/arrow/Schema.h" +#include "velox/type/DecimalUtil.h" +#include "velox/type/HugeInt.h" using arrow::default_memory_pool; using arrow::MemoryPool; @@ -94,9 +97,15 @@ struct CompareHelper { "T is an unsigned numeric"); constexpr static T DefaultMin() { + if constexpr (std::is_floating_point_v) { + return std::numeric_limits::infinity(); + } return std::numeric_limits::max(); } constexpr static T DefaultMax() { + if constexpr (std::is_floating_point_v) { + return -std::numeric_limits::infinity(); + } return std::numeric_limits::lowest(); } @@ -361,6 +370,7 @@ CleanStatistic(std::pair min_max) { // In case of floating point types, the following rules are applied (as per // upstream parquet-mr): // - If any of min/max is NaN, return nothing. +// - If min is infinity and max is -infinity, return nothing. // - If min is 0.0f, replace with -0.0f // - If max is -0.0f, replace with 0.0f template @@ -375,8 +385,8 @@ ::arrow:: return ::std::nullopt; } - if (min == std::numeric_limits::max() && - max == std::numeric_limits::lowest()) { + if (min == std::numeric_limits::infinity() && + max == -std::numeric_limits::infinity()) { return ::std::nullopt; } @@ -547,6 +557,17 @@ TypedComparatorImpl::GetMinMax( return GetMinMaxBinaryHelper(*this, values); } +template +std::string encodeDecimalToBigEndian(T value) { + uint8_t buffer[sizeof(T)]; + if constexpr (std::is_same_v) { + *reinterpret_cast(buffer) = ::arrow::bit_util::ToBigEndian(value); + } else if constexpr (std::is_same_v) { + *reinterpret_cast(buffer) = DecimalUtil::bigEndian(value); + } + return std::string(reinterpret_cast(buffer), sizeof(T)); +} + template class TypedStatisticsImpl : public TypedStatistics { public: @@ -590,9 +611,11 @@ class TypedStatisticsImpl : public TypedStatistics { int64_t num_values, int64_t null_count, int64_t distinct_count, + int64_t nan_count, bool has_min_max, bool has_null_count, bool has_distinct_count, + bool has_nan_count, MemoryPool* pool) : TypedStatisticsImpl(descr, pool) { TypedStatisticsImpl::IncrementNumValues(num_values); @@ -607,6 +630,12 @@ class TypedStatisticsImpl : public TypedStatistics { has_distinct_count_ = false; } + if (has_nan_count) { + IncrementNaNValues(nan_count); + } else { + has_nan_count_ = false; + } + if (!encoded_min.empty()) { PlainDecode(encoded_min, &min_); } @@ -619,6 +648,11 @@ class TypedStatisticsImpl : public TypedStatistics { bool HasDistinctCount() const override { return has_distinct_count_; }; + + bool HasNaNCount() const override { + return has_nan_count_; + }; + bool HasMinMax() const override { return has_min_max_; } @@ -635,6 +669,13 @@ class TypedStatisticsImpl : public TypedStatistics { num_values_ += n; } + void IncrementNaNValues(int64_t n) override { + if (n > 0) { + statistics_.nan_count += n; + has_nan_count_ = true; + } + } + bool Equals(const Statistics& raw_other) const override { if (physical_type() != raw_other.physical_type()) return false; @@ -673,6 +714,10 @@ class TypedStatisticsImpl : public TypedStatistics { } else { this->has_null_count_ = false; } + if (other.HasNaNCount()) { + this->statistics_.nan_count += other.nan_count(); + this->has_nan_count_ = true; + } if (has_distinct_count_ && other.HasDistinctCount() && (distinct_count() == 0 || other.distinct_count() == 0)) { // We can merge distinct counts if either side is zero. @@ -742,6 +787,55 @@ class TypedStatisticsImpl : public TypedStatistics { return s; } + std::string MinValue() const override { + if constexpr (std::is_same_v) { + if (descr_->logical_type()->is_decimal()) { + return encodeDecimalToBigEndian(min_); + } + } + if constexpr (std::is_same_v) { + return encodeDecimalToBigEndian(min_); + } + if constexpr (std::is_same_v) { + // TODO: 16 is default value. See DEFAULT_WRITE_METRICS_MODE_DEFAULT in + // org.apache.iceberg.TableProperties. Need to support this table + // property. + const auto truncatedMin = UnicodeUtil::truncateStringMin( + reinterpret_cast(min_.ptr), min_.len, 16); + std::string s; + this->PlainEncode( + ByteArray( + truncatedMin.size(), + reinterpret_cast(truncatedMin.data())), + &s); + return s; + } + return EncodeMin(); + } + + std::string MaxValue() const override { + if constexpr (std::is_same_v) { + if (descr_->logical_type()->is_decimal()) { + return encodeDecimalToBigEndian(max_); + } + } + if constexpr (std::is_same_v) { + return encodeDecimalToBigEndian(max_); + } + if constexpr (std::is_same_v) { + const auto truncatedMax = UnicodeUtil::truncateStringMax( + reinterpret_cast(max_.ptr), max_.len, 16); + std::string s; + this->PlainEncode( + ByteArray( + truncatedMax.size(), + reinterpret_cast(truncatedMax.data())), + &s); + return s; + } + return EncodeMax(); + } + EncodedStatistics Encode() override { EncodedStatistics s; if (HasMinMax()) { @@ -756,6 +850,9 @@ class TypedStatisticsImpl : public TypedStatistics { if (HasDistinctCount()) { s.set_distinct_count(this->distinct_count()); } + if (has_nan_count_) { + s.set_nan_count(this->nan_count()); + } return s; } @@ -768,12 +865,26 @@ class TypedStatisticsImpl : public TypedStatistics { int64_t num_values() const override { return num_values_; } + int64_t nan_count() const override { + return statistics_.nan_count; + } + + bool CompareMax(const Statistics& other) const override { + auto typedStats = dynamic_cast*>(&other); + return comparator_->Compare(max_, typedStats->max_) ? false : true; + } + + bool CompareMin(const Statistics& other) const override { + auto typedStats = dynamic_cast*>(&other); + return comparator_->Compare(min_, typedStats->min_) ? true : false; + } private: const ColumnDescriptor* descr_; bool has_min_max_ = false; bool has_null_count_ = false; bool has_distinct_count_ = false; + bool has_nan_count_ = false; T min_; T max_; ::arrow::MemoryPool* pool_; @@ -803,6 +914,7 @@ class TypedStatisticsImpl : public TypedStatistics { void ResetCounts() { this->statistics_.null_count = 0; this->statistics_.distinct_count = 0; + this->statistics_.nan_count = 0; this->num_values_ = 0; } @@ -815,6 +927,7 @@ class TypedStatisticsImpl : public TypedStatistics { this->has_distinct_count_ = false; // Null count calculation is cheap and enabled by default. this->has_null_count_ = true; + this->has_nan_count_ = false; } void SetMinMaxPair(std::pair min_max) { @@ -841,6 +954,46 @@ class TypedStatisticsImpl : public TypedStatistics { max_buffer_.get()); } } + + int64_t CountNaN(const T* values, int64_t length) { + if constexpr (!std::is_floating_point_v) { + return 0; + } else { + int64_t count = 0; + for (auto i = 0; i < length; i++) { + const auto val = SafeLoad(values + i); + if (std::isnan(val)) { + count++; + } + } + return count; + } + } + + int64_t CountNaNSpaced( + const T* values, + int64_t length, + const uint8_t* valid_bits, + int64_t valid_bits_offset) { + if constexpr (!std::is_floating_point_v) { + return 0; + } else { + int64_t count = 0; + ::arrow::internal::VisitSetBitRunsVoid( + valid_bits, + valid_bits_offset, + length, + [&](int64_t position, int64_t run_length) { + for (auto i = 0; i < run_length; i++) { + const auto val = SafeLoad(values + i + position); + if (std::isnan(val)) { + count++; + } + } + }); + return count; + } + } }; template <> @@ -896,6 +1049,7 @@ void TypedStatisticsImpl::Update( if (num_values == 0) return; SetMinMaxPair(comparator_->GetMinMax(values, num_values)); + IncrementNaNValues(CountNaN(values, num_values)); } template @@ -916,6 +1070,8 @@ void TypedStatisticsImpl::UpdateSpaced( return; SetMinMaxPair(comparator_->GetMinMaxSpaced( values, num_spaced_values, valid_bits, valid_bits_offset)); + IncrementNaNValues( + CountNaNSpaced(values, num_spaced_values, valid_bits, valid_bits_offset)); } template @@ -1080,9 +1236,11 @@ std::shared_ptr Statistics::Make( num_values, encoded_stats->null_count, encoded_stats->distinct_count, + encoded_stats->nan_count, encoded_stats->has_min && encoded_stats->has_max, encoded_stats->has_null_count, encoded_stats->has_distinct_count, + encoded_stats->has_nan_count, pool); } @@ -1093,9 +1251,11 @@ std::shared_ptr Statistics::Make( int64_t num_values, int64_t null_count, int64_t distinct_count, + int64_t nan_count, bool has_min_max, bool has_null_count, bool has_distinct_count, + bool has_nan_count, ::arrow::MemoryPool* pool) { #define MAKE_STATS(CAP_TYPE, KLASS) \ case Type::CAP_TYPE: \ @@ -1106,9 +1266,11 @@ std::shared_ptr Statistics::Make( num_values, \ null_count, \ distinct_count, \ + nan_count, \ has_min_max, \ has_null_count, \ has_distinct_count, \ + has_nan_count, \ pool) switch (descr->physical_type()) { diff --git a/velox/dwio/parquet/writer/arrow/Statistics.h b/velox/dwio/parquet/writer/arrow/Statistics.h index 6abf66b0b20a..0b7b738c253a 100644 --- a/velox/dwio/parquet/writer/arrow/Statistics.h +++ b/velox/dwio/parquet/writer/arrow/Statistics.h @@ -141,11 +141,13 @@ class PARQUET_EXPORT EncodedStatistics { int64_t null_count = 0; int64_t distinct_count = 0; + int64_t nan_count = 0; bool has_min = false; bool has_max = false; bool has_null_count = false; bool has_distinct_count = false; + bool has_nan_count = false; // When all values in the statistics are null, it is set to true. // Otherwise, at least one value is not null, or we are not sure at all. @@ -170,7 +172,8 @@ class PARQUET_EXPORT EncodedStatistics { } bool is_set() const { - return has_min || has_max || has_null_count || has_distinct_count; + return has_min || has_max || has_null_count || has_distinct_count || + has_nan_count; } bool is_signed() const { @@ -204,6 +207,12 @@ class PARQUET_EXPORT EncodedStatistics { has_distinct_count = true; return *this; } + + EncodedStatistics& set_nan_count(int64_t value) { + nan_count = value; + has_nan_count = true; + return *this; + } }; /// \brief Base type for computing column statistics while writing a file @@ -227,10 +236,12 @@ class PARQUET_EXPORT Statistics { /// \param[in] num_values total number of values /// \param[in] null_count number of null values /// \param[in] distinct_count number of distinct values + /// \param[in] nan_count number of nan values /// \param[in] has_min_max whether the min/max statistics are set /// \param[in] has_null_count whether the null_count statistics are set /// \param[in] has_distinct_count whether the distinct_count statistics are - /// set \param[in] pool a memory pool to use for any memory allocations, + /// set \param[in] has_nan_count whether the nan_count statistics are set + /// \param[in] pool a memory pool to use for any memory allocations, /// optional static std::shared_ptr Make( const ColumnDescriptor* descr, @@ -239,9 +250,11 @@ class PARQUET_EXPORT Statistics { int64_t num_values, int64_t null_count, int64_t distinct_count, + int64_t nan_count, bool has_min_max, bool has_null_count, bool has_distinct_count, + bool has_nan_count, ::arrow::MemoryPool* pool = ::arrow::default_memory_pool()); // Helper function to convert EncodedStatistics to Statistics. @@ -268,6 +281,12 @@ class PARQUET_EXPORT Statistics { /// \brief The number of non-null values in the column virtual int64_t num_values() const = 0; + /// \brief Return true if the count of nan values is set + virtual bool HasNaNCount() const = 0; + + /// \brief The number of NaN values, may not be set + virtual int64_t nan_count() const = 0; + /// \brief Return true if the min and max statistics are set. Obtain /// with TypedStatistics::min and max virtual bool HasMinMax() const = 0; @@ -281,6 +300,12 @@ class PARQUET_EXPORT Statistics { /// \brief Plain-encoded maximum value virtual std::string EncodeMax() const = 0; + /// \brief Compatible minimum value with iceberg + virtual std::string MinValue() const = 0; + + /// \brief Compatible maximum value with iceberg + virtual std::string MaxValue() const = 0; + /// \brief The finalized encoded form of the statistics for transport virtual EncodedStatistics Encode() = 0; @@ -293,6 +318,24 @@ class PARQUET_EXPORT Statistics { /// \brief Check two Statistics for equality virtual bool Equals(const Statistics& other) const = 0; + /// \brief Return true if this object is greater than other + virtual bool CompareMax(const Statistics& other) const = 0; + + /// \brief Return true if this object is less than other + virtual bool CompareMin(const Statistics& other) const = 0; + + static std::shared_ptr CompareAndGetMax( + const std::shared_ptr& stats1, + const std::shared_ptr& stats2) { + return stats1->CompareMax(*stats2) ? stats1 : stats2; + } + + static std::shared_ptr CompareAndGetMin( + const std::shared_ptr& stats1, + const std::shared_ptr& stats2) { + return stats1->CompareMin(*stats2) ? stats1 : stats2; + } + protected: static std::shared_ptr Make( Type::type physical_type, @@ -369,6 +412,9 @@ class TypedStatistics : public Statistics { /// \brief Increments the number of values directly /// The same note on IncrementNullCount applies here virtual void IncrementNumValues(int64_t n) = 0; + + /// \brief Increments the NaN count directly + virtual void IncrementNaNValues(int64_t n) = 0; }; using BoolStatistics = TypedStatistics; @@ -414,9 +460,11 @@ std::shared_ptr> MakeStatistics( int64_t num_values, int64_t null_count, int64_t distinct_count, + int64_t nan_count, bool has_min_max, bool has_null_count, bool has_distinct_count, + bool has_nan_count, ::arrow::MemoryPool* pool = ::arrow::default_memory_pool()) { return std::static_pointer_cast>(Statistics::Make( descr, @@ -425,9 +473,11 @@ std::shared_ptr> MakeStatistics( num_values, null_count, distinct_count, + nan_count, has_min_max, has_null_count, has_distinct_count, + has_nan_count, pool)); } diff --git a/velox/dwio/parquet/writer/arrow/ThriftInternal.h b/velox/dwio/parquet/writer/arrow/ThriftInternal.h index a6383474d23f..43c2aad60918 100644 --- a/velox/dwio/parquet/writer/arrow/ThriftInternal.h +++ b/velox/dwio/parquet/writer/arrow/ThriftInternal.h @@ -384,6 +384,9 @@ static inline facebook::velox::parquet::thrift::Statistics ToThrift( if (stats.has_distinct_count) { statistics.__set_distinct_count(stats.distinct_count); } + if (stats.has_nan_count) { + statistics.__set_nan_count(stats.nan_count); + } return statistics; } diff --git a/velox/dwio/parquet/writer/arrow/tests/StatisticsTest.cpp b/velox/dwio/parquet/writer/arrow/tests/StatisticsTest.cpp index da4aede6a841..fe3a87da64b7 100644 --- a/velox/dwio/parquet/writer/arrow/tests/StatisticsTest.cpp +++ b/velox/dwio/parquet/writer/arrow/tests/StatisticsTest.cpp @@ -344,9 +344,11 @@ class TestStatistics : public PrimitiveTypedTest { this->values_.size(), 0, 0, + 0, + true, true, true, - true); + false); auto statistics3 = MakeStatistics(this->schema_.Column(0)); std::vector valid_bits( @@ -610,9 +612,11 @@ void TestStatistics::TestMinMaxEncode() { this->values_.size(), 0, 0, + 0, true, true, - true); + true, + false); ASSERT_EQ(encoded_min, statistics2->EncodeMin()); ASSERT_EQ(encoded_max, statistics2->EncodeMax()); @@ -1533,6 +1537,7 @@ void CheckNaNs() { auto some_nan_stats = MakeStatistics(&descr); // Ingesting only nans should not yield valid min max AssertUnsetMinMax(some_nan_stats, all_nans); + EXPECT_EQ(some_nan_stats->nan_count(), all_nans.size()); // Ingesting a mix of NaNs and non-NaNs should not yield valid min max. AssertMinMaxAre(some_nan_stats, some_nans, min, max); // Ingesting only nans after a valid min/max, should have not effect @@ -1550,6 +1555,7 @@ void CheckNaNs() { 1.5f, max, -3.0f, -1.0f, nan, 2.0f, min, nan}; auto other_stats = MakeStatistics(&descr); AssertMinMaxAre(other_stats, other_nans, min, max); + EXPECT_EQ(other_stats->nan_count(), 2); } TEST(TestStatistic, NaNFloatValues) { @@ -1636,6 +1642,172 @@ TEST(TestStatistics, DoubleNegativeZero) { CheckNegativeZeroStats(); } +// Test infinity handling in statistics. +template +void CheckInfinityStats() { + using T = typename ParquetType::c_type; + + constexpr int32_t kNumValues = 8; + NodePtr node = PrimitiveNode::Make( + "infinity_test", Repetition::OPTIONAL, ParquetType::type_num); + ColumnDescriptor descr(node, 1, 1); + + constexpr T posInf = std::numeric_limits::infinity(); + constexpr T negInf = -std::numeric_limits::infinity(); + constexpr T min = -1.0f; + constexpr T max = 1.0f; + + { + std::array allPosInf{ + posInf, posInf, posInf, posInf, posInf, posInf, posInf, posInf}; + auto stats = MakeStatistics(&descr); + AssertMinMaxAre(stats, allPosInf, posInf, posInf); + } + + { + std::array allNegInf{ + negInf, negInf, negInf, negInf, negInf, negInf, negInf, negInf}; + auto stats = MakeStatistics(&descr); + AssertMinMaxAre(stats, allNegInf, negInf, negInf); + } + + { + std::array mixedInf{ + posInf, negInf, posInf, negInf, posInf, negInf, posInf, negInf}; + auto stats = MakeStatistics(&descr); + AssertMinMaxAre(stats, mixedInf, negInf, posInf); + } + + { + std::array mixedValues{ + posInf, max, min, min, negInf, max, min, posInf}; + auto stats = MakeStatistics(&descr); + AssertMinMaxAre(stats, mixedValues, negInf, posInf); + } + + { + constexpr T nan = std::numeric_limits::quiet_NaN(); + std::array mixedWithNan{ + posInf, nan, max, negInf, nan, min, posInf, nan}; + auto stats = MakeStatistics(&descr); + AssertMinMaxAre(stats, mixedWithNan, negInf, posInf); + } +} + +TEST(TestStatistics, FloatInfinityValues) { + CheckInfinityStats(); +} + +TEST(TestStatistics, DoubleInfinityValues) { + CheckInfinityStats(); +} + +// Test infinity values with validity bitmap. +TEST(TestStatistics, InfinityWithNullBitmap) { + constexpr int kNumValues = 8; + NodePtr node = PrimitiveNode::Make( + "infinity_null_test", Repetition::OPTIONAL, Type::FLOAT); + ColumnDescriptor descr(node, 1, 1); + + constexpr float posInf = std::numeric_limits::infinity(); + constexpr float negInf = -std::numeric_limits::infinity(); + + // Test with some infinity values marked as null. + std::array valuesWithNulls{ + posInf, negInf, 1.0f, 2.0f, posInf, -1.0f, 3.0f, negInf}; + + // Bitmap: exclude first posInf and last negInf (01111110 = 0x7E). + uint8_t validBitmap = 0x7E; + + auto stats = MakeStatistics(&descr); + AssertMinMaxAre(stats, valuesWithNulls, &validBitmap, negInf, posInf); + valuesWithNulls = {posInf, 0.0f, 1.0f, 2.0f, -2.0f, -1.0f, 3.0f, negInf}; + + stats = MakeStatistics(&descr); + AssertMinMaxAre(stats, valuesWithNulls, &validBitmap, -2.0f, 3.0f); +} + +// Test merging statistics with infinity values. +TEST(TestStatistics, MergeInfinityStatistics) { + NodePtr node = + PrimitiveNode::Make("merge_infinity", Repetition::OPTIONAL, Type::DOUBLE); + ColumnDescriptor descr(node, 1, 1); + + constexpr double posInf = std::numeric_limits::infinity(); + constexpr double negInf = -std::numeric_limits::infinity(); + + auto stats1 = MakeStatistics(&descr); + std::array normalValues{-1.0f, 0.0f, 1.0f}; + AssertMinMaxAre(stats1, normalValues, -1.0f, 1.0f); + + auto stats2 = MakeStatistics(&descr); + std::array infinityValues{negInf, posInf}; + AssertMinMaxAre(stats2, infinityValues, negInf, posInf); + + auto mergedStats = MakeStatistics(&descr); + mergedStats->Merge(*stats1); + mergedStats->Merge(*stats2); + + // Result should have infinity bounds. + ASSERT_TRUE(mergedStats->HasMinMax()); + ASSERT_EQ(negInf, mergedStats->min()); + ASSERT_EQ(posInf, mergedStats->max()); +} + +TEST(TestStatistics, CleanInfinityStatistics) { + constexpr int kNumValues = 4; + NodePtr node = PrimitiveNode::Make( + "clean_stat_nullopt", Repetition::OPTIONAL, Type::FLOAT); + ColumnDescriptor descr(node, 1, 1); + + constexpr float nan = std::numeric_limits::quiet_NaN(); + + { + std::array allNans{nan, nan, nan, nan}; + auto stats = MakeStatistics(&descr); + AssertUnsetMinMax(stats, allNans); + } + + { + std::array values{1.0f, 2.0f, 3.0f, 4.0f}; + uint8_t allNullBitmap = 0x00; + + auto stats = MakeStatistics(&descr); + AssertUnsetMinMax(stats, values, &allNullBitmap); + } + + { + std::array mixedNans{nan, 1.0f, nan, 2.0f}; + uint8_t partialNullBitmap = 0x05; + + auto stats = MakeStatistics(&descr); + AssertUnsetMinMax(stats, mixedNans, &partialNullBitmap); + } +} + +TEST(TestStatistics, InfinityCleanStatisticValid) { + constexpr int kNumValues = 4; + NodePtr node = PrimitiveNode::Make( + "clean_stat_valid", Repetition::OPTIONAL, Type::DOUBLE); + ColumnDescriptor descr(node, 1, 1); + + constexpr double posInf = std::numeric_limits::infinity(); + constexpr double negInf = -std::numeric_limits::infinity(); + constexpr double nan = std::numeric_limits::quiet_NaN(); + + { + std::array mixedValues{posInf, nan, negInf, nan}; + auto stats = MakeStatistics(&descr); + AssertMinMaxAre(stats, mixedValues, negInf, posInf); + } + + { + std::array singleInf{negInf}; + auto stats = MakeStatistics(&descr); + AssertMinMaxAre(stats, singleInf, negInf, negInf); + } +} + // TODO: disabled as it requires Arrow parquet data dir. // Test statistics for binary column with UNSIGNED sort order /* 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 f61df31824bc..f4ff151c845d 100644 --- a/velox/type/Filter.cpp +++ b/velox/type/Filter.cpp @@ -1214,7 +1214,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( @@ -1230,7 +1230,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 ee1a5f74d72d..15ee3f14b550 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, diff --git a/velox/type/Timestamp.cpp b/velox/type/Timestamp.cpp index 562c738d961d..61d07249fd2a 100644 --- a/velox/type/Timestamp.cpp +++ b/velox/type/Timestamp.cpp @@ -268,17 +268,22 @@ StringView Timestamp::tmToStringView( *writePosition++ = ':'; writePosition += appendDigits(tmValue.tm_min, 2, writePosition); - // Second. - *writePosition++ = ':'; - writePosition += appendDigits(tmValue.tm_sec, 2, writePosition); - if (options.precision == TimestampToStringOptions::Precision::kMilliseconds) { nanos /= 1'000'000; } else if ( options.precision == TimestampToStringOptions::Precision::kMicroseconds) { nanos /= 1'000; } - if (options.skipTrailingZeros && nanos == 0) { + + // Second. + const bool shouldSkipSeconds = + options.skipTrailingZeroSeconds && tmValue.tm_sec == 0 && nanos == 0; + if (!shouldSkipSeconds) { + *writePosition++ = ':'; + writePosition += appendDigits(tmValue.tm_sec, 2, writePosition); + } + + if ((options.skipTrailingZeros && nanos == 0) || shouldSkipSeconds) { return StringView(startPosition, writePosition - startPosition); } diff --git a/velox/type/Timestamp.h b/velox/type/Timestamp.h index bd749f11420f..d85bd070c868 100644 --- a/velox/type/Timestamp.h +++ b/velox/type/Timestamp.h @@ -48,6 +48,12 @@ struct TimestampToStringOptions { /// '2000-01-01 12:21:56.129000' becomes '2000-01-01 12:21:56.129'. bool skipTrailingZeros = false; + /// Whether to skip trailing zeros of second part. E.g. when true, + /// '2000-01-01 12:21:00' becomes '2000-01-01 12:21'. + /// '2000-01-01 12:21:00.000' becomes '2000-01-01 12:21'. + /// '2000-01-01 12:21:00.123' will not be impacted by this option. + bool skipTrailingZeroSeconds = false; + /// Whether padding zeros are added when the digits of year is less than 4. /// E.g. when true, '1-01-01 05:17:32.000' becomes '0001-01-01 05:17:32.000', /// '-03-24 13:20:00.000' becomes '0000-03-24 13:20:00.000', and '-1-11-29 diff --git a/velox/type/tests/TimestampTest.cpp b/velox/type/tests/TimestampTest.cpp index 9e3ee43e9ed7..b1fdbdd73239 100644 --- a/velox/type/tests/TimestampTest.cpp +++ b/velox/type/tests/TimestampTest.cpp @@ -578,5 +578,44 @@ TEST(TimestampTest, skipTrailingZeros) { "0384-01-01 08:00:00.7266"); } +TEST(TimestampTest, skipTrailingZeroSeconds) { + TimestampToStringOptions options = { + .precision = TimestampToStringOptions::Precision::kMicroseconds, + .skipTrailingZeros = true, + .skipTrailingZeroSeconds = true, + .zeroPaddingYear = true, + .dateTimeSeparator = ' ', + }; + + ASSERT_EQ( + timestampToString(Timestamp(-946684800, 0), options), "1940-01-02 00:00"); + ASSERT_EQ(timestampToString(Timestamp(0, 0), options), "1970-01-01 00:00"); + ASSERT_EQ(timestampToString(Timestamp(0, 365), options), "1970-01-01 00:00"); + ASSERT_EQ( + timestampToString(Timestamp(0, 65873), options), + "1970-01-01 00:00:00.000065"); + ASSERT_EQ( + timestampToString(Timestamp(94668480000, 0), options), + "4969-12-04 00:00"); + ASSERT_EQ( + timestampToString(Timestamp(946729316, 129999999), options), + "2000-01-01 12:21:56.129999"); + ASSERT_EQ( + timestampToString(Timestamp(946729316, 129900000), options), + "2000-01-01 12:21:56.1299"); + ASSERT_EQ( + timestampToString(Timestamp(946729316, 129000000), options), + "2000-01-01 12:21:56.129"); + ASSERT_EQ( + timestampToString(Timestamp(946729316, 0), options), + "2000-01-01 12:21:56"); + ASSERT_EQ( + timestampToString(Timestamp(946729316, 129001000), options), + "2000-01-01 12:21:56.129001"); + ASSERT_EQ( + timestampToString(Timestamp(-50049331200, 726600000), options), + "0384-01-01 08:00:00.7266"); +} + } // namespace } // namespace facebook::velox