diff --git a/velox/connectors/lakehouse/CMakeLists.txt b/velox/connectors/lakehouse/CMakeLists.txt new file mode 100644 index 000000000000..5a49821913e3 --- /dev/null +++ b/velox/connectors/lakehouse/CMakeLists.txt @@ -0,0 +1,17 @@ +# Copyright (c) Facebook, Inc. and its affiliates. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +add_subdirectory(storage_adapters) +add_subdirectory(common) +add_subdirectory(iceberg) \ No newline at end of file diff --git a/velox/connectors/lakehouse/common/CMakeLists.txt b/velox/connectors/lakehouse/common/CMakeLists.txt new file mode 100644 index 000000000000..6c8aa05487b4 --- /dev/null +++ b/velox/connectors/lakehouse/common/CMakeLists.txt @@ -0,0 +1,53 @@ +# Copyright (c) Facebook, Inc. and its affiliates. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +velox_add_library(velox_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 + FileHandle.cpp + HiveConfig.cpp + HiveConnector.cpp + HiveConnectorUtil.cpp + HiveConnectorSplit.cpp + HiveDataSink.cpp + HiveDataSource.cpp + HivePartitionUtil.cpp + PartitionIdGenerator.cpp + SplitReader.cpp + TableHandle.cpp) + +velox_link_libraries( + velox_hive_connector + PUBLIC velox_hive_iceberg_splitreader + PRIVATE velox_common_io velox_connector velox_dwio_catalog_fbhive + velox_hive_partition_function) + +velox_add_library(velox_hive_partition_function HivePartitionFunction.cpp) + +velox_link_libraries(velox_hive_partition_function velox_core velox_exec) + +add_subdirectory(storage_adapters) + +if(${VELOX_BUILD_TESTING}) + add_subdirectory(tests) +endif() + +if(${VELOX_ENABLE_BENCHMARKS}) + add_subdirectory(benchmarks) +endif() diff --git a/velox/connectors/lakehouse/common/FileHandle.cpp b/velox/connectors/lakehouse/common/FileHandle.cpp new file mode 100644 index 000000000000..267691cce2ee --- /dev/null +++ b/velox/connectors/lakehouse/common/FileHandle.cpp @@ -0,0 +1,78 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "velox/connectors/hive/FileHandle.h" +#include "velox/common/base/Counters.h" +#include "velox/common/base/StatsReporter.h" +#include "velox/common/file/FileSystems.h" +#include "velox/common/time/Timer.h" + +#include + +namespace facebook::velox { + +uint64_t FileHandleSizer::operator()(const FileHandle& fileHandle) { + // TODO: add to support variable file cache size support when the file system + // underneath supports. + return 1; +} + +namespace { +// The group tracking is at the level of the directory, i.e. Hive partition. +std::string groupName(const std::string& filename) { + const char* slash = strrchr(filename.c_str(), '/'); + return slash ? std::string(filename.data(), slash - filename.data()) + : filename; +} +} // namespace + +std::unique_ptr FileHandleGenerator::operator()( + const FileHandleKey& key, + const FileProperties* properties, + filesystems::File::IoStats* stats) { + // We have seen cases where drivers are stuck when creating file handles. + // Adding a trace here to spot this more easily in future. + process::TraceContext trace("FileHandleGenerator::operator()"); + uint64_t elapsedTimeUs{0}; + std::unique_ptr fileHandle; + { + MicrosecondTimer timer(&elapsedTimeUs); + fileHandle = std::make_unique(); + filesystems::FileOptions options; + options.stats = stats; + options.tokenProvider = key.tokenProvider; + if (properties) { + options.fileSize = properties->fileSize; + options.readRangeHint = properties->readRangeHint; + options.extraFileInfo = properties->extraFileInfo; + } + const auto& filename = key.filename; + fileHandle->file = filesystems::getFileSystem(filename, properties_) + ->openFileForRead(filename, options); + fileHandle->uuid = StringIdLease(fileIds(), filename); + fileHandle->groupId = StringIdLease(fileIds(), groupName(filename)); + VLOG(1) << "Generating file handle for: " << filename + << " uuid: " << fileHandle->uuid.id(); + } + RECORD_HISTOGRAM_METRIC_VALUE( + kMetricHiveFileHandleGenerateLatencyMs, elapsedTimeUs / 1000); + // TODO: build the hash map/etc per file type -- presumably after reading + // the appropriate magic number from the file, or perhaps we include the file + // type in the file handle key. + return fileHandle; +} + +} // namespace facebook::velox diff --git a/velox/connectors/lakehouse/common/FileHandle.h b/velox/connectors/lakehouse/common/FileHandle.h new file mode 100644 index 000000000000..6f9b4050c31e --- /dev/null +++ b/velox/connectors/lakehouse/common/FileHandle.h @@ -0,0 +1,130 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// A FileHandle is a File pointer plus some (optional, file-type-dependent) +// extra information for speeding up loading columnar data. For example, when +// we open a file we might build a hash map saying what region(s) on disk +// correspond to a given column in a given stripe. +// +// The FileHandle will normally be used in conjunction with a CachedFactory +// to speed up queries that hit the same files repeatedly; see the +// FileHandleCache and FileHandleFactory. + +#pragma once + +#include "velox/common/base/BitUtil.h" +#include "velox/common/caching/CachedFactory.h" +#include "velox/common/caching/FileIds.h" +#include "velox/common/config/Config.h" +#include "velox/common/file/File.h" +#include "velox/common/file/TokenProvider.h" +#include "velox/connectors/hive/FileProperties.h" + +namespace facebook::velox { + +// See the file comment. +struct FileHandle { + std::shared_ptr file; + + // Each time we make a new FileHandle we assign it a uuid and use that id as + // the identifier in downstream data caching structures. This saves a lot of + // memory compared to using the filename as the identifier. + StringIdLease uuid; + + // Id for the group of files this belongs to, e.g. its + // directory. Used for coarse granularity access tracking, for + // example to decide placing on SSD. + StringIdLease groupId; + + // We'll want to have a hash map here to record the identifier->byte range + // mappings. Different formats may have different identifiers, so we may need + // a union of maps. For example in orc you need 3 integers (I think, to be + // confirmed with xldb): the row bundle, the node, and the sequence. For the + // first diff we'll not include the map. +}; + +/// Estimates the memory usage of a FileHandle object. +struct FileHandleSizer { + uint64_t operator()(const FileHandle& a); +}; + +struct FileHandleKey { + std::string filename; + std::shared_ptr tokenProvider{nullptr}; + + bool operator==(const FileHandleKey& other) const { + if (filename != other.filename) { + return false; + } + + if (tokenProvider == other.tokenProvider) { + return true; + } + + if (!tokenProvider || !other.tokenProvider) { + return false; + } + + return tokenProvider->equals(*other.tokenProvider); + } +}; + +} // namespace facebook::velox + +namespace std { +template <> +struct hash { + size_t operator()(const facebook::velox::FileHandleKey& key) const noexcept { + size_t filenameHash = std::hash()(key.filename); + return key.tokenProvider ? facebook::velox::bits::hashMix( + filenameHash, key.tokenProvider->hash()) + : filenameHash; + } +}; +} // namespace std + +namespace facebook::velox { +using FileHandleCache = + SimpleLRUCache; + +// Creates FileHandles via the Generator interface the CachedFactory requires. +class FileHandleGenerator { + public: + FileHandleGenerator() {} + FileHandleGenerator(std::shared_ptr properties) + : properties_(std::move(properties)) {} + std::unique_ptr operator()( + const FileHandleKey& filename, + const FileProperties* properties, + filesystems::File::IoStats* stats); + + private: + const std::shared_ptr properties_; +}; + +using FileHandleFactory = CachedFactory< + FileHandleKey, + FileHandle, + FileHandleGenerator, + FileProperties, + filesystems::File::IoStats, + FileHandleSizer>; + +using FileHandleCachedPtr = CachedPtr; + +using FileHandleCacheStats = SimpleLRUCacheStats; + +} // namespace facebook::velox diff --git a/velox/connectors/lakehouse/common/FileProperties.h b/velox/connectors/lakehouse/common/FileProperties.h new file mode 100644 index 000000000000..d3ed9e3cbd6b --- /dev/null +++ b/velox/connectors/lakehouse/common/FileProperties.h @@ -0,0 +1,39 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// A FileHandle is a File pointer plus some (optional, file-type-dependent) +// extra information for speeding up loading columnar data. For example, when +// we open a file we might build a hash map saying what region(s) on disk +// correspond to a given column in a given stripe. +// +// The FileHandle will normally be used in conjunction with a CachedFactory +// to speed up queries that hit the same files repeatedly; see the +// FileHandleCache and FileHandleFactory. + +#pragma once + +#include + +namespace facebook::velox { + +struct FileProperties { + std::optional fileSize; + std::optional modificationTime; + std::optional readRangeHint{std::nullopt}; + std::shared_ptr extraFileInfo{nullptr}; +}; + +} // namespace facebook::velox diff --git a/velox/connectors/lakehouse/common/HiveConfig.cpp b/velox/connectors/lakehouse/common/HiveConfig.cpp new file mode 100644 index 000000000000..2cee1cae84ef --- /dev/null +++ b/velox/connectors/lakehouse/common/HiveConfig.cpp @@ -0,0 +1,248 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "velox/connectors/hive/HiveConfig.h" +#include "velox/common/config/Config.h" +#include "velox/core/QueryConfig.h" + +#include + +namespace facebook::velox::connector::hive { + +namespace { + +HiveConfig::InsertExistingPartitionsBehavior +stringToInsertExistingPartitionsBehavior(const std::string& strValue) { + auto upperValue = boost::algorithm::to_upper_copy(strValue); + if (upperValue == "ERROR") { + return HiveConfig::InsertExistingPartitionsBehavior::kError; + } + if (upperValue == "OVERWRITE") { + return HiveConfig::InsertExistingPartitionsBehavior::kOverwrite; + } + VELOX_UNSUPPORTED( + "Unsupported insert existing partitions behavior: {}.", strValue); +} + +} // namespace + +// static +std::string HiveConfig::insertExistingPartitionsBehaviorString( + InsertExistingPartitionsBehavior behavior) { + switch (behavior) { + case InsertExistingPartitionsBehavior::kError: + return "ERROR"; + case InsertExistingPartitionsBehavior::kOverwrite: + return "OVERWRITE"; + default: + return fmt::format("UNKNOWN BEHAVIOR {}", static_cast(behavior)); + } +} + +HiveConfig::InsertExistingPartitionsBehavior +HiveConfig::insertExistingPartitionsBehavior( + const config::ConfigBase* session) const { + return stringToInsertExistingPartitionsBehavior(session->get( + kInsertExistingPartitionsBehaviorSession, + config_->get(kInsertExistingPartitionsBehavior, "ERROR"))); +} + +uint32_t HiveConfig::maxPartitionsPerWriters( + const config::ConfigBase* session) const { + return session->get( + kMaxPartitionsPerWritersSession, + config_->get(kMaxPartitionsPerWriters, 128)); +} + +uint32_t HiveConfig::maxBucketCount(const config::ConfigBase* session) const { + return session->get( + kMaxBucketCountSession, config_->get(kMaxBucketCount, 100'000)); +} + +bool HiveConfig::immutablePartitions() const { + return config_->get(kImmutablePartitions, false); +} + +std::string HiveConfig::gcsEndpoint() const { + return config_->get(kGcsEndpoint, std::string("")); +} + +std::string HiveConfig::gcsCredentialsPath() const { + return config_->get(kGcsCredentialsPath, std::string("")); +} + +std::optional HiveConfig::gcsMaxRetryCount() const { + return static_cast>(config_->get(kGcsMaxRetryCount)); +} + +std::optional HiveConfig::gcsMaxRetryTime() const { + return static_cast>( + config_->get(kGcsMaxRetryTime)); +} + +bool HiveConfig::isOrcUseColumnNames(const config::ConfigBase* session) const { + return session->get( + kOrcUseColumnNamesSession, config_->get(kOrcUseColumnNames, false)); +} + +bool HiveConfig::isParquetUseColumnNames( + const config::ConfigBase* session) const { + return session->get( + kParquetUseColumnNamesSession, + config_->get(kParquetUseColumnNames, false)); +} + +bool HiveConfig::isFileColumnNamesReadAsLowerCase( + const config::ConfigBase* session) const { + return session->get( + kFileColumnNamesReadAsLowerCaseSession, + config_->get(kFileColumnNamesReadAsLowerCase, false)); +} + +bool HiveConfig::isPartitionPathAsLowerCase( + const config::ConfigBase* session) const { + return session->get(kPartitionPathAsLowerCaseSession, true); +} + +bool HiveConfig::allowNullPartitionKeys( + const config::ConfigBase* session) const { + return session->get( + kAllowNullPartitionKeysSession, + config_->get(kAllowNullPartitionKeys, true)); +} + +bool HiveConfig::ignoreMissingFiles(const config::ConfigBase* session) const { + return session->get(kIgnoreMissingFilesSession, false); +} + +int64_t HiveConfig::maxCoalescedBytes(const config::ConfigBase* session) const { + return session->get( + kMaxCoalescedBytesSession, + config_->get(kMaxCoalescedBytes, 128 << 20)); // 128MB +} + +int32_t HiveConfig::maxCoalescedDistanceBytes( + const config::ConfigBase* session) const { + const auto distance = config::toCapacity( + session->get( + kMaxCoalescedDistanceSession, + config_->get(kMaxCoalescedDistance, "512kB")), + config::CapacityUnit::BYTE); + VELOX_USER_CHECK_LE( + distance, + std::numeric_limits::max(), + "The max merge distance to combine read requests must be less than 2GB." + " Got {} bytes.", + distance); + return int32_t(distance); +} + +int32_t HiveConfig::prefetchRowGroups() const { + return config_->get(kPrefetchRowGroups, 1); +} + +int32_t HiveConfig::loadQuantum(const config::ConfigBase* session) const { + return session->get( + kLoadQuantumSession, config_->get(kLoadQuantum, 8 << 20)); +} + +int32_t HiveConfig::numCacheFileHandles() const { + return config_->get(kNumCacheFileHandles, 20'000); +} + +uint64_t HiveConfig::fileHandleExpirationDurationMs() const { + return config_->get(kFileHandleExpirationDurationMs, 0); +} + +bool HiveConfig::isFileHandleCacheEnabled() const { + return config_->get(kEnableFileHandleCache, true); +} + +std::string HiveConfig::writeFileCreateConfig() const { + return config_->get(kWriteFileCreateConfig, ""); +} + +uint32_t HiveConfig::sortWriterMaxOutputRows( + const config::ConfigBase* session) const { + return session->get( + kSortWriterMaxOutputRowsSession, + config_->get(kSortWriterMaxOutputRows, 1024)); +} + +uint64_t HiveConfig::sortWriterMaxOutputBytes( + const config::ConfigBase* session) const { + return config::toCapacity( + session->get( + kSortWriterMaxOutputBytesSession, + config_->get(kSortWriterMaxOutputBytes, "10MB")), + config::CapacityUnit::BYTE); +} + +uint64_t HiveConfig::sortWriterFinishTimeSliceLimitMs( + const config::ConfigBase* session) const { + return session->get( + kSortWriterFinishTimeSliceLimitMsSession, + config_->get(kSortWriterFinishTimeSliceLimitMs, 5'000)); +} + +uint64_t HiveConfig::footerEstimatedSize() const { + return config_->get(kFooterEstimatedSize, 256UL << 10); +} + +uint64_t HiveConfig::filePreloadThreshold() const { + return config_->get(kFilePreloadThreshold, 8UL << 20); +} + +uint8_t HiveConfig::readTimestampUnit(const config::ConfigBase* session) const { + const auto unit = session->get( + kReadTimestampUnitSession, + config_->get(kReadTimestampUnit, 3 /*milli*/)); + VELOX_CHECK( + unit == 3 || unit == 6 /*micro*/ || unit == 9 /*nano*/, + "Invalid timestamp unit."); + return unit; +} + +bool HiveConfig::readTimestampPartitionValueAsLocalTime( + const config::ConfigBase* session) const { + return session->get( + kReadTimestampPartitionValueAsLocalTimeSession, + config_->get(kReadTimestampPartitionValueAsLocalTime, true)); +} + +bool HiveConfig::readStatsBasedFilterReorderDisabled( + const config::ConfigBase* session) const { + return session->get( + kReadStatsBasedFilterReorderDisabledSession, + config_->get(kReadStatsBasedFilterReorderDisabled, false)); +} + +bool HiveConfig::isRequestedTypeCheckEnabled( + const config::ConfigBase* session) const { + return session->get( + kEnableRequestedTypeCheckSession, + config_->get(kEnableRequestedTypeCheck, true)); +} + +std::string HiveConfig::hiveLocalDataPath() const { + return config_->get(kLocalDataPath, ""); +} + +std::string HiveConfig::hiveLocalFileFormat() const { + return config_->get(kLocalFileFormat, ""); +} + +} // namespace facebook::velox::connector::hive diff --git a/velox/connectors/lakehouse/common/HiveConfig.h b/velox/connectors/lakehouse/common/HiveConfig.h new file mode 100644 index 000000000000..d61e8cc42411 --- /dev/null +++ b/velox/connectors/lakehouse/common/HiveConfig.h @@ -0,0 +1,294 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#pragma once + +#include +#include +#include "velox/common/base/Exceptions.h" + +namespace facebook::velox::config { +class ConfigBase; +} + +namespace facebook::velox::connector::hive { + +/// Hive connector configs. +class HiveConfig { + public: + enum class InsertExistingPartitionsBehavior { + kError, + kOverwrite, + }; + + static std::string insertExistingPartitionsBehaviorString( + InsertExistingPartitionsBehavior behavior); + + /// Behavior on insert into existing partitions. + static constexpr const char* kInsertExistingPartitionsBehaviorSession = + "insert_existing_partitions_behavior"; + static constexpr const char* kInsertExistingPartitionsBehavior = + "insert-existing-partitions-behavior"; + + /// Maximum number of (bucketed) partitions per a single table writer + /// instance. + static constexpr const char* kMaxPartitionsPerWriters = + "max-partitions-per-writers"; + static constexpr const char* kMaxPartitionsPerWritersSession = + "max_partitions_per_writers"; + + /// Maximum number of buckets allowed to output by the table writers. + static constexpr const char* kMaxBucketCount = "hive.max-bucket-count"; + static constexpr const char* kMaxBucketCountSession = "hive.max_bucket_count"; + + /// Whether new data can be inserted into an unpartition table. + /// Velox currently does not support appending data to existing partitions. + static constexpr const char* kImmutablePartitions = + "hive.immutable-partitions"; + + /// The GCS storage endpoint server. + static constexpr const char* kGcsEndpoint = "hive.gcs.endpoint"; + + /// The GCS service account configuration JSON key file. + static constexpr const char* kGcsCredentialsPath = + "hive.gcs.json-key-file-path"; + + /// The GCS maximum retry counter of transient errors. + static constexpr const char* kGcsMaxRetryCount = "hive.gcs.max-retry-count"; + + /// The GCS maximum time allowed to retry transient errors. + static constexpr const char* kGcsMaxRetryTime = "hive.gcs.max-retry-time"; + + /// Maps table field names to file field names using names, not indices. + // TODO: remove hive_orc_use_column_names since it doesn't exist in presto, + // right now this is only used for testing. + static constexpr const char* kOrcUseColumnNames = "hive.orc.use-column-names"; + static constexpr const char* kOrcUseColumnNamesSession = + "hive_orc_use_column_names"; + + /// Maps table field names to file field names using names, not indices. + static constexpr const char* kParquetUseColumnNames = + "hive.parquet.use-column-names"; + static constexpr const char* kParquetUseColumnNamesSession = + "parquet_use_column_names"; + + /// Reads the source file column name as lower case. + static constexpr const char* kFileColumnNamesReadAsLowerCase = + "file-column-names-read-as-lower-case"; + static constexpr const char* kFileColumnNamesReadAsLowerCaseSession = + "file_column_names_read_as_lower_case"; + + static constexpr const char* kPartitionPathAsLowerCaseSession = + "partition_path_as_lower_case"; + + static constexpr const char* kAllowNullPartitionKeys = + "allow-null-partition-keys"; + static constexpr const char* kAllowNullPartitionKeysSession = + "allow_null_partition_keys"; + + static constexpr const char* kIgnoreMissingFilesSession = + "ignore_missing_files"; + + /// The max coalesce bytes for a request. + static constexpr const char* kMaxCoalescedBytes = "max-coalesced-bytes"; + static constexpr const char* kMaxCoalescedBytesSession = + "max-coalesced-bytes"; + + /// The max merge distance to combine read requests. + /// Note: The session property name differs from the constant name for + /// backward compatibility with Presto. + static constexpr const char* kMaxCoalescedDistance = "max-coalesced-distance"; + static constexpr const char* kMaxCoalescedDistanceSession = + "orc_max_merge_distance"; + + /// The number of prefetch rowgroups + static constexpr const char* kPrefetchRowGroups = "prefetch-rowgroups"; + + /// The total size in bytes for a direct coalesce request. Up to 8MB load + /// quantum size is supported when SSD cache is enabled. + static constexpr const char* kLoadQuantum = "load-quantum"; + static constexpr const char* kLoadQuantumSession = "load-quantum"; + + /// Maximum number of entries in the file handle cache. + static constexpr const char* kNumCacheFileHandles = "num_cached_file_handles"; + + /// Expiration time in ms for a file handle in the cache. A value of 0 + /// means cache will not evict the handle after kFileHandleExprationDurationMs + /// has passed. + static constexpr const char* kFileHandleExpirationDurationMs = + "file-handle-expiration-duration-ms"; + + /// Enable file handle cache. + static constexpr const char* kEnableFileHandleCache = + "file-handle-cache-enabled"; + + /// The size in bytes to be fetched with Meta data together, used when the + /// data after meta data will be used later. Optimization to decrease small IO + /// request + static constexpr const char* kFooterEstimatedSize = "footer-estimated-size"; + + /// The threshold of file size in bytes when the whole file is fetched with + /// meta data together. Optimization to decrease the small IO requests + static constexpr const char* kFilePreloadThreshold = "file-preload-threshold"; + + /// Config used to create write files. This config is provided to underlying + /// file system through hive connector and data sink. The config is free form. + /// The form should be defined by the underlying file system. + static constexpr const char* kWriteFileCreateConfig = + "hive.write_file_create_config"; + + /// Maximum number of rows for sort writer in one batch of output. + static constexpr const char* kSortWriterMaxOutputRows = + "sort-writer-max-output-rows"; + static constexpr const char* kSortWriterMaxOutputRowsSession = + "sort_writer_max_output_rows"; + + /// Maximum bytes for sort writer in one batch of output. + static constexpr const char* kSortWriterMaxOutputBytes = + "sort-writer-max-output-bytes"; + static constexpr const char* kSortWriterMaxOutputBytesSession = + "sort_writer_max_output_bytes"; + + /// Sort Writer will exit finish() method after this many milliseconds even if + /// it has not completed its work yet. Zero means no time limit. + static constexpr const char* kSortWriterFinishTimeSliceLimitMs = + "sort-writer_finish_time_slice_limit_ms"; + static constexpr const char* kSortWriterFinishTimeSliceLimitMsSession = + "sort_writer_finish_time_slice_limit_ms"; + + // The unit for reading timestamps from files. + static constexpr const char* kReadTimestampUnit = + "hive.reader.timestamp-unit"; + static constexpr const char* kReadTimestampUnitSession = + "hive.reader.timestamp_unit"; + + static constexpr const char* kReadTimestampPartitionValueAsLocalTime = + "hive.reader.timestamp-partition-value-as-local-time"; + static constexpr const char* kReadTimestampPartitionValueAsLocalTimeSession = + "hive.reader.timestamp_partition_value_as_local_time"; + + static constexpr const char* kReadStatsBasedFilterReorderDisabled = + "stats-based-filter-reorder-disabled"; + static constexpr const char* kReadStatsBasedFilterReorderDisabledSession = + "stats_based_filter_reorder_disabled"; + + static constexpr const char* kLocalDataPath = "hive_local_data_path"; + static constexpr const char* kLocalFileFormat = "hive_local_file_format"; + + static constexpr const char* kEnableRequestedTypeCheck = + "enable-requested-type-check"; + static constexpr const char* kEnableRequestedTypeCheckSession = + "enable_requested_type_check"; + + InsertExistingPartitionsBehavior insertExistingPartitionsBehavior( + const config::ConfigBase* session) const; + + uint32_t maxPartitionsPerWriters(const config::ConfigBase* session) const; + + uint32_t maxBucketCount(const config::ConfigBase* session) const; + + bool immutablePartitions() const; + + std::string gcsEndpoint() const; + + std::string gcsCredentialsPath() const; + + std::optional gcsMaxRetryCount() const; + + std::optional gcsMaxRetryTime() const; + + bool isOrcUseColumnNames(const config::ConfigBase* session) const; + + bool isParquetUseColumnNames(const config::ConfigBase* session) const; + + bool isFileColumnNamesReadAsLowerCase( + const config::ConfigBase* session) const; + + bool isPartitionPathAsLowerCase(const config::ConfigBase* session) const; + + bool allowNullPartitionKeys(const config::ConfigBase* session) const; + + bool ignoreMissingFiles(const config::ConfigBase* session) const; + + int64_t maxCoalescedBytes(const config::ConfigBase* session) const; + + int32_t maxCoalescedDistanceBytes(const config::ConfigBase* session) const; + + int32_t prefetchRowGroups() const; + + int32_t loadQuantum(const config::ConfigBase* session) const; + + int32_t numCacheFileHandles() const; + + uint64_t fileHandleExpirationDurationMs() const; + + bool isFileHandleCacheEnabled() const; + + uint64_t fileWriterFlushThresholdBytes() const; + + std::string writeFileCreateConfig() const; + + uint32_t sortWriterMaxOutputRows(const config::ConfigBase* session) const; + + uint64_t sortWriterMaxOutputBytes(const config::ConfigBase* session) const; + + uint64_t sortWriterFinishTimeSliceLimitMs( + const config::ConfigBase* session) const; + + uint64_t footerEstimatedSize() const; + + uint64_t filePreloadThreshold() const; + + // Returns the timestamp unit used when reading timestamps from files. + uint8_t readTimestampUnit(const config::ConfigBase* session) const; + + // Whether to read timestamp partition value as local time. If false, read as + // UTC. + bool readTimestampPartitionValueAsLocalTime( + const config::ConfigBase* session) const; + + /// Returns true if the stats based filter reorder for read is disabled. + bool readStatsBasedFilterReorderDisabled( + const config::ConfigBase* session) const; + + /// Whether to enable requested type check in the ReaderBase::convertType. + /// Returns true by default. + bool isRequestedTypeCheckEnabled(const config::ConfigBase* session) const; + + /// Returns the file system path containing local data. If non-empty, + /// initializes LocalHiveConnectorMetadata to provide metadata for the tables + /// in the directory. + std::string hiveLocalDataPath() const; + + /// Returns the name of the file format to use in interpreting the contents of + /// hiveLocalDataPath(). + std::string hiveLocalFileFormat() const; + + HiveConfig(std::shared_ptr config) { + VELOX_CHECK_NOT_NULL( + config, "Config is null for HiveConfig initialization"); + config_ = std::move(config); + // TODO: add sanity check + } + + const std::shared_ptr& config() const { + return config_; + } + + private: + std::shared_ptr config_; +}; + +} // namespace facebook::velox::connector::hive diff --git a/velox/connectors/lakehouse/common/HiveConnector.cpp b/velox/connectors/lakehouse/common/HiveConnector.cpp new file mode 100644 index 000000000000..3f28309a3d7e --- /dev/null +++ b/velox/connectors/lakehouse/common/HiveConnector.cpp @@ -0,0 +1,214 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "velox/connectors/hive/HiveConnector.h" + +#include "velox/connectors/hive/HiveConfig.h" +#include "velox/connectors/hive/HiveDataSink.h" +#include "velox/connectors/hive/HiveDataSource.h" +#include "velox/connectors/hive/HivePartitionFunction.h" +#include "velox/connectors/hive/iceberg/IcebergDataSink.h" + +#include +#include + +using namespace facebook::velox::exec; + +namespace facebook::velox::connector::hive { + +namespace { +std::vector>& +hiveConnectorMetadataFactories() { + static std::vector> factories; + return factories; +} +} // namespace + +HiveConnector::HiveConnector( + const std::string& id, + std::shared_ptr config, + folly::Executor* executor) + : Connector(id), + hiveConfig_(std::make_shared(config)), + fileHandleFactory_( + hiveConfig_->isFileHandleCacheEnabled() + ? std::make_unique>( + hiveConfig_->numCacheFileHandles()) + : nullptr, + std::make_unique(config)), + executor_(executor) { + if (hiveConfig_->isFileHandleCacheEnabled()) { + LOG(INFO) << "Hive connector " << connectorId() + << " created with maximum of " + << hiveConfig_->numCacheFileHandles() + << " cached file handles with expiration of " + << hiveConfig_->fileHandleExpirationDurationMs() << "ms."; + } else { + LOG(INFO) << "Hive connector " << connectorId() + << " created with file handle cache disabled"; + } + for (auto& factory : hiveConnectorMetadataFactories()) { + metadata_ = factory->create(this); + if (metadata_ != nullptr) { + break; + } + } +} + +std::unique_ptr HiveConnector::createDataSource( + const RowTypePtr& outputType, + const ConnectorTableHandlePtr& tableHandle, + const std::unordered_map& columnHandles, + ConnectorQueryCtx* connectorQueryCtx) { + return std::make_unique( + outputType, + tableHandle, + columnHandles, + &fileHandleFactory_, + executor_, + connectorQueryCtx, + hiveConfig_); +} + +std::unique_ptr HiveConnector::createDataSink( + RowTypePtr inputType, + ConnectorInsertTableHandlePtr connectorInsertTableHandle, + ConnectorQueryCtx* connectorQueryCtx, + CommitStrategy commitStrategy) { + if (auto icebergInsertHandle = + std::dynamic_pointer_cast( + connectorInsertTableHandle)) { + return std::make_unique( + inputType, + icebergInsertHandle, + connectorQueryCtx, + commitStrategy, + hiveConfig_); + } else { + auto hiveInsertHandle = + std::dynamic_pointer_cast( + connectorInsertTableHandle); + + VELOX_CHECK_NOT_NULL( + hiveInsertHandle, "Hive connector expecting hive write handle!"); + return std::make_unique( + inputType, + hiveInsertHandle, + connectorQueryCtx, + commitStrategy, + hiveConfig_); + } +} + +std::unique_ptr HivePartitionFunctionSpec::create( + int numPartitions, + bool localExchange) const { + std::vector bucketToPartitions; + if (bucketToPartition_.empty()) { + // NOTE: if hive partition function spec doesn't specify bucket to partition + // mapping, then we do round-robin mapping based on the actual number of + // partitions. + bucketToPartitions.resize(numBuckets_); + for (int bucket = 0; bucket < numBuckets_; ++bucket) { + bucketToPartitions[bucket] = bucket % numPartitions; + } + if (localExchange) { + // Shuffle the map from bucket to partition for local exchange so we don't + // use the same map for remote shuffle. + std::shuffle( + bucketToPartitions.begin(), + bucketToPartitions.end(), + std::mt19937{0}); + } + } + return std::make_unique( + numBuckets_, + bucketToPartition_.empty() ? std::move(bucketToPartitions) + : bucketToPartition_, + channels_, + constValues_); +} + +std::string HivePartitionFunctionSpec::toString() const { + std::ostringstream keys; + size_t constIndex = 0; + for (auto i = 0; i < channels_.size(); ++i) { + if (i > 0) { + keys << ", "; + } + auto channel = channels_[i]; + if (channel == kConstantChannel) { + keys << "\"" << constValues_[constIndex++]->toString(0) << "\""; + } else { + keys << channel; + } + } + + return fmt::format("HIVE(({}) buckets: {})", keys.str(), numBuckets_); +} + +folly::dynamic HivePartitionFunctionSpec::serialize() const { + folly::dynamic obj = folly::dynamic::object; + obj["name"] = "HivePartitionFunctionSpec"; + obj["numBuckets"] = ISerializable::serialize(numBuckets_); + obj["bucketToPartition"] = ISerializable::serialize(bucketToPartition_); + obj["keys"] = ISerializable::serialize(channels_); + std::vector constValueExprs; + constValueExprs.reserve(constValues_.size()); + for (const auto& value : constValues_) { + constValueExprs.emplace_back(value); + } + obj["constants"] = ISerializable::serialize(constValueExprs); + return obj; +} + +// static +core::PartitionFunctionSpecPtr HivePartitionFunctionSpec::deserialize( + const folly::dynamic& obj, + void* context) { + std::vector channels = + ISerializable::deserialize>( + obj["keys"], context); + const auto constTypedValues = + ISerializable::deserialize>( + obj["constants"], context); + std::vector constValues; + constValues.reserve(constTypedValues.size()); + auto* pool = static_cast(context); + for (const auto& value : constTypedValues) { + constValues.emplace_back(value->toConstantVector(pool)); + } + return std::make_shared( + ISerializable::deserialize(obj["numBuckets"], context), + ISerializable::deserialize>( + obj["bucketToPartition"], context), + std::move(channels), + std::move(constValues)); +} + +void registerHivePartitionFunctionSerDe() { + auto& registry = DeserializationWithContextRegistryForSharedPtr(); + registry.Register( + "HivePartitionFunctionSpec", HivePartitionFunctionSpec::deserialize); +} + +bool registerHiveConnectorMetadataFactory( + std::unique_ptr factory) { + hiveConnectorMetadataFactories().push_back(std::move(factory)); + return true; +} + +} // namespace facebook::velox::connector::hive diff --git a/velox/connectors/lakehouse/common/HiveConnector.h b/velox/connectors/lakehouse/common/HiveConnector.h new file mode 100644 index 000000000000..e3622cdee052 --- /dev/null +++ b/velox/connectors/lakehouse/common/HiveConnector.h @@ -0,0 +1,174 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#pragma once + +#include "velox/connectors/Connector.h" +#include "velox/connectors/hive/FileHandle.h" +#include "velox/connectors/hive/HiveConfig.h" +#include "velox/core/PlanNode.h" + +namespace facebook::velox::dwio::common { +class DataSink; +class DataSource; +} // namespace facebook::velox::dwio::common + +namespace facebook::velox::connector::hive { + +class HiveConnector : public Connector { + public: + HiveConnector( + const std::string& id, + std::shared_ptr config, + folly::Executor* executor); + + const std::shared_ptr& connectorConfig() + const override { + return hiveConfig_->config(); + } + + bool canAddDynamicFilter() const override { + return true; + } + + ConnectorMetadata* metadata() const override { + VELOX_CHECK_NOT_NULL(metadata_); + return metadata_.get(); + } + + std::unique_ptr createDataSource( + const RowTypePtr& outputType, + const ConnectorTableHandlePtr& tableHandle, + const connector::ColumnHandleMap& columnHandles, + ConnectorQueryCtx* connectorQueryCtx) override; + + bool supportsSplitPreload() override { + return true; + } + + std::unique_ptr createDataSink( + RowTypePtr inputType, + ConnectorInsertTableHandlePtr connectorInsertTableHandle, + ConnectorQueryCtx* connectorQueryCtx, + CommitStrategy commitStrategy) override; + + folly::Executor* executor() const override { + return executor_; + } + + FileHandleCacheStats fileHandleCacheStats() { + return fileHandleFactory_.cacheStats(); + } + + // NOTE: this is to clear file handle cache which might affect performance, + // and is only used for operational purposes. + FileHandleCacheStats clearFileHandleCache() { + return fileHandleFactory_.clearCache(); + } + + protected: + const std::shared_ptr hiveConfig_; + FileHandleFactory fileHandleFactory_; + folly::Executor* executor_; + std::shared_ptr metadata_; +}; + +class HiveConnectorFactory : public ConnectorFactory { + public: + static constexpr const char* kHiveConnectorName = "hive"; + + HiveConnectorFactory() : ConnectorFactory(kHiveConnectorName) {} + + explicit HiveConnectorFactory(const char* connectorName) + : ConnectorFactory(connectorName) {} + + std::shared_ptr newConnector( + const std::string& id, + std::shared_ptr config, + folly::Executor* ioExecutor = nullptr, + folly::Executor* cpuExecutor = nullptr) override { + return std::make_shared(id, config, ioExecutor); + } +}; + +class HivePartitionFunctionSpec : public core::PartitionFunctionSpec { + public: + HivePartitionFunctionSpec( + int numBuckets, + std::vector bucketToPartition, + std::vector channels, + std::vector constValues) + : numBuckets_(numBuckets), + bucketToPartition_(std::move(bucketToPartition)), + channels_(std::move(channels)), + constValues_(std::move(constValues)) {} + + /// The constructor without 'bucketToPartition' input is used in case that + /// we don't know the actual number of partitions until we create the + /// partition function instance. The hive partition function spec then builds + /// a bucket to partition map based on the actual number of partitions with + /// round-robin partitioning scheme to create the function instance. For + /// instance, when we create the local partition node with hive bucket + /// function to support multiple table writer drivers, we don't know the the + /// actual number of table writer drivers until start the task. + HivePartitionFunctionSpec( + int numBuckets, + std::vector channels, + std::vector constValues) + : HivePartitionFunctionSpec( + numBuckets, + {}, + std::move(channels), + std::move(constValues)) {} + + std::unique_ptr create( + int numPartitions, + bool localExchange) const override; + + std::string toString() const override; + + folly::dynamic serialize() const override; + + static core::PartitionFunctionSpecPtr deserialize( + const folly::dynamic& obj, + void* context); + + private: + const int numBuckets_; + const std::vector bucketToPartition_; + const std::vector channels_; + const std::vector constValues_; +}; + +void registerHivePartitionFunctionSerDe(); + +/// Hook for connecting metadata functions to a HiveConnector. Each registered +/// factory is called after initializing a HiveConnector until one of these +/// returns a ConnectorMetadata instance. +class HiveConnectorMetadataFactory { + public: + virtual ~HiveConnectorMetadataFactory() = default; + + /// Returns a ConnectorMetadata to complete'hiveConnector' if 'this' + /// recognizes a data source, e.g. local file system or remote metadata + /// service associated to configs in 'hiveConnector'. + virtual std::shared_ptr create( + HiveConnector* connector) = 0; +}; + +bool registerHiveConnectorMetadataFactory( + std::unique_ptr); + +} // namespace facebook::velox::connector::hive diff --git a/velox/connectors/lakehouse/common/HiveConnectorSplit.cpp b/velox/connectors/lakehouse/common/HiveConnectorSplit.cpp new file mode 100644 index 000000000000..f38336d92ab1 --- /dev/null +++ b/velox/connectors/lakehouse/common/HiveConnectorSplit.cpp @@ -0,0 +1,225 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "velox/connectors/hive/HiveConnectorSplit.h" + +namespace facebook::velox::connector::hive { + +std::string HiveConnectorSplit::toString() const { + if (tableBucketNumber.has_value()) { + return fmt::format( + "Hive: {} {} - {} {}", + filePath, + start, + length, + tableBucketNumber.value()); + } + return fmt::format("Hive: {} {} - {}", filePath, start, length); +} + +uint64_t HiveConnectorSplit::size() const { + return length; +} + +std::string HiveConnectorSplit::getFileName() const { + const auto i = filePath.rfind('/'); + return i == std::string::npos ? filePath : filePath.substr(i + 1); +} + +folly::dynamic HiveConnectorSplit::serialize() const { + folly::dynamic obj = folly::dynamic::object; + obj["name"] = "HiveConnectorSplit"; + obj["connectorId"] = connectorId; + obj["splitWeight"] = splitWeight; + obj["cacheable"] = cacheable; + obj["filePath"] = filePath; + obj["fileFormat"] = dwio::common::toString(fileFormat); + obj["start"] = start; + obj["length"] = length; + + folly::dynamic partitionKeysObj = folly::dynamic::object; + for (const auto& [key, value] : partitionKeys) { + partitionKeysObj[key] = + value.has_value() ? folly::dynamic(value.value()) : nullptr; + } + obj["partitionKeys"] = partitionKeysObj; + + obj["tableBucketNumber"] = tableBucketNumber.has_value() + ? folly::dynamic(tableBucketNumber.value()) + : nullptr; + + if (bucketConversion.has_value()) { + folly::dynamic bucketConversionObj = folly::dynamic::object; + bucketConversionObj["tableBucketCount"] = + bucketConversion->tableBucketCount; + bucketConversionObj["partitionBucketCount"] = + bucketConversion->partitionBucketCount; + folly::dynamic bucketColumnHandlesArray = folly::dynamic::array; + for (const auto& handle : bucketConversion->bucketColumnHandles) { + bucketColumnHandlesArray.push_back(handle->serialize()); + } + bucketConversionObj["bucketColumnHandles"] = bucketColumnHandlesArray; + obj["bucketConversion"] = bucketConversionObj; + } else { + obj["bucketConversion"] = nullptr; + } + + folly::dynamic customSplitInfoObj = folly::dynamic::object; + for (const auto& [key, value] : customSplitInfo) { + customSplitInfoObj[key] = value; + } + obj["customSplitInfo"] = customSplitInfoObj; + obj["extraFileInfo"] = + extraFileInfo == nullptr ? nullptr : folly::dynamic(*extraFileInfo); + + folly::dynamic serdeParametersObj = folly::dynamic::object; + for (const auto& [key, value] : serdeParameters) { + serdeParametersObj[key] = value; + } + obj["serdeParameters"] = serdeParametersObj; + + folly::dynamic infoColumnsObj = folly::dynamic::object; + for (const auto& [key, value] : infoColumns) { + infoColumnsObj[key] = value; + } + obj["infoColumns"] = infoColumnsObj; + + if (properties.has_value()) { + folly::dynamic propertiesObj = folly::dynamic::object; + propertiesObj["fileSize"] = properties->fileSize.has_value() + ? folly::dynamic(properties->fileSize.value()) + : nullptr; + propertiesObj["modificationTime"] = properties->modificationTime.has_value() + ? folly::dynamic(properties->modificationTime.value()) + : nullptr; + obj["properties"] = propertiesObj; + } + + if (rowIdProperties.has_value()) { + folly::dynamic rowIdObj = folly::dynamic::object; + rowIdObj["metadataVersion"] = rowIdProperties->metadataVersion; + rowIdObj["partitionId"] = rowIdProperties->partitionId; + rowIdObj["tableGuid"] = rowIdProperties->tableGuid; + obj["rowIdProperties"] = rowIdObj; + } + + return obj; +} + +// static +std::shared_ptr HiveConnectorSplit::create( + const folly::dynamic& obj) { + const auto connectorId = obj["connectorId"].asString(); + const auto splitWeight = obj["splitWeight"].asInt(); + const bool cacheable = obj["cacheable"].asBool(); + const auto filePath = obj["filePath"].asString(); + const auto fileFormat = + dwio::common::toFileFormat(obj["fileFormat"].asString()); + const auto start = static_cast(obj["start"].asInt()); + const auto length = static_cast(obj["length"].asInt()); + + std::unordered_map> partitionKeys; + for (const auto& [key, value] : obj["partitionKeys"].items()) { + partitionKeys[key.asString()] = value.isNull() + ? std::nullopt + : std::optional(value.asString()); + } + + const auto tableBucketNumber = obj["tableBucketNumber"].isNull() + ? std::nullopt + : std::optional(obj["tableBucketNumber"].asInt()); + + std::optional bucketConversion = std::nullopt; + if (obj.count("bucketConversion") && !obj["bucketConversion"].isNull()) { + const auto& bucketConversionObj = obj["bucketConversion"]; + std::vector> bucketColumnHandles; + for (const auto& bucketColumnHandleObj : + bucketConversionObj["bucketColumnHandles"]) { + bucketColumnHandles.push_back(std::const_pointer_cast( + ISerializable::deserialize(bucketColumnHandleObj))); + } + bucketConversion = HiveBucketConversion{ + .tableBucketCount = static_cast( + bucketConversionObj["tableBucketCount"].asInt()), + .partitionBucketCount = static_cast( + bucketConversionObj["partitionBucketCount"].asInt()), + .bucketColumnHandles = bucketColumnHandles}; + } + + std::unordered_map customSplitInfo; + for (const auto& [key, value] : obj["customSplitInfo"].items()) { + customSplitInfo[key.asString()] = value.asString(); + } + + std::shared_ptr extraFileInfo = obj["extraFileInfo"].isNull() + ? nullptr + : std::make_shared(obj["extraFileInfo"].asString()); + std::unordered_map serdeParameters; + for (const auto& [key, value] : obj["serdeParameters"].items()) { + serdeParameters[key.asString()] = value.asString(); + } + + std::unordered_map infoColumns; + for (const auto& [key, value] : obj["infoColumns"].items()) { + infoColumns[key.asString()] = value.asString(); + } + + std::optional properties = std::nullopt; + const auto& propertiesObj = obj.getDefault("properties", nullptr); + if (propertiesObj != nullptr) { + properties = FileProperties{ + .fileSize = propertiesObj["fileSize"].isNull() + ? std::nullopt + : std::optional(propertiesObj["fileSize"].asInt()), + .modificationTime = propertiesObj["modificationTime"].isNull() + ? std::nullopt + : std::optional(propertiesObj["modificationTime"].asInt())}; + } + + std::optional rowIdProperties = std::nullopt; + const auto& rowIdObj = obj.getDefault("rowIdProperties", nullptr); + if (rowIdObj != nullptr) { + rowIdProperties = RowIdProperties{ + .metadataVersion = rowIdObj["metadataVersion"].asInt(), + .partitionId = rowIdObj["partitionId"].asInt(), + .tableGuid = rowIdObj["tableGuid"].asString()}; + } + + return std::make_shared( + connectorId, + filePath, + fileFormat, + start, + length, + partitionKeys, + tableBucketNumber, + customSplitInfo, + extraFileInfo, + serdeParameters, + splitWeight, + cacheable, + infoColumns, + properties, + rowIdProperties, + bucketConversion); +} + +// static +void HiveConnectorSplit::registerSerDe() { + auto& registry = DeserializationRegistryForSharedPtr(); + registry.Register("HiveConnectorSplit", HiveConnectorSplit::create); +} +} // namespace facebook::velox::connector::hive diff --git a/velox/connectors/lakehouse/common/HiveConnectorSplit.h b/velox/connectors/lakehouse/common/HiveConnectorSplit.h new file mode 100644 index 000000000000..3485c2330fa5 --- /dev/null +++ b/velox/connectors/lakehouse/common/HiveConnectorSplit.h @@ -0,0 +1,255 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#pragma once + +#include +#include +#include "velox/connectors/Connector.h" +#include "velox/connectors/hive/FileProperties.h" +#include "velox/connectors/hive/TableHandle.h" +#include "velox/dwio/common/Options.h" + +namespace facebook::velox::connector::hive { + +/// A bucket conversion that should happen on the split. This happens when we +/// increase the bucket count of a table, but the old partitions are still +/// generated using the old bucket count, so that multiple new buckets can exist +/// in the same file, and we need to apply extra filter when we read these files +/// to make sure we read the rows corresponding to the selected bucket number +/// only. +struct HiveBucketConversion { + int32_t tableBucketCount; + int32_t partitionBucketCount; + std::vector> bucketColumnHandles; +}; + +struct RowIdProperties { + int64_t metadataVersion; + int64_t partitionId; + std::string tableGuid; +}; + +struct HiveConnectorSplit : public connector::ConnectorSplit { + const std::string filePath; + dwio::common::FileFormat fileFormat; + const uint64_t start; + const uint64_t length; + + /// Mapping from partition keys to values. Values are specified as strings + /// formatted the same way as CAST(x as VARCHAR). Null values are specified as + /// std::nullopt. Date values must be formatted using ISO 8601 as YYYY-MM-DD. + /// All scalar types and date type are supported. + const std::unordered_map> + partitionKeys; + std::optional tableBucketNumber; + std::unordered_map customSplitInfo; + std::shared_ptr extraFileInfo; + // Parameters that are provided as the serialization options. + std::unordered_map serdeParameters; + + /// These represent columns like $file_size, $file_modified_time that are + /// associated with the HiveSplit. + std::unordered_map infoColumns; + + /// These represent file properties like file size that are used while opening + /// the file handle. + std::optional properties; + + std::optional rowIdProperties; + + std::optional bucketConversion; + + HiveConnectorSplit( + const std::string& connectorId, + const std::string& _filePath, + dwio::common::FileFormat _fileFormat, + uint64_t _start = 0, + uint64_t _length = std::numeric_limits::max(), + const std::unordered_map>& + _partitionKeys = {}, + std::optional _tableBucketNumber = std::nullopt, + const std::unordered_map& _customSplitInfo = {}, + const std::shared_ptr& _extraFileInfo = {}, + const std::unordered_map& _serdeParameters = {}, + int64_t splitWeight = 0, + bool cacheable = true, + const std::unordered_map& _infoColumns = {}, + std::optional _properties = std::nullopt, + std::optional _rowIdProperties = std::nullopt, + const std::optional& _bucketConversion = + std::nullopt) + : ConnectorSplit(connectorId, splitWeight, cacheable), + filePath(_filePath), + fileFormat(_fileFormat), + start(_start), + length(_length), + partitionKeys(_partitionKeys), + tableBucketNumber(_tableBucketNumber), + customSplitInfo(_customSplitInfo), + extraFileInfo(_extraFileInfo), + serdeParameters(_serdeParameters), + infoColumns(_infoColumns), + properties(_properties), + rowIdProperties(_rowIdProperties), + bucketConversion(_bucketConversion) {} + + ~HiveConnectorSplit() = default; + + uint64_t size() const override; + + std::string toString() const override; + + std::string getFileName() const; + + folly::dynamic serialize() const override; + + static std::shared_ptr create(const folly::dynamic& obj); + + static void registerSerDe(); +}; + +class HiveConnectorSplitBuilder { + public: + explicit HiveConnectorSplitBuilder(std::string filePath) + : filePath_{std::move(filePath)} { + infoColumns_["$path"] = filePath_; + } + + HiveConnectorSplitBuilder& start(uint64_t start) { + start_ = start; + return *this; + } + + HiveConnectorSplitBuilder& length(uint64_t length) { + length_ = length; + return *this; + } + + HiveConnectorSplitBuilder& splitWeight(int64_t splitWeight) { + splitWeight_ = splitWeight; + return *this; + } + + HiveConnectorSplitBuilder& cacheable(bool cacheable) { + cacheable_ = cacheable; + return *this; + } + + HiveConnectorSplitBuilder& fileFormat(dwio::common::FileFormat format) { + fileFormat_ = format; + return *this; + } + + HiveConnectorSplitBuilder& infoColumn( + const std::string& name, + const std::string& value) { + infoColumns_.emplace(std::move(name), std::move(value)); + return *this; + } + + HiveConnectorSplitBuilder& partitionKey( + std::string name, + std::optional value) { + partitionKeys_.emplace(std::move(name), std::move(value)); + return *this; + } + + HiveConnectorSplitBuilder& tableBucketNumber(int32_t bucket) { + tableBucketNumber_ = bucket; + infoColumns_["$bucket"] = std::to_string(bucket); + return *this; + } + + HiveConnectorSplitBuilder& bucketConversion( + const HiveBucketConversion& bucketConversion) { + bucketConversion_ = bucketConversion; + return *this; + } + + HiveConnectorSplitBuilder& customSplitInfo( + const std::unordered_map& customSplitInfo) { + customSplitInfo_ = customSplitInfo; + return *this; + } + + HiveConnectorSplitBuilder& extraFileInfo( + const std::shared_ptr& extraFileInfo) { + extraFileInfo_ = extraFileInfo; + return *this; + } + + HiveConnectorSplitBuilder& serdeParameters( + const std::unordered_map& serdeParameters) { + serdeParameters_ = serdeParameters; + return *this; + } + + HiveConnectorSplitBuilder& connectorId(const std::string& connectorId) { + connectorId_ = connectorId; + return *this; + } + + HiveConnectorSplitBuilder& fileProperties(FileProperties fileProperties) { + fileProperties_ = fileProperties; + return *this; + } + + HiveConnectorSplitBuilder& rowIdProperties( + const RowIdProperties& rowIdProperties) { + rowIdProperties_ = rowIdProperties; + return *this; + } + + std::shared_ptr build() const { + return std::make_shared( + connectorId_, + filePath_, + fileFormat_, + start_, + length_, + partitionKeys_, + tableBucketNumber_, + customSplitInfo_, + extraFileInfo_, + serdeParameters_, + splitWeight_, + cacheable_, + infoColumns_, + fileProperties_, + rowIdProperties_, + bucketConversion_); + } + + private: + const std::string filePath_; + dwio::common::FileFormat fileFormat_{dwio::common::FileFormat::DWRF}; + uint64_t start_{0}; + uint64_t length_{std::numeric_limits::max()}; + std::unordered_map> partitionKeys_; + std::optional tableBucketNumber_; + std::optional bucketConversion_; + std::unordered_map customSplitInfo_ = {}; + std::shared_ptr extraFileInfo_ = {}; + std::unordered_map serdeParameters_ = {}; + std::unordered_map infoColumns_ = {}; + std::string connectorId_; + int64_t splitWeight_{0}; + bool cacheable_{true}; + std::optional fileProperties_; + std::optional rowIdProperties_ = std::nullopt; +}; + +} // namespace facebook::velox::connector::hive diff --git a/velox/connectors/lakehouse/common/HiveConnectorUtil.cpp b/velox/connectors/lakehouse/common/HiveConnectorUtil.cpp new file mode 100644 index 000000000000..d2b29a3c9a6c --- /dev/null +++ b/velox/connectors/lakehouse/common/HiveConnectorUtil.cpp @@ -0,0 +1,937 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "velox/connectors/hive/HiveConnectorUtil.h" + +#include "velox/connectors/hive/HiveConfig.h" +#include "velox/connectors/hive/HiveConnectorSplit.h" +#include "velox/dwio/common/CachedBufferedInput.h" +#include "velox/dwio/common/DirectBufferedInput.h" +#include "velox/expression/Expr.h" +#include "velox/expression/ExprToSubfieldFilter.h" + +#include +#include +#include + +namespace facebook::velox::connector::hive { +namespace { + +struct SubfieldSpec { + const common::Subfield* subfield; + bool filterOnly; +}; + +template +void deduplicate(std::vector& values) { + std::sort(values.begin(), values.end()); + values.erase(std::unique(values.begin(), values.end()), values.end()); +} + +// Floating point map key subscripts are truncated toward 0 in Presto. For +// example given `a' as a map with floating point key, if user queries a[0.99], +// Presto coordinator will generate a required subfield a[0]; for a[-1.99] it +// will generate a[-1]; for anything larger than 9223372036854775807, it +// generates a[9223372036854775807]; for anything smaller than +// -9223372036854775808 it generates a[-9223372036854775808]. +template +std::unique_ptr makeFloatingPointMapKeyFilter( + const std::vector& subscripts) { + std::vector> filters; + for (auto subscript : subscripts) { + T lower = subscript; + T upper = subscript; + bool lowerUnbounded = subscript == std::numeric_limits::min(); + bool upperUnbounded = subscript == std::numeric_limits::max(); + bool lowerExclusive = false; + bool upperExclusive = false; + if (lower <= 0 && !lowerUnbounded) { + if (lower > subscript - 1) { + lower = subscript - 1; + } else { + lower = std::nextafter(lower, -std::numeric_limits::infinity()); + } + lowerExclusive = true; + } + if (upper >= 0 && !upperUnbounded) { + if (upper < subscript + 1) { + upper = subscript + 1; + } else { + upper = std::nextafter(upper, std::numeric_limits::infinity()); + } + upperExclusive = true; + } + if (lowerUnbounded && upperUnbounded) { + continue; + } + filters.push_back(std::make_unique>( + lower, + lowerUnbounded, + lowerExclusive, + upper, + upperUnbounded, + upperExclusive, + false)); + } + if (filters.size() == 1) { + return std::move(filters[0]); + } + return std::make_unique(std::move(filters), false); +} + +// Recursively add subfields to scan spec. +void addSubfields( + const Type& type, + std::vector& subfields, + int level, + memory::MemoryPool* pool, + common::ScanSpec& spec) { + int newSize = 0; + for (int i = 0; i < subfields.size(); ++i) { + if (level < subfields[i].subfield->path().size()) { + subfields[newSize++] = subfields[i]; + } else if (!subfields[i].filterOnly) { + spec.addAllChildFields(type); + return; + } + } + subfields.resize(newSize); + switch (type.kind()) { + case TypeKind::ROW: { + folly::F14FastMap> required; + for (auto& subfield : subfields) { + auto* element = subfield.subfield->path()[level].get(); + auto* nestedField = + dynamic_cast(element); + VELOX_CHECK( + nestedField, + "Unsupported for row subfields pruning: {}", + element->toString()); + required[nestedField->name()].push_back(subfield); + } + auto& rowType = type.asRow(); + for (int i = 0; i < rowType.size(); ++i) { + auto& childName = rowType.nameOf(i); + auto& childType = rowType.childAt(i); + auto* child = spec.addField(childName, i); + auto it = required.find(childName); + if (it == required.end()) { + child->setConstantValue( + BaseVector::createNullConstant(childType, 1, pool)); + } else { + addSubfields(*childType, it->second, level + 1, pool, *child); + } + } + break; + } + case TypeKind::MAP: { + auto& keyType = type.childAt(0); + auto* keys = spec.addMapKeyFieldRecursively(*keyType); + addSubfields( + *type.childAt(1), + subfields, + level + 1, + pool, + *spec.addMapValueField()); + if (subfields.empty()) { + return; + } + bool stringKey = keyType->isVarchar() || keyType->isVarbinary(); + std::vector stringSubscripts; + std::vector longSubscripts; + for (auto& subfield : subfields) { + auto* element = subfield.subfield->path()[level].get(); + if (dynamic_cast(element)) { + return; + } + if (stringKey) { + auto* subscript = + dynamic_cast(element); + VELOX_CHECK( + subscript, + "Unsupported for string map pruning: {}", + element->toString()); + stringSubscripts.push_back(subscript->index()); + } else { + auto* subscript = + dynamic_cast(element); + VELOX_CHECK( + subscript, + "Unsupported for long map pruning: {}", + element->toString()); + longSubscripts.push_back(subscript->index()); + } + } + std::unique_ptr filter; + if (stringKey) { + deduplicate(stringSubscripts); + filter = std::make_unique(stringSubscripts, false); + spec.setFlatMapFeatureSelection(std::move(stringSubscripts)); + } else { + deduplicate(longSubscripts); + if (keyType->isReal()) { + filter = makeFloatingPointMapKeyFilter(longSubscripts); + } else if (keyType->isDouble()) { + filter = makeFloatingPointMapKeyFilter(longSubscripts); + } else { + filter = common::createBigintValues(longSubscripts, false); + } + std::vector features; + for (auto num : longSubscripts) { + features.push_back(std::to_string(num)); + } + spec.setFlatMapFeatureSelection(std::move(features)); + } + keys->setFilter(std::move(filter)); + break; + } + case TypeKind::ARRAY: { + addSubfields( + *type.childAt(0), + subfields, + level + 1, + pool, + *spec.addArrayElementField()); + if (subfields.empty()) { + return; + } + constexpr long kMaxIndex = std::numeric_limits::max(); + long maxIndex = -1; + for (auto& subfield : subfields) { + auto* element = subfield.subfield->path()[level].get(); + if (dynamic_cast(element)) { + return; + } + auto* subscript = + dynamic_cast(element); + VELOX_CHECK( + subscript, + "Unsupported for array pruning: {}", + element->toString()); + VELOX_USER_CHECK_GT( + subscript->index(), + 0, + "Non-positive array subscript cannot be push down"); + maxIndex = std::max(maxIndex, std::min(kMaxIndex, subscript->index())); + } + spec.setMaxArrayElementsCount(maxIndex); + break; + } + default: + break; + } +} + +inline uint8_t parseDelimiter(const std::string& delim) { + for (char const& ch : delim) { + if (!std::isdigit(ch)) { + return delim[0]; + } + } + return stoi(delim); +} + +inline bool isSynthesizedColumn( + const std::string& name, + const std::unordered_map& infoColumns) { + return infoColumns.count(name) != 0; +} + +bool isSpecialColumn( + const std::string& name, + const std::optional& specialName) { + return specialName.has_value() && name == *specialName; +} + +} // namespace + +const std::string& getColumnName(const common::Subfield& subfield) { + VELOX_CHECK_GT(subfield.path().size(), 0); + auto* field = dynamic_cast( + subfield.path()[0].get()); + VELOX_CHECK_NOT_NULL(field); + return field->name(); +} + +void checkColumnNameLowerCase(const TypePtr& type) { + switch (type->kind()) { + case TypeKind::ARRAY: + checkColumnNameLowerCase(type->asArray().elementType()); + break; + case TypeKind::MAP: { + checkColumnNameLowerCase(type->asMap().keyType()); + checkColumnNameLowerCase(type->asMap().valueType()); + + } break; + case TypeKind::ROW: { + for (const auto& outputName : type->asRow().names()) { + VELOX_CHECK( + !std::any_of(outputName.begin(), outputName.end(), isupper)); + } + for (auto& childType : type->asRow().children()) { + checkColumnNameLowerCase(childType); + } + } break; + default: + VLOG(1) << "No need to check type lowercase mode" << type->toString(); + } +} + +void checkColumnNameLowerCase( + const common::SubfieldFilters& filters, + const std::unordered_map& infoColumns) { + for (const auto& filterIt : filters) { + const auto name = filterIt.first.toString(); + if (isSynthesizedColumn(name, infoColumns)) { + continue; + } + const auto& path = filterIt.first.path(); + + for (int i = 0; i < path.size(); ++i) { + auto* nestedField = + dynamic_cast(path[i].get()); + if (nestedField == nullptr) { + continue; + } + VELOX_CHECK(!std::any_of( + nestedField->name().begin(), nestedField->name().end(), isupper)); + } + } +} + +void checkColumnNameLowerCase(const core::TypedExprPtr& typeExpr) { + if (typeExpr == nullptr) { + return; + } + checkColumnNameLowerCase(typeExpr->type()); + for (auto& type : typeExpr->inputs()) { + checkColumnNameLowerCase(type); + } +} + +namespace { + +void processFieldSpec( + const RowTypePtr& dataColumns, + const TypePtr& outputType, + common::ScanSpec& fieldSpec) { + fieldSpec.visit(*outputType, [](const Type& type, common::ScanSpec& spec) { + if (type.isMap() && !spec.isConstant()) { + auto* keys = spec.childByName(common::ScanSpec::kMapKeysFieldName); + VELOX_CHECK_NOT_NULL(keys); + if (keys->filter()) { + VELOX_CHECK(!keys->filter()->testNull()); + } else { + keys->setFilter(std::make_shared()); + } + } + }); + if (dataColumns) { + auto i = dataColumns->getChildIdxIfExists(fieldSpec.fieldName()); + if (i.has_value()) { + if (dataColumns->childAt(*i)->isMap() && outputType->isRow()) { + fieldSpec.setFlatMapAsStruct(true); + } + } + } +} + +} // namespace + +std::shared_ptr makeScanSpec( + const RowTypePtr& rowType, + const folly::F14FastMap>& + outputSubfields, + const common::SubfieldFilters& filters, + const RowTypePtr& dataColumns, + const std::unordered_map& partitionKeys, + const std::unordered_map& infoColumns, + const SpecialColumnNames& specialColumns, + bool disableStatsBasedFilterReorder, + memory::MemoryPool* pool) { + auto spec = std::make_shared("root"); + folly::F14FastMap> + filterSubfields; + std::vector subfieldSpecs; + for (auto& [subfield, _] : filters) { + if (auto name = subfield.toString(); + !isSynthesizedColumn(name, infoColumns) && + partitionKeys.count(name) == 0) { + VELOX_CHECK(!isSpecialColumn(name, specialColumns.rowIndex)); + VELOX_CHECK(!isSpecialColumn(name, specialColumns.rowId)); + filterSubfields[getColumnName(subfield)].push_back(&subfield); + } + } + + // Process columns that will be projected out. + for (int i = 0; i < rowType->size(); ++i) { + auto& name = rowType->nameOf(i); + auto& type = rowType->childAt(i); + if (isSpecialColumn(name, specialColumns.rowIndex)) { + VELOX_CHECK(type->isBigint()); + auto* fieldSpec = spec->addField(name, i); + fieldSpec->setColumnType(common::ScanSpec::ColumnType::kRowIndex); + continue; + } + if (isSpecialColumn(name, specialColumns.rowId)) { + VELOX_CHECK(type->isRow() && type->size() == 5); + auto& rowIdType = type->asRow(); + auto* fieldSpec = spec->addFieldRecursively(name, rowIdType, i); + fieldSpec->setColumnType(common::ScanSpec::ColumnType::kComposite); + fieldSpec->childByName(rowIdType.nameOf(0)) + ->setColumnType(common::ScanSpec::ColumnType::kRowIndex); + continue; + } + auto it = outputSubfields.find(name); + if (it == outputSubfields.end()) { + auto* fieldSpec = spec->addFieldRecursively(name, *type, i); + processFieldSpec(dataColumns, type, *fieldSpec); + filterSubfields.erase(name); + continue; + } + for (auto* subfield : it->second) { + subfieldSpecs.push_back({subfield, false}); + } + it = filterSubfields.find(name); + if (it != filterSubfields.end()) { + for (auto* subfield : it->second) { + subfieldSpecs.push_back({subfield, true}); + } + filterSubfields.erase(it); + } + auto* fieldSpec = spec->addField(name, i); + addSubfields(*type, subfieldSpecs, 1, pool, *fieldSpec); + processFieldSpec(dataColumns, type, *fieldSpec); + subfieldSpecs.clear(); + } + + // Now process the columns that will not be projected out. + if (!filterSubfields.empty()) { + VELOX_CHECK_NOT_NULL(dataColumns); + for (auto& [fieldName, subfields] : filterSubfields) { + for (auto* subfield : subfields) { + subfieldSpecs.push_back({subfield, true}); + } + auto& type = dataColumns->findChild(fieldName); + auto* fieldSpec = spec->getOrCreateChild(fieldName); + addSubfields(*type, subfieldSpecs, 1, pool, *fieldSpec); + processFieldSpec(dataColumns, type, *fieldSpec); + subfieldSpecs.clear(); + } + } + + for (auto& pair : filters) { + const auto name = pair.first.toString(); + // SelectiveColumnReader doesn't support constant columns with filters, + // hence, we can't have a filter for a $path or $bucket column. + // + // Unfortunately, Presto happens to specify a filter for $path, $file_size, + // $file_modified_time or $bucket column. This filter is redundant and needs + // to be removed. + // TODO Remove this check when Presto is fixed to not specify a filter + // on $path and $bucket column. + if (isSynthesizedColumn(name, infoColumns)) { + continue; + } + auto fieldSpec = spec->getOrCreateChild(pair.first); + VELOX_CHECK_NULL(spec->filter()); + fieldSpec->setFilter(pair.second); + } + + if (disableStatsBasedFilterReorder) { + spec->disableStatsBasedFilterReorder(); + } + return spec; +} + +std::unique_ptr parseSerdeParameters( + const std::unordered_map& serdeParameters, + const std::unordered_map& tableParameters) { + auto fieldIt = serdeParameters.find(dwio::common::SerDeOptions::kFieldDelim); + if (fieldIt == serdeParameters.end()) { + fieldIt = serdeParameters.find("serialization.format"); + } + auto collectionIt = + serdeParameters.find(dwio::common::SerDeOptions::kCollectionDelim); + if (collectionIt == serdeParameters.end()) { + // For collection delimiter, Hive 1.x, 2.x uses "colelction.delim", but + // Hive 3.x uses "collection.delim". + // See: https://issues.apache.org/jira/browse/HIVE-16922) + collectionIt = serdeParameters.find("colelction.delim"); + } + auto mapKeyIt = + serdeParameters.find(dwio::common::SerDeOptions::kMapKeyDelim); + + auto escapeCharIt = + serdeParameters.find(dwio::common::SerDeOptions::kEscapeChar); + + auto nullStringIt = tableParameters.find( + dwio::common::TableParameter::kSerializationNullFormat); + + if (fieldIt == serdeParameters.end() && + collectionIt == serdeParameters.end() && + mapKeyIt == serdeParameters.end() && + escapeCharIt == serdeParameters.end() && + nullStringIt == tableParameters.end()) { + return nullptr; + } + + uint8_t fieldDelim = '\1'; + uint8_t collectionDelim = '\2'; + uint8_t mapKeyDelim = '\3'; + if (fieldIt != serdeParameters.end()) { + fieldDelim = parseDelimiter(fieldIt->second); + } + if (collectionIt != serdeParameters.end()) { + collectionDelim = parseDelimiter(collectionIt->second); + } + if (mapKeyIt != serdeParameters.end()) { + mapKeyDelim = parseDelimiter(mapKeyIt->second); + } + + // If escape character is specified then we use it, unless it is empty - in + // which case we default to '\\'. + // If escape character is not specified (not in the map) we turn escaping off. + // Logic is based on apache hive java code: + // https://github.com/apache/hive/blob/3f6f940af3f60cc28834268e5d7f5612e3b13c30/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazySerDeParameters.java#L105-L108 + uint8_t escapeChar = '\\'; + const bool hasEscapeChar = (escapeCharIt != serdeParameters.end()); + if (hasEscapeChar) { + if (!escapeCharIt->second.empty()) { + // If delim is convertible to uint8_t then we use it as character code, + // otherwise we use the 1st character of the string. + escapeChar = folly::tryTo(escapeCharIt->second) + .value_or(escapeCharIt->second[0]); + } + } + + auto serDeOptions = hasEscapeChar + ? std::make_unique( + fieldDelim, collectionDelim, mapKeyDelim, escapeChar, true) + : std::make_unique( + fieldDelim, collectionDelim, mapKeyDelim); + if (nullStringIt != tableParameters.end()) { + serDeOptions->nullString = nullStringIt->second; + } + return serDeOptions; +} + +void configureReaderOptions( + const std::shared_ptr& hiveConfig, + const ConnectorQueryCtx* connectorQueryCtx, + const std::shared_ptr& hiveTableHandle, + const std::shared_ptr& hiveSplit, + dwio::common::ReaderOptions& readerOptions) { + configureReaderOptions( + hiveConfig, + connectorQueryCtx, + hiveTableHandle->dataColumns(), + hiveSplit, + hiveTableHandle->tableParameters(), + readerOptions); +} + +void configureReaderOptions( + const std::shared_ptr& hiveConfig, + const ConnectorQueryCtx* connectorQueryCtx, + const RowTypePtr& fileSchema, + const std::shared_ptr& hiveSplit, + const std::unordered_map& tableParameters, + dwio::common::ReaderOptions& readerOptions) { + auto sessionProperties = connectorQueryCtx->sessionProperties(); + readerOptions.setLoadQuantum(hiveConfig->loadQuantum(sessionProperties)); + readerOptions.setMaxCoalesceBytes( + hiveConfig->maxCoalescedBytes(sessionProperties)); + readerOptions.setMaxCoalesceDistance( + hiveConfig->maxCoalescedDistanceBytes(sessionProperties)); + readerOptions.setFileColumnNamesReadAsLowerCase( + hiveConfig->isFileColumnNamesReadAsLowerCase(sessionProperties)); + readerOptions.setAllowEmptyFile(true); + bool useColumnNamesForColumnMapping = false; + switch (hiveSplit->fileFormat) { + case dwio::common::FileFormat::DWRF: + case dwio::common::FileFormat::ORC: { + useColumnNamesForColumnMapping = + hiveConfig->isOrcUseColumnNames(sessionProperties); + break; + } + case dwio::common::FileFormat::PARQUET: { + useColumnNamesForColumnMapping = + hiveConfig->isParquetUseColumnNames(sessionProperties); + break; + } + default: + useColumnNamesForColumnMapping = false; + } + + readerOptions.setUseColumnNamesForColumnMapping( + useColumnNamesForColumnMapping); + readerOptions.setFileSchema(fileSchema); + readerOptions.setFooterEstimatedSize(hiveConfig->footerEstimatedSize()); + readerOptions.setFilePreloadThreshold(hiveConfig->filePreloadThreshold()); + readerOptions.setPrefetchRowGroups(hiveConfig->prefetchRowGroups()); + readerOptions.setNoCacheRetention(!hiveSplit->cacheable); + const auto& sessionTzName = connectorQueryCtx->sessionTimezone(); + if (!sessionTzName.empty()) { + const auto timezone = tz::locateZone(sessionTzName); + readerOptions.setSessionTimezone(timezone); + } + readerOptions.setAdjustTimestampToTimezone( + connectorQueryCtx->adjustTimestampToTimezone()); + readerOptions.setSelectiveNimbleReaderEnabled( + connectorQueryCtx->selectiveNimbleReaderEnabled()); + + if (readerOptions.fileFormat() != dwio::common::FileFormat::UNKNOWN) { + VELOX_CHECK( + readerOptions.fileFormat() == hiveSplit->fileFormat, + "HiveDataSource received splits of different formats: {} and {}", + dwio::common::toString(readerOptions.fileFormat()), + dwio::common::toString(hiveSplit->fileFormat)); + } else { + auto serDeOptions = + parseSerdeParameters(hiveSplit->serdeParameters, tableParameters); + if (serDeOptions) { + readerOptions.setSerDeOptions(*serDeOptions); + } + + readerOptions.setFileFormat(hiveSplit->fileFormat); + readerOptions.setEnableRequestedTypeCheck( + hiveConfig->isRequestedTypeCheckEnabled(sessionProperties)); + } +} + +void configureRowReaderOptions( + const std::unordered_map& tableParameters, + const std::shared_ptr& scanSpec, + std::shared_ptr metadataFilter, + const RowTypePtr& rowType, + const std::shared_ptr& hiveSplit, + const std::shared_ptr& hiveConfig, + const config::ConfigBase* sessionProperties, + dwio::common::RowReaderOptions& rowReaderOptions) { + auto skipRowsIt = + tableParameters.find(dwio::common::TableParameter::kSkipHeaderLineCount); + if (skipRowsIt != tableParameters.end()) { + rowReaderOptions.setSkipRows(folly::to(skipRowsIt->second)); + } + rowReaderOptions.setScanSpec(scanSpec); + rowReaderOptions.setMetadataFilter(std::move(metadataFilter)); + rowReaderOptions.setRequestedType(rowType); + rowReaderOptions.range(hiveSplit->start, hiveSplit->length); + if (hiveConfig && sessionProperties) { + rowReaderOptions.setTimestampPrecision(static_cast( + hiveConfig->readTimestampUnit(sessionProperties))); + } + rowReaderOptions.setSerdeParameters(hiveSplit->serdeParameters); +} + +namespace { + +bool applyPartitionFilter( + const TypePtr& type, + const std::string& partitionValue, + bool isPartitionDateDaysSinceEpoch, + const common::Filter* filter, + bool asLocalTime) { + if (type->isDate()) { + int32_t result = 0; + // days_since_epoch partition values are integers in string format. Eg. + // Iceberg partition values. + if (isPartitionDateDaysSinceEpoch) { + result = folly::to(partitionValue); + } else { + result = DATE()->toDays(static_cast(partitionValue)); + } + return applyFilter(*filter, result); + } + + switch (type->kind()) { + case TypeKind::BIGINT: + case TypeKind::INTEGER: + case TypeKind::SMALLINT: + case TypeKind::TINYINT: { + return applyFilter(*filter, folly::to(partitionValue)); + } + case TypeKind::REAL: + case TypeKind::DOUBLE: { + return applyFilter(*filter, folly::to(partitionValue)); + } + case TypeKind::BOOLEAN: { + return applyFilter(*filter, folly::to(partitionValue)); + } + case TypeKind::TIMESTAMP: { + auto result = util::fromTimestampString( + StringView(partitionValue), util::TimestampParseMode::kPrestoCast); + VELOX_CHECK(!result.hasError()); + if (asLocalTime) { + result.value().toGMT(Timestamp::defaultTimezone()); + } + return applyFilter(*filter, result.value()); + } + case TypeKind::VARCHAR: { + return applyFilter(*filter, partitionValue); + } + default: + VELOX_FAIL( + "Bad type {} for partition value: {}", type->kind(), partitionValue); + } +} + +} // namespace + +bool testFilters( + const common::ScanSpec* scanSpec, + const dwio::common::Reader* reader, + const std::string& filePath, + const std::unordered_map>& + partitionKeys, + const std::unordered_map& + partitionKeysHandle, + bool asLocalTime) { + const auto totalRows = reader->numberOfRows(); + const auto& fileTypeWithId = reader->typeWithId(); + const auto& rowType = reader->rowType(); + for (const auto& child : scanSpec->children()) { + if (child->filter()) { + const auto& name = child->fieldName(); + auto iter = partitionKeys.find(name); + // By design, the partition key columns for Iceberg tables are included in + // the data files to facilitate partition transform and partition + // evolution, so we need to test both cases. + if (!rowType->containsChild(name) || iter != partitionKeys.end()) { + if (iter != partitionKeys.end() && iter->second.has_value()) { + const auto handlesIter = partitionKeysHandle.find(name); + VELOX_CHECK(handlesIter != partitionKeysHandle.end()); + + // This is a non-null partition key + return applyPartitionFilter( + handlesIter->second->dataType(), + iter->second.value(), + handlesIter->second->isPartitionDateValueDaysSinceEpoch(), + child->filter(), + asLocalTime); + } + // Column is missing, most likely due to schema evolution. Or it's a + // partition key but the partition value is NULL. + if (child->filter()->isDeterministic() && + !child->filter()->testNull()) { + VLOG(1) << "Skipping " << filePath + << " because the filter testNull() failed for column " + << child->fieldName(); + return false; + } + } else { + const auto& typeWithId = fileTypeWithId->childByName(name); + const auto columnStats = reader->columnStatistics(typeWithId->id()); + if (columnStats != nullptr && + !testFilter( + child->filter(), + columnStats.get(), + totalRows.value(), + typeWithId->type())) { + VLOG(1) << "Skipping " << filePath + << " based on stats and filter for column " + << child->fieldName(); + return false; + } + } + } + } + + return true; +} + +std::unique_ptr createBufferedInput( + const FileHandle& fileHandle, + const dwio::common::ReaderOptions& readerOpts, + const ConnectorQueryCtx* connectorQueryCtx, + std::shared_ptr ioStats, + std::shared_ptr fsStats, + folly::Executor* executor) { + if (connectorQueryCtx->cache()) { + return std::make_unique( + fileHandle.file, + dwio::common::MetricsLog::voidLog(), + fileHandle.uuid, + connectorQueryCtx->cache(), + Connector::getTracker( + connectorQueryCtx->scanId(), readerOpts.loadQuantum()), + fileHandle.groupId, + ioStats, + std::move(fsStats), + executor, + readerOpts); + } + if (readerOpts.fileFormat() == dwio::common::FileFormat::NIMBLE) { + // Nimble streams (in case of single chunk) are compressed as whole and need + // to be fully fetched in order to do decompression, so there is no point to + // fetch them by quanta. Just use BufferedInput to fetch streams as whole + // to reduce memory footprint. + return std::make_unique( + fileHandle.file, + readerOpts.memoryPool(), + dwio::common::MetricsLog::voidLog(), + ioStats.get(), + fsStats.get()); + } + return std::make_unique( + fileHandle.file, + dwio::common::MetricsLog::voidLog(), + fileHandle.uuid, + Connector::getTracker( + connectorQueryCtx->scanId(), readerOpts.loadQuantum()), + fileHandle.groupId, + std::move(ioStats), + std::move(fsStats), + executor, + readerOpts); +} + +namespace { + +core::CallTypedExprPtr replaceInputs( + const core::CallTypedExpr* call, + std::vector&& inputs) { + return std::make_shared( + call->type(), std::move(inputs), call->name()); +} + +bool endWith(const std::string& str, const char* suffix) { + int len = strlen(suffix); + if (str.size() < len) { + return false; + } + for (int i = 0, j = str.size() - len; i < len; ++i, ++j) { + if (str[j] != suffix[i]) { + return false; + } + } + return true; +} + +bool isNotExpr( + const core::TypedExprPtr& expr, + const core::CallTypedExpr* call, + core::ExpressionEvaluator* evaluator) { + if (!endWith(call->name(), "not")) { + return false; + } + auto exprs = evaluator->compile(expr); + VELOX_CHECK_EQ(exprs->size(), 1); + auto& compiled = exprs->expr(0); + return compiled->vectorFunction() && + compiled->vectorFunction()->getCanonicalName() == + exec::FunctionCanonicalName::kNot; +} + +double getPrestoSampleRate( + const core::TypedExprPtr& expr, + const core::CallTypedExpr* call, + core::ExpressionEvaluator* evaluator) { + if (!endWith(call->name(), "lt")) { + return -1; + } + VELOX_CHECK_EQ(call->inputs().size(), 2); + auto exprs = evaluator->compile(expr); + VELOX_CHECK_EQ(exprs->size(), 1); + auto& lt = exprs->expr(0); + if (!(lt->vectorFunction() && + lt->vectorFunction()->getCanonicalName() == + exec::FunctionCanonicalName::kLt)) { + return -1; + } + auto& rand = lt->inputs()[0]; + if (!(rand->inputs().empty() && rand->vectorFunction() && + rand->vectorFunction()->getCanonicalName() == + exec::FunctionCanonicalName::kRand)) { + return -1; + } + auto* rate = + dynamic_cast(call->inputs()[1].get()); + if (!(rate && rate->type()->kind() == TypeKind::DOUBLE)) { + return -1; + } + return std::max(0.0, std::min(1.0, rate->value().value())); +} + +} // namespace + +core::TypedExprPtr extractFiltersFromRemainingFilter( + const core::TypedExprPtr& expr, + core::ExpressionEvaluator* evaluator, + bool negated, + common::SubfieldFilters& filters, + double& sampleRate) { + auto* call = dynamic_cast(expr.get()); + if (call == nullptr) { + return expr; + } + common::Filter* oldFilter = nullptr; + try { + common::Subfield subfield; + if (auto filter = exec::ExprToSubfieldFilterParser::getInstance() + ->leafCallToSubfieldFilter( + *call, subfield, evaluator, negated)) { + if (auto it = filters.find(subfield); it != filters.end()) { + oldFilter = it->second.get(); + filter = filter->mergeWith(oldFilter); + } + filters.insert_or_assign(std::move(subfield), std::move(filter)); + return nullptr; + } + } catch (const VeloxException&) { + LOG(WARNING) << "Unexpected failure when extracting filter for: " + << expr->toString(); + if (oldFilter) { + LOG(WARNING) << "Merging with " << oldFilter->toString(); + } + } + + if (isNotExpr(expr, call, evaluator)) { + auto inner = extractFiltersFromRemainingFilter( + call->inputs()[0], evaluator, !negated, filters, sampleRate); + return inner ? replaceInputs(call, {inner}) : nullptr; + } + + if ((call->name() == "and" && !negated) || + (call->name() == "or" && negated)) { + auto lhs = extractFiltersFromRemainingFilter( + call->inputs()[0], evaluator, negated, filters, sampleRate); + auto rhs = extractFiltersFromRemainingFilter( + call->inputs()[1], evaluator, negated, filters, sampleRate); + if (!lhs) { + return rhs; + } + if (!rhs) { + return lhs; + } + return replaceInputs(call, {lhs, rhs}); + } + if (!negated) { + double rate = getPrestoSampleRate(expr, call, evaluator); + if (rate != -1) { + sampleRate *= rate; + return nullptr; + } + } + return expr; +} + +std::string makeUuid() { + return boost::lexical_cast(boost::uuids::random_generator()()); +} + +} // namespace facebook::velox::connector::hive diff --git a/velox/connectors/lakehouse/common/HiveConnectorUtil.h b/velox/connectors/lakehouse/common/HiveConnectorUtil.h new file mode 100644 index 000000000000..75906343bd07 --- /dev/null +++ b/velox/connectors/lakehouse/common/HiveConnectorUtil.h @@ -0,0 +1,119 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#pragma once +#include +#include + +#include "velox/connectors/Connector.h" +#include "velox/connectors/hive/FileHandle.h" +#include "velox/dwio/common/BufferedInput.h" +#include "velox/dwio/common/Reader.h" + +namespace facebook::velox::connector::hive { + +class HiveColumnHandle; +class HiveTableHandle; +class HiveConfig; +struct HiveConnectorSplit; + +const std::string& getColumnName(const common::Subfield& subfield); + +void checkColumnNameLowerCase(const std::shared_ptr& type); + +void checkColumnNameLowerCase( + const common::SubfieldFilters& filters, + const std::unordered_map< + std::string, + std::shared_ptr>& infoColumns); + +void checkColumnNameLowerCase(const core::TypedExprPtr& typeExpr); + +struct SpecialColumnNames { + std::optional rowIndex; + std::optional rowId; +}; + +std::shared_ptr makeScanSpec( + const RowTypePtr& rowType, + const folly::F14FastMap>& + outputSubfields, + const common::SubfieldFilters& filters, + const RowTypePtr& dataColumns, + const std::unordered_map< + std::string, + std::shared_ptr>& partitionKeys, + const std::unordered_map< + std::string, + std::shared_ptr>& infoColumns, + const SpecialColumnNames& specialColumns, + bool disableStatsBasedFilterReorder, + memory::MemoryPool* pool); + +void configureReaderOptions( + const std::shared_ptr& config, + const ConnectorQueryCtx* connectorQueryCtx, + const std::shared_ptr& hiveTableHandle, + const std::shared_ptr& hiveSplit, + dwio::common::ReaderOptions& readerOptions); + +void configureReaderOptions( + const std::shared_ptr& hiveConfig, + const ConnectorQueryCtx* connectorQueryCtx, + const RowTypePtr& fileSchema, + const std::shared_ptr& hiveSplit, + const std::unordered_map& tableParameters, + dwio::common::ReaderOptions& readerOptions); + +void configureRowReaderOptions( + const std::unordered_map& tableParameters, + const std::shared_ptr& scanSpec, + std::shared_ptr metadataFilter, + const RowTypePtr& rowType, + const std::shared_ptr& hiveSplit, + const std::shared_ptr& hiveConfig, + const config::ConfigBase* sessionProperties, + dwio::common::RowReaderOptions& rowReaderOptions); + +bool testFilters( + const common::ScanSpec* scanSpec, + const dwio::common::Reader* reader, + const std::string& filePath, + const std::unordered_map>& + partitionKey, + const std::unordered_map< + std::string, + std::shared_ptr>& partitionKeysHandle, + bool asLocalTime); + +std::unique_ptr createBufferedInput( + const FileHandle& fileHandle, + const dwio::common::ReaderOptions& readerOpts, + const ConnectorQueryCtx* connectorQueryCtx, + std::shared_ptr ioStats, + std::shared_ptr fsStats, + folly::Executor* executor); + +core::TypedExprPtr extractFiltersFromRemainingFilter( + const core::TypedExprPtr& expr, + core::ExpressionEvaluator* evaluator, + bool negated, + common::SubfieldFilters& filters, + double& sampleRate); + +std::string makeUuid(); + +} // namespace facebook::velox::connector::hive diff --git a/velox/connectors/lakehouse/common/HiveDataSink.cpp b/velox/connectors/lakehouse/common/HiveDataSink.cpp new file mode 100644 index 000000000000..a530d5d6c140 --- /dev/null +++ b/velox/connectors/lakehouse/common/HiveDataSink.cpp @@ -0,0 +1,1302 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "velox/connectors/hive/HiveDataSink.h" + +#include "velox/common/base/Counters.h" +#include "velox/common/base/Fs.h" +#include "velox/common/base/StatsReporter.h" +#include "velox/common/testutil/TestValue.h" +#include "velox/connectors/hive/HiveConfig.h" +#include "velox/connectors/hive/HiveConnectorUtil.h" +#include "velox/connectors/hive/HivePartitionFunction.h" +#include "velox/connectors/hive/TableHandle.h" +#include "velox/core/ITypedExpr.h" +#include "velox/dwio/common/Options.h" +#include "velox/dwio/common/SortingWriter.h" +#include "velox/exec/OperatorUtils.h" +#include "velox/exec/SortBuffer.h" + +using facebook::velox::common::testutil::TestValue; + +namespace facebook::velox::connector::hive { +namespace { +#define WRITER_NON_RECLAIMABLE_SECTION_GUARD(index) \ + memory::NonReclaimableSectionGuard nonReclaimableGuard( \ + writerInfo_[(index)]->nonReclaimableSectionHolder.get()) + +// Returns the type of non-partition data columns. +RowTypePtr getNonPartitionTypes( + const std::vector& dataCols, + const RowTypePtr& inputType) { + std::vector childNames; + std::vector childTypes; + const auto& dataSize = dataCols.size(); + childNames.reserve(dataSize); + childTypes.reserve(dataSize); + for (int dataCol : dataCols) { + childNames.push_back(inputType->nameOf(dataCol)); + childTypes.push_back(inputType->childAt(dataCol)); + } + + return ROW(std::move(childNames), std::move(childTypes)); +} + +// Filters out partition columns if there is any. +RowVectorPtr makeDataInput( + const std::vector& dataCols, + const RowVectorPtr& input) { + std::vector childVectors; + childVectors.reserve(dataCols.size()); + for (int dataCol : dataCols) { + childVectors.push_back(input->childAt(dataCol)); + } + + return std::make_shared( + input->pool(), + getNonPartitionTypes(dataCols, asRowType(input->type())), + input->nulls(), + input->size(), + std::move(childVectors), + input->getNullCount()); +} + +// Returns a subset of column indices corresponding to partition keys. +std::vector getPartitionChannels( + const std::shared_ptr& insertTableHandle) { + std::vector channels; + + for (column_index_t i = 0; i < insertTableHandle->inputColumns().size(); + i++) { + if (insertTableHandle->inputColumns()[i]->isPartitionKey()) { + channels.push_back(i); + } + } + + return channels; +} + +// Returns the column indices of non-partition data columns. +std::vector getNonPartitionChannels( + const std::shared_ptr& insertTableHandle) { + std::vector dataChannels; + + for (column_index_t i = 0; i < insertTableHandle->inputColumns().size(); + i++) { + if (!insertTableHandle->inputColumns()[i]->isPartitionKey()) { + dataChannels.push_back(i); + } + } + + return dataChannels; +} + +std::string makePartitionDirectory( + const std::string& tableDirectory, + const std::optional& partitionSubdirectory) { + if (partitionSubdirectory.has_value()) { + return fs::path(tableDirectory) / partitionSubdirectory.value(); + } + return tableDirectory; +} + +std::unordered_map tableTypeNames() { + return { + {LocationHandle::TableType::kNew, "kNew"}, + {LocationHandle::TableType::kExisting, "kExisting"}, + }; +} + +template +std::unordered_map invertMap(const std::unordered_map& mapping) { + std::unordered_map inverted; + for (const auto& [key, value] : mapping) { + inverted.emplace(value, key); + } + return inverted; +} + +std::unique_ptr createBucketFunction( + const HiveBucketProperty& bucketProperty, + const RowTypePtr& inputType) { + const auto& bucketedBy = bucketProperty.bucketedBy(); + const auto& bucketedTypes = bucketProperty.bucketedTypes(); + std::vector bucketedByChannels; + bucketedByChannels.reserve(bucketedBy.size()); + for (int32_t i = 0; i < bucketedBy.size(); ++i) { + const auto& bucketColumn = bucketedBy[i]; + const auto& bucketType = bucketedTypes[i]; + const auto inputChannel = inputType->getChildIdx(bucketColumn); + if (FOLLY_UNLIKELY( + !inputType->childAt(inputChannel)->equivalent(*bucketType))) { + VELOX_USER_FAIL( + "Input column {} type {} doesn't match bucket type {}", + inputType->nameOf(inputChannel), + inputType->childAt(inputChannel)->toString(), + bucketType->toString()); + } + bucketedByChannels.push_back(inputChannel); + } + return std::make_unique( + bucketProperty.bucketCount(), bucketedByChannels); +} + +std::string computeBucketedFileName( + const std::string& queryId, + uint32_t maxBucketCount, + uint32_t bucket) { + const uint32_t kMaxBucketCountPadding = + std::to_string(maxBucketCount - 1).size(); + const std::string bucketValueStr = std::to_string(bucket); + return fmt::format( + "0{:0>{}}_0_{}", bucketValueStr, kMaxBucketCountPadding, queryId); +} + +std::shared_ptr createSinkPool( + const std::shared_ptr& writerPool) { + return writerPool->addLeafChild(fmt::format("{}.sink", writerPool->name())); +} + +std::shared_ptr createSortPool( + const std::shared_ptr& writerPool) { + return writerPool->addLeafChild(fmt::format("{}.sort", writerPool->name())); +} + +uint64_t getFinishTimeSliceLimitMsFromHiveConfig( + const std::shared_ptr& config, + const config::ConfigBase* sessions) { + const uint64_t flushTimeSliceLimitMsFromConfig = + config->sortWriterFinishTimeSliceLimitMs(sessions); + // NOTE: if the flush time slice limit is set to 0, then we treat it as no + // limit. + return flushTimeSliceLimitMsFromConfig == 0 + ? std::numeric_limits::max() + : flushTimeSliceLimitMsFromConfig; +} + +FOLLY_ALWAYS_INLINE int32_t +getBucketCount(const HiveBucketProperty* bucketProperty) { + return bucketProperty == nullptr ? 0 : bucketProperty->bucketCount(); +} +} // namespace + +const HiveWriterId& HiveWriterId::unpartitionedId() { + static const HiveWriterId writerId{0}; + return writerId; +} + +std::string HiveWriterId::toString() const { + if (partitionId.has_value() && bucketId.has_value()) { + return fmt::format("part[{}.{}]", partitionId.value(), bucketId.value()); + } + + if (partitionId.has_value() && !bucketId.has_value()) { + return fmt::format("part[{}]", partitionId.value()); + } + + // This WriterId is used to add an identifier in the MemoryPools. This could + // indicate unpart, but the bucket number needs to be disambiguated. So + // creating a new label using bucket. + if (!partitionId.has_value() && bucketId.has_value()) { + return fmt::format("bucket[{}]", bucketId.value()); + } + + return "unpart"; +} + +const std::string LocationHandle::tableTypeName( + LocationHandle::TableType type) { + static const auto tableTypes = tableTypeNames(); + return tableTypes.at(type); +} + +LocationHandle::TableType LocationHandle::tableTypeFromName( + const std::string& name) { + static const auto nameTableTypes = invertMap(tableTypeNames()); + return nameTableTypes.at(name); +} + +HiveSortingColumn::HiveSortingColumn( + const std::string& sortColumn, + const core::SortOrder& sortOrder) + : sortColumn_(sortColumn), sortOrder_(sortOrder) { + VELOX_USER_CHECK(!sortColumn_.empty(), "hive sort column must be set"); + + if (FOLLY_UNLIKELY( + (sortOrder_.isAscending() && !sortOrder_.isNullsFirst()) || + (!sortOrder_.isAscending() && sortOrder_.isNullsFirst()))) { + VELOX_USER_FAIL("Bad hive sort order: {}", toString()); + } +} + +folly::dynamic HiveSortingColumn::serialize() const { + folly::dynamic obj = folly::dynamic::object; + obj["name"] = "HiveSortingColumn"; + obj["columnName"] = sortColumn_; + obj["sortOrder"] = sortOrder_.serialize(); + return obj; +} + +std::shared_ptr HiveSortingColumn::deserialize( + const folly::dynamic& obj, + void* context) { + const std::string columnName = obj["columnName"].asString(); + const auto sortOrder = core::SortOrder::deserialize(obj["sortOrder"]); + return std::make_shared(columnName, sortOrder); +} + +std::string HiveSortingColumn::toString() const { + return fmt::format( + "[COLUMN[{}] ORDER[{}]]", sortColumn_, sortOrder_.toString()); +} + +void HiveSortingColumn::registerSerDe() { + auto& registry = DeserializationWithContextRegistryForSharedPtr(); + registry.Register("HiveSortingColumn", HiveSortingColumn::deserialize); +} + +HiveBucketProperty::HiveBucketProperty( + Kind kind, + int32_t bucketCount, + const std::vector& bucketedBy, + const std::vector& bucketTypes, + const std::vector>& sortedBy) + : kind_(kind), + bucketCount_(bucketCount), + bucketedBy_(bucketedBy), + bucketTypes_(bucketTypes), + sortedBy_(sortedBy) { + validate(); +} + +void HiveBucketProperty::validate() const { + VELOX_USER_CHECK_GT(bucketCount_, 0, "Hive bucket count can't be zero"); + VELOX_USER_CHECK(!bucketedBy_.empty(), "Hive bucket columns must be set"); + VELOX_USER_CHECK_EQ( + bucketedBy_.size(), + bucketTypes_.size(), + "The number of hive bucket columns and types do not match {}", + toString()); +} + +std::string HiveBucketProperty::kindString(Kind kind) { + switch (kind) { + case Kind::kHiveCompatible: + return "HIVE_COMPATIBLE"; + case Kind::kPrestoNative: + return "PRESTO_NATIVE"; + default: + return fmt::format("UNKNOWN {}", static_cast(kind)); + } +} + +folly::dynamic HiveBucketProperty::serialize() const { + folly::dynamic obj = folly::dynamic::object; + obj["name"] = "HiveBucketProperty"; + obj["kind"] = static_cast(kind_); + obj["bucketCount"] = bucketCount_; + obj["bucketedBy"] = ISerializable::serialize(bucketedBy_); + obj["bucketedTypes"] = ISerializable::serialize(bucketTypes_); + obj["sortedBy"] = ISerializable::serialize(sortedBy_); + return obj; +} + +std::shared_ptr HiveBucketProperty::deserialize( + const folly::dynamic& obj, + void* context) { + const Kind kind = static_cast(obj["kind"].asInt()); + const int32_t bucketCount = obj["bucketCount"].asInt(); + const auto buckectedBy = + ISerializable::deserialize>(obj["bucketedBy"]); + const auto bucketedTypes = ISerializable::deserialize>( + obj["bucketedTypes"], context); + const auto sortedBy = + ISerializable::deserialize>( + obj["sortedBy"], context); + return std::make_shared( + kind, bucketCount, buckectedBy, bucketedTypes, sortedBy); +} + +void HiveBucketProperty::registerSerDe() { + auto& registry = DeserializationWithContextRegistryForSharedPtr(); + registry.Register("HiveBucketProperty", HiveBucketProperty::deserialize); +} + +std::string HiveBucketProperty::toString() const { + std::stringstream out; + out << "\nHiveBucketProperty[<" << kind_ << " " << bucketCount_ << ">\n"; + out << "\tBucket Columns:\n"; + for (const auto& column : bucketedBy_) { + out << "\t\t" << column << "\n"; + } + out << "\tBucket Types:\n"; + for (const auto& type : bucketTypes_) { + out << "\t\t" << type->toString() << "\n"; + } + if (!sortedBy_.empty()) { + out << "\tSortedBy Columns:\n"; + for (const auto& sortColum : sortedBy_) { + out << "\t\t" << sortColum->toString() << "\n"; + } + } + out << "]\n"; + return out.str(); +} + +HiveDataSink::HiveDataSink( + RowTypePtr inputType, + std::shared_ptr insertTableHandle, + const ConnectorQueryCtx* connectorQueryCtx, + CommitStrategy commitStrategy, + const std::shared_ptr& hiveConfig) + : HiveDataSink( + inputType, + insertTableHandle, + connectorQueryCtx, + commitStrategy, + hiveConfig, + getBucketCount(insertTableHandle->bucketProperty()), + getBucketCount(insertTableHandle->bucketProperty()) > 0 + ? createBucketFunction( + *insertTableHandle->bucketProperty(), + inputType) + : nullptr, + getNonPartitionChannels(insertTableHandle)) {} + +HiveDataSink::HiveDataSink( + RowTypePtr inputType, + std::shared_ptr insertTableHandle, + const ConnectorQueryCtx* connectorQueryCtx, + CommitStrategy commitStrategy, + const std::shared_ptr& hiveConfig, + uint32_t bucketCount, + std::unique_ptr bucketFunction, + const std::vector& dataChannels) + : inputType_(std::move(inputType)), + insertTableHandle_(std::move(insertTableHandle)), + connectorQueryCtx_(connectorQueryCtx), + commitStrategy_(commitStrategy), + hiveConfig_(hiveConfig), + updateMode_(getUpdateMode()), + maxOpenWriters_(hiveConfig_->maxPartitionsPerWriters( + connectorQueryCtx->sessionProperties())), + partitionChannels_(getPartitionChannels(insertTableHandle_)), + partitionIdGenerator_( + !partitionChannels_.empty() + ? std::make_unique( + inputType_, + partitionChannels_, + maxOpenWriters_, + connectorQueryCtx_->memoryPool(), + hiveConfig_->isPartitionPathAsLowerCase( + connectorQueryCtx->sessionProperties())) + : nullptr), + dataChannels_(dataChannels), + bucketCount_(static_cast(bucketCount)), + bucketFunction_(std::move(bucketFunction)), + writerFactory_( + dwio::common::getWriterFactory(insertTableHandle_->storageFormat())), + spillConfig_(connectorQueryCtx->spillConfig()), + sortWriterFinishTimeSliceLimitMs_(getFinishTimeSliceLimitMsFromHiveConfig( + hiveConfig_, + connectorQueryCtx->sessionProperties())), + fileNameGenerator_(insertTableHandle_->fileNameGenerator()) { + if (isBucketed()) { + VELOX_USER_CHECK_LT( + bucketCount_, + hiveConfig_->maxBucketCount(connectorQueryCtx->sessionProperties()), + "bucketCount exceeds the limit"); + } + VELOX_USER_CHECK( + (commitStrategy_ == CommitStrategy::kNoCommit) || + (commitStrategy_ == CommitStrategy::kTaskCommit), + "Unsupported commit strategy: {}", + commitStrategyToString(commitStrategy_)); + + if (insertTableHandle_->ensureFiles()) { + VELOX_CHECK( + !isPartitioned() && !isBucketed(), + "ensureFiles is not supported with bucketing or partition keys in the data"); + ensureWriter(HiveWriterId::unpartitionedId()); + } + + if (!isBucketed()) { + return; + } + const auto& sortedProperty = insertTableHandle_->bucketProperty()->sortedBy(); + if (!sortedProperty.empty()) { + sortColumnIndices_.reserve(sortedProperty.size()); + sortCompareFlags_.reserve(sortedProperty.size()); + for (int i = 0; i < sortedProperty.size(); ++i) { + auto columnIndex = + getNonPartitionTypes(dataChannels_, inputType_) + ->getChildIdxIfExists(sortedProperty.at(i)->sortColumn()); + if (columnIndex.has_value()) { + sortColumnIndices_.push_back(columnIndex.value()); + sortCompareFlags_.push_back( + {sortedProperty.at(i)->sortOrder().isNullsFirst(), + sortedProperty.at(i)->sortOrder().isAscending(), + false, + CompareFlags::NullHandlingMode::kNullAsValue}); + } + } + } +} + +bool HiveDataSink::canReclaim() const { + // Currently, we only support memory reclaim on dwrf file writer. + return (spillConfig_ != nullptr) && + (insertTableHandle_->storageFormat() == dwio::common::FileFormat::DWRF); +} + +void HiveDataSink::appendData(RowVectorPtr input) { + checkRunning(); + + // Lazy load all the input columns. + input->loadedVector(); + + // Write to unpartitioned (and unbucketed) table. + if (!isPartitioned() && !isBucketed()) { + const auto index = ensureWriter(HiveWriterId::unpartitionedId()); + write(index, input); + return; + } + + // Compute partition and bucket numbers. + computePartitionAndBucketIds(input); + + splitInputRowsAndEnsureWriters(input); + + // All inputs belong to a single non-bucketed partition. The partition id + // must be zero. + if (!isBucketed() && partitionIdGenerator_->numPartitions() == 1) { + const auto index = ensureWriter(HiveWriterId{0}); + write(index, input); + return; + } + + for (auto index = 0; index < writers_.size(); ++index) { + const vector_size_t partitionSize = partitionSizes_[index]; + if (partitionSize == 0) { + continue; + } + + RowVectorPtr writerInput = partitionSize == input->size() + ? input + : exec::wrap(partitionSize, partitionRows_[index], input); + write(index, writerInput); + } +} + +void HiveDataSink::write(size_t index, RowVectorPtr input) { + WRITER_NON_RECLAIMABLE_SECTION_GUARD(index); + auto dataInput = makeDataInput(dataChannels_, input); + + writers_[index]->write(dataInput); + writerInfo_[index]->inputSizeInBytes += dataInput->estimateFlatSize(); + writerInfo_[index]->numWrittenRows += dataInput->size(); +} + +std::string HiveDataSink::stateString(State state) { + switch (state) { + case State::kRunning: + return "RUNNING"; + case State::kFinishing: + return "FLUSHING"; + case State::kClosed: + return "CLOSED"; + case State::kAborted: + return "ABORTED"; + default: + VELOX_UNREACHABLE("BAD STATE: {}", static_cast(state)); + } +} + +void HiveDataSink::computePartitionAndBucketIds(const RowVectorPtr& input) { + VELOX_CHECK(isPartitioned() || isBucketed()); + if (isPartitioned()) { + if (!hiveConfig_->allowNullPartitionKeys( + connectorQueryCtx_->sessionProperties())) { + // Check that there are no nulls in the partition keys. + for (auto& partitionIdx : partitionChannels_) { + auto col = input->childAt(partitionIdx); + if (col->mayHaveNulls()) { + for (auto i = 0; i < col->size(); ++i) { + VELOX_USER_CHECK( + !col->isNullAt(i), + "Partition key must not be null: {}", + input->type()->asRow().nameOf(partitionIdx)); + } + } + } + } + partitionIdGenerator_->run(input, partitionIds_); + } + + if (isBucketed()) { + bucketFunction_->partition(*input, bucketIds_); + } +} + +DataSink::Stats HiveDataSink::stats() const { + Stats stats; + if (state_ == State::kAborted) { + return stats; + } + + int64_t numWrittenBytes{0}; + int64_t writeIOTimeUs{0}; + for (const auto& ioStats : ioStats_) { + numWrittenBytes += ioStats->rawBytesWritten(); + writeIOTimeUs += ioStats->writeIOTimeUs(); + } + stats.numWrittenBytes = numWrittenBytes; + stats.writeIOTimeUs = writeIOTimeUs; + + if (state_ != State::kClosed) { + return stats; + } + + stats.numWrittenFiles = writers_.size(); + for (int i = 0; i < writerInfo_.size(); ++i) { + const auto& info = writerInfo_.at(i); + VELOX_CHECK_NOT_NULL(info); + const auto spillStats = info->spillStats->rlock(); + if (!spillStats->empty()) { + stats.spillStats += *spillStats; + } + } + return stats; +} + +std::shared_ptr HiveDataSink::createWriterPool( + const HiveWriterId& writerId) { + auto* connectorPool = connectorQueryCtx_->connectorMemoryPool(); + return connectorPool->addAggregateChild( + fmt::format("{}.{}", connectorPool->name(), writerId.toString())); +} + +void HiveDataSink::setMemoryReclaimers( + HiveWriterInfo* writerInfo, + io::IoStatistics* ioStats) { + auto* connectorPool = connectorQueryCtx_->connectorMemoryPool(); + if (connectorPool->reclaimer() == nullptr) { + return; + } + writerInfo->writerPool->setReclaimer( + WriterReclaimer::create(this, writerInfo, ioStats)); + writerInfo->sinkPool->setReclaimer(exec::MemoryReclaimer::create()); + // NOTE: we set the memory reclaimer for sort pool when we construct the sort + // writer. +} + +void HiveDataSink::setState(State newState) { + checkStateTransition(state_, newState); + state_ = newState; +} + +/// Validates the state transition from 'oldState' to 'newState'. +void HiveDataSink::checkStateTransition(State oldState, State newState) { + switch (oldState) { + case State::kRunning: + if (newState == State::kAborted || newState == State::kFinishing) { + return; + } + break; + case State::kFinishing: + if (newState == State::kAborted || newState == State::kClosed || + // The finishing state is reentry state if we yield in the middle of + // finish processing if a single run takes too long. + newState == State::kFinishing) { + return; + } + [[fallthrough]]; + case State::kAborted: + case State::kClosed: + default: + break; + } + VELOX_FAIL("Unexpected state transition from {} to {}", oldState, newState); +} + +bool HiveDataSink::finish() { + // Flush is reentry state. + setState(State::kFinishing); + + // As for now, only sorted writer needs flush buffered data. For non-sorted + // writer, data is directly written to the underlying file writer. + if (!sortWrite()) { + return true; + } + + // TODO: we might refactor to move the data sorting logic into hive data sink. + const uint64_t startTimeMs = getCurrentTimeMs(); + for (auto i = 0; i < writers_.size(); ++i) { + WRITER_NON_RECLAIMABLE_SECTION_GUARD(i); + if (!writers_[i]->finish()) { + return false; + } + if (getCurrentTimeMs() - startTimeMs > sortWriterFinishTimeSliceLimitMs_) { + return false; + } + } + return true; +} + +std::vector HiveDataSink::close() { + setState(State::kClosed); + closeInternal(); + return commitMessage(); +} + +std::vector HiveDataSink::commitMessage() const { + std::vector partitionUpdates; + partitionUpdates.reserve(writerInfo_.size()); + for (int i = 0; i < writerInfo_.size(); ++i) { + const auto& info = writerInfo_.at(i); + VELOX_CHECK_NOT_NULL(info); + // clang-format off + auto partitionUpdateJson = folly::toJson( + folly::dynamic::object + ("name", info->writerParameters.partitionName().value_or("")) + ("updateMode", + HiveWriterParameters::updateModeToString( + info->writerParameters.updateMode())) + ("writePath", info->writerParameters.writeDirectory()) + ("targetPath", info->writerParameters.targetDirectory()) + ("fileWriteInfos", folly::dynamic::array( + folly::dynamic::object + ("writeFileName", info->writerParameters.writeFileName()) + ("targetFileName", info->writerParameters.targetFileName()) + ("fileSize", ioStats_.at(i)->rawBytesWritten()))) + ("rowCount", info->numWrittenRows) + ("inMemoryDataSizeInBytes", info->inputSizeInBytes) + ("onDiskDataSizeInBytes", ioStats_.at(i)->rawBytesWritten()) + ("containsNumberedFileNames", true)); + // clang-format on + partitionUpdates.push_back(partitionUpdateJson); + } + return partitionUpdates; +} + +void HiveDataSink::abort() { + setState(State::kAborted); + closeInternal(); +} + +void HiveDataSink::closeInternal() { + VELOX_CHECK_NE(state_, State::kRunning); + VELOX_CHECK_NE(state_, State::kFinishing); + + TestValue::adjust( + "facebook::velox::connector::hive::HiveDataSink::closeInternal", this); + + if (state_ == State::kClosed) { + for (int i = 0; i < writers_.size(); ++i) { + WRITER_NON_RECLAIMABLE_SECTION_GUARD(i); + writers_[i]->close(); + } + } else { + for (int i = 0; i < writers_.size(); ++i) { + WRITER_NON_RECLAIMABLE_SECTION_GUARD(i); + writers_[i]->abort(); + } + } +} + +uint32_t HiveDataSink::ensureWriter(const HiveWriterId& id) { + auto it = writerIndexMap_.find(id); + if (it != writerIndexMap_.end()) { + return it->second; + } + return appendWriter(id); +} + +uint32_t HiveDataSink::appendWriter(const HiveWriterId& id) { + // Check max open writers. + VELOX_USER_CHECK_LE( + writers_.size(), maxOpenWriters_, "Exceeded open writer limit"); + VELOX_CHECK_EQ(writers_.size(), writerInfo_.size()); + VELOX_CHECK_EQ(writerIndexMap_.size(), writerInfo_.size()); + + std::optional partitionName = getPartitionName(id); + + // Without explicitly setting flush policy, the default memory based flush + // policy is used. + auto writerParameters = getWriterParameters(partitionName, id.bucketId); + const auto writePath = fs::path(writerParameters.writeDirectory()) / + writerParameters.writeFileName(); + auto writerPool = createWriterPool(id); + auto sinkPool = createSinkPool(writerPool); + std::shared_ptr sortPool{nullptr}; + if (sortWrite()) { + sortPool = createSortPool(writerPool); + } + writerInfo_.emplace_back(std::make_shared( + std::move(writerParameters), + std::move(writerPool), + std::move(sinkPool), + std::move(sortPool))); + ioStats_.emplace_back(std::make_shared()); + setMemoryReclaimers(writerInfo_.back().get(), ioStats_.back().get()); + + // Take the writer options provided by the user as a starting point, or + // allocate a new one. + auto options = insertTableHandle_->writerOptions(); + if (!options) { + options = writerFactory_->createWriterOptions(); + } + + const auto* connectorSessionProperties = + connectorQueryCtx_->sessionProperties(); + + // Only overwrite options in case they were not already provided. + if (options->schema == nullptr) { + options->schema = getNonPartitionTypes(dataChannels_, inputType_); + } + + if (options->memoryPool == nullptr) { + options->memoryPool = writerInfo_.back()->writerPool.get(); + } + + if (!options->compressionKind) { + options->compressionKind = insertTableHandle_->compressionKind(); + } + + if (options->spillConfig == nullptr && canReclaim()) { + options->spillConfig = spillConfig_; + } + + if (options->nonReclaimableSection == nullptr) { + options->nonReclaimableSection = + writerInfo_.back()->nonReclaimableSectionHolder.get(); + } + + if (options->memoryReclaimerFactory == nullptr || + options->memoryReclaimerFactory() == nullptr) { + options->memoryReclaimerFactory = []() { + return exec::MemoryReclaimer::create(); + }; + } + + if (options->serdeParameters.empty()) { + options->serdeParameters = std::map( + insertTableHandle_->serdeParameters().begin(), + insertTableHandle_->serdeParameters().end()); + } + + options->sessionTimezoneName = connectorQueryCtx_->sessionTimezone(); + options->adjustTimestampToTimezone = + connectorQueryCtx_->adjustTimestampToTimezone(); + options->processConfigs(*hiveConfig_->config(), *connectorSessionProperties); + + // Prevents the memory allocation during the writer creation. + WRITER_NON_RECLAIMABLE_SECTION_GUARD(writerInfo_.size() - 1); + auto writer = writerFactory_->createWriter( + dwio::common::FileSink::create( + writePath, + { + .bufferWrite = false, + .connectorProperties = hiveConfig_->config(), + .fileCreateConfig = hiveConfig_->writeFileCreateConfig(), + .pool = writerInfo_.back()->sinkPool.get(), + .metricLogger = dwio::common::MetricsLog::voidLog(), + .stats = ioStats_.back().get(), + }), + options); + writer = maybeCreateBucketSortWriter(std::move(writer)); + writers_.emplace_back(std::move(writer)); + + extendBuffersForPartitionedTables(); + + writerIndexMap_.emplace(id, writers_.size() - 1); + return writerIndexMap_[id]; +} + +std::optional HiveDataSink::getPartitionName( + const HiveWriterId& id) const { + std::optional partitionName; + if (isPartitioned()) { + partitionName = + partitionIdGenerator_->partitionName(id.partitionId.value()); + } + return partitionName; +} + +std::unique_ptr +HiveDataSink::maybeCreateBucketSortWriter( + std::unique_ptr writer) { + if (!sortWrite()) { + return writer; + } + auto* sortPool = writerInfo_.back()->sortPool.get(); + VELOX_CHECK_NOT_NULL(sortPool); + auto sortBuffer = std::make_unique( + getNonPartitionTypes(dataChannels_, inputType_), + sortColumnIndices_, + sortCompareFlags_, + sortPool, + writerInfo_.back()->nonReclaimableSectionHolder.get(), + connectorQueryCtx_->prefixSortConfig(), + spillConfig_, + writerInfo_.back()->spillStats.get()); + return std::make_unique( + std::move(writer), + std::move(sortBuffer), + hiveConfig_->sortWriterMaxOutputRows( + connectorQueryCtx_->sessionProperties()), + hiveConfig_->sortWriterMaxOutputBytes( + connectorQueryCtx_->sessionProperties()), + sortWriterFinishTimeSliceLimitMs_); +} + +void HiveDataSink::extendBuffersForPartitionedTables() { + // Extends the buffer used for partition rows calculations. + partitionSizes_.emplace_back(0); + partitionRows_.emplace_back(nullptr); + rawPartitionRows_.emplace_back(nullptr); +} + +HiveWriterId HiveDataSink::getWriterId(size_t row) const { + std::optional partitionId; + if (isPartitioned()) { + VELOX_CHECK_LT(partitionIds_[row], std::numeric_limits::max()); + partitionId = static_cast(partitionIds_[row]); + } + + std::optional bucketId; + if (isBucketed()) { + bucketId = bucketIds_[row]; + } + return HiveWriterId{partitionId, bucketId}; +} + +void HiveDataSink::updatePartitionRows( + uint32_t index, + vector_size_t numRows, + vector_size_t row) { + VELOX_DCHECK_LT(index, partitionSizes_.size()); + VELOX_DCHECK_EQ(partitionSizes_.size(), partitionRows_.size()); + VELOX_DCHECK_EQ(partitionRows_.size(), rawPartitionRows_.size()); + if (FOLLY_UNLIKELY(partitionRows_[index] == nullptr) || + (partitionRows_[index]->capacity() < numRows * sizeof(vector_size_t))) { + partitionRows_[index] = + allocateIndices(numRows, connectorQueryCtx_->memoryPool()); + rawPartitionRows_[index] = + partitionRows_[index]->asMutable(); + } + rawPartitionRows_[index][partitionSizes_[index]] = row; + ++partitionSizes_[index]; +} + +void HiveDataSink::splitInputRowsAndEnsureWriters(RowVectorPtr /* input */) { + VELOX_CHECK(isPartitioned() || isBucketed()); + if (isBucketed() && isPartitioned()) { + VELOX_CHECK_EQ(bucketIds_.size(), partitionIds_.size()); + } + + std::fill(partitionSizes_.begin(), partitionSizes_.end(), 0); + + const auto numRows = + isPartitioned() ? partitionIds_.size() : bucketIds_.size(); + for (auto row = 0; row < numRows; ++row) { + const auto id = getWriterId(row); + const uint32_t index = ensureWriter(id); + + updatePartitionRows(index, numRows, row); + } + + for (uint32_t i = 0; i < partitionSizes_.size(); ++i) { + if (partitionSizes_[i] != 0) { + VELOX_CHECK_NOT_NULL(partitionRows_[i]); + partitionRows_[i]->setSize(partitionSizes_[i] * sizeof(vector_size_t)); + } + } +} + +std::string HiveDataSink::makePartitionDirectory( + const std::string& tableDirectory, + const std::optional& partitionSubdirectory) const { + if (partitionSubdirectory.has_value()) { + return fs::path(tableDirectory) / partitionSubdirectory.value(); + } + return tableDirectory; +} + +HiveWriterParameters HiveDataSink::getWriterParameters( + const std::optional& partition, + std::optional bucketId) const { + auto [targetFileName, writeFileName] = getWriterFileNames(bucketId); + + return HiveWriterParameters{ + updateMode_, + partition, + targetFileName, + makePartitionDirectory( + insertTableHandle_->locationHandle()->targetPath(), partition), + writeFileName, + makePartitionDirectory( + insertTableHandle_->locationHandle()->writePath(), partition)}; +} + +std::pair HiveDataSink::getWriterFileNames( + std::optional bucketId) const { + if (auto hiveInsertFileNameGenerator = + std::dynamic_pointer_cast( + fileNameGenerator_)) { + return hiveInsertFileNameGenerator->gen( + bucketId, + insertTableHandle_, + *connectorQueryCtx_, + hiveConfig_, + isCommitRequired()); + } + + return fileNameGenerator_->gen( + bucketId, insertTableHandle_, *connectorQueryCtx_, isCommitRequired()); +} + +std::pair HiveInsertFileNameGenerator::gen( + std::optional bucketId, + const std::shared_ptr insertTableHandle, + const ConnectorQueryCtx& connectorQueryCtx, + bool commitRequired) const { + auto defaultHiveConfig = + std::make_shared(std::make_shared( + std::unordered_map())); + + return this->gen( + bucketId, + insertTableHandle, + connectorQueryCtx, + defaultHiveConfig, + commitRequired); +} + +std::pair HiveInsertFileNameGenerator::gen( + std::optional bucketId, + const std::shared_ptr insertTableHandle, + const ConnectorQueryCtx& connectorQueryCtx, + const std::shared_ptr& hiveConfig, + bool commitRequired) const { + auto targetFileName = insertTableHandle->locationHandle()->targetFileName(); + const bool generateFileName = targetFileName.empty(); + if (bucketId.has_value()) { + VELOX_CHECK(generateFileName); + // TODO: add hive.file_renaming_enabled support. + targetFileName = computeBucketedFileName( + connectorQueryCtx.queryId(), + hiveConfig->maxBucketCount(connectorQueryCtx.sessionProperties()), + bucketId.value()); + } else if (generateFileName) { + // targetFileName includes planNodeId and Uuid. As a result, different + // table writers run by the same task driver or the same table writer + // run in different task tries would have different targetFileNames. + targetFileName = fmt::format( + "{}_{}_{}_{}", + connectorQueryCtx.taskId(), + connectorQueryCtx.driverId(), + connectorQueryCtx.planNodeId(), + makeUuid()); + } + VELOX_CHECK(!targetFileName.empty()); + const std::string writeFileName = commitRequired + ? fmt::format(".tmp.velox.{}_{}", targetFileName, makeUuid()) + : targetFileName; + if (generateFileName && + insertTableHandle->storageFormat() == dwio::common::FileFormat::PARQUET) { + return { + fmt::format("{}{}", targetFileName, ".parquet"), + fmt::format("{}{}", writeFileName, ".parquet")}; + } + return {targetFileName, writeFileName}; +} + +folly::dynamic HiveInsertFileNameGenerator::serialize() const { + folly::dynamic obj = folly::dynamic::object; + obj["name"] = "HiveInsertFileNameGenerator"; + return obj; +} + +std::shared_ptr +HiveInsertFileNameGenerator::deserialize( + const folly::dynamic& /* obj */, + void* /* context */) { + return std::make_shared(); +} + +void HiveInsertFileNameGenerator::registerSerDe() { + auto& registry = DeserializationWithContextRegistryForSharedPtr(); + registry.Register( + "HiveInsertFileNameGenerator", HiveInsertFileNameGenerator::deserialize); +} + +std::string HiveInsertFileNameGenerator::toString() const { + return "HiveInsertFileNameGenerator"; +} + +HiveWriterParameters::UpdateMode HiveDataSink::getUpdateMode() const { + if (insertTableHandle_->isExistingTable()) { + if (insertTableHandle_->isPartitioned()) { + const auto insertBehavior = hiveConfig_->insertExistingPartitionsBehavior( + connectorQueryCtx_->sessionProperties()); + switch (insertBehavior) { + case HiveConfig::InsertExistingPartitionsBehavior::kOverwrite: + return HiveWriterParameters::UpdateMode::kOverwrite; + case HiveConfig::InsertExistingPartitionsBehavior::kError: + return HiveWriterParameters::UpdateMode::kNew; + default: + VELOX_UNSUPPORTED( + "Unsupported insert existing partitions behavior: {}", + HiveConfig::insertExistingPartitionsBehaviorString( + insertBehavior)); + } + } else { + if (hiveConfig_->immutablePartitions()) { + VELOX_USER_FAIL("Unpartitioned Hive tables are immutable."); + } + return HiveWriterParameters::UpdateMode::kAppend; + } + } else { + return HiveWriterParameters::UpdateMode::kNew; + } +} + +bool HiveInsertTableHandle::isPartitioned() const { + return std::any_of( + inputColumns_.begin(), inputColumns_.end(), [](auto column) { + return column->isPartitionKey(); + }); +} + +const HiveBucketProperty* HiveInsertTableHandle::bucketProperty() const { + return bucketProperty_.get(); +} + +bool HiveInsertTableHandle::isBucketed() const { + return bucketProperty() != nullptr; +} + +bool HiveInsertTableHandle::isExistingTable() const { + return locationHandle_->tableType() == LocationHandle::TableType::kExisting; +} + +folly::dynamic HiveInsertTableHandle::serialize() const { + folly::dynamic obj = folly::dynamic::object; + obj["name"] = "HiveInsertTableHandle"; + folly::dynamic arr = folly::dynamic::array; + for (const auto& ic : inputColumns_) { + arr.push_back(ic->serialize()); + } + + obj["inputColumns"] = arr; + obj["locationHandle"] = locationHandle_->serialize(); + obj["tableStorageFormat"] = dwio::common::toString(storageFormat_); + + if (bucketProperty_) { + obj["bucketProperty"] = bucketProperty_->serialize(); + } + + if (compressionKind_.has_value()) { + obj["compressionKind"] = common::compressionKindToString(*compressionKind_); + } + + folly::dynamic params = folly::dynamic::object; + for (const auto& [key, value] : serdeParameters_) { + params[key] = value; + } + obj["serdeParameters"] = params; + obj["ensureFiles"] = ensureFiles_; + obj["fileNameGenerator"] = fileNameGenerator_->serialize(); + return obj; +} + +HiveInsertTableHandlePtr HiveInsertTableHandle::create( + const folly::dynamic& obj) { + auto inputColumns = ISerializable::deserialize>( + obj["inputColumns"]); + auto locationHandle = + ISerializable::deserialize(obj["locationHandle"]); + auto storageFormat = + dwio::common::toFileFormat(obj["tableStorageFormat"].asString()); + + std::optional compressionKind = std::nullopt; + if (obj.count("compressionKind") > 0) { + compressionKind = + common::stringToCompressionKind(obj["compressionKind"].asString()); + } + + std::shared_ptr bucketProperty; + if (obj.count("bucketProperty") > 0) { + bucketProperty = + ISerializable::deserialize(obj["bucketProperty"]); + } + + std::unordered_map serdeParameters; + for (const auto& pair : obj["serdeParameters"].items()) { + serdeParameters.emplace(pair.first.asString(), pair.second.asString()); + } + + bool ensureFiles = obj["ensureFiles"].asBool(); + + auto fileNameGenerator = + ISerializable::deserialize(obj["fileNameGenerator"]); + return std::make_shared( + inputColumns, + locationHandle, + storageFormat, + bucketProperty, + compressionKind, + serdeParameters, + nullptr, // writerOptions is not serializable + ensureFiles, + fileNameGenerator); +} + +void HiveInsertTableHandle::registerSerDe() { + auto& registry = DeserializationRegistryForSharedPtr(); + registry.Register("HiveInsertTableHandle", HiveInsertTableHandle::create); +} + +std::string HiveInsertTableHandle::toString() const { + std::ostringstream out; + out << "HiveInsertTableHandle [" << dwio::common::toString(storageFormat_); + if (compressionKind_.has_value()) { + out << " " << common::compressionKindToString(compressionKind_.value()); + } else { + out << " none"; + } + out << "], [inputColumns: ["; + for (const auto& i : inputColumns_) { + out << " " << i->toString(); + } + out << " ], locationHandle: " << locationHandle_->toString(); + if (bucketProperty_) { + out << ", bucketProperty: " << bucketProperty_->toString(); + } + + if (serdeParameters_.size() > 0) { + std::map sortedSerdeParams( + serdeParameters_.begin(), serdeParameters_.end()); + out << ", serdeParameters: "; + for (const auto& [key, value] : sortedSerdeParams) { + out << "[" << key << ", " << value << "] "; + } + } + out << ", fileNameGenerator: " << fileNameGenerator_->toString(); + out << "]"; + return out.str(); +} + +std::string LocationHandle::toString() const { + return fmt::format( + "LocationHandle [targetPath: {}, writePath: {}, tableType: {}, tableFileName: {}]", + targetPath_, + writePath_, + tableTypeName(tableType_), + targetFileName_); +} + +void LocationHandle::registerSerDe() { + auto& registry = DeserializationRegistryForSharedPtr(); + registry.Register("LocationHandle", LocationHandle::create); +} + +folly::dynamic LocationHandle::serialize() const { + folly::dynamic obj = folly::dynamic::object; + obj["name"] = "LocationHandle"; + obj["targetPath"] = targetPath_; + obj["writePath"] = writePath_; + obj["tableType"] = tableTypeName(tableType_); + obj["targetFileName"] = targetFileName_; + return obj; +} + +LocationHandlePtr LocationHandle::create(const folly::dynamic& obj) { + auto targetPath = obj["targetPath"].asString(); + auto writePath = obj["writePath"].asString(); + auto tableType = tableTypeFromName(obj["tableType"].asString()); + auto targetFileName = obj["targetFileName"].asString(); + return std::make_shared( + targetPath, writePath, tableType, targetFileName); +} + +std::unique_ptr HiveDataSink::WriterReclaimer::create( + HiveDataSink* dataSink, + HiveWriterInfo* writerInfo, + io::IoStatistics* ioStats) { + return std::unique_ptr( + new HiveDataSink::WriterReclaimer(dataSink, writerInfo, ioStats)); +} + +bool HiveDataSink::WriterReclaimer::reclaimableBytes( + const memory::MemoryPool& pool, + uint64_t& reclaimableBytes) const { + VELOX_CHECK_EQ(pool.name(), writerInfo_->writerPool->name()); + reclaimableBytes = 0; + if (!dataSink_->canReclaim()) { + return false; + } + return exec::MemoryReclaimer::reclaimableBytes(pool, reclaimableBytes); +} + +uint64_t HiveDataSink::WriterReclaimer::reclaim( + memory::MemoryPool* pool, + uint64_t targetBytes, + uint64_t maxWaitMs, + memory::MemoryReclaimer::Stats& stats) { + VELOX_CHECK_EQ(pool->name(), writerInfo_->writerPool->name()); + if (!dataSink_->canReclaim()) { + return 0; + } + + if (*writerInfo_->nonReclaimableSectionHolder.get()) { + RECORD_METRIC_VALUE(kMetricMemoryNonReclaimableCount); + LOG(WARNING) << "Can't reclaim from hive writer pool " << pool->name() + << " which is under non-reclaimable section, " + << " reserved memory: " + << succinctBytes(pool->reservedBytes()); + ++stats.numNonReclaimableAttempts; + return 0; + } + + const uint64_t memoryUsageBeforeReclaim = pool->reservedBytes(); + const std::string memoryUsageTreeBeforeReclaim = pool->treeMemoryUsage(); + const auto writtenBytesBeforeReclaim = ioStats_->rawBytesWritten(); + const auto reclaimedBytes = + exec::MemoryReclaimer::reclaim(pool, targetBytes, maxWaitMs, stats); + const auto earlyFlushedRawBytes = + ioStats_->rawBytesWritten() - writtenBytesBeforeReclaim; + addThreadLocalRuntimeStat( + kEarlyFlushedRawBytes, + RuntimeCounter(earlyFlushedRawBytes, RuntimeCounter::Unit::kBytes)); + if (earlyFlushedRawBytes > 0) { + RECORD_METRIC_VALUE( + kMetricFileWriterEarlyFlushedRawBytes, earlyFlushedRawBytes); + } + const uint64_t memoryUsageAfterReclaim = pool->reservedBytes(); + if (memoryUsageAfterReclaim > memoryUsageBeforeReclaim) { + VELOX_FAIL( + "Unexpected memory growth after memory reclaim from {}, the memory usage before reclaim: {}, after reclaim: {}\nThe memory tree usage before reclaim:\n{}\nThe memory tree usage after reclaim:\n{}", + pool->name(), + succinctBytes(memoryUsageBeforeReclaim), + succinctBytes(memoryUsageAfterReclaim), + memoryUsageTreeBeforeReclaim, + pool->treeMemoryUsage()); + } + return reclaimedBytes; +} +} // namespace facebook::velox::connector::hive diff --git a/velox/connectors/lakehouse/common/HiveDataSink.h b/velox/connectors/lakehouse/common/HiveDataSink.h new file mode 100644 index 000000000000..9fad76ec5ec0 --- /dev/null +++ b/velox/connectors/lakehouse/common/HiveDataSink.h @@ -0,0 +1,755 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#pragma once + +#include "velox/common/compression/Compression.h" +#include "velox/connectors/Connector.h" +#include "velox/connectors/hive/HiveConfig.h" +#include "velox/connectors/hive/PartitionIdGenerator.h" +#include "velox/connectors/hive/TableHandle.h" +#include "velox/dwio/common/Options.h" +#include "velox/dwio/common/Writer.h" +#include "velox/dwio/common/WriterFactory.h" +#include "velox/exec/MemoryReclaimer.h" + +namespace facebook::velox::connector::hive { + +class LocationHandle; +using LocationHandlePtr = std::shared_ptr; + +/// Location related properties of the Hive table to be written. +class LocationHandle : public ISerializable { + public: + enum class TableType { + /// Write to a new table to be created. + kNew, + /// Write to an existing table. + kExisting, + }; + + LocationHandle( + std::string targetPath, + std::string writePath, + TableType tableType, + std::string targetFileName = "") + : targetPath_(std::move(targetPath)), + targetFileName_(std::move(targetFileName)), + writePath_(std::move(writePath)), + tableType_(tableType) {} + + const std::string& targetPath() const { + return targetPath_; + } + + const std::string& targetFileName() const { + return targetFileName_; + } + + const std::string& writePath() const { + return writePath_; + } + + TableType tableType() const { + return tableType_; + } + + std::string toString() const; + + static void registerSerDe(); + + folly::dynamic serialize() const override; + + static LocationHandlePtr create(const folly::dynamic& obj); + + static const std::string tableTypeName(LocationHandle::TableType type); + + static LocationHandle::TableType tableTypeFromName(const std::string& name); + + private: + // Target directory path. + const std::string targetPath_; + // If non-empty, use this name instead of generating our own. + const std::string targetFileName_; + // Staging directory path. + const std::string writePath_; + // Whether the table to be written is new, already existing or temporary. + const TableType tableType_; +}; + +class HiveSortingColumn : public ISerializable { + public: + HiveSortingColumn( + const std::string& sortColumn, + const core::SortOrder& sortOrder); + + const std::string& sortColumn() const { + return sortColumn_; + } + + core::SortOrder sortOrder() const { + return sortOrder_; + } + + folly::dynamic serialize() const override; + + static std::shared_ptr deserialize( + const folly::dynamic& obj, + void* context); + + std::string toString() const; + + static void registerSerDe(); + + private: + const std::string sortColumn_; + const core::SortOrder sortOrder_; +}; + +class HiveBucketProperty : public ISerializable { + public: + enum class Kind { kHiveCompatible, kPrestoNative }; + + HiveBucketProperty( + Kind kind, + int32_t bucketCount, + const std::vector& bucketedBy, + const std::vector& bucketedTypes, + const std::vector>& sortedBy); + + Kind kind() const { + return kind_; + } + + static std::string kindString(Kind kind); + + /// Returns the number of bucket count. + int32_t bucketCount() const { + return bucketCount_; + } + + /// Returns the bucketed by column names. + const std::vector& bucketedBy() const { + return bucketedBy_; + } + + /// Returns the bucketed by column types. + const std::vector& bucketedTypes() const { + return bucketTypes_; + } + + /// Returns the hive sorting columns if not empty. + const std::vector>& sortedBy() + const { + return sortedBy_; + } + + folly::dynamic serialize() const override; + + static std::shared_ptr deserialize( + const folly::dynamic& obj, + void* context); + + bool operator==(const HiveBucketProperty& other) const { + return true; + } + + static void registerSerDe(); + + std::string toString() const; + + private: + void validate() const; + + const Kind kind_; + const int32_t bucketCount_; + const std::vector bucketedBy_; + const std::vector bucketTypes_; + const std::vector> sortedBy_; +}; + +FOLLY_ALWAYS_INLINE std::ostream& operator<<( + std::ostream& os, + HiveBucketProperty::Kind kind) { + os << HiveBucketProperty::kindString(kind); + return os; +} + +class HiveInsertTableHandle; +using HiveInsertTableHandlePtr = std::shared_ptr; + +class FileNameGenerator : public ISerializable { + public: + virtual ~FileNameGenerator() = default; + + virtual std::pair gen( + std::optional bucketId, + const std::shared_ptr insertTableHandle, + const ConnectorQueryCtx& connectorQueryCtx, + bool commitRequired) const = 0; + + virtual std::string toString() const = 0; +}; + +class HiveInsertFileNameGenerator : public FileNameGenerator { + public: + HiveInsertFileNameGenerator() {} + + std::pair gen( + std::optional bucketId, + const std::shared_ptr insertTableHandle, + const ConnectorQueryCtx& connectorQueryCtx, + bool commitRequired) const override; + + /// Version of file generation that takes hiveConfig into account when + /// generating file names + std::pair gen( + std::optional bucketId, + const std::shared_ptr insertTableHandle, + const ConnectorQueryCtx& connectorQueryCtx, + const std::shared_ptr& hiveConfig, + bool commitRequired) const; + + static void registerSerDe(); + + folly::dynamic serialize() const override; + + static std::shared_ptr deserialize( + const folly::dynamic& obj, + void* context); + + std::string toString() const override; +}; + +/// Represents a request for Hive write. +class HiveInsertTableHandle : public ConnectorInsertTableHandle { + public: + HiveInsertTableHandle( + std::vector> inputColumns, + std::shared_ptr locationHandle, + dwio::common::FileFormat storageFormat = dwio::common::FileFormat::DWRF, + std::shared_ptr bucketProperty = nullptr, + std::optional compressionKind = {}, + const std::unordered_map& serdeParameters = {}, + const std::shared_ptr& writerOptions = + nullptr, + // When this option is set the HiveDataSink will always write a file even + // if there's no data. This is useful when the table is bucketed, but the + // engine handles ensuring a 1 to 1 mapping from task to bucket. + const bool ensureFiles = false, + std::shared_ptr fileNameGenerator = + std::make_shared()) + : inputColumns_(std::move(inputColumns)), + locationHandle_(std::move(locationHandle)), + storageFormat_(storageFormat), + bucketProperty_(std::move(bucketProperty)), + compressionKind_(compressionKind), + serdeParameters_(serdeParameters), + writerOptions_(writerOptions), + ensureFiles_(ensureFiles), + fileNameGenerator_(std::move(fileNameGenerator)) { + if (compressionKind.has_value()) { + VELOX_CHECK( + compressionKind.value() != common::CompressionKind_MAX, + "Unsupported compression type: CompressionKind_MAX"); + } + + if (ensureFiles_) { + // If ensureFiles is set and either the bucketProperty is set or some + // partition keys are in the data, there is not a 1:1 mapping from Task to + // files so we can't proactively create writers. + VELOX_CHECK( + bucketProperty_ == nullptr || bucketProperty_->bucketCount() == 0, + "ensureFiles is not supported with bucketing"); + + for (const auto& inputColumn : inputColumns_) { + VELOX_CHECK( + !inputColumn->isPartitionKey(), + "ensureFiles is not supported with partition keys in the data"); + } + } + } + + virtual ~HiveInsertTableHandle() = default; + + const std::vector>& inputColumns() + const { + return inputColumns_; + } + + const std::shared_ptr& locationHandle() const { + return locationHandle_; + } + + std::optional compressionKind() const { + return compressionKind_; + } + + dwio::common::FileFormat storageFormat() const { + return storageFormat_; + } + + const std::unordered_map& serdeParameters() const { + return serdeParameters_; + } + + const std::shared_ptr& writerOptions() const { + return writerOptions_; + } + + bool ensureFiles() const { + return ensureFiles_; + } + + const std::shared_ptr& fileNameGenerator() const { + return fileNameGenerator_; + } + + bool supportsMultiThreading() const override { + return true; + } + + bool isPartitioned() const; + + bool isBucketed() const; + + const HiveBucketProperty* bucketProperty() const; + + bool isExistingTable() const; + + folly::dynamic serialize() const override; + + static HiveInsertTableHandlePtr create(const folly::dynamic& obj); + + static void registerSerDe(); + + std::string toString() const override; + + private: + const std::vector> inputColumns_; + const std::shared_ptr locationHandle_; + const dwio::common::FileFormat storageFormat_; + const std::shared_ptr bucketProperty_; + const std::optional compressionKind_; + const std::unordered_map serdeParameters_; + const std::shared_ptr writerOptions_; + const bool ensureFiles_; + const std::shared_ptr fileNameGenerator_; +}; + +/// Parameters for Hive writers. +class HiveWriterParameters { + public: + enum class UpdateMode { + kNew, // Write files to a new directory. + kOverwrite, // Overwrite an existing directory. + // Append mode is currently only supported for unpartitioned tables. + kAppend, // Append to an unpartitioned table. + }; + + /// @param updateMode Write the files to a new directory, or append to an + /// existing directory or overwrite an existing directory. + /// @param partitionName Partition name in the typical Hive style, which is + /// also the partition subdirectory part of the partition path. + /// @param targetFileName The final name of a file after committing. + /// @param targetDirectory The final directory that a file should be in after + /// committing. + /// @param writeFileName The temporary name of the file that a running writer + /// writes to. If a running writer writes directory to the target file, set + /// writeFileName to targetFileName by default. + /// @param writeDirectory The temporary directory that a running writer writes + /// to. If a running writer writes directory to the target directory, set + /// writeDirectory to targetDirectory by default. + HiveWriterParameters( + UpdateMode updateMode, + std::optional partitionName, + std::string targetFileName, + std::string targetDirectory, + std::optional writeFileName = std::nullopt, + std::optional writeDirectory = std::nullopt) + : updateMode_(updateMode), + partitionName_(std::move(partitionName)), + targetFileName_(std::move(targetFileName)), + targetDirectory_(std::move(targetDirectory)), + writeFileName_(writeFileName.value_or(targetFileName_)), + writeDirectory_(writeDirectory.value_or(targetDirectory_)) {} + + UpdateMode updateMode() const { + return updateMode_; + } + + static std::string updateModeToString(UpdateMode updateMode) { + switch (updateMode) { + case UpdateMode::kNew: + return "NEW"; + case UpdateMode::kOverwrite: + return "OVERWRITE"; + case UpdateMode::kAppend: + return "APPEND"; + default: + VELOX_UNSUPPORTED("Unsupported update mode."); + } + } + + const std::optional& partitionName() const { + return partitionName_; + } + + const std::string& targetFileName() const { + return targetFileName_; + } + + const std::string& writeFileName() const { + return writeFileName_; + } + + const std::string& targetDirectory() const { + return targetDirectory_; + } + + const std::string& writeDirectory() const { + return writeDirectory_; + } + + private: + const UpdateMode updateMode_; + const std::optional partitionName_; + const std::string targetFileName_; + const std::string targetDirectory_; + const std::string writeFileName_; + const std::string writeDirectory_; +}; + +struct HiveWriterInfo { + HiveWriterInfo( + HiveWriterParameters parameters, + std::shared_ptr _writerPool, + std::shared_ptr _sinkPool, + std::shared_ptr _sortPool) + : writerParameters(std::move(parameters)), + nonReclaimableSectionHolder(new tsan_atomic(false)), + spillStats(std::make_unique>()), + writerPool(std::move(_writerPool)), + sinkPool(std::move(_sinkPool)), + sortPool(std::move(_sortPool)) {} + + const HiveWriterParameters writerParameters; + const std::unique_ptr> nonReclaimableSectionHolder; + /// Collects the spill stats from sort writer if the spilling has been + /// triggered. + const std::unique_ptr> spillStats; + const std::shared_ptr writerPool; + const std::shared_ptr sinkPool; + const std::shared_ptr sortPool; + int64_t numWrittenRows = 0; + int64_t inputSizeInBytes = 0; +}; + +/// Identifies a hive writer. +struct HiveWriterId { + std::optional partitionId{std::nullopt}; + std::optional bucketId{std::nullopt}; + + HiveWriterId() = default; + + HiveWriterId( + std::optional _partitionId, + std::optional _bucketId = std::nullopt) + : partitionId(_partitionId), bucketId(_bucketId) {} + + /// Returns the special writer id for the un-partitioned (and non-bucketed) + /// table. + static const HiveWriterId& unpartitionedId(); + + std::string toString() const; + + bool operator==(const HiveWriterId& other) const { + return std::tie(partitionId, bucketId) == + std::tie(other.partitionId, other.bucketId); + } +}; + +struct HiveWriterIdHasher { + std::size_t operator()(const HiveWriterId& id) const { + return bits::hashMix( + id.partitionId.value_or(std::numeric_limits::max()), + id.bucketId.value_or(std::numeric_limits::max())); + } +}; + +struct HiveWriterIdEq { + bool operator()(const HiveWriterId& lhs, const HiveWriterId& rhs) const { + return lhs == rhs; + } +}; + +class HiveDataSink : public DataSink { + public: + /// The list of runtime stats reported by hive data sink + static constexpr const char* kEarlyFlushedRawBytes = "earlyFlushedRawBytes"; + + /// Defines the execution states of a hive data sink running internally. + enum class State { + /// The data sink accepts new append data in this state. + kRunning = 0, + /// The data sink flushes any buffered data to the underlying file writer + /// but no more data can be appended. + kFinishing = 1, + /// The data sink is aborted on error and no more data can be appended. + kAborted = 2, + /// The data sink is closed on error and no more data can be appended. + kClosed = 3 + }; + static std::string stateString(State state); + + HiveDataSink( + RowTypePtr inputType, + std::shared_ptr insertTableHandle, + const ConnectorQueryCtx* connectorQueryCtx, + CommitStrategy commitStrategy, + const std::shared_ptr& hiveConfig); + + HiveDataSink( + RowTypePtr inputType, + std::shared_ptr insertTableHandle, + const ConnectorQueryCtx* connectorQueryCtx, + CommitStrategy commitStrategy, + const std::shared_ptr& hiveConfig, + uint32_t bucketCount, + std::unique_ptr bucketFunction, + const std::vector& dataChannels); + + void appendData(RowVectorPtr input) override; + + bool finish() override; + + Stats stats() const override; + + std::vector close() override; + + void abort() override; + + bool canReclaim() const; + + protected: + // Validates the state transition from 'oldState' to 'newState'. + void checkStateTransition(State oldState, State newState); + + void setState(State newState); + + virtual std::vector commitMessage() const; + + class WriterReclaimer : public exec::MemoryReclaimer { + public: + static std::unique_ptr create( + HiveDataSink* dataSink, + HiveWriterInfo* writerInfo, + io::IoStatistics* ioStats); + + bool reclaimableBytes( + const memory::MemoryPool& pool, + uint64_t& reclaimableBytes) const override; + + uint64_t reclaim( + memory::MemoryPool* pool, + uint64_t targetBytes, + uint64_t maxWaitMs, + memory::MemoryReclaimer::Stats& stats) override; + + private: + WriterReclaimer( + HiveDataSink* dataSink, + HiveWriterInfo* writerInfo, + io::IoStatistics* ioStats) + : exec::MemoryReclaimer(0), + dataSink_(dataSink), + writerInfo_(writerInfo), + ioStats_(ioStats) { + VELOX_CHECK_NOT_NULL(dataSink_); + VELOX_CHECK_NOT_NULL(writerInfo_); + VELOX_CHECK_NOT_NULL(ioStats_); + } + + HiveDataSink* const dataSink_; + HiveWriterInfo* const writerInfo_; + io::IoStatistics* const ioStats_; + }; + + FOLLY_ALWAYS_INLINE bool sortWrite() const { + return !sortColumnIndices_.empty(); + } + + // Returns true if the table is partitioned. + FOLLY_ALWAYS_INLINE bool isPartitioned() const { + return partitionIdGenerator_ != nullptr; + } + + // Returns true if the table is bucketed. + FOLLY_ALWAYS_INLINE bool isBucketed() const { + return bucketCount_ != 0; + } + + FOLLY_ALWAYS_INLINE bool isCommitRequired() const { + return commitStrategy_ != CommitStrategy::kNoCommit; + } + + std::shared_ptr createWriterPool( + const HiveWriterId& writerId); + + void setMemoryReclaimers( + HiveWriterInfo* writerInfo, + io::IoStatistics* ioStats); + + // Compute the partition id and bucket id for each row in 'input'. + void computePartitionAndBucketIds(const RowVectorPtr& input); + + // Get the hive writer id corresponding to the row + // from partitionIds and bucketIds. + FOLLY_ALWAYS_INLINE HiveWriterId getWriterId(size_t row) const; + + // Computes the number of input rows as well as the actual input row indices + // to each corresponding (bucketed) partition based on the partition and + // bucket ids calculated by 'computePartitionAndBucketIds'. The function also + // ensures that there is a writer created for each (bucketed) partition. + virtual void splitInputRowsAndEnsureWriters(RowVectorPtr input); + + // Makes sure the writer is created for the given writer id. The function + // returns the corresponding index in 'writers_'. + uint32_t ensureWriter(const HiveWriterId& id); + + // Appends a new writer for the given 'id'. The function returns the index of + // the newly created writer in 'writers_'. + uint32_t appendWriter(const HiveWriterId& id); + + virtual std::optional getPartitionName( + const HiveWriterId& id) const; + + std::unique_ptr + maybeCreateBucketSortWriter( + std::unique_ptr writer); + + std::string makePartitionDirectory( + const std::string& tableDirectory, + const std::optional& partitionSubdirectory) const; + + void + updatePartitionRows(uint32_t index, vector_size_t numRows, vector_size_t row); + + void extendBuffersForPartitionedTables(); + + HiveWriterParameters getWriterParameters( + const std::optional& partition, + std::optional bucketId) const; + + // Gets write and target file names for a writer based on the table commit + // strategy as well as table partitioned type. If commit is not required, the + // write file and target file has the same name. If not, add a temp file + // prefix to the target file for write file name. The coordinator (or driver + // for Presto on spark) will rename the write file to target file to commit + // the table write when update the metadata store. If it is a bucketed table, + // the file name encodes the corresponding bucket id. + std::pair getWriterFileNames( + std::optional bucketId) const; + + HiveWriterParameters::UpdateMode getUpdateMode() const; + + FOLLY_ALWAYS_INLINE void checkRunning() const { + VELOX_CHECK_EQ(state_, State::kRunning, "Hive data sink is not running"); + } + + // Invoked to write 'input' to the specified file writer. + void write(size_t index, RowVectorPtr input); + + void closeInternal(); + + const RowTypePtr inputType_; + const std::shared_ptr insertTableHandle_; + const ConnectorQueryCtx* const connectorQueryCtx_; + const CommitStrategy commitStrategy_; + const std::shared_ptr hiveConfig_; + const HiveWriterParameters::UpdateMode updateMode_; + const uint32_t maxOpenWriters_; + const std::vector partitionChannels_; + const std::unique_ptr partitionIdGenerator_; + // Indices of dataChannel are stored in ascending order + const std::vector dataChannels_; + const int32_t bucketCount_{0}; + const std::unique_ptr bucketFunction_; + const std::shared_ptr writerFactory_; + const common::SpillConfig* const spillConfig_; + const uint64_t sortWriterFinishTimeSliceLimitMs_{0}; + + std::vector sortColumnIndices_; + std::vector sortCompareFlags_; + + State state_{State::kRunning}; + + tsan_atomic nonReclaimableSection_{false}; + + // The map from writer id to the writer index in 'writers_' and 'writerInfo_'. + folly::F14FastMap + writerIndexMap_; + + // Below are structures for partitions from all inputs. writerInfo_ and + // writers_ are both indexed by partitionId. + std::vector> writerInfo_; + std::vector> writers_; + // IO statistics collected for each writer. + std::vector> ioStats_; + + // Below are structures updated when processing current input. partitionIds_ + // are indexed by the row of input_. partitionRows_, rawPartitionRows_ and + // partitionSizes_ are indexed by partitionId. + raw_vector partitionIds_; + std::vector partitionRows_; + std::vector rawPartitionRows_; + std::vector partitionSizes_; + + // Reusable buffers for bucket id calculations. + std::vector bucketIds_; + + // Strategy for naming writer files + std::shared_ptr fileNameGenerator_; +}; + +FOLLY_ALWAYS_INLINE std::ostream& operator<<( + std::ostream& os, + HiveDataSink::State state) { + os << HiveDataSink::stateString(state); + return os; +} +} // namespace facebook::velox::connector::hive + +template <> +struct fmt::formatter + : formatter { + auto format( + facebook::velox::connector::hive::HiveDataSink::State s, + format_context& ctx) const { + return formatter::format( + facebook::velox::connector::hive::HiveDataSink::stateString(s), ctx); + } +}; + +template <> +struct fmt::formatter< + facebook::velox::connector::hive::LocationHandle::TableType> + : formatter { + auto format( + facebook::velox::connector::hive::LocationHandle::TableType s, + format_context& ctx) const { + return formatter::format(static_cast(s), ctx); + } +}; diff --git a/velox/connectors/lakehouse/common/HiveDataSource.cpp b/velox/connectors/lakehouse/common/HiveDataSource.cpp new file mode 100644 index 000000000000..6bd5a129946c --- /dev/null +++ b/velox/connectors/lakehouse/common/HiveDataSource.cpp @@ -0,0 +1,572 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "velox/connectors/hive/HiveDataSource.h" + +#include +#include +#include + +#include "velox/common/testutil/TestValue.h" +#include "velox/connectors/hive/HiveConfig.h" +#include "velox/dwio/common/ReaderFactory.h" +#include "velox/expression/FieldReference.h" + +using facebook::velox::common::testutil::TestValue; + +namespace facebook::velox::connector::hive { + +class HiveTableHandle; +class HiveColumnHandle; + +namespace { + +bool isMember( + const std::vector& fields, + const exec::FieldReference& field) { + return std::find(fields.begin(), fields.end(), &field) != fields.end(); +} + +bool shouldEagerlyMaterialize( + const exec::Expr& remainingFilter, + const exec::FieldReference& field) { + if (!remainingFilter.evaluatesArgumentsOnNonIncreasingSelection()) { + return true; + } + for (auto& input : remainingFilter.inputs()) { + if (isMember(input->distinctFields(), field) && input->hasConditionals()) { + return true; + } + } + return false; +} + +} // namespace + +HiveDataSource::HiveDataSource( + const RowTypePtr& outputType, + const connector::ConnectorTableHandlePtr& tableHandle, + const connector::ColumnHandleMap& columnHandles, + FileHandleFactory* fileHandleFactory, + folly::Executor* executor, + const ConnectorQueryCtx* connectorQueryCtx, + const std::shared_ptr& hiveConfig) + : fileHandleFactory_(fileHandleFactory), + executor_(executor), + connectorQueryCtx_(connectorQueryCtx), + hiveConfig_(hiveConfig), + pool_(connectorQueryCtx->memoryPool()), + outputType_(outputType), + expressionEvaluator_(connectorQueryCtx->expressionEvaluator()) { + // Column handled keyed on the column alias, the name used in the query. + for (const auto& [canonicalizedName, columnHandle] : columnHandles) { + auto handle = + std::dynamic_pointer_cast(columnHandle); + VELOX_CHECK_NOT_NULL( + handle, + "ColumnHandle must be an instance of HiveColumnHandle for {}", + canonicalizedName); + switch (handle->columnType()) { + case HiveColumnHandle::ColumnType::kRegular: + break; + case HiveColumnHandle::ColumnType::kPartitionKey: + partitionKeys_.emplace(handle->name(), handle); + break; + case HiveColumnHandle::ColumnType::kSynthesized: + infoColumns_.emplace(handle->name(), handle); + break; + case HiveColumnHandle::ColumnType::kRowIndex: + specialColumns_.rowIndex = handle->name(); + break; + case HiveColumnHandle::ColumnType::kRowId: + specialColumns_.rowId = handle->name(); + break; + } + } + + std::vector readColumnNames; + auto readColumnTypes = outputType_->children(); + for (const auto& outputName : outputType_->names()) { + auto it = columnHandles.find(outputName); + VELOX_CHECK( + it != columnHandles.end(), + "ColumnHandle is missing for output column: {}", + outputName); + + auto* handle = static_cast(it->second.get()); + readColumnNames.push_back(handle->name()); + for (auto& subfield : handle->requiredSubfields()) { + VELOX_USER_CHECK_EQ( + getColumnName(subfield), + handle->name(), + "Required subfield does not match column name"); + subfields_[handle->name()].push_back(&subfield); + } + } + + hiveTableHandle_ = + std::dynamic_pointer_cast(tableHandle); + VELOX_CHECK_NOT_NULL( + hiveTableHandle_, "TableHandle must be an instance of HiveTableHandle"); + if (hiveConfig_->isFileColumnNamesReadAsLowerCase( + connectorQueryCtx->sessionProperties())) { + checkColumnNameLowerCase(outputType_); + checkColumnNameLowerCase(hiveTableHandle_->subfieldFilters(), infoColumns_); + checkColumnNameLowerCase(hiveTableHandle_->remainingFilter()); + } + + for (const auto& [k, v] : hiveTableHandle_->subfieldFilters()) { + filters_.emplace(k.clone(), v); + } + double sampleRate = 1; + auto remainingFilter = extractFiltersFromRemainingFilter( + hiveTableHandle_->remainingFilter(), + expressionEvaluator_, + false, + filters_, + sampleRate); + if (sampleRate != 1) { + randomSkip_ = std::make_shared(sampleRate); + } + + if (remainingFilter) { + remainingFilterExprSet_ = expressionEvaluator_->compile(remainingFilter); + auto& remainingFilterExpr = remainingFilterExprSet_->expr(0); + folly::F14FastMap columnNames; + for (int i = 0; i < readColumnNames.size(); ++i) { + columnNames[readColumnNames[i]] = i; + } + for (auto& input : remainingFilterExpr->distinctFields()) { + auto it = columnNames.find(input->field()); + if (it != columnNames.end()) { + if (shouldEagerlyMaterialize(*remainingFilterExpr, *input)) { + multiReferencedFields_.push_back(it->second); + } + continue; + } + // Remaining filter may reference columns that are not used otherwise, + // e.g. are not being projected out and are not used in range filters. + // Make sure to add these columns to readerOutputType_. + readColumnNames.push_back(input->field()); + readColumnTypes.push_back(input->type()); + } + remainingFilterSubfields_ = remainingFilterExpr->extractSubfields(); + if (VLOG_IS_ON(1)) { + VLOG(1) << fmt::format( + "Extracted subfields from remaining filter: [{}]", + fmt::join(remainingFilterSubfields_, ", ")); + } + for (auto& subfield : remainingFilterSubfields_) { + const auto& name = getColumnName(subfield); + auto it = subfields_.find(name); + if (it != subfields_.end()) { + // Some subfields of the column are already projected out, we append the + // remainingFilter subfield + it->second.push_back(&subfield); + } else if (columnNames.count(name) == 0) { + // remainingFilter subfield's column is not projected out, we add the + // column and append the subfield + subfields_[name].push_back(&subfield); + } + } + } + + readerOutputType_ = + ROW(std::move(readColumnNames), std::move(readColumnTypes)); + scanSpec_ = makeScanSpec( + readerOutputType_, + subfields_, + filters_, + hiveTableHandle_->dataColumns(), + partitionKeys_, + infoColumns_, + specialColumns_, + hiveConfig_->readStatsBasedFilterReorderDisabled( + connectorQueryCtx_->sessionProperties()), + pool_); + if (remainingFilter) { + metadataFilter_ = std::make_shared( + *scanSpec_, *remainingFilter, expressionEvaluator_); + } + + ioStats_ = std::make_shared(); + fsStats_ = std::make_shared(); +} + +std::unique_ptr HiveDataSource::createSplitReader() { + return SplitReader::create( + split_, + hiveTableHandle_, + &partitionKeys_, + connectorQueryCtx_, + hiveConfig_, + readerOutputType_, + ioStats_, + fsStats_, + fileHandleFactory_, + executor_, + scanSpec_, + expressionEvaluator_, + totalRemainingFilterTime_); +} + +std::vector HiveDataSource::setupBucketConversion() { + VELOX_CHECK_NE( + split_->bucketConversion->tableBucketCount, + split_->bucketConversion->partitionBucketCount); + VELOX_CHECK(split_->tableBucketNumber.has_value()); + VELOX_CHECK_NOT_NULL(hiveTableHandle_->dataColumns()); + ++numBucketConversion_; + bool rebuildScanSpec = false; + std::vector names; + std::vector types; + std::vector bucketChannels; + for (auto& handle : split_->bucketConversion->bucketColumnHandles) { + VELOX_CHECK(handle->columnType() == HiveColumnHandle::ColumnType::kRegular); + if (subfields_.erase(handle->name()) > 0) { + rebuildScanSpec = true; + } + auto index = readerOutputType_->getChildIdxIfExists(handle->name()); + if (!index.has_value()) { + if (names.empty()) { + names = readerOutputType_->names(); + types = readerOutputType_->children(); + } + index = names.size(); + names.push_back(handle->name()); + types.push_back( + hiveTableHandle_->dataColumns()->findChild(handle->name())); + rebuildScanSpec = true; + } + bucketChannels.push_back(*index); + } + if (!names.empty()) { + readerOutputType_ = ROW(std::move(names), std::move(types)); + } + if (rebuildScanSpec) { + auto newScanSpec = makeScanSpec( + readerOutputType_, + subfields_, + filters_, + hiveTableHandle_->dataColumns(), + partitionKeys_, + infoColumns_, + specialColumns_, + hiveConfig_->readStatsBasedFilterReorderDisabled( + connectorQueryCtx_->sessionProperties()), + pool_); + newScanSpec->moveAdaptationFrom(*scanSpec_); + scanSpec_ = std::move(newScanSpec); + } + return bucketChannels; +} + +void HiveDataSource::setupRowIdColumn() { + VELOX_CHECK(split_->rowIdProperties.has_value()); + const auto& props = *split_->rowIdProperties; + auto* rowId = scanSpec_->childByName(*specialColumns_.rowId); + VELOX_CHECK_NOT_NULL(rowId); + auto& rowIdType = + readerOutputType_->findChild(*specialColumns_.rowId)->asRow(); + auto rowGroupId = split_->getFileName(); + rowId->childByName(rowIdType.nameOf(1)) + ->setConstantValue( + StringView(rowGroupId), VARCHAR(), connectorQueryCtx_->memoryPool()); + rowId->childByName(rowIdType.nameOf(2)) + ->setConstantValue( + props.metadataVersion, BIGINT(), connectorQueryCtx_->memoryPool()); + rowId->childByName(rowIdType.nameOf(3)) + ->setConstantValue( + props.partitionId, BIGINT(), connectorQueryCtx_->memoryPool()); + rowId->childByName(rowIdType.nameOf(4)) + ->setConstantValue( + StringView(props.tableGuid), + VARCHAR(), + connectorQueryCtx_->memoryPool()); +} + +void HiveDataSource::addSplit(std::shared_ptr split) { + VELOX_CHECK_NULL( + split_, + "Previous split has not been processed yet. Call next to process the split."); + split_ = std::dynamic_pointer_cast(split); + VELOX_CHECK_NOT_NULL(split_, "Wrong type of split"); + + VLOG(1) << "Adding split " << split_->toString(); + + if (splitReader_) { + splitReader_.reset(); + } + + std::vector bucketChannels; + if (split_->bucketConversion.has_value()) { + bucketChannels = setupBucketConversion(); + } + if (specialColumns_.rowId.has_value()) { + setupRowIdColumn(); + } + + splitReader_ = createSplitReader(); + + if (!bucketChannels.empty()) { + splitReader_->setBucketConversion(std::move(bucketChannels)); + } + // Split reader subclasses may need to use the reader options in prepareSplit + // so we initialize it beforehand. + splitReader_->configureReaderOptions(randomSkip_); + splitReader_->prepareSplit(metadataFilter_, runtimeStats_); + readerOutputType_ = splitReader_->readerOutputType(); +} + +std::optional HiveDataSource::next( + uint64_t size, + velox::ContinueFuture& /*future*/) { + VELOX_CHECK(split_ != nullptr, "No split to process. Call addSplit first."); + VELOX_CHECK_NOT_NULL(splitReader_, "No split reader present"); + + TestValue::adjust( + "facebook::velox::connector::hive::HiveDataSource::next", this); + + if (splitReader_->emptySplit()) { + resetSplit(); + return nullptr; + } + + // Bucket conversion or delta update could add extra column to reader output. + auto needsExtraColumn = [&] { + return output_->asUnchecked()->childrenSize() < + readerOutputType_->size(); + }; + if (!output_ || needsExtraColumn()) { + output_ = BaseVector::create(readerOutputType_, 0, pool_); + } + + const auto rowsScanned = splitReader_->next(size, output_); + completedRows_ += rowsScanned; + if (rowsScanned == 0) { + splitReader_->updateRuntimeStats(runtimeStats_); + resetSplit(); + return nullptr; + } + + VELOX_CHECK( + !output_->mayHaveNulls(), "Top-level row vector cannot have nulls"); + auto rowsRemaining = output_->size(); + if (rowsRemaining == 0) { + // no rows passed the pushed down filters. + return getEmptyOutput(); + } + + auto rowVector = std::dynamic_pointer_cast(output_); + + // In case there is a remaining filter that excludes some but not all + // rows, collect the indices of the passing rows. If there is no filter, + // or it passes on all rows, leave this as null and let exec::wrap skip + // wrapping the results. + BufferPtr remainingIndices; + filterRows_.resize(rowVector->size()); + + if (remainingFilterExprSet_) { + rowsRemaining = evaluateRemainingFilter(rowVector); + VELOX_CHECK_LE(rowsRemaining, rowsScanned); + if (rowsRemaining == 0) { + // No rows passed the remaining filter. + return getEmptyOutput(); + } + + if (rowsRemaining < rowVector->size()) { + // Some, but not all rows passed the remaining filter. + remainingIndices = filterEvalCtx_.selectedIndices; + } + } + + if (outputType_->size() == 0) { + return exec::wrap(rowsRemaining, remainingIndices, rowVector); + } + + std::vector outputColumns; + outputColumns.reserve(outputType_->size()); + for (int i = 0; i < outputType_->size(); ++i) { + auto& child = rowVector->childAt(i); + if (remainingIndices) { + // Disable dictionary values caching in expression eval so that we + // don't need to reallocate the result for every batch. + child->disableMemo(); + } + outputColumns.emplace_back( + exec::wrapChild(rowsRemaining, remainingIndices, child)); + } + + return std::make_shared( + pool_, outputType_, BufferPtr(nullptr), rowsRemaining, outputColumns); +} + +void HiveDataSource::addDynamicFilter( + column_index_t outputChannel, + const std::shared_ptr& filter) { + auto& fieldSpec = scanSpec_->getChildByChannel(outputChannel); + fieldSpec.setFilter(filter); + scanSpec_->resetCachedValues(true); + if (splitReader_) { + splitReader_->resetFilterCaches(); + } +} + +std::unordered_map HiveDataSource::runtimeStats() { + auto res = runtimeStats_.toMap(); + res.insert( + {{"numPrefetch", RuntimeCounter(ioStats_->prefetch().count())}, + {"prefetchBytes", + RuntimeCounter( + ioStats_->prefetch().sum(), RuntimeCounter::Unit::kBytes)}, + {"totalScanTime", + RuntimeCounter( + ioStats_->totalScanTime(), RuntimeCounter::Unit::kNanos)}, + {"totalRemainingFilterTime", + RuntimeCounter( + totalRemainingFilterTime_.load(std::memory_order_relaxed), + RuntimeCounter::Unit::kNanos)}, + {"ioWaitWallNanos", + RuntimeCounter( + ioStats_->queryThreadIoLatency().sum() * 1000, + RuntimeCounter::Unit::kNanos)}, + {"maxSingleIoWaitWallNanos", + RuntimeCounter( + ioStats_->queryThreadIoLatency().max() * 1000, + RuntimeCounter::Unit::kNanos)}, + {"overreadBytes", + RuntimeCounter( + ioStats_->rawOverreadBytes(), RuntimeCounter::Unit::kBytes)}}); + if (ioStats_->read().count() > 0) { + res.insert({"numStorageRead", RuntimeCounter(ioStats_->read().count())}); + res.insert( + {"storageReadBytes", + RuntimeCounter(ioStats_->read().sum(), RuntimeCounter::Unit::kBytes)}); + } + if (ioStats_->ssdRead().count() > 0) { + res.insert({"numLocalRead", RuntimeCounter(ioStats_->ssdRead().count())}); + res.insert( + {"localReadBytes", + RuntimeCounter( + ioStats_->ssdRead().sum(), RuntimeCounter::Unit::kBytes)}); + } + if (ioStats_->ramHit().count() > 0) { + res.insert({"numRamRead", RuntimeCounter(ioStats_->ramHit().count())}); + res.insert( + {"ramReadBytes", + RuntimeCounter( + ioStats_->ramHit().sum(), RuntimeCounter::Unit::kBytes)}); + } + if (numBucketConversion_ > 0) { + res.insert({"numBucketConversion", RuntimeCounter(numBucketConversion_)}); + } + + const auto fsStats = fsStats_->stats(); + for (const auto& storageStats : fsStats) { + res.emplace( + storageStats.first, + RuntimeCounter(storageStats.second.sum, storageStats.second.unit)); + } + return res; +} + +void HiveDataSource::setFromDataSource( + std::unique_ptr sourceUnique) { + auto source = dynamic_cast(sourceUnique.get()); + VELOX_CHECK_NOT_NULL(source, "Bad DataSource type"); + + split_ = std::move(source->split_); + runtimeStats_.skippedSplits += source->runtimeStats_.skippedSplits; + runtimeStats_.processedSplits += source->runtimeStats_.processedSplits; + runtimeStats_.skippedSplitBytes += source->runtimeStats_.skippedSplitBytes; + readerOutputType_ = std::move(source->readerOutputType_); + source->scanSpec_->moveAdaptationFrom(*scanSpec_); + scanSpec_ = std::move(source->scanSpec_); + splitReader_ = std::move(source->splitReader_); + splitReader_->setConnectorQueryCtx(connectorQueryCtx_); + // New io will be accounted on the stats of 'source'. Add the existing + // balance to that. + source->ioStats_->merge(*ioStats_); + ioStats_ = std::move(source->ioStats_); + source->fsStats_->merge(*fsStats_); + fsStats_ = std::move(source->fsStats_); + + numBucketConversion_ += source->numBucketConversion_; +} + +int64_t HiveDataSource::estimatedRowSize() { + if (!splitReader_) { + return kUnknownRowSize; + } + return splitReader_->estimatedRowSize(); +} + +vector_size_t HiveDataSource::evaluateRemainingFilter(RowVectorPtr& rowVector) { + for (auto fieldIndex : multiReferencedFields_) { + LazyVector::ensureLoadedRows( + rowVector->childAt(fieldIndex), + filterRows_, + filterLazyDecoded_, + filterLazyBaseRows_); + } + uint64_t filterTimeUs{0}; + vector_size_t rowsRemaining{0}; + { + MicrosecondTimer timer(&filterTimeUs); + expressionEvaluator_->evaluate( + remainingFilterExprSet_.get(), filterRows_, *rowVector, filterResult_); + rowsRemaining = exec::processFilterResults( + filterResult_, filterRows_, filterEvalCtx_, pool_); + } + totalRemainingFilterTime_.fetch_add( + filterTimeUs * 1000, std::memory_order_relaxed); + return rowsRemaining; +} + +void HiveDataSource::resetSplit() { + split_.reset(); + splitReader_->resetSplit(); + // Keep readers around to hold adaptation. +} + +HiveDataSource::WaveDelegateHookFunction HiveDataSource::waveDelegateHook_; + +std::shared_ptr HiveDataSource::toWaveDataSource() { + VELOX_CHECK_NOT_NULL(waveDelegateHook_); + if (!waveDataSource_) { + waveDataSource_ = waveDelegateHook_( + hiveTableHandle_, + scanSpec_, + readerOutputType_, + &partitionKeys_, + fileHandleFactory_, + executor_, + connectorQueryCtx_, + hiveConfig_, + ioStats_, + remainingFilterExprSet_.get(), + metadataFilter_); + } + return waveDataSource_; +} + +// static +void HiveDataSource::registerWaveDelegateHook(WaveDelegateHookFunction hook) { + waveDelegateHook_ = hook; +} +std::shared_ptr toWaveDataSource(); + +} // namespace facebook::velox::connector::hive diff --git a/velox/connectors/lakehouse/common/HiveDataSource.h b/velox/connectors/lakehouse/common/HiveDataSource.h new file mode 100644 index 000000000000..e10228727871 --- /dev/null +++ b/velox/connectors/lakehouse/common/HiveDataSource.h @@ -0,0 +1,187 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#pragma once + +#include "velox/common/base/RandomUtil.h" +#include "velox/common/file/FileSystems.h" +#include "velox/common/io/IoStatistics.h" +#include "velox/connectors/Connector.h" +#include "velox/connectors/hive/FileHandle.h" +#include "velox/connectors/hive/HiveConnectorSplit.h" +#include "velox/connectors/hive/HiveConnectorUtil.h" +#include "velox/connectors/hive/SplitReader.h" +#include "velox/connectors/hive/TableHandle.h" +#include "velox/dwio/common/Statistics.h" +#include "velox/exec/OperatorUtils.h" +#include "velox/expression/Expr.h" + +namespace facebook::velox::connector::hive { + +class HiveConfig; + +class HiveDataSource : public DataSource { + public: + HiveDataSource( + const RowTypePtr& outputType, + const connector::ConnectorTableHandlePtr& tableHandle, + const connector::ColumnHandleMap& columnHandles, + FileHandleFactory* fileHandleFactory, + folly::Executor* executor, + const ConnectorQueryCtx* connectorQueryCtx, + const std::shared_ptr& hiveConfig); + + void addSplit(std::shared_ptr split) override; + + std::optional next(uint64_t size, velox::ContinueFuture& future) + override; + + void addDynamicFilter( + column_index_t outputChannel, + const std::shared_ptr& filter) override; + + uint64_t getCompletedBytes() override { + return ioStats_->rawBytesRead(); + } + + uint64_t getCompletedRows() override { + return completedRows_; + } + + std::unordered_map runtimeStats() override; + + bool allPrefetchIssued() const override { + return splitReader_ && splitReader_->allPrefetchIssued(); + } + + void setFromDataSource(std::unique_ptr sourceUnique) override; + + int64_t estimatedRowSize() override; + + const common::SubfieldFilters* getFilters() const override { + return &filters_; + } + + std::shared_ptr toWaveDataSource() override; + + using WaveDelegateHookFunction = + std::function( + const HiveTableHandlePtr& hiveTableHandle, + const std::shared_ptr& scanSpec, + const RowTypePtr& readerOutputType, + std::unordered_map* partitionKeys, + FileHandleFactory* fileHandleFactory, + folly::Executor* executor, + const ConnectorQueryCtx* connectorQueryCtx, + const std::shared_ptr& hiveConfig, + const std::shared_ptr& ioStats, + const exec::ExprSet* remainingFilter, + std::shared_ptr metadataFilter)>; + + static WaveDelegateHookFunction waveDelegateHook_; + + static void registerWaveDelegateHook(WaveDelegateHookFunction hook); + + const ConnectorQueryCtx* testingConnectorQueryCtx() const { + return connectorQueryCtx_; + } + + protected: + virtual std::unique_ptr createSplitReader(); + + FileHandleFactory* const fileHandleFactory_; + folly::Executor* const executor_; + const ConnectorQueryCtx* const connectorQueryCtx_; + const std::shared_ptr hiveConfig_; + memory::MemoryPool* const pool_; + + std::shared_ptr split_; + HiveTableHandlePtr hiveTableHandle_; + std::shared_ptr scanSpec_; + VectorPtr output_; + std::unique_ptr splitReader_; + + // Output type from file reader. This is different from outputType_ that it + // contains column names before assignment, and columns that only used in + // remaining filter. + RowTypePtr readerOutputType_; + + // Column handles for the partition key columns keyed on partition key column + // name. + std::unordered_map partitionKeys_; + + std::shared_ptr ioStats_; + std::shared_ptr fsStats_; + + private: + std::vector setupBucketConversion(); + + void setupRowIdColumn(); + + // Evaluates remainingFilter_ on the specified vector. Returns number of rows + // passed. Populates filterEvalCtx_.selectedIndices and selectedBits if only + // some rows passed the filter. If none or all rows passed + // filterEvalCtx_.selectedIndices and selectedBits are not updated. + vector_size_t evaluateRemainingFilter(RowVectorPtr& rowVector); + + // Clear split_ after split has been fully processed. Keep readers around to + // hold adaptation. + void resetSplit(); + + const RowVectorPtr& getEmptyOutput() { + if (!emptyOutput_) { + emptyOutput_ = RowVector::createEmpty(outputType_, pool_); + } + return emptyOutput_; + } + + // The row type for the data source output, not including filter-only columns + const RowTypePtr outputType_; + core::ExpressionEvaluator* const expressionEvaluator_; + + // Column handles for the Split info columns keyed on their column names. + std::unordered_map infoColumns_; + SpecialColumnNames specialColumns_{}; + std::vector remainingFilterSubfields_; + folly::F14FastMap> + subfields_; + common::SubfieldFilters filters_; + std::shared_ptr metadataFilter_; + std::shared_ptr remainingFilterExprSet_; + RowVectorPtr emptyOutput_; + dwio::common::RuntimeStatistics runtimeStats_; + std::atomic totalRemainingFilterTime_{0}; + uint64_t completedRows_ = 0; + + // Field indices referenced in both remaining filter and output type. These + // columns need to be materialized eagerly to avoid missing values in output. + std::vector multiReferencedFields_; + + std::shared_ptr randomSkip_; + + int64_t numBucketConversion_ = 0; + + // Reusable memory for remaining filter evaluation. + VectorPtr filterResult_; + SelectivityVector filterRows_; + DecodedVector filterLazyDecoded_; + SelectivityVector filterLazyBaseRows_; + exec::FilterEvalCtx filterEvalCtx_; + + // Remembers the WaveDataSource. Successive calls to toWaveDataSource() will + // return the same. + std::shared_ptr waveDataSource_; +}; +} // namespace facebook::velox::connector::hive diff --git a/velox/connectors/lakehouse/common/HivePartitionFunction.cpp b/velox/connectors/lakehouse/common/HivePartitionFunction.cpp new file mode 100644 index 000000000000..d273cc8163e3 --- /dev/null +++ b/velox/connectors/lakehouse/common/HivePartitionFunction.cpp @@ -0,0 +1,579 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#include "velox/connectors/hive/HivePartitionFunction.h" + +namespace facebook::velox::connector::hive { + +namespace { +void mergeHash(bool mix, uint32_t oneHash, uint32_t& aggregateHash) { + aggregateHash = mix ? aggregateHash * 31 + oneHash : oneHash; +} + +int32_t hashInt64(int64_t value) { + return ((*reinterpret_cast(&value)) >> 32) ^ value; +} + +#if defined(__has_feature) +#if __has_feature(__address_sanitizer__) +__attribute__((no_sanitize("integer"))) +#endif +#endif +uint32_t +hashBytes(StringView bytes, int32_t initialValue) { + uint32_t hash = initialValue; + auto* data = bytes.data(); + for (auto i = 0; i < bytes.size(); ++i) { + hash = hash * 31 + *reinterpret_cast(data + i); + } + return hash; +} + +int32_t hashTimestamp(const Timestamp& ts) { + return hashInt64((ts.getSeconds() << 30) | ts.getNanos()); +} + +template +inline uint32_t hashOne(typename TypeTraits::NativeType /* value */) { + VELOX_UNSUPPORTED( + "Hive partitioning function doesn't support {} type", + TypeTraits::name); + return 0; // Make compiler happy. +} + +template <> +inline uint32_t hashOne(bool value) { + return value ? 1 : 0; +} + +template <> +inline uint32_t hashOne(int8_t value) { + return static_cast(value); +} + +template <> +inline uint32_t hashOne(int16_t value) { + return static_cast(value); +} + +template <> +inline uint32_t hashOne(int32_t value) { + return static_cast(value); +} + +template <> +inline uint32_t hashOne(float value) { + return static_cast(*reinterpret_cast(&value)); +} + +template <> +inline uint32_t hashOne(int64_t value) { + return hashInt64(value); +} + +template <> +inline uint32_t hashOne(double value) { + return hashInt64(*reinterpret_cast(&value)); +} + +template <> +inline uint32_t hashOne(StringView value) { + return hashBytes(value, 0); +} + +template <> +inline uint32_t hashOne(StringView value) { + return hashBytes(value, 0); +} + +template <> +inline uint32_t hashOne(Timestamp value) { + return hashTimestamp(value); +} + +template <> +inline uint32_t hashOne(UnknownValue /*value*/) { + VELOX_FAIL("Unknown values cannot be non-NULL"); +} + +template +void hashPrimitive( + const DecodedVector& values, + const SelectivityVector& rows, + bool mix, + std::vector& hashes) { + if (rows.isAllSelected()) { + // The compiler seems to be a little fickle with optimizations. + // Although rows.applyToSelected should do roughly the same thing, doing + // this here along with assigning rows.size() to a variable seems to help + // the compiler to inline hashOne showing a 50% performance improvement in + // benchmarks. + vector_size_t numRows = rows.size(); + for (auto i = 0; i < numRows; ++i) { + const uint32_t hash = values.isNullAt(i) + ? 0 + : hashOne( + values.valueAt::NativeType>(i)); + mergeHash(mix, hash, hashes[i]); + } + } else { + rows.applyToSelected([&](auto row) INLINE_LAMBDA { + const uint32_t hash = values.isNullAt(row) + ? 0 + : hashOne( + values.valueAt::NativeType>(row)); + mergeHash(mix, hash, hashes[row]); + }); + } +} + +void hashPrecomputed( + uint32_t precomputedHash, + vector_size_t numRows, + bool mix, + std::vector& hashes) { + for (auto i = 0; i < numRows; ++i) { + hashes[i] = mix ? hashes[i] * 31 + precomputedHash : precomputedHash; + } +} +} // namespace + +template <> +void HivePartitionFunction::hashTyped( + const DecodedVector& values, + const SelectivityVector& rows, + bool mix, + std::vector& hashes, + size_t /* poolIndex */) { + hashPrimitive(values, rows, mix, hashes); +} + +template <> +void HivePartitionFunction::hashTyped( + const DecodedVector& values, + const SelectivityVector& rows, + bool mix, + std::vector& hashes, + size_t /* poolIndex */) { + hashPrimitive(values, rows, mix, hashes); +} + +template <> +void HivePartitionFunction::hashTyped( + const DecodedVector& values, + const SelectivityVector& rows, + bool mix, + std::vector& hashes, + size_t /* poolIndex */) { + hashPrimitive(values, rows, mix, hashes); +} + +template <> +void HivePartitionFunction::hashTyped( + const DecodedVector& values, + const SelectivityVector& rows, + bool mix, + std::vector& hashes, + size_t /* poolIndex */) { + hashPrimitive(values, rows, mix, hashes); +} + +template <> +void HivePartitionFunction::hashTyped( + const DecodedVector& values, + const SelectivityVector& rows, + bool mix, + std::vector& hashes, + size_t /* poolIndex */) { + hashPrimitive(values, rows, mix, hashes); +} + +template <> +void HivePartitionFunction::hashTyped( + const DecodedVector& values, + const SelectivityVector& rows, + bool mix, + std::vector& hashes, + size_t /* poolIndex */) { + hashPrimitive(values, rows, mix, hashes); +} + +template <> +void HivePartitionFunction::hashTyped( + const DecodedVector& values, + const SelectivityVector& rows, + bool mix, + std::vector& hashes, + size_t /* poolIndex */) { + hashPrimitive(values, rows, mix, hashes); +} + +template <> +void HivePartitionFunction::hashTyped( + const DecodedVector& values, + const SelectivityVector& rows, + bool mix, + std::vector& hashes, + size_t /* poolIndex */) { + hashPrimitive(values, rows, mix, hashes); +} + +template <> +void HivePartitionFunction::hashTyped( + const DecodedVector& values, + const SelectivityVector& rows, + bool mix, + std::vector& hashes, + size_t /* poolIndex */) { + hashPrimitive(values, rows, mix, hashes); +} + +template <> +void HivePartitionFunction::hashTyped( + const DecodedVector& values, + const SelectivityVector& rows, + bool mix, + std::vector& hashes, + size_t /* poolIndex */) { + hashPrimitive(values, rows, mix, hashes); +} + +template <> +void HivePartitionFunction::hashTyped( + const DecodedVector& values, + const SelectivityVector& rows, + bool mix, + std::vector& hashes, + size_t /* poolIndex */) { + hashPrimitive(values, rows, mix, hashes); +} + +template <> +void HivePartitionFunction::hashTyped( + const DecodedVector& /*values*/, + const SelectivityVector& /*rows*/, + bool /*mix*/, + std::vector& /*hashes*/, + size_t /* poolIndex */) { + VELOX_UNSUPPORTED("Hive partitioning function doesn't support OPAQUE type"); +} + +template <> +void HivePartitionFunction::hashTyped( + const DecodedVector& values, + const SelectivityVector& rows, + bool mix, + std::vector& hashes, + size_t poolIndex) { + auto& elementsDecoded = getDecodedVector(poolIndex); + auto& elementsRows = getRows(poolIndex); + auto& elementsHashes = getHashes(poolIndex); + + const auto* arrayVector = values.base()->as(); + const vector_size_t elementsSize = arrayVector->elements()->size(); + elementsRows.resizeFill(elementsSize, false); + elementsHashes.resize(elementsSize); + + rows.applyToSelected([&](auto row) { + if (!values.isNullAt(row)) { + const auto index = values.index(row); + const auto offset = arrayVector->offsetAt(index); + const auto length = arrayVector->sizeAt(index); + + elementsRows.setValidRange(offset, offset + length, true); + } + }); + + elementsRows.updateBounds(); + + elementsDecoded.decode(*arrayVector->elements(), elementsRows); + + hash( + elementsDecoded, + elementsDecoded.base()->typeKind(), + elementsRows, + false, + elementsHashes, + poolIndex + 1); + + rows.applyToSelected([&](auto row) { + uint32_t hash = 0; + + if (!values.isNullAt(row)) { + const auto index = values.index(row); + const auto offset = arrayVector->offsetAt(index); + const auto length = arrayVector->sizeAt(index); + + for (size_t i = offset; i < offset + length; ++i) { + mergeHash(true, elementsHashes[i], hash); + } + } + + mergeHash(mix, hash, hashes[row]); + }); +} + +template <> +void HivePartitionFunction::hashTyped( + const DecodedVector& values, + const SelectivityVector& rows, + bool mix, + std::vector& hashes, + size_t poolIndex) { + auto& valuesDecoded = getDecodedVector(poolIndex); + auto& keysDecoded = getDecodedVector(poolIndex + 1); + auto& elementsRows = getRows(poolIndex); + auto& valuesHashes = getHashes(poolIndex); + auto& keysHashes = getHashes(poolIndex + 1); + + const auto* mapVector = values.base()->as(); + const vector_size_t elementsSize = mapVector->mapKeys()->size(); + elementsRows.resizeFill(elementsSize, false); + keysHashes.resize(elementsSize); + valuesHashes.resize(elementsSize); + + rows.applyToSelected([&](auto row) { + if (!values.isNullAt(row)) { + const auto index = values.index(row); + const auto offset = mapVector->offsetAt(index); + const auto length = mapVector->sizeAt(index); + + elementsRows.setValidRange(offset, offset + length, true); + } + }); + + elementsRows.updateBounds(); + + keysDecoded.decode(*mapVector->mapKeys(), elementsRows); + valuesDecoded.decode(*mapVector->mapValues(), elementsRows); + + hash( + keysDecoded, + keysDecoded.base()->typeKind(), + elementsRows, + false, + keysHashes, + poolIndex + 2); + + hash( + valuesDecoded, + valuesDecoded.base()->typeKind(), + elementsRows, + false, + valuesHashes, + poolIndex + 2); + + rows.applyToSelected([&](auto row) { + uint32_t hash = 0; + + if (!values.isNullAt(row)) { + const auto index = values.index(row); + const auto offset = mapVector->offsetAt(index); + const auto length = mapVector->sizeAt(index); + + for (size_t i = offset; i < offset + length; ++i) { + hash += keysHashes[i] ^ valuesHashes[i]; + } + } + + mergeHash(mix, hash, hashes[row]); + }); +} + +template <> +void HivePartitionFunction::hashTyped( + const DecodedVector& values, + const SelectivityVector& rows, + bool mix, + std::vector& hashes, + size_t poolIndex) { + auto& childDecodedVector = getDecodedVector(poolIndex); + auto& childRows = getRows(poolIndex); + auto& childHashes = getHashes(poolIndex); + + const auto* rowVector = values.base()->as(); + childRows.resizeFill(rowVector->size(), false); + childHashes.resize(rowVector->size()); + + rows.applyToSelected([&](auto row) { + if (!values.isNullAt(row)) { + childRows.setValid(values.index(row), true); + } + }); + + childRows.updateBounds(); + + for (vector_size_t i = 0; i < rowVector->childrenSize(); ++i) { + auto& child = rowVector->childAt(i); + childDecodedVector.decode(*child, childRows); + hash( + childDecodedVector, + child->typeKind(), + childRows, + i > 0, + childHashes, + poolIndex + 1); + } + + rows.applyToSelected([&](auto row) { + mergeHash( + mix, + values.isNullAt(row) ? 0 : childHashes[values.index(row)], + hashes[row]); + }); +} + +void HivePartitionFunction::hash( + const DecodedVector& values, + TypeKind typeKind, + const SelectivityVector& rows, + bool mix, + std::vector& hashes, + size_t poolIndex) { + // This function mirrors the behavior of function hashCode in + // HIVE-12025 ba83fd7bff + // serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspectorUtils.java + // https://github.com/apache/hive/blob/ba83fd7bff/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspectorUtils.java + + // HIVE-7148 proposed change to bucketing hash algorithms. If that + // gets implemented, this function will need to change + // significantly. + + VELOX_DYNAMIC_TYPE_DISPATCH_ALL( + hashTyped, typeKind, values, rows, mix, hashes, poolIndex); +} + +HivePartitionFunction::HivePartitionFunction( + int numBuckets, + std::vector bucketToPartition, + std::vector keyChannels, + const std::vector& constValues) + : numBuckets_{numBuckets}, + bucketToPartition_{bucketToPartition}, + keyChannels_{std::move(keyChannels)} { + precomputedHashes_.resize(keyChannels_.size()); + size_t constChannel{0}; + for (auto i = 0; i < keyChannels_.size(); ++i) { + if (keyChannels_[i] == kConstantChannel) { + precompute(*(constValues[constChannel++]), i); + } + } +} + +std::optional HivePartitionFunction::partition( + const RowVector& input, + std::vector& partitions) { + const auto numRows = input.size(); + + auto& decodedVector = getDecodedVector(); + auto& rows = getRows(); + auto& hashes = getHashes(); + rows.resizeFill(numRows, true); + if (numRows > hashes.size()) { + hashes.resize(numRows); + } + partitions.resize(numRows); + for (auto i = 0; i < keyChannels_.size(); ++i) { + if (keyChannels_[i] != kConstantChannel) { + const auto& keyVector = input.childAt(keyChannels_[i]); + decodedVector.decode(*keyVector, rows); + hash(decodedVector, keyVector->typeKind(), rows, i > 0, hashes, 1); + } else { + hashPrecomputed(precomputedHashes_[i], numRows, i > 0, hashes); + } + } + + static const int32_t kInt32Max = std::numeric_limits::max(); + + if (bucketToPartition_.empty()) { + // NOTE: if bucket to partition mapping is empty, then we do + // identical mapping. + for (auto i = 0; i < numRows; ++i) { + partitions[i] = (hashes[i] & kInt32Max) % numBuckets_; + } + } else { + for (auto i = 0; i < numRows; ++i) { + partitions[i] = + bucketToPartition_[((hashes[i] & kInt32Max) % numBuckets_)]; + } + } + + return std::nullopt; +} + +void HivePartitionFunction::precompute( + const BaseVector& value, + size_t channelIndex) { + if (value.isNullAt(0)) { + precomputedHashes_[channelIndex] = 0; + return; + } + + const SelectivityVector rows(1, true); + DecodedVector& decodedVector = getDecodedVector(); + decodedVector.decode(value, rows); + + std::vector hashes{1}; + hash(decodedVector, value.typeKind(), rows, false, hashes, 1); + precomputedHashes_[channelIndex] = hashes[0]; +} + +DecodedVector& HivePartitionFunction::getDecodedVector(size_t poolIndex) { + while (poolIndex >= decodedVectorsPool_.size()) { + decodedVectorsPool_.push_back(std::make_unique()); + } + + return *decodedVectorsPool_[poolIndex]; +} + +SelectivityVector& HivePartitionFunction::getRows(size_t poolIndex) { + while (poolIndex >= rowsPool_.size()) { + rowsPool_.push_back(std::make_unique()); + } + + return *rowsPool_[poolIndex]; +} + +std::vector& HivePartitionFunction::getHashes(size_t poolIndex) { + while (poolIndex >= hashesPool_.size()) { + hashesPool_.push_back(std::make_unique>()); + } + + return *hashesPool_[poolIndex]; +} + +HiveIdentityPartitionFunction::HiveIdentityPartitionFunction( + int numBuckets, + column_index_t keyChannel) + : numBuckets_(numBuckets), keyChannel_(keyChannel) {} + +std::optional HiveIdentityPartitionFunction::partition( + const RowVector& input, + std::vector& partitions) { + const auto& keyVector = input.childAt(keyChannel_); + VELOX_CHECK( + keyVector->typeKind() == TypeKind::BIGINT, + "Only BIGINT type is supported for identity partition function"); + decodedVector_->decode(*keyVector); + + const auto numRows = input.size(); + partitions.resize(numRows); + for (auto i = 0; i < numRows; i++) { + partitions[i] = decodedVector_->valueAt(i) % numBuckets_; + } + return std::nullopt; +} + +} // namespace facebook::velox::connector::hive diff --git a/velox/connectors/lakehouse/common/HivePartitionFunction.h b/velox/connectors/lakehouse/common/HivePartitionFunction.h new file mode 100644 index 000000000000..c44d200a128d --- /dev/null +++ b/velox/connectors/lakehouse/common/HivePartitionFunction.h @@ -0,0 +1,109 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#pragma once + +#include "velox/core/PlanNode.h" +#include "velox/vector/DecodedVector.h" + +namespace facebook::velox::connector::hive { + +class HivePartitionFunction : public core::PartitionFunction { + public: + HivePartitionFunction( + int numBuckets, + std::vector bucketToPartition, + std::vector keyChannels, + const std::vector& constValues = {}); + + HivePartitionFunction( + int numBuckets, + std::vector keyChannels, + const std::vector& constValues = {}) + : HivePartitionFunction( + numBuckets, + {}, + std::move(keyChannels), + constValues) {} + + ~HivePartitionFunction() override = default; + + std::optional partition( + const RowVector& input, + std::vector& partitions) override; + + const std::vector& testingBucketToPartition() const { + return bucketToPartition_; + } + + private: + // Precompute single value hive hash for a constant partition key. + void precompute(const BaseVector& value, size_t column_index_t); + + void hash( + const DecodedVector& values, + TypeKind typeKind, + const SelectivityVector& rows, + bool mix, + std::vector& hashes, + size_t poolIndex); + + template + void hashTyped( + const DecodedVector& /* values */, + const SelectivityVector& /* rows */, + bool /* mix */, + std::vector& /* hashes */, + size_t /* poolIndex */) { + VELOX_UNSUPPORTED( + "Hive partitioning function doesn't support {} type", + TypeTraits::name); + } + + // Helper functions to retrieve reusable memory from pools. + DecodedVector& getDecodedVector(size_t poolIndex = 0); + SelectivityVector& getRows(size_t poolIndex = 0); + std::vector& getHashes(size_t poolIndex = 0); + + const int numBuckets_; + const std::vector bucketToPartition_; + const std::vector keyChannels_; + + // Pools of reusable memory. + std::vector>> hashesPool_; + std::vector> rowsPool_; + std::vector> decodedVectorsPool_; + // Precomputed hashes for constant partition keys (one per key). + std::vector precomputedHashes_; +}; + +// PartitionFunction implementation which uses the value extracted +// from a column channel as the partition index. +class HiveIdentityPartitionFunction : public core::PartitionFunction { + public: + HiveIdentityPartitionFunction(int numBuckets, column_index_t keyChannel); + + std::optional partition( + const RowVector& input, + std::vector& partitions) override; + + private: + const int numBuckets_; + const column_index_t keyChannel_; + std::unique_ptr decodedVector_ = + std::make_unique(); +}; + +} // namespace facebook::velox::connector::hive diff --git a/velox/connectors/lakehouse/common/HivePartitionUtil.cpp b/velox/connectors/lakehouse/common/HivePartitionUtil.cpp new file mode 100644 index 000000000000..acf6f4c89eea --- /dev/null +++ b/velox/connectors/lakehouse/common/HivePartitionUtil.cpp @@ -0,0 +1,119 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "velox/connectors/hive/HivePartitionUtil.h" +#include "velox/vector/SimpleVector.h" + +namespace facebook::velox::connector::hive { + +#define PARTITION_TYPE_DISPATCH(TEMPLATE_FUNC, typeKind, ...) \ + [&]() { \ + switch (typeKind) { \ + case TypeKind::BOOLEAN: \ + case TypeKind::TINYINT: \ + case TypeKind::SMALLINT: \ + case TypeKind::INTEGER: \ + case TypeKind::BIGINT: \ + case TypeKind::VARCHAR: \ + case TypeKind::VARBINARY: \ + case TypeKind::TIMESTAMP: \ + return VELOX_DYNAMIC_SCALAR_TYPE_DISPATCH( \ + TEMPLATE_FUNC, typeKind, __VA_ARGS__); \ + default: \ + VELOX_UNSUPPORTED( \ + "Unsupported partition type: {}", mapTypeKindToName(typeKind)); \ + } \ + }() + +namespace { +template +inline std::string makePartitionValueString(T value) { + return folly::to(value); +} + +template <> +inline std::string makePartitionValueString(bool value) { + return value ? "true" : "false"; +} + +template <> +inline std::string makePartitionValueString(Timestamp value) { + value.toTimezone(Timestamp::defaultTimezone()); + TimestampToStringOptions options; + options.dateTimeSeparator = ' '; + // Set the precision to milliseconds, and enable the skipTrailingZeros match + // the timestamp precision and truncation behavior of Presto. + options.precision = TimestampPrecision::kMilliseconds; + options.skipTrailingZeros = true; + + auto result = value.toString(options); + + // Presto's java.sql.Timestamp.toString() always keeps at least one decimal + // place even when all fractional seconds are zero. + // If skipTrailingZeros removed all fractional digits, add back ".0" to match + // Presto's behavior. + if (auto dotPos = result.find_last_of('.'); dotPos == std::string::npos) { + // No decimal point found, add ".0" + result += ".0"; + } + + return result; +} + +template +std::pair makePartitionKeyValueString( + const BaseVector* partitionVector, + vector_size_t row, + const std::string& name, + bool isDate, + const std::string& nullValueName) { + using T = typename TypeTraits::NativeType; + if (partitionVector->as>()->isNullAt(row)) { + return std::make_pair(name, nullValueName); + } + if (isDate) { + return std::make_pair( + name, + DATE()->toString( + partitionVector->as>()->valueAt(row))); + } + return std::make_pair( + name, + makePartitionValueString( + partitionVector->as>()->valueAt(row))); +} + +} // namespace + +std::vector> extractPartitionKeyValues( + const RowVectorPtr& partitionsVector, + vector_size_t row, + const std::string& nullValueName) { + std::vector> partitionKeyValues; + for (auto i = 0; i < partitionsVector->childrenSize(); i++) { + partitionKeyValues.push_back(PARTITION_TYPE_DISPATCH( + makePartitionKeyValueString, + partitionsVector->childAt(i)->typeKind(), + partitionsVector->childAt(i)->loadedVector(), + row, + asRowType(partitionsVector->type())->nameOf(i), + partitionsVector->childAt(i)->type()->isDate(), + nullValueName)); + } + return partitionKeyValues; +} + +} // namespace facebook::velox::connector::hive diff --git a/velox/connectors/lakehouse/common/HivePartitionUtil.h b/velox/connectors/lakehouse/common/HivePartitionUtil.h new file mode 100644 index 000000000000..3ce10dbef705 --- /dev/null +++ b/velox/connectors/lakehouse/common/HivePartitionUtil.h @@ -0,0 +1,27 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#pragma once + +#include "velox/vector/ComplexVector.h" + +namespace facebook::velox::connector::hive { + +std::vector> extractPartitionKeyValues( + const RowVectorPtr& partitionsVector, + vector_size_t row, + const std::string& nullValueName = ""); + +} // namespace facebook::velox::connector::hive diff --git a/velox/connectors/lakehouse/common/PartitionIdGenerator.cpp b/velox/connectors/lakehouse/common/PartitionIdGenerator.cpp new file mode 100644 index 000000000000..d412456a17be --- /dev/null +++ b/velox/connectors/lakehouse/common/PartitionIdGenerator.cpp @@ -0,0 +1,185 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "velox/connectors/hive/PartitionIdGenerator.h" + +#include "velox/connectors/hive/HivePartitionUtil.h" +#include "velox/dwio/catalog/fbhive/FileUtils.h" + +using namespace facebook::velox::dwio::catalog::fbhive; + +namespace facebook::velox::connector::hive { + +PartitionIdGenerator::PartitionIdGenerator( + const RowTypePtr& inputType, + std::vector partitionChannels, + uint32_t maxPartitions, + memory::MemoryPool* pool, + bool partitionPathAsLowerCase) + : pool_(pool), + partitionChannels_(std::move(partitionChannels)), + maxPartitions_(maxPartitions), + partitionPathAsLowerCase_(partitionPathAsLowerCase) { + VELOX_USER_CHECK( + !partitionChannels_.empty(), "There must be at least one partition key."); + for (auto channel : partitionChannels_) { + hashers_.emplace_back( + exec::VectorHasher::create(inputType->childAt(channel), channel)); + } + + std::vector partitionKeyTypes; + std::vector partitionKeyNames; + for (auto channel : partitionChannels_) { + VELOX_USER_CHECK( + exec::VectorHasher::typeKindSupportsValueIds( + inputType->childAt(channel)->kind()), + "Unsupported partition type: {}.", + inputType->childAt(channel)->toString()); + partitionKeyTypes.push_back(inputType->childAt(channel)); + partitionKeyNames.push_back(inputType->nameOf(channel)); + } + + partitionValues_ = BaseVector::create( + ROW(std::move(partitionKeyNames), std::move(partitionKeyTypes)), + maxPartitions_, + pool); + for (auto& key : partitionValues_->children()) { + key->resize(maxPartitions_); + } +} + +void PartitionIdGenerator::run( + const RowVectorPtr& input, + raw_vector& result) { + const auto numRows = input->size(); + result.resize(numRows); + + // Compute value IDs using VectorHashers and store these in 'result'. + computeValueIds(input, result); + + // Convert value IDs in 'result' into partition IDs using partitionIds + // mapping. Update 'result' in place. + + // TODO Optimize common use case where all records belong to the same + // partition. VectorHashers keep track of the number of unique values, hence, + // we can find out if there is only one unique value for each partition key. + for (auto i = 0; i < numRows; ++i) { + auto valueId = result[i]; + auto it = partitionIds_.find(valueId); + if (it != partitionIds_.end()) { + result[i] = it->second; + } else { + uint64_t nextPartitionId = partitionIds_.size(); + VELOX_USER_CHECK_LT( + nextPartitionId, + maxPartitions_, + "Exceeded limit of {} distinct partitions.", + maxPartitions_); + + partitionIds_.emplace(valueId, nextPartitionId); + savePartitionValues(nextPartitionId, input, i); + + result[i] = nextPartitionId; + } + } +} + +std::string PartitionIdGenerator::partitionName( + uint64_t partitionId, + const std::string& nullValueName) const { + return FileUtils::makePartName( + extractPartitionKeyValues(partitionValues_, partitionId, nullValueName), + partitionPathAsLowerCase_); +} + +void PartitionIdGenerator::computeValueIds( + const RowVectorPtr& input, + raw_vector& valueIds) { + allRows_.resize(input->size()); + allRows_.setAll(); + + bool rehash = false; + for (auto& hasher : hashers_) { + // NOTE: for boolean column type, computeValueIds() always returns true and + // this might cause problem in case of multiple boolean partition columns as + // we might not set the multiplier properly. + auto partitionVector = input->childAt(hasher->channel())->loadedVector(); + hasher->decode(*partitionVector, allRows_); + if (!hasher->computeValueIds(allRows_, valueIds)) { + rehash = true; + } + } + + if (!rehash && hasMultiplierSet_) { + return; + } + + uint64_t multiplier = 1; + for (auto& hasher : hashers_) { + hasMultiplierSet_ = true; + multiplier = hasher->typeKind() == TypeKind::BOOLEAN + ? hasher->enableValueRange(multiplier, 50) + : hasher->enableValueIds(multiplier, 50); + + VELOX_CHECK_NE( + multiplier, + exec::VectorHasher::kRangeTooLarge, + "Number of requested IDs is out of range."); + } + + for (auto& hasher : hashers_) { + const bool ok = hasher->computeValueIds(allRows_, valueIds); + VELOX_CHECK(ok); + } + + updateValueToPartitionIdMapping(); +} + +void PartitionIdGenerator::updateValueToPartitionIdMapping() { + if (partitionIds_.empty()) { + return; + } + + const auto numPartitions = partitionIds_.size(); + + partitionIds_.clear(); + + raw_vector newValueIds(numPartitions, pool_); + SelectivityVector rows(numPartitions); + for (auto i = 0; i < hashers_.size(); ++i) { + auto& hasher = hashers_[i]; + hasher->decode(*partitionValues_->childAt(i), rows); + const bool ok = hasher->computeValueIds(rows, newValueIds); + VELOX_CHECK(ok); + } + + for (auto i = 0; i < numPartitions; ++i) { + partitionIds_.emplace(newValueIds[i], i); + } +} + +void PartitionIdGenerator::savePartitionValues( + uint64_t partitionId, + const RowVectorPtr& input, + vector_size_t row) { + for (auto i = 0; i < partitionChannels_.size(); ++i) { + auto channel = partitionChannels_[i]; + partitionValues_->childAt(i)->copy( + input->childAt(channel).get(), partitionId, row, 1); + } +} + +} // namespace facebook::velox::connector::hive diff --git a/velox/connectors/lakehouse/common/PartitionIdGenerator.h b/velox/connectors/lakehouse/common/PartitionIdGenerator.h new file mode 100644 index 000000000000..b9474957107f --- /dev/null +++ b/velox/connectors/lakehouse/common/PartitionIdGenerator.h @@ -0,0 +1,102 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#pragma once + +#include "velox/exec/VectorHasher.h" + +namespace facebook::velox::connector::hive { +/// Generate sequential integer IDs for distinct partition values, which could +/// be used as vector index. +class PartitionIdGenerator { + public: + /// @param inputType RowType of the input. + /// @param partitionChannels Channels of partition keys in the input + /// RowVector. + /// @param maxPartitions The max number of distinct partitions. + /// @param pool Memory pool. Used to allocate memory for storing unique + /// partition key values. + /// @param partitionPathAsLowerCase Used to control whether the partition path + /// need to convert to lower case. + PartitionIdGenerator( + const RowTypePtr& inputType, + std::vector partitionChannels, + uint32_t maxPartitions, + memory::MemoryPool* pool, + bool partitionPathAsLowerCase); + + /// Generate sequential partition IDs for input vector. + /// @param input Input RowVector. + /// @param result Generated integer IDs indexed by input row number. + void run(const RowVectorPtr& input, raw_vector& result); + + /// Return the total number of distinct partitions processed so far. + uint64_t numPartitions() const { + return partitionIds_.size(); + } + + /// Return partition name for the given partition id in the typical Hive + /// style. It is derived from the partitionValues_ at index partitionId. + /// Partition keys appear in the order of partition columns in the table + /// schema. + std::string partitionName( + uint64_t partitionId, + const std::string& nullValueName = "") const; + + private: + static constexpr const int32_t kHasherReservePct = 20; + + // Computes value IDs using VectorHashers for all rows in 'input'. + void computeValueIds( + const RowVectorPtr& input, + raw_vector& valueIds); + + // In case of rehash (when value IDs produced by VectorHashers change), we + // update value id for pre-existing partitions while keeping partition ids. + // This method rebuilds 'partitionIds_' by re-calculating the value ids using + // updated 'hashers_'. + void updateValueToPartitionIdMapping(); + + // Copies partition values of 'row' from 'input' into 'partitionId' row in + // 'partitionValues_'. + void savePartitionValues( + uint64_t partitionId, + const RowVectorPtr& input, + vector_size_t row); + + memory::MemoryPool* const pool_; + + const std::vector partitionChannels_; + + const uint32_t maxPartitions_; + + const bool partitionPathAsLowerCase_; + + std::vector> hashers_; + bool hasMultiplierSet_ = false; + + // A mapping from value ID produced by VectorHashers to a partition ID. + std::unordered_map partitionIds_; + + // A vector holding unique partition key values. One row per partition. Row + // numbers match partition IDs. + RowVectorPtr partitionValues_; + + // All rows are set valid to compute partition IDs for all input rows. + SelectivityVector allRows_; +}; + +} // namespace facebook::velox::connector::hive diff --git a/velox/connectors/lakehouse/common/SplitReader.cpp b/velox/connectors/lakehouse/common/SplitReader.cpp new file mode 100644 index 000000000000..e68b66b4f8d7 --- /dev/null +++ b/velox/connectors/lakehouse/common/SplitReader.cpp @@ -0,0 +1,483 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "velox/connectors/hive/SplitReader.h" + +#include "velox/common/caching/CacheTTLController.h" +#include "velox/connectors/hive/HiveConfig.h" +#include "velox/connectors/hive/HiveConnectorSplit.h" +#include "velox/connectors/hive/HiveConnectorUtil.h" +#include "velox/connectors/hive/TableHandle.h" +#include "velox/connectors/hive/iceberg/IcebergSplitReader.h" +#include "velox/dwio/common/ReaderFactory.h" + +namespace facebook::velox::connector::hive { +namespace { + +template +VectorPtr newConstantFromString( + const TypePtr& type, + const std::optional& value, + vector_size_t size, + velox::memory::MemoryPool* pool, + const std::string& sessionTimezone, + bool asLocalTime, + bool isPartitionDateDaysSinceEpoch = false) { + using T = typename TypeTraits::NativeType; + if (!value.has_value()) { + return std::make_shared>(pool, size, true, type, T()); + } + + if (type->isDate()) { + int32_t days = 0; + // For Iceberg, the date partition values are already in daysSinceEpoch + // form. + if (isPartitionDateDaysSinceEpoch) { + days = folly::to(value.value()); + } else { + days = DATE()->toDays(static_cast(value.value())); + } + return std::make_shared>( + pool, size, false, type, std::move(days)); + } + + if constexpr (std::is_same_v) { + return std::make_shared>( + pool, size, false, type, StringView(value.value())); + } else { + auto copy = velox::util::Converter::tryCast(value.value()) + .thenOrThrow(folly::identity, [&](const Status& status) { + VELOX_USER_FAIL("{}", status.message()); + }); + if constexpr (kind == TypeKind::TIMESTAMP) { + if (asLocalTime) { + copy.toGMT(Timestamp::defaultTimezone()); + } + } + return std::make_shared>( + pool, size, false, type, std::move(copy)); + } +} +} // namespace + +std::unique_ptr SplitReader::create( + const std::shared_ptr& hiveSplit, + const HiveTableHandlePtr& hiveTableHandle, + const std::unordered_map* partitionKeys, + const ConnectorQueryCtx* connectorQueryCtx, + const std::shared_ptr& hiveConfig, + const RowTypePtr& readerOutputType, + const std::shared_ptr& ioStats, + const std::shared_ptr& fsStats, + FileHandleFactory* fileHandleFactory, + folly::Executor* executor, + const std::shared_ptr& scanSpec, + core::ExpressionEvaluator* expressionEvaluator, + std::atomic& totalRemainingFilterTime) { + // Create the SplitReader based on hiveSplit->customSplitInfo["table_format"] + if (hiveSplit->customSplitInfo.count("table_format") > 0 && + hiveSplit->customSplitInfo["table_format"] == "hive-iceberg") { + return std::make_unique( + hiveSplit, + hiveTableHandle, + partitionKeys, + connectorQueryCtx, + hiveConfig, + readerOutputType, + ioStats, + fsStats, + fileHandleFactory, + executor, + scanSpec, + expressionEvaluator, + totalRemainingFilterTime); + } else { + return std::unique_ptr(new SplitReader( + hiveSplit, + hiveTableHandle, + partitionKeys, + connectorQueryCtx, + hiveConfig, + readerOutputType, + ioStats, + fsStats, + fileHandleFactory, + executor, + scanSpec)); + } +} + +SplitReader::SplitReader( + const std::shared_ptr& hiveSplit, + const HiveTableHandlePtr& hiveTableHandle, + const std::unordered_map* partitionKeys, + const ConnectorQueryCtx* connectorQueryCtx, + const std::shared_ptr& hiveConfig, + const RowTypePtr& readerOutputType, + const std::shared_ptr& ioStats, + const std::shared_ptr& fsStats, + FileHandleFactory* fileHandleFactory, + folly::Executor* executor, + const std::shared_ptr& scanSpec) + : hiveSplit_(hiveSplit), + hiveTableHandle_(hiveTableHandle), + partitionKeys_(partitionKeys), + connectorQueryCtx_(connectorQueryCtx), + hiveConfig_(hiveConfig), + readerOutputType_(readerOutputType), + ioStats_(ioStats), + fsStats_(fsStats), + fileHandleFactory_(fileHandleFactory), + executor_(executor), + pool_(connectorQueryCtx->memoryPool()), + scanSpec_(scanSpec), + baseReaderOpts_(connectorQueryCtx->memoryPool()), + emptySplit_(false) {} + +void SplitReader::configureReaderOptions( + std::shared_ptr randomSkip) { + hive::configureReaderOptions( + hiveConfig_, + connectorQueryCtx_, + hiveTableHandle_, + hiveSplit_, + baseReaderOpts_); + baseReaderOpts_.setRandomSkip(std::move(randomSkip)); + baseReaderOpts_.setScanSpec(scanSpec_); + baseReaderOpts_.setFileFormat(hiveSplit_->fileFormat); +} + +void SplitReader::prepareSplit( + std::shared_ptr metadataFilter, + dwio::common::RuntimeStatistics& runtimeStats) { + createReader(); + if (emptySplit_) { + return; + } + auto rowType = getAdaptedRowType(); + + if (checkIfSplitIsEmpty(runtimeStats)) { + VELOX_CHECK(emptySplit_); + return; + } + + createRowReader(std::move(metadataFilter), std::move(rowType)); +} + +void SplitReader::setBucketConversion( + std::vector bucketChannels) { + bucketChannels_ = {bucketChannels.begin(), bucketChannels.end()}; + partitionFunction_ = std::make_unique( + hiveSplit_->bucketConversion->tableBucketCount, + std::move(bucketChannels)); +} + +std::vector SplitReader::bucketConversionRows( + const RowVector& vector) { + partitions_.clear(); + partitionFunction_->partition(vector, partitions_); + const auto bucketToKeep = *hiveSplit_->tableBucketNumber; + const auto partitionBucketCount = + hiveSplit_->bucketConversion->partitionBucketCount; + std::vector ranges; + for (vector_size_t i = 0; i < vector.size(); ++i) { + VELOX_CHECK_EQ((partitions_[i] - bucketToKeep) % partitionBucketCount, 0); + if (partitions_[i] == bucketToKeep) { + auto& r = ranges.emplace_back(); + r.sourceIndex = i; + r.targetIndex = ranges.size() - 1; + r.count = 1; + } + } + return ranges; +} + +void SplitReader::applyBucketConversion( + VectorPtr& output, + const std::vector& ranges) { + auto filtered = + BaseVector::create(output->type(), ranges.size(), output->pool()); + filtered->copyRanges(output.get(), ranges); + output = std::move(filtered); +} + +uint64_t SplitReader::next(uint64_t size, VectorPtr& output) { + uint64_t numScanned; + if (!baseReaderOpts_.randomSkip()) { + numScanned = baseRowReader_->next(size, output); + } else { + dwio::common::Mutation mutation; + mutation.randomSkip = baseReaderOpts_.randomSkip().get(); + numScanned = baseRowReader_->next(size, output, &mutation); + } + if (numScanned > 0 && output->size() > 0 && partitionFunction_) { + applyBucketConversion( + output, bucketConversionRows(*output->asChecked())); + } + return numScanned; +} + +void SplitReader::resetFilterCaches() { + if (baseRowReader_) { + baseRowReader_->resetFilterCaches(); + } +} + +bool SplitReader::emptySplit() const { + return emptySplit_; +} + +void SplitReader::resetSplit() { + hiveSplit_.reset(); +} + +int64_t SplitReader::estimatedRowSize() const { + if (!baseRowReader_) { + return DataSource::kUnknownRowSize; + } + + const auto size = baseRowReader_->estimatedRowSize(); + return size.value_or(DataSource::kUnknownRowSize); +} + +void SplitReader::updateRuntimeStats( + dwio::common::RuntimeStatistics& stats) const { + if (baseRowReader_) { + baseRowReader_->updateRuntimeStats(stats); + } +} + +bool SplitReader::allPrefetchIssued() const { + return baseRowReader_ && baseRowReader_->allPrefetchIssued(); +} + +void SplitReader::setConnectorQueryCtx( + const ConnectorQueryCtx* connectorQueryCtx) { + connectorQueryCtx_ = connectorQueryCtx; +} + +std::string SplitReader::toString() const { + std::string partitionKeys; + std::for_each( + partitionKeys_->begin(), partitionKeys_->end(), [&](const auto& column) { + partitionKeys += " " + column.second->toString(); + }); + return fmt::format( + "SplitReader: hiveSplit_{} scanSpec_{} readerOutputType_{} partitionKeys_{} reader{} rowReader{}", + hiveSplit_->toString(), + scanSpec_->toString(), + readerOutputType_->toString(), + partitionKeys, + static_cast(baseReader_.get()), + static_cast(baseRowReader_.get())); +} + +void SplitReader::createReader() { + VELOX_CHECK_NE( + baseReaderOpts_.fileFormat(), dwio::common::FileFormat::UNKNOWN); + + FileHandleCachedPtr fileHandleCachePtr; + FileHandleKey fileHandleKey{ + .filename = hiveSplit_->filePath, + .tokenProvider = connectorQueryCtx_->fsTokenProvider()}; + try { + fileHandleCachePtr = fileHandleFactory_->generate( + fileHandleKey, + hiveSplit_->properties.has_value() ? &*hiveSplit_->properties : nullptr, + fsStats_ ? fsStats_.get() : nullptr); + VELOX_CHECK_NOT_NULL(fileHandleCachePtr.get()); + } catch (const VeloxRuntimeError& e) { + if (e.errorCode() == error_code::kFileNotFound && + hiveConfig_->ignoreMissingFiles( + connectorQueryCtx_->sessionProperties())) { + emptySplit_ = true; + return; + } + throw; + } + + // Here we keep adding new entries to CacheTTLController when new fileHandles + // are generated, if CacheTTLController was created. Creator of + // CacheTTLController needs to make sure a size control strategy was available + // such as removing aged out entries. + if (auto* cacheTTLController = cache::CacheTTLController::getInstance()) { + cacheTTLController->addOpenFileInfo(fileHandleCachePtr->uuid.id()); + } + auto baseFileInput = createBufferedInput( + *fileHandleCachePtr, + baseReaderOpts_, + connectorQueryCtx_, + ioStats_, + fsStats_, + executor_); + + baseReader_ = dwio::common::getReaderFactory(baseReaderOpts_.fileFormat()) + ->createReader(std::move(baseFileInput), baseReaderOpts_); + if (!baseReader_) { + emptySplit_ = true; + } +} + +RowTypePtr SplitReader::getAdaptedRowType() const { + auto& fileType = baseReader_->rowType(); + auto columnTypes = adaptColumns(fileType, baseReaderOpts_.fileSchema()); + auto columnNames = fileType->names(); + return ROW(std::move(columnNames), std::move(columnTypes)); +} + +bool SplitReader::filterOnStats( + dwio::common::RuntimeStatistics& runtimeStats) const { + if (testFilters( + scanSpec_.get(), + baseReader_.get(), + hiveSplit_->filePath, + hiveSplit_->partitionKeys, + *partitionKeys_, + hiveConfig_->readTimestampPartitionValueAsLocalTime( + connectorQueryCtx_->sessionProperties()))) { + ++runtimeStats.processedSplits; + return true; + } + ++runtimeStats.skippedSplits; + runtimeStats.skippedSplitBytes += hiveSplit_->length; + return false; +} + +bool SplitReader::checkIfSplitIsEmpty( + dwio::common::RuntimeStatistics& runtimeStats) { + // emptySplit_ may already be set if the data file is not found. In this case + // we don't need to test further. + if (emptySplit_) { + return true; + } + if (!baseReader_ || baseReader_->numberOfRows() == 0 || + !filterOnStats(runtimeStats)) { + emptySplit_ = true; + } + return emptySplit_; +} + +void SplitReader::createRowReader( + std::shared_ptr metadataFilter, + RowTypePtr rowType) { + VELOX_CHECK_NULL(baseRowReader_); + configureRowReaderOptions( + hiveTableHandle_->tableParameters(), + scanSpec_, + std::move(metadataFilter), + std::move(rowType), + hiveSplit_, + hiveConfig_, + connectorQueryCtx_->sessionProperties(), + baseRowReaderOpts_); + baseRowReader_ = baseReader_->createRowReader(baseRowReaderOpts_); +} + +std::vector SplitReader::adaptColumns( + const RowTypePtr& fileType, + const std::shared_ptr& tableSchema) const { + // Keep track of schema types for columns in file, used by ColumnSelector. + std::vector columnTypes = fileType->children(); + + auto& childrenSpecs = scanSpec_->children(); + for (size_t i = 0; i < childrenSpecs.size(); ++i) { + auto* childSpec = childrenSpecs[i].get(); + const std::string& fieldName = childSpec->fieldName(); + + if (auto it = hiveSplit_->partitionKeys.find(fieldName); + it != hiveSplit_->partitionKeys.end()) { + setPartitionValue(childSpec, fieldName, it->second); + } else if (auto iter = hiveSplit_->infoColumns.find(fieldName); + iter != hiveSplit_->infoColumns.end()) { + auto infoColumnType = + readerOutputType_->childAt(readerOutputType_->getChildIdx(fieldName)); + auto constant = VELOX_DYNAMIC_SCALAR_TYPE_DISPATCH_ALL( + newConstantFromString, + infoColumnType->kind(), + infoColumnType, + iter->second, + 1, + connectorQueryCtx_->memoryPool(), + connectorQueryCtx_->sessionTimezone(), + hiveConfig_->readTimestampPartitionValueAsLocalTime( + connectorQueryCtx_->sessionProperties())); + childSpec->setConstantValue(constant); + } else if ( + childSpec->columnType() == common::ScanSpec::ColumnType::kRegular) { + auto fileTypeIdx = fileType->getChildIdxIfExists(fieldName); + if (!fileTypeIdx.has_value()) { + // Column is missing. Most likely due to schema evolution. + auto outputTypeIdx = readerOutputType_->getChildIdxIfExists(fieldName); + TypePtr fieldType; + if (outputTypeIdx.has_value()) { + // Field name exists in the user-specified output type. + fieldType = readerOutputType_->childAt(outputTypeIdx.value()); + } else { + VELOX_CHECK(tableSchema, "Unable to resolve column '{}'", fieldName); + fieldType = tableSchema->findChild(fieldName); + } + childSpec->setConstantValue(BaseVector::createNullConstant( + fieldType, 1, connectorQueryCtx_->memoryPool())); + } else { + // Column no longer missing, reset constant value set on the spec. + childSpec->setConstantValue(nullptr); + auto outputTypeIdx = readerOutputType_->getChildIdxIfExists(fieldName); + if (outputTypeIdx.has_value()) { + auto& outputType = readerOutputType_->childAt(*outputTypeIdx); + auto& columnType = columnTypes[*fileTypeIdx]; + if (childSpec->isFlatMapAsStruct()) { + // Flat map column read as struct. Leave the schema type as MAP. + VELOX_CHECK(outputType->isRow() && columnType->isMap()); + } else { + // We know the fieldName exists in the file, make the type at that + // position match what we expect in the output. + columnType = outputType; + } + } + } + } + } + + scanSpec_->resetCachedValues(false); + + return columnTypes; +} + +void SplitReader::setPartitionValue( + common::ScanSpec* spec, + const std::string& partitionKey, + const std::optional& value) const { + auto it = partitionKeys_->find(partitionKey); + VELOX_CHECK( + it != partitionKeys_->end(), + "ColumnHandle is missing for partition key {}", + partitionKey); + auto type = it->second->dataType(); + auto constant = VELOX_DYNAMIC_SCALAR_TYPE_DISPATCH_ALL( + newConstantFromString, + type->kind(), + type, + value, + 1, + connectorQueryCtx_->memoryPool(), + connectorQueryCtx_->sessionTimezone(), + hiveConfig_->readTimestampPartitionValueAsLocalTime( + connectorQueryCtx_->sessionProperties()), + it->second->isPartitionDateValueDaysSinceEpoch()); + spec->setConstantValue(constant); +} + +} // namespace facebook::velox::connector::hive diff --git a/velox/connectors/lakehouse/common/SplitReader.h b/velox/connectors/lakehouse/common/SplitReader.h new file mode 100644 index 000000000000..d0055082e7e5 --- /dev/null +++ b/velox/connectors/lakehouse/common/SplitReader.h @@ -0,0 +1,206 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#pragma once + +#include "velox/common/base/RandomUtil.h" +#include "velox/common/file/FileSystems.h" +#include "velox/connectors/hive/FileHandle.h" +#include "velox/connectors/hive/HivePartitionFunction.h" +#include "velox/dwio/common/Options.h" +#include "velox/dwio/common/Reader.h" + +namespace facebook::velox { +class BaseVector; +using VectorPtr = std::shared_ptr; +} // namespace facebook::velox + +namespace facebook::velox::common { +class MetadataFilter; +class ScanSpec; +} // namespace facebook::velox::common + +namespace facebook::velox::connector { +class ConnectorQueryCtx; +} // namespace facebook::velox::connector + +namespace facebook::velox::dwio::common { +struct RuntimeStatistics; +} // namespace facebook::velox::dwio::common + +namespace facebook::velox::memory { +class MemoryPool; +} + +namespace facebook::velox::connector::hive { + +struct HiveConnectorSplit; +class HiveTableHandle; +class HiveColumnHandle; +class HiveConfig; + +class SplitReader { + public: + static std::unique_ptr create( + const std::shared_ptr& hiveSplit, + const std::shared_ptr& hiveTableHandle, + const std::unordered_map< + std::string, + std::shared_ptr>* partitionKeys, + const ConnectorQueryCtx* connectorQueryCtx, + const std::shared_ptr& hiveConfig, + const RowTypePtr& readerOutputType, + const std::shared_ptr& ioStats, + const std::shared_ptr& fsStats, + FileHandleFactory* fileHandleFactory, + folly::Executor* executor, + const std::shared_ptr& scanSpec, + core::ExpressionEvaluator* expressionEvaluator, + std::atomic& totalRemainingFilterTime); + + virtual ~SplitReader() = default; + + void configureReaderOptions( + std::shared_ptr randomSkip); + + /// This function is used by different table formats like Iceberg and Hudi to + /// do additional preparations before reading the split, e.g. Open delete + /// files or log files, and add column adapatations for metadata columns. It + /// would be called only once per incoming split + virtual void prepareSplit( + std::shared_ptr metadataFilter, + dwio::common::RuntimeStatistics& runtimeStats); + + virtual uint64_t next(uint64_t size, VectorPtr& output); + + void resetFilterCaches(); + + bool emptySplit() const; + + void resetSplit(); + + int64_t estimatedRowSize() const; + + void updateRuntimeStats(dwio::common::RuntimeStatistics& stats) const; + + bool allPrefetchIssued() const; + + void setConnectorQueryCtx(const ConnectorQueryCtx* connectorQueryCtx); + + void setBucketConversion(std::vector bucketChannels); + + const RowTypePtr& readerOutputType() const { + return readerOutputType_; + } + + std::string toString() const; + + protected: + SplitReader( + const std::shared_ptr& hiveSplit, + const std::shared_ptr& hiveTableHandle, + const std::unordered_map< + std::string, + std::shared_ptr>* partitionKeys, + const ConnectorQueryCtx* connectorQueryCtx, + const std::shared_ptr& hiveConfig, + const RowTypePtr& readerOutputType, + const std::shared_ptr& ioStats, + const std::shared_ptr& fsStats, + FileHandleFactory* fileHandleFactory, + folly::Executor* executor, + const std::shared_ptr& scanSpec); + + /// Create the dwio::common::Reader object baseReader_, which will be used to + /// read the data file's metadata and schema + void createReader(); + + // Adjust the scan spec according to the current split, then return the + // adapted row type. + RowTypePtr getAdaptedRowType() const; + + // Check if the filters pass on the column statistics. When delta update is + // present, the corresonding filter should be disabled before calling this + // function. + bool filterOnStats(dwio::common::RuntimeStatistics& runtimeStats) const; + + /// Check if the hiveSplit_ is empty. The split is considered empty when + /// 1) The data file is missing but the user chooses to ignore it + /// 2) The file does not contain any rows + /// 3) The data in the file does not pass the filters. The test is based on + /// the file metadata and partition key values + /// This function needs to be called after baseReader_ is created. + bool checkIfSplitIsEmpty(dwio::common::RuntimeStatistics& runtimeStats); + + /// Create the dwio::common::RowReader object baseRowReader_, which owns the + /// ColumnReaders that will be used to read the data + void createRowReader( + std::shared_ptr metadataFilter, + RowTypePtr rowType); + + const folly::F14FastSet& bucketChannels() const { + return bucketChannels_; + } + + std::vector bucketConversionRows( + const RowVector& vector); + + void applyBucketConversion( + VectorPtr& output, + const std::vector& ranges); + + private: + /// Different table formats may have different meatadata columns. + /// This function will be used to update the scanSpec for these columns. + std::vector adaptColumns( + const RowTypePtr& fileType, + const std::shared_ptr& tableSchema) const; + + void setPartitionValue( + common::ScanSpec* spec, + const std::string& partitionKey, + const std::optional& value) const; + + protected: + std::shared_ptr hiveSplit_; + const std::shared_ptr hiveTableHandle_; + const std::unordered_map< + std::string, + std::shared_ptr>* const partitionKeys_; + const ConnectorQueryCtx* connectorQueryCtx_; + const std::shared_ptr hiveConfig_; + + RowTypePtr readerOutputType_; + const std::shared_ptr ioStats_; + const std::shared_ptr fsStats_; + FileHandleFactory* const fileHandleFactory_; + folly::Executor* const executor_; + memory::MemoryPool* const pool_; + + std::shared_ptr scanSpec_; + std::unique_ptr baseReader_; + std::unique_ptr baseRowReader_; + dwio::common::ReaderOptions baseReaderOpts_; + dwio::common::RowReaderOptions baseRowReaderOpts_; + bool emptySplit_; + + private: + folly::F14FastSet bucketChannels_; + std::unique_ptr partitionFunction_; + std::vector partitions_; +}; + +} // namespace facebook::velox::connector::hive diff --git a/velox/connectors/lakehouse/common/TableHandle.cpp b/velox/connectors/lakehouse/common/TableHandle.cpp new file mode 100644 index 000000000000..3f7c8b6f93d5 --- /dev/null +++ b/velox/connectors/lakehouse/common/TableHandle.cpp @@ -0,0 +1,244 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "velox/connectors/hive/TableHandle.h" + +namespace facebook::velox::connector::hive { + +namespace { +std::unordered_map +columnTypeNames() { + return { + {HiveColumnHandle::ColumnType::kPartitionKey, "PartitionKey"}, + {HiveColumnHandle::ColumnType::kRegular, "Regular"}, + {HiveColumnHandle::ColumnType::kSynthesized, "Synthesized"}, + {HiveColumnHandle::ColumnType::kRowIndex, "RowIndex"}, + }; +} + +template +std::unordered_map invertMap(const std::unordered_map& mapping) { + std::unordered_map inverted; + for (const auto& [key, value] : mapping) { + inverted.emplace(value, key); + } + return inverted; +} + +} // namespace + +std::string HiveColumnHandle::columnTypeName( + HiveColumnHandle::ColumnType type) { + static const auto ctNames = columnTypeNames(); + return ctNames.at(type); +} + +HiveColumnHandle::ColumnType HiveColumnHandle::columnTypeFromName( + const std::string& name) { + static const auto nameColumnTypes = invertMap(columnTypeNames()); + return nameColumnTypes.at(name); +} + +folly::dynamic HiveColumnHandle::serialize() const { + folly::dynamic obj = ColumnHandle::serializeBase("HiveColumnHandle"); + obj["hiveColumnHandleName"] = name_; + obj["columnType"] = columnTypeName(columnType_); + obj["dataType"] = dataType_->serialize(); + obj["hiveType"] = hiveType_->serialize(); + folly::dynamic requiredSubfields = folly::dynamic::array; + for (const auto& subfield : requiredSubfields_) { + requiredSubfields.push_back(subfield.toString()); + } + obj["requiredSubfields"] = requiredSubfields; + return obj; +} + +std::string HiveColumnHandle::toString() const { + std::ostringstream out; + out << fmt::format( + "HiveColumnHandle [name: {}, columnType: {}, dataType: {},", + name_, + columnTypeName(columnType_), + dataType_->toString()); + out << " requiredSubfields: ["; + for (const auto& subfield : requiredSubfields_) { + out << " " << subfield.toString(); + } + out << " ]]"; + return out.str(); +} + +ColumnHandlePtr HiveColumnHandle::create(const folly::dynamic& obj) { + auto name = obj["hiveColumnHandleName"].asString(); + auto columnType = columnTypeFromName(obj["columnType"].asString()); + auto dataType = ISerializable::deserialize(obj["dataType"]); + auto hiveType = ISerializable::deserialize(obj["hiveType"]); + + const auto& arr = obj["requiredSubfields"]; + std::vector requiredSubfields; + requiredSubfields.reserve(arr.size()); + for (auto& s : arr) { + requiredSubfields.emplace_back(s.asString()); + } + + return std::make_shared( + name, columnType, dataType, hiveType, std::move(requiredSubfields)); +} + +void HiveColumnHandle::registerSerDe() { + auto& registry = DeserializationRegistryForSharedPtr(); + registry.Register("HiveColumnHandle", HiveColumnHandle::create); +} + +HiveTableHandle::HiveTableHandle( + std::string connectorId, + const std::string& tableName, + bool filterPushdownEnabled, + common::SubfieldFilters subfieldFilters, + const core::TypedExprPtr& remainingFilter, + const RowTypePtr& dataColumns, + const std::unordered_map& tableParameters) + : ConnectorTableHandle(std::move(connectorId)), + tableName_(tableName), + filterPushdownEnabled_(filterPushdownEnabled), + subfieldFilters_(std::move(subfieldFilters)), + remainingFilter_(remainingFilter), + dataColumns_(dataColumns), + tableParameters_(tableParameters) {} + +std::string HiveTableHandle::toString() const { + std::stringstream out; + out << "table: " << tableName_; + if (!subfieldFilters_.empty()) { + // Sort filters by subfield for deterministic output. + std::map orderedFilters; + for (const auto& [field, filter] : subfieldFilters_) { + orderedFilters[field.toString()] = filter.get(); + } + out << ", range filters: ["; + bool notFirstFilter = false; + for (const auto& [field, filter] : orderedFilters) { + if (notFirstFilter) { + out << ", "; + } + out << "(" << field << ", " << filter->toString() << ")"; + notFirstFilter = true; + } + out << "]"; + } + if (remainingFilter_) { + out << ", remaining filter: (" << remainingFilter_->toString() << ")"; + } + if (dataColumns_) { + out << ", data columns: " << dataColumns_->toString(); + } + if (!tableParameters_.empty()) { + std::map orderedTableParameters{ + tableParameters_.begin(), tableParameters_.end()}; + out << ", table parameters: ["; + bool firstParam = true; + for (const auto& param : orderedTableParameters) { + if (!firstParam) { + out << ", "; + } + out << param.first << ":" << param.second; + firstParam = false; + } + out << "]"; + } + return out.str(); +} + +folly::dynamic HiveTableHandle::serialize() const { + folly::dynamic obj = ConnectorTableHandle::serializeBase("HiveTableHandle"); + obj["tableName"] = tableName_; + obj["filterPushdownEnabled"] = filterPushdownEnabled_; + + folly::dynamic subfieldFilters = folly::dynamic::array; + for (const auto& [subfield, filter] : subfieldFilters_) { + folly::dynamic pair = folly::dynamic::object; + pair["subfield"] = subfield.toString(); + pair["filter"] = filter->serialize(); + subfieldFilters.push_back(pair); + } + + obj["subfieldFilters"] = subfieldFilters; + if (remainingFilter_) { + obj["remainingFilter"] = remainingFilter_->serialize(); + } + if (dataColumns_) { + obj["dataColumns"] = dataColumns_->serialize(); + } + folly::dynamic tableParameters = folly::dynamic::object; + for (const auto& param : tableParameters_) { + tableParameters[param.first] = param.second; + } + obj["tableParameters"] = tableParameters; + + return obj; +} + +ConnectorTableHandlePtr HiveTableHandle::create( + const folly::dynamic& obj, + void* context) { + auto connectorId = obj["connectorId"].asString(); + auto tableName = obj["tableName"].asString(); + auto filterPushdownEnabled = obj["filterPushdownEnabled"].asBool(); + + core::TypedExprPtr remainingFilter; + if (auto it = obj.find("remainingFilter"); it != obj.items().end()) { + remainingFilter = + ISerializable::deserialize(it->second, context); + } + + common::SubfieldFilters subfieldFilters; + folly::dynamic subfieldFiltersObj = obj["subfieldFilters"]; + for (const auto& subfieldFilter : subfieldFiltersObj) { + common::Subfield subfield(subfieldFilter["subfield"].asString()); + auto filter = + ISerializable::deserialize(subfieldFilter["filter"]); + subfieldFilters[common::Subfield(std::move(subfield.path()))] = + filter->clone(); + } + + RowTypePtr dataColumns; + if (auto it = obj.find("dataColumns"); it != obj.items().end()) { + dataColumns = ISerializable::deserialize(it->second, context); + } + + std::unordered_map tableParameters{}; + const auto& tableParametersObj = obj["tableParameters"]; + for (const auto& key : tableParametersObj.keys()) { + const auto& value = tableParametersObj[key]; + tableParameters.emplace(key.asString(), value.asString()); + } + + return std::make_shared( + connectorId, + tableName, + filterPushdownEnabled, + std::move(subfieldFilters), + remainingFilter, + dataColumns, + tableParameters); +} + +void HiveTableHandle::registerSerDe() { + auto& registry = DeserializationWithContextRegistryForSharedPtr(); + registry.Register("HiveTableHandle", create); +} + +} // namespace facebook::velox::connector::hive diff --git a/velox/connectors/lakehouse/common/TableHandle.h b/velox/connectors/lakehouse/common/TableHandle.h new file mode 100644 index 000000000000..8ce89445bb0e --- /dev/null +++ b/velox/connectors/lakehouse/common/TableHandle.h @@ -0,0 +1,200 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#pragma once + +#include "velox/connectors/Connector.h" +#include "velox/core/ITypedExpr.h" +#include "velox/type/Filter.h" +#include "velox/type/Subfield.h" +#include "velox/type/Type.h" + +namespace facebook::velox::connector::hive { + +class HiveColumnHandle : public ColumnHandle { + public: + enum class ColumnType { + kPartitionKey, + kRegular, + kSynthesized, + /// A zero-based row number of type BIGINT auto-generated by the connector. + /// Rows numbers are unique within a single file only. + kRowIndex, + kRowId, + }; + + struct ColumnParseParameters { + enum PartitionDateValueFormat { + kISO8601, + kDaysSinceEpoch, + } partitionDateValueFormat; + }; + + /// NOTE: 'dataType' is the column type in target write table. 'hiveType' is + /// converted type of the corresponding column in source table which might not + /// be the same type, and the table scan needs to do data coercion if needs. + /// The table writer also needs to respect the type difference when processing + /// input data such as bucket id calculation. + HiveColumnHandle( + const std::string& name, + ColumnType columnType, + TypePtr dataType, + TypePtr hiveType, + std::vector requiredSubfields = {}, + ColumnParseParameters columnParseParameters = {}) + : name_(name), + columnType_(columnType), + dataType_(std::move(dataType)), + hiveType_(std::move(hiveType)), + requiredSubfields_(std::move(requiredSubfields)), + columnParseParameters_(columnParseParameters) { + VELOX_USER_CHECK( + dataType_->equivalent(*hiveType_), + "data type {} and hive type {} do not match", + dataType_->toString(), + hiveType_->toString()); + } + + const std::string& name() const override { + return name_; + } + + ColumnType columnType() const { + return columnType_; + } + + const TypePtr& dataType() const { + return dataType_; + } + + const TypePtr& hiveType() const { + return hiveType_; + } + + /// Applies to columns of complex types: arrays, maps and structs. When a + /// query uses only some of the subfields, the engine provides the complete + /// list of required subfields and the connector is free to prune the rest. + /// + /// Examples: + /// - SELECT a[1], b['x'], x.y FROM t + /// - SELECT a FROM t WHERE b['y'] > 10 + /// + /// Pruning a struct means populating some of the members with null values. + /// + /// Pruning a map means dropping keys not listed in the required subfields. + /// + /// Pruning arrays means dropping values with indices larger than maximum + /// required index. + const std::vector& requiredSubfields() const { + return requiredSubfields_; + } + + bool isPartitionKey() const { + return columnType_ == ColumnType::kPartitionKey; + } + + bool isPartitionDateValueDaysSinceEpoch() const { + return columnParseParameters_.partitionDateValueFormat == + ColumnParseParameters::kDaysSinceEpoch; + } + + std::string toString() const; + + folly::dynamic serialize() const override; + + static ColumnHandlePtr create(const folly::dynamic& obj); + + static std::string columnTypeName(HiveColumnHandle::ColumnType columnType); + + static HiveColumnHandle::ColumnType columnTypeFromName( + const std::string& name); + + static void registerSerDe(); + + private: + const std::string name_; + const ColumnType columnType_; + const TypePtr dataType_; + const TypePtr hiveType_; + const std::vector requiredSubfields_; + const ColumnParseParameters columnParseParameters_; +}; + +using HiveColumnHandlePtr = std::shared_ptr; +using HiveColumnHandleMap = + std::unordered_map; + +class HiveTableHandle : public ConnectorTableHandle { + public: + HiveTableHandle( + std::string connectorId, + const std::string& tableName, + bool filterPushdownEnabled, + common::SubfieldFilters subfieldFilters, + const core::TypedExprPtr& remainingFilter, + const RowTypePtr& dataColumns = nullptr, + const std::unordered_map& tableParameters = {}); + + const std::string& tableName() const { + return tableName_; + } + + const std::string& name() const override { + return tableName(); + } + + bool isFilterPushdownEnabled() const { + return filterPushdownEnabled_; + } + + const common::SubfieldFilters& subfieldFilters() const { + return subfieldFilters_; + } + + const core::TypedExprPtr& remainingFilter() const { + return remainingFilter_; + } + + // Schema of the table. Need this for reading TEXTFILE. + const RowTypePtr& dataColumns() const { + return dataColumns_; + } + + const std::unordered_map& tableParameters() const { + return tableParameters_; + } + + std::string toString() const override; + + folly::dynamic serialize() const override; + + static ConnectorTableHandlePtr create( + const folly::dynamic& obj, + void* context); + + static void registerSerDe(); + + private: + const std::string tableName_; + const bool filterPushdownEnabled_; + const common::SubfieldFilters subfieldFilters_; + const core::TypedExprPtr remainingFilter_; + const RowTypePtr dataColumns_; + const std::unordered_map tableParameters_; +}; + +using HiveTableHandlePtr = std::shared_ptr; + +} // namespace facebook::velox::connector::hive diff --git a/velox/connectors/lakehouse/iceberg/CMakeLists.txt b/velox/connectors/lakehouse/iceberg/CMakeLists.txt new file mode 100644 index 000000000000..28856884e278 --- /dev/null +++ b/velox/connectors/lakehouse/iceberg/CMakeLists.txt @@ -0,0 +1,30 @@ +# Copyright (c) Facebook, Inc. and its affiliates. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +velox_add_library( + velox_hive_iceberg_splitreader + EqualityDeleteFileReader.cpp + FilterUtil.cpp + IcebergDeleteFile.cpp + IcebergSplitReader.cpp + IcebergSplit.cpp + PositionalDeleteFileReader.cpp + IcebergDataSink.cpp) + +velox_link_libraries(velox_hive_iceberg_splitreader velox_connector + velox_dwio_common Folly::folly) + +if(${VELOX_BUILD_TESTING}) + add_subdirectory(tests) +endif() diff --git a/velox/connectors/lakehouse/iceberg/EqualityDeleteFileReader.cpp b/velox/connectors/lakehouse/iceberg/EqualityDeleteFileReader.cpp new file mode 100644 index 000000000000..47b6bdf1b2e6 --- /dev/null +++ b/velox/connectors/lakehouse/iceberg/EqualityDeleteFileReader.cpp @@ -0,0 +1,224 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "velox/connectors/hive/iceberg/EqualityDeleteFileReader.h" + +#include "velox/connectors/hive/HiveConnectorUtil.h" +#include "velox/connectors/hive/iceberg/FilterUtil.h" +#include "velox/connectors/hive/iceberg/IcebergDeleteFile.h" +#include "velox/core/Expressions.h" +#include "velox/dwio/common/ReaderFactory.h" + +using namespace facebook::velox::common; +using namespace facebook::velox::core; +using namespace facebook::velox::exec; + +namespace facebook::velox::connector::hive::iceberg { + +static constexpr const int kMaxBatchRows = 10'000; + +EqualityDeleteFileReader::EqualityDeleteFileReader( + const IcebergDeleteFile& deleteFile, + std::shared_ptr baseFileSchema, + FileHandleFactory* fileHandleFactory, + folly::Executor* executor, + const ConnectorQueryCtx* connectorQueryCtx, + const std::shared_ptr& hiveConfig, + const std::shared_ptr ioStats, + const std::shared_ptr& fsStats, + const std::string& connectorId) + : deleteFile_(deleteFile), + baseFileSchema_(baseFileSchema), + fileHandleFactory_(fileHandleFactory), + pool_(connectorQueryCtx->memoryPool()), + deleteSplit_(nullptr), + deleteRowReader_(nullptr) { + VELOX_CHECK_EQ(deleteFile_.content, FileContent::kEqualityDeletes); + + if (deleteFile_.recordCount == 0) { + return; + } + + // TODO: push down filter if previous delete file contains this one. E.g. + // previous equality delete file has a=1, and this file also contains + // columns a, then a!=1 can be pushed as a filter when reading this delete + // file. + + deleteSplit_ = std::make_shared( + connectorId, + deleteFile_.filePath, + deleteFile_.fileFormat, + 0, + deleteFile_.fileSizeInBytes); + + // Create the Reader and RowReader for the equality delete file + + dwio::common::ReaderOptions deleteReaderOpts(pool_); + configureReaderOptions( + hiveConfig, + connectorQueryCtx, + nullptr, + deleteSplit_, + {}, + deleteReaderOpts); + + const FileHandleKey fileHandleKey{ + .filename = deleteFile_.filePath, + .tokenProvider = connectorQueryCtx->fsTokenProvider()}; + auto deleteFileHandleCachePtr = fileHandleFactory_->generate(fileHandleKey); + auto deleteFileInput = createBufferedInput( + *deleteFileHandleCachePtr, + deleteReaderOpts, + connectorQueryCtx, + ioStats, + fsStats, + executor); + + auto deleteReader = + dwio::common::getReaderFactory(deleteReaderOpts.fileFormat()) + ->createReader(std::move(deleteFileInput), deleteReaderOpts); + + // For now, we assume only the delete columns are written in the delete file + deleteFileRowType_ = deleteReader->rowType(); + auto scanSpec = std::make_shared(""); + scanSpec->addAllChildFields(deleteFileRowType_->asRow()); + + dwio::common::RowReaderOptions deleteRowReaderOpts; + configureRowReaderOptions( + {}, + scanSpec, + nullptr, + deleteFileRowType_, + deleteSplit_, + hiveConfig, + connectorQueryCtx->sessionProperties(), + deleteRowReaderOpts); + + deleteRowReader_.reset(); + deleteRowReader_ = deleteReader->createRowReader(deleteRowReaderOpts); +} + +void EqualityDeleteFileReader::readDeleteValues( + SubfieldFilters& subfieldFilters, + std::vector& expressionInputs) { + VELOX_CHECK(deleteRowReader_); + VELOX_CHECK(deleteSplit_); + + if (!deleteValuesOutput_) { + deleteValuesOutput_ = BaseVector::create(deleteFileRowType_, 0, pool_); + } + + // TODO: verfiy if the field is an Iceberg RowId. Velox currently doesn't + // support pushing down filters to non-RowId types, i.e. sub-fields of Array + // or Map + if (deleteFileRowType_->size() == 1) { + // Construct the IN list filter that can be pushed down to the base file + // readers, then update the baseFileScanSpec. + buildDomainFilter(subfieldFilters); + } else { + // Build the filter functions that will be evaluated after all base file + // read is done + buildFilterFunctions(expressionInputs); + } + + deleteSplit_.reset(); +} + +void EqualityDeleteFileReader::buildDomainFilter( + SubfieldFilters& subfieldFilters) { + std::unique_ptr filter = std::make_unique(); + auto name = baseFileSchema_->childByFieldId(deleteFile_.equalityFieldIds[0]) + ->fullName(); + while (deleteRowReader_->next(kMaxBatchRows, deleteValuesOutput_)) { + if (deleteValuesOutput_->size() == 0) { + continue; + } + + deleteValuesOutput_->loadedVector(); + auto vector = + std::dynamic_pointer_cast(deleteValuesOutput_)->childAt(0); + + auto typeKind = vector->type()->kind(); + VELOX_CHECK( + typeKind != TypeKind::DOUBLE && typeKind != TypeKind::REAL, + "Iceberg does not allow DOUBLE or REAL columns as the equality delete columns: {} : {}", + name, + typeKind); + + auto notExistsFilter = + createNotInFilter(vector, 0, deleteValuesOutput_->size(), typeKind); + filter = filter->mergeWith(notExistsFilter.get()); + } + + if (filter->kind() != FilterKind::kAlwaysTrue) { + if (subfieldFilters.find(common::Subfield(name)) != subfieldFilters.end()) { + subfieldFilters[common::Subfield(name)] = + subfieldFilters[common::Subfield(name)]->mergeWith(filter.get()); + } else { + subfieldFilters[common::Subfield(name)] = std::move(filter); + } + } +} + +void EqualityDeleteFileReader::buildFilterFunctions( + std::vector& expressionInputs) { + auto numDeleteFields = deleteFileRowType_->size(); + VELOX_CHECK_GT( + numDeleteFields, + 0, + "Iceberg equality delete file should have at least one field."); + + // TODO: logical expression simplifications + while (deleteRowReader_->next(kMaxBatchRows, deleteValuesOutput_)) { + if (deleteValuesOutput_->size() == 0) { + continue; + } + + deleteValuesOutput_->loadedVector(); + auto rowVector = std::dynamic_pointer_cast(deleteValuesOutput_); + auto numDeletedValues = rowVector->childAt(0)->size(); + + for (int i = 0; i < numDeletedValues; i++) { + std::vector disconjunctInputs; + + for (int j = 0; j < numDeleteFields; j++) { + auto type = deleteFileRowType_->childAt(j); + auto name = + baseFileSchema_->childByFieldId(deleteFile_.equalityFieldIds[j]) + ->fullName(); + auto value = BaseVector::wrapInConstant(1, i, rowVector->childAt(j)); + + std::vector isNotEqualInputs; + isNotEqualInputs.push_back( + std::make_shared(type, name)); + isNotEqualInputs.push_back(std::make_shared(value)); + // TODO: generalize this to support different engines. Currently, only + // Presto "neq" is supported. Spark does not register the "neq" function + // but does support "not" and "equalto" functions. + auto isNotEqualExpr = + std::make_shared(BOOLEAN(), isNotEqualInputs, "neq"); + + disconjunctInputs.push_back(isNotEqualExpr); + } + + auto disconjunctNotEqualExpr = + std::make_shared(BOOLEAN(), disconjunctInputs, "or"); + expressionInputs.push_back(disconjunctNotEqualExpr); + } + } +} + +} // namespace facebook::velox::connector::hive::iceberg diff --git a/velox/connectors/lakehouse/iceberg/EqualityDeleteFileReader.h b/velox/connectors/lakehouse/iceberg/EqualityDeleteFileReader.h new file mode 100644 index 000000000000..99ce239b9f9d --- /dev/null +++ b/velox/connectors/lakehouse/iceberg/EqualityDeleteFileReader.h @@ -0,0 +1,87 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#pragma once + +#include "velox/connectors/Connector.h" +#include "velox/connectors/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/lakehouse/iceberg/FilterUtil.cpp b/velox/connectors/lakehouse/iceberg/FilterUtil.cpp new file mode 100644 index 000000000000..7ef26f5e18db --- /dev/null +++ b/velox/connectors/lakehouse/iceberg/FilterUtil.cpp @@ -0,0 +1,87 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#include "velox/connectors/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/lakehouse/iceberg/FilterUtil.h b/velox/connectors/lakehouse/iceberg/FilterUtil.h new file mode 100644 index 000000000000..96b8255511c2 --- /dev/null +++ b/velox/connectors/lakehouse/iceberg/FilterUtil.h @@ -0,0 +1,30 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#pragma once + +#include "velox/expression/Expr.h" +#include "velox/type/Filter.h" +#include "velox/vector/ComplexVector.h" + +namespace facebook::velox::connector::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/lakehouse/iceberg/IcebergDataSink.cpp b/velox/connectors/lakehouse/iceberg/IcebergDataSink.cpp new file mode 100644 index 000000000000..0825781640a4 --- /dev/null +++ b/velox/connectors/lakehouse/iceberg/IcebergDataSink.cpp @@ -0,0 +1,254 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "velox/connectors/hive/iceberg/IcebergDataSink.h" +#include "velox/common/base/Fs.h" +#include "velox/connectors/hive/HiveConnectorUtil.h" + +namespace facebook::velox::connector::hive::iceberg { + +namespace { + +std::string toJson(const std::vector& partitionValues) { + folly::dynamic jsonObject = folly::dynamic::object(); + folly::dynamic valuesArray = folly::dynamic::array(); + for (const auto& value : partitionValues) { + valuesArray.push_back(value); + } + jsonObject["partitionValues"] = valuesArray; + return folly::toJson(jsonObject); +} + +template +folly::dynamic extractPartitionValue( + const DecodedVector* block, + vector_size_t row) { + using T = typename TypeTraits::NativeType; + return block->valueAt(row); +} + +template <> +folly::dynamic extractPartitionValue( + const DecodedVector* block, + vector_size_t row) { + return block->toString(row); +} + +template <> +folly::dynamic extractPartitionValue( + const DecodedVector* block, + vector_size_t row) { + VELOX_NYI("Partition on varbinary column is not supported yet."); +} + +class IcebergFileNameGenerator : public 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::shared_ptr locationHandle, + std::shared_ptr partitionSpec, + dwio::common::FileFormat tableStorageFormat, + std::shared_ptr bucketProperty, + std::optional compressionKind, + const std::unordered_map& serdeParameters) + : HiveInsertTableHandle( + std::move(inputColumns), + std::move(locationHandle), + tableStorageFormat, + std::move(bucketProperty), + compressionKind, + serdeParameters, + nullptr, + false, + std::make_shared()), + partitionSpec_(std::move(partitionSpec)) {} + +IcebergDataSink::IcebergDataSink( + facebook::velox::RowTypePtr inputType, + std::shared_ptr insertTableHandle, + const facebook::velox::connector::ConnectorQueryCtx* connectorQueryCtx, + facebook::velox::connector::CommitStrategy commitStrategy, + const std::shared_ptr& hiveConfig) + : IcebergDataSink( + std::move(inputType), + insertTableHandle, + connectorQueryCtx, + commitStrategy, + hiveConfig, + [&insertTableHandle]() { + std::vector channels( + insertTableHandle->inputColumns().size()); + std::iota(channels.begin(), channels.end(), 0); + return channels; + }()) {} + +IcebergDataSink::IcebergDataSink( + RowTypePtr inputType, + std::shared_ptr insertTableHandle, + const ConnectorQueryCtx* connectorQueryCtx, + CommitStrategy commitStrategy, + const std::shared_ptr& hiveConfig, + const std::vector& dataChannels) + : HiveDataSink( + inputType, + insertTableHandle, + connectorQueryCtx, + commitStrategy, + hiveConfig, + 0, + nullptr, + dataChannels) { + if (isPartitioned()) { + partitionData_.resize(maxOpenWriters_); + } +} + +std::vector IcebergDataSink::commitMessage() const { + auto icebergInsertTableHandle = + std::dynamic_pointer_cast( + insertTableHandle_); + + std::vector commitTasks; + commitTasks.reserve(writerInfo_.size()); + std::string fileFormat(toString(insertTableHandle_->storageFormat())); + std::transform( + fileFormat.begin(), fileFormat.end(), fileFormat.begin(), ::toupper); + + for (auto i = 0; i < writerInfo_.size(); ++i) { + const auto& info = writerInfo_.at(i); + VELOX_CHECK_NOT_NULL(info); + // Following metadata (json format) is consumed by Presto CommitTaskData. + // It contains the minimal subset of metadata. + // Complete metrics is missing now and this could lead to suboptimal query + // plan, will collect full iceberg metrics in following PR. + // clang-format off + folly::dynamic commitData = folly::dynamic::object( + "path", info->writerParameters.writeDirectory() + "/" + + info->writerParameters.writeFileName()) + ("fileSizeInBytes", ioStats_.at(i)->rawBytesWritten()) + ("metrics", + folly::dynamic::object("recordCount", info->numWrittenRows)) + ("partitionSpecJson", icebergInsertTableHandle->partitionSpec()->specId) + ("fileFormat", fileFormat) + ("content", "DATA"); + // clang-format on + if (!(partitionData_.empty() || partitionData_[i].empty())) { + commitData["partitionDataJson"] = toJson(partitionData_[i]); + } + auto commitDataJson = folly::toJson(commitData); + commitTasks.push_back(commitDataJson); + } + return commitTasks; +} + +void IcebergDataSink::splitInputRowsAndEnsureWriters(RowVectorPtr input) { + VELOX_CHECK(isPartitioned()); + + std::fill(partitionSizes_.begin(), partitionSizes_.end(), 0); + + const auto numRows = partitionIds_.size(); + for (auto row = 0; row < numRows; ++row) { + auto id = getIcebergWriterId(row); + uint32_t index = ensureWriter(id); + + updatePartitionRows(index, numRows, row); + + if (!partitionData_[index].empty()) { + continue; + } + + std::vector partitionValues(partitionChannels_.size()); + + for (auto i = 0; i < partitionChannels_.size(); ++i) { + auto block = input->childAt(partitionChannels_[i]); + if (block->type()->isDecimal()) { + VELOX_NYI("Partition on decimal column is not supported yet."); + } + DecodedVector decoded(*block); + partitionValues[i] = VELOX_DYNAMIC_SCALAR_TYPE_DISPATCH( + extractPartitionValue, block->typeKind(), &decoded, row); + } + + partitionData_[index] = partitionValues; + } + + for (auto i = 0; i < partitionSizes_.size(); ++i) { + if (partitionSizes_[i] != 0) { + VELOX_CHECK_NOT_NULL(partitionRows_[i]); + partitionRows_[i]->setSize(partitionSizes_[i] * sizeof(vector_size_t)); + } + } +} + +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::optional IcebergDataSink::getPartitionName( + const HiveWriterId& id) const { + std::optional partitionName; + if (isPartitioned()) { + partitionName = + partitionIdGenerator_->partitionName(id.partitionId.value(), "null"); + } + return partitionName; +} + +} // namespace facebook::velox::connector::hive::iceberg diff --git a/velox/connectors/lakehouse/iceberg/IcebergDataSink.h b/velox/connectors/lakehouse/iceberg/IcebergDataSink.h new file mode 100644 index 000000000000..ea3daadde56b --- /dev/null +++ b/velox/connectors/lakehouse/iceberg/IcebergDataSink.h @@ -0,0 +1,79 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#pragma once + +#include "velox/connectors/hive/HiveDataSink.h" +#include "velox/connectors/hive/iceberg/PartitionSpec.h" + +namespace facebook::velox::connector::hive::iceberg { + +// Represents a request for Iceberg write. +class IcebergInsertTableHandle final : public HiveInsertTableHandle { + public: + IcebergInsertTableHandle( + std::vector> inputColumns, + std::shared_ptr locationHandle, + std::shared_ptr partitionSpec, + dwio::common::FileFormat tableStorageFormat = + dwio::common::FileFormat::PARQUET, + std::shared_ptr bucketProperty = nullptr, + std::optional compressionKind = {}, + const std::unordered_map& serdeParameters = {}); + + ~IcebergInsertTableHandle() = default; + + std::shared_ptr partitionSpec() const { + return partitionSpec_; + } + + private: + std::shared_ptr partitionSpec_; +}; + +class IcebergDataSink : public HiveDataSink { + public: + IcebergDataSink( + RowTypePtr inputType, + std::shared_ptr insertTableHandle, + const ConnectorQueryCtx* connectorQueryCtx, + CommitStrategy commitStrategy, + const std::shared_ptr& hiveConfig); + + private: + IcebergDataSink( + RowTypePtr inputType, + std::shared_ptr insertTableHandle, + const ConnectorQueryCtx* connectorQueryCtx, + CommitStrategy commitStrategy, + const std::shared_ptr& hiveConfig, + const std::vector& dataChannels); + + void splitInputRowsAndEnsureWriters(RowVectorPtr input) override; + + std::vector commitMessage() const override; + + HiveWriterId getIcebergWriterId(size_t row) const; + + std::optional getPartitionName( + const HiveWriterId& id) const override; + + // Below are structures for partitions from all inputs. partitionData_ + // is indexed by partitionId. + std::vector> partitionData_; +}; + +} // namespace facebook::velox::connector::hive::iceberg diff --git a/velox/connectors/lakehouse/iceberg/IcebergDeleteFile.cpp b/velox/connectors/lakehouse/iceberg/IcebergDeleteFile.cpp new file mode 100644 index 000000000000..4237785e8ca9 --- /dev/null +++ b/velox/connectors/lakehouse/iceberg/IcebergDeleteFile.cpp @@ -0,0 +1,40 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "velox/connectors/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/lakehouse/iceberg/IcebergDeleteFile.h b/velox/connectors/lakehouse/iceberg/IcebergDeleteFile.h new file mode 100644 index 000000000000..995578e129c6 --- /dev/null +++ b/velox/connectors/lakehouse/iceberg/IcebergDeleteFile.h @@ -0,0 +1,72 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#pragma once + +#include +#include +#include + +#include "velox/dwio/common/Options.h" + +namespace facebook::velox::connector::hive::iceberg { + +enum class FileContent { + kData, + kPositionalDeletes, + kEqualityDeletes, +}; + +/// Formatter for fmt. +fmt::underlying_t format_as(FileContent f); + +struct IcebergDeleteFile { + FileContent content; + const std::string filePath; + dwio::common::FileFormat fileFormat; + uint64_t recordCount; + uint64_t fileSizeInBytes; + // The field ids for the delete columns for equality delete files + std::vector equalityFieldIds; + // The lower bounds of the in-file positions for the deleted rows, identified + // by each column's field id. E.g. The deleted rows for a column with field id + // 1 is in range [10, 50], where 10 and 50 are the deleted row positions in + // the data file, then lowerBounds would contain entry <1, "10"> + std::unordered_map lowerBounds; + // The upper bounds of the in-file positions for the deleted rows, identified + // by each column's field id. E.g. The deleted rows for a column with field id + // 1 is in range [10, 50], then upperBounds will contain entry <1, "50"> + std::unordered_map upperBounds; + + IcebergDeleteFile( + FileContent _content, + const std::string& _filePath, + dwio::common::FileFormat _fileFormat, + uint64_t _recordCount, + uint64_t _fileSizeInBytes, + std::vector _equalityFieldIds = {}, + std::unordered_map _lowerBounds = {}, + std::unordered_map _upperBounds = {}) + : content(_content), + filePath(_filePath), + fileFormat(_fileFormat), + recordCount(_recordCount), + fileSizeInBytes(_fileSizeInBytes), + equalityFieldIds(_equalityFieldIds), + lowerBounds(_lowerBounds), + upperBounds(_upperBounds) {} +}; + +} // namespace facebook::velox::connector::hive::iceberg diff --git a/velox/connectors/lakehouse/iceberg/IcebergMetadataColumns.h b/velox/connectors/lakehouse/iceberg/IcebergMetadataColumns.h new file mode 100644 index 000000000000..4cbf2a7862b3 --- /dev/null +++ b/velox/connectors/lakehouse/iceberg/IcebergMetadataColumns.h @@ -0,0 +1,55 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#pragma once + +#include + +#include "velox/type/Type.h" + +namespace facebook::velox::connector::hive::iceberg { + +struct IcebergMetadataColumn { + int id; + std::string name; + std::shared_ptr type; + std::string doc; + + IcebergMetadataColumn( + int _id, + const std::string& _name, + std::shared_ptr _type, + const std::string& _doc) + : id(_id), name(_name), type(_type), doc(_doc) {} + + static std::shared_ptr icebergDeleteFilePathColumn() { + return std::make_shared( + 2147483546, + "file_path", + VARCHAR(), + "Path of a file in which a deleted row is stored"); + } + + static std::shared_ptr icebergDeletePosColumn() { + return std::make_shared( + 2147483545, + "pos", + BIGINT(), + "Ordinal position of a deleted row in the data file"); + } +}; + +} // namespace facebook::velox::connector::hive::iceberg diff --git a/velox/connectors/lakehouse/iceberg/IcebergSplit.cpp b/velox/connectors/lakehouse/iceberg/IcebergSplit.cpp new file mode 100644 index 000000000000..e5cdf63c33b4 --- /dev/null +++ b/velox/connectors/lakehouse/iceberg/IcebergSplit.cpp @@ -0,0 +1,91 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "velox/connectors/hive/iceberg/IcebergSplit.h" + +#include "velox/connectors/hive/iceberg/IcebergDeleteFile.h" + +namespace facebook::velox::connector::hive::iceberg { + +HiveIcebergSplit::HiveIcebergSplit( + const std::string& connectorId, + const std::string& filePath, + dwio::common::FileFormat fileFormat, + uint64_t start, + uint64_t length, + const std::unordered_map>& + partitionKeys, + std::optional tableBucketNumber, + const std::unordered_map& customSplitInfo, + const std::shared_ptr& extraFileInfo, + bool cacheable, + const std::unordered_map& infoColumns, + std::optional properties) + : HiveConnectorSplit( + connectorId, + filePath, + fileFormat, + start, + length, + partitionKeys, + tableBucketNumber, + customSplitInfo, + extraFileInfo, + /*serdeParameters=*/{}, + /*splitWeight=*/0, + cacheable, + infoColumns, + properties, + std::nullopt, + std::nullopt) { + // TODO: Deserialize _extraFileInfo to get deleteFiles; +} + +// For tests only +HiveIcebergSplit::HiveIcebergSplit( + const std::string& connectorId, + const std::string& filePath, + dwio::common::FileFormat fileFormat, + uint64_t start, + uint64_t length, + const std::unordered_map>& + partitionKeys, + std::optional tableBucketNumber, + const std::unordered_map& customSplitInfo, + const std::shared_ptr& extraFileInfo, + bool cacheable, + std::vector deletes, + const std::unordered_map& infoColumns, + std::optional properties) + : HiveConnectorSplit( + connectorId, + filePath, + fileFormat, + start, + length, + partitionKeys, + tableBucketNumber, + customSplitInfo, + extraFileInfo, + /*serdeParameters=*/{}, + 0, + cacheable, + infoColumns, + properties, + std::nullopt, + std::nullopt), + deleteFiles(std::move(deletes)) {} +} // namespace facebook::velox::connector::hive::iceberg diff --git a/velox/connectors/lakehouse/iceberg/IcebergSplit.h b/velox/connectors/lakehouse/iceberg/IcebergSplit.h new file mode 100644 index 000000000000..eb2448dabd1c --- /dev/null +++ b/velox/connectors/lakehouse/iceberg/IcebergSplit.h @@ -0,0 +1,61 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#pragma once + +#include + +#include "velox/connectors/hive/HiveConnectorSplit.h" +#include "velox/connectors/hive/iceberg/IcebergDeleteFile.h" + +namespace facebook::velox::connector::hive::iceberg { + +struct HiveIcebergSplit : public connector::hive::HiveConnectorSplit { + std::vector deleteFiles; + + HiveIcebergSplit( + const std::string& connectorId, + const std::string& filePath, + dwio::common::FileFormat fileFormat, + uint64_t start = 0, + uint64_t length = std::numeric_limits::max(), + const std::unordered_map>& + partitionKeys = {}, + std::optional tableBucketNumber = std::nullopt, + const std::unordered_map& customSplitInfo = {}, + const std::shared_ptr& extraFileInfo = {}, + bool cacheable = true, + const std::unordered_map& infoColumns = {}, + std::optional fileProperties = std::nullopt); + + // For tests only + HiveIcebergSplit( + const std::string& connectorId, + const std::string& filePath, + dwio::common::FileFormat fileFormat, + uint64_t start = 0, + uint64_t length = std::numeric_limits::max(), + const std::unordered_map>& + partitionKeys = {}, + std::optional tableBucketNumber = std::nullopt, + const std::unordered_map& customSplitInfo = {}, + const std::shared_ptr& extraFileInfo = {}, + bool cacheable = true, + std::vector deletes = {}, + const std::unordered_map& infoColumns = {}, + std::optional fileProperties = std::nullopt); +}; + +} // namespace facebook::velox::connector::hive::iceberg diff --git a/velox/connectors/lakehouse/iceberg/IcebergSplitReader.cpp b/velox/connectors/lakehouse/iceberg/IcebergSplitReader.cpp new file mode 100644 index 000000000000..dc9055fdcd53 --- /dev/null +++ b/velox/connectors/lakehouse/iceberg/IcebergSplitReader.cpp @@ -0,0 +1,243 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "velox/connectors/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" + +using namespace facebook::velox::dwio::common; + +namespace facebook::velox::connector::hive::iceberg { + +IcebergSplitReader::IcebergSplitReader( + const std::shared_ptr& hiveSplit, + const HiveTableHandlePtr& hiveTableHandle, + const std::unordered_map* partitionKeys, + const ConnectorQueryCtx* connectorQueryCtx, + const std::shared_ptr& hiveConfig, + const RowTypePtr& readerOutputType, + const std::shared_ptr& ioStats, + const std::shared_ptr& fsStats, + FileHandleFactory* const fileHandleFactory, + folly::Executor* executor, + const std::shared_ptr& scanSpec, + core::ExpressionEvaluator* expressionEvaluator, + std::atomic& totalRemainingFilterTime) + : SplitReader( + hiveSplit, + hiveTableHandle, + partitionKeys, + connectorQueryCtx, + hiveConfig, + readerOutputType, + ioStats, + fsStats, + fileHandleFactory, + executor, + scanSpec), + baseReadOffset_(0), + splitOffset_(0), + deleteBitmap_(nullptr), + deleteExprSet_(nullptr), + expressionEvaluator_(expressionEvaluator), + totalRemainingFilterMs_(totalRemainingFilterTime) {} + +IcebergSplitReader::~IcebergSplitReader() {} + +void IcebergSplitReader::prepareSplit( + std::shared_ptr metadataFilter, + dwio::common::RuntimeStatistics& runtimeStats) { + createReader(); + if (emptySplit_) { + return; + } + auto rowType = getAdaptedRowType(); + + std::shared_ptr icebergSplit = + std::dynamic_pointer_cast(hiveSplit_); + const auto& deleteFiles = icebergSplit->deleteFiles; + std::unordered_set equalityFieldIds; + for (const auto& deleteFile : deleteFiles) { + if (deleteFile.content == FileContent::kEqualityDeletes && + deleteFile.recordCount > 0) { + equalityFieldIds.insert( + deleteFile.equalityFieldIds.begin(), + deleteFile.equalityFieldIds.end()); + } + } + + // checkIfSplitIsEmpty needs to use the base reader's schemaWithId_. For that + // we need to update the base RowReader to include these extra fields from the + // equality delete file first, so that the schemaWithId_ of the base file is + // updated when we call baseFileSchema() later. + baseReader_->setRequiredExtraFieldIds(equalityFieldIds); + + if (checkIfSplitIsEmpty(runtimeStats)) { + VELOX_CHECK(emptySplit_); + return; + } + + // Process the equality delete files to update the scan spec and remaining + // filters. It needs to be done after creating the Reader and before creating + // the RowReader. + + SubfieldFilters subfieldFilters; + std::vector conjunctInputs; + + for (const auto& deleteFile : deleteFiles) { + if (deleteFile.content == FileContent::kEqualityDeletes && + deleteFile.recordCount > 0) { + // TODO: build cache of to avoid repeating file + // parsing across partitions. Within a single partition, the splits should + // be with the same equality delete files and only need to be parsed once. + auto equalityDeleteReader = std::make_unique( + deleteFile, + baseFileSchema(), + fileHandleFactory_, + executor_, + connectorQueryCtx_, + hiveConfig_, + ioStats_, + fsStats_, + hiveSplit_->connectorId); + equalityDeleteReader->readDeleteValues(subfieldFilters, conjunctInputs); + } + } + + if (!subfieldFilters.empty()) { + for (const auto& [key, filter] : subfieldFilters) { + auto childSpec = scanSpec_->getOrCreateChild(key, true); + childSpec->addFilter(*filter); + childSpec->setHasTempFilter(true); + childSpec->setSubscript(scanSpec_->children().size() - 1); + } + } + + if (!conjunctInputs.empty()) { + core::TypedExprPtr expression = + std::make_shared(BOOLEAN(), conjunctInputs, "and"); + deleteExprSet_ = expressionEvaluator_->compile(expression); + VELOX_CHECK_EQ(deleteExprSet_->size(), 1); + } + + createRowReader(std::move(metadataFilter), std::move(rowType)); + + baseReadOffset_ = 0; + splitOffset_ = baseRowReader_->nextRowNumber(); + + // Create the positional deletes file readers. They need to be created after + // the RowReader is created. + positionalDeleteFileReaders_.clear(); + for (const auto& deleteFile : deleteFiles) { + if (deleteFile.content == FileContent::kPositionalDeletes) { + if (deleteFile.recordCount > 0) { + positionalDeleteFileReaders_.push_back( + std::make_unique( + deleteFile, + hiveSplit_->filePath, + fileHandleFactory_, + connectorQueryCtx_, + executor_, + hiveConfig_, + ioStats_, + fsStats_, + runtimeStats, + splitOffset_, + hiveSplit_->connectorId)); + } + } + } +} + +std::shared_ptr +IcebergSplitReader::baseFileSchema() { + VELOX_CHECK_NOT_NULL(baseReader_.get()); + return baseReader_->typeWithId(); +} + +uint64_t IcebergSplitReader::next(uint64_t size, VectorPtr& output) { + Mutation mutation; + mutation.randomSkip = baseReaderOpts_.randomSkip().get(); + mutation.deletedRows = nullptr; + + if (deleteBitmap_) { + std::memset( + (void*)(deleteBitmap_->asMutable()), 0L, deleteBitmap_->size()); + } + + const auto actualSize = baseRowReader_->nextReadSize(size); + + if (actualSize == dwio::common::RowReader::kAtEnd) { + return 0; + } + + if (!positionalDeleteFileReaders_.empty()) { + auto numBytes = bits::nbytes(actualSize); + dwio::common::ensureCapacity( + deleteBitmap_, numBytes, connectorQueryCtx_->memoryPool(), false, true); + + for (auto iter = positionalDeleteFileReaders_.begin(); + iter != positionalDeleteFileReaders_.end();) { + (*iter)->readDeletePositions(baseReadOffset_, actualSize, deleteBitmap_); + + if ((*iter)->noMoreData()) { + iter = positionalDeleteFileReaders_.erase(iter); + } else { + ++iter; + } + } + } + + mutation.deletedRows = deleteBitmap_ && deleteBitmap_->size() > 0 + ? deleteBitmap_->as() + : nullptr; + + auto rowsScanned = baseRowReader_->next(actualSize, output, &mutation); + + // Evaluate the remaining filter deleteExprSet_ for every batch and update the + // output vector if it reduces any rows. + if (deleteExprSet_) { + auto filterStartMs = getCurrentTimeMs(); + + filterRows_.resize(output->size()); + auto rowVector = std::dynamic_pointer_cast(output); + expressionEvaluator_->evaluate( + deleteExprSet_.get(), filterRows_, *rowVector, filterResult_); + auto numRemainingRows = exec::processFilterResults( + filterResult_, filterRows_, filterEvalCtx_, pool_); + + if (numRemainingRows < output->size()) { + output = exec::wrap( + numRemainingRows, filterEvalCtx_.selectedIndices, rowVector); + } + + totalRemainingFilterMs_.fetch_add( + (getCurrentTimeMs() - filterStartMs), std::memory_order_relaxed); + } + + baseReadOffset_ += rowsScanned; + + if (rowsScanned == 0) { + scanSpec_->deleteTempNodes(); + } + + return rowsScanned; +} + +} // namespace facebook::velox::connector::hive::iceberg diff --git a/velox/connectors/lakehouse/iceberg/IcebergSplitReader.h b/velox/connectors/lakehouse/iceberg/IcebergSplitReader.h new file mode 100644 index 000000000000..a984b3770621 --- /dev/null +++ b/velox/connectors/lakehouse/iceberg/IcebergSplitReader.h @@ -0,0 +1,74 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#pragma once + +#include "velox/connectors/Connector.h" +#include "velox/connectors/hive/SplitReader.h" +#include "velox/connectors/hive/iceberg/PositionalDeleteFileReader.h" +#include "velox/exec/OperatorUtils.h" + +namespace facebook::velox::connector::hive::iceberg { + +struct IcebergDeleteFile; + +class IcebergSplitReader : public SplitReader { + public: + IcebergSplitReader( + const std::shared_ptr& hiveSplit, + const HiveTableHandlePtr& hiveTableHandle, + const std::unordered_map* partitionKeys, + const ConnectorQueryCtx* connectorQueryCtx, + const std::shared_ptr& hiveConfig, + const RowTypePtr& readerOutputType, + const std::shared_ptr& ioStats, + const std::shared_ptr& fsStats, + FileHandleFactory* fileHandleFactory, + folly::Executor* executor, + const std::shared_ptr& scanSpec, + core::ExpressionEvaluator* expressionEvaluator, + std::atomic& totalRemainingFilterTime); + + ~IcebergSplitReader() override; + + void prepareSplit( + std::shared_ptr metadataFilter, + dwio::common::RuntimeStatistics& runtimeStats) override; + + uint64_t next(uint64_t size, VectorPtr& output) override; + + std::shared_ptr baseFileSchema(); + + private: + // The read offset to the beginning of the split in number of rows for the + // current batch for the base data file + uint64_t baseReadOffset_; + // The file position for the first row in the split + uint64_t splitOffset_; + std::list> + positionalDeleteFileReaders_; + BufferPtr deleteBitmap_; + + std::unique_ptr deleteExprSet_; + core::ExpressionEvaluator* expressionEvaluator_; + std::atomic& totalRemainingFilterMs_; + + // Reusable memory for remaining filter evaluation. + VectorPtr filterResult_; + SelectivityVector filterRows_; + exec::FilterEvalCtx filterEvalCtx_; +}; +} // namespace facebook::velox::connector::hive::iceberg diff --git a/velox/connectors/lakehouse/iceberg/PartitionSpec.h b/velox/connectors/lakehouse/iceberg/PartitionSpec.h new file mode 100644 index 000000000000..5d0b3a71ac6e --- /dev/null +++ b/velox/connectors/lakehouse/iceberg/PartitionSpec.h @@ -0,0 +1,63 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#pragma once + +#include +#include +#include + +namespace facebook::velox::connector::hive::iceberg { + +enum class TransformType { + kIdentity, + kHour, + kDay, + kMonth, + kYear, + kBucket, + kTruncate +}; + +struct IcebergPartitionSpec { + struct Field { + // The column name and type of this partition field as it appears in the + // partition spec. The column can be a nested column in struct field. + std::string name; + + // The transform type applied to the source field (e.g., kIdentity, kBucket, + // kTruncate, etc.). + TransformType transformType; + + // Optional parameter for transforms that require configuration + // (e.g., bucket count or truncate width). + std::optional parameter; + + Field( + const std::string& _name, + TransformType _transform, + std::optional _parameter) + : name(_name), transformType(_transform), parameter(_parameter) {} + }; + + const int32_t specId; + const std::vector fields; + + IcebergPartitionSpec(int32_t _specId, const std::vector& _fields) + : specId(_specId), fields(_fields) {} +}; + +} // namespace facebook::velox::connector::hive::iceberg diff --git a/velox/connectors/lakehouse/iceberg/PositionalDeleteFileReader.cpp b/velox/connectors/lakehouse/iceberg/PositionalDeleteFileReader.cpp new file mode 100644 index 000000000000..d36550ac66a8 --- /dev/null +++ b/velox/connectors/lakehouse/iceberg/PositionalDeleteFileReader.cpp @@ -0,0 +1,287 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "velox/connectors/hive/iceberg/PositionalDeleteFileReader.h" + +#include "velox/connectors/hive/HiveConnectorUtil.h" +#include "velox/connectors/hive/TableHandle.h" +#include "velox/connectors/hive/iceberg/IcebergDeleteFile.h" +#include "velox/connectors/hive/iceberg/IcebergMetadataColumns.h" +#include "velox/dwio/common/ReaderFactory.h" + +namespace facebook::velox::connector::hive::iceberg { + +PositionalDeleteFileReader::PositionalDeleteFileReader( + const IcebergDeleteFile& deleteFile, + const std::string& baseFilePath, + FileHandleFactory* fileHandleFactory, + const ConnectorQueryCtx* connectorQueryCtx, + folly::Executor* executor, + const std::shared_ptr& hiveConfig, + const std::shared_ptr& ioStats, + const std::shared_ptr& fsStats, + dwio::common::RuntimeStatistics& runtimeStats, + uint64_t splitOffset, + const std::string& connectorId) + : deleteFile_(deleteFile), + baseFilePath_(baseFilePath), + fileHandleFactory_(fileHandleFactory), + executor_(executor), + connectorQueryCtx_(connectorQueryCtx), + hiveConfig_(hiveConfig), + ioStats_(ioStats), + fsStats_(fsStats), + pool_(connectorQueryCtx->memoryPool()), + filePathColumn_(IcebergMetadataColumn::icebergDeleteFilePathColumn()), + posColumn_(IcebergMetadataColumn::icebergDeletePosColumn()), + splitOffset_(splitOffset), + deleteSplit_(nullptr), + deleteRowReader_(nullptr), + deletePositionsOutput_(nullptr), + deletePositionsOffset_(0), + totalNumRowsScanned_(0) { + VELOX_CHECK(deleteFile_.content == FileContent::kPositionalDeletes); + VELOX_CHECK(deleteFile_.recordCount); + + // TODO: check if the lowerbounds and upperbounds in deleteFile overlap with + // this batch. If not, no need to proceed. + + // Create the ScanSpec for this delete file + auto scanSpec = std::make_shared(""); + scanSpec->addField(posColumn_->name, 0); + auto* pathSpec = scanSpec->getOrCreateChild(filePathColumn_->name); + pathSpec->setFilter(std::make_unique( + std::vector({baseFilePath_}), false)); + + // Create the file schema (in RowType) and split that will be used by readers + std::vector deleteColumnNames( + {filePathColumn_->name, posColumn_->name}); + std::vector> deleteColumnTypes( + {filePathColumn_->type, posColumn_->type}); + RowTypePtr deleteFileSchema = + ROW(std::move(deleteColumnNames), std::move(deleteColumnTypes)); + + deleteSplit_ = std::make_shared( + connectorId, + deleteFile_.filePath, + deleteFile_.fileFormat, + 0, + deleteFile_.fileSizeInBytes); + + // Create the Reader and RowReader + + dwio::common::ReaderOptions deleteReaderOpts(pool_); + configureReaderOptions( + hiveConfig_, + connectorQueryCtx, + deleteFileSchema, + deleteSplit_, + /*tableParameters=*/{}, + deleteReaderOpts); + + const 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); + + // Check if the whole delete file split can be skipped. This could happen when + // 1) the delete file doesn't contain the base file that is being read; 2) The + // delete file does not contain the positions in the current batch for the + // base file. + if (!testFilters( + scanSpec.get(), + deleteReader.get(), + deleteSplit_->filePath, + deleteSplit_->partitionKeys, + {}, + hiveConfig_->readTimestampPartitionValueAsLocalTime( + connectorQueryCtx_->sessionProperties()))) { + // We only count the number of base splits skipped as skippedSplits runtime + // statistics in Velox. Skipped delta split is only counted as skipped + // bytes. + runtimeStats.skippedSplitBytes += deleteSplit_->length; + deleteSplit_.reset(); + return; + } + + dwio::common::RowReaderOptions deleteRowReaderOpts; + configureRowReaderOptions( + {}, + scanSpec, + nullptr, + deleteFileSchema, + deleteSplit_, + nullptr, + nullptr, + deleteRowReaderOpts); + + deleteRowReader_.reset(); + deleteRowReader_ = deleteReader->createRowReader(deleteRowReaderOpts); +} + +void PositionalDeleteFileReader::readDeletePositions( + uint64_t baseReadOffset, + uint64_t size, + BufferPtr deleteBitmapBuffer) { + // We are going to read to the row number up to the end of the batch. For the + // same base file, the deleted rows are in ascending order in the same delete + // file. rowNumberUpperBound is the upperbound for the row number in this + // batch, excluding boundaries + int64_t rowNumberUpperBound = splitOffset_ + baseReadOffset + size; + + // Finish unused delete positions from last batch. + if (deletePositionsOutput_ && + deletePositionsOffset_ < deletePositionsOutput_->size()) { + updateDeleteBitmap( + std::dynamic_pointer_cast(deletePositionsOutput_) + ->childAt(0), + baseReadOffset, + rowNumberUpperBound, + deleteBitmapBuffer); + + if (readFinishedForBatch(rowNumberUpperBound)) { + return; + } + } + + if (!deleteRowReader_ || !deleteSplit_) { + return; + } + + // Read the new delete positions for this batch into deletePositionsOutput_ + // and update the delete bitmap + auto outputType = posColumn_->type; + RowTypePtr outputRowType = ROW({posColumn_->name}, {posColumn_->type}); + if (!deletePositionsOutput_) { + deletePositionsOutput_ = BaseVector::create(outputRowType, 0, pool_); + } + + do { + auto rowsScanned = deleteRowReader_->next(size, deletePositionsOutput_); + totalNumRowsScanned_ += rowsScanned; + + if (rowsScanned > 0) { + VELOX_CHECK( + !deletePositionsOutput_->mayHaveNulls(), + "Iceberg delete file pos column cannot have nulls"); + + auto numDeletedRows = deletePositionsOutput_->size(); + if (numDeletedRows > 0) { + deletePositionsOutput_->loadedVector(); + deletePositionsOffset_ = 0; + + // Convert the row numbers to set bits, up to rowNumberUpperBound. + // Beyond that the buffer of deleteBitMap is not available. + updateDeleteBitmap( + std::dynamic_pointer_cast(deletePositionsOutput_) + ->childAt(0), + baseReadOffset, + rowNumberUpperBound, + deleteBitmapBuffer); + } + } else { + // Reaching the end of the file + deleteSplit_.reset(); + break; + } + } while (!readFinishedForBatch(rowNumberUpperBound)); +} + +bool PositionalDeleteFileReader::noMoreData() { + return totalNumRowsScanned_ >= deleteFile_.recordCount && + deletePositionsOutput_ && + deletePositionsOffset_ >= deletePositionsOutput_->size(); +} + +void PositionalDeleteFileReader::updateDeleteBitmap( + VectorPtr deletePositionsVector, + uint64_t baseReadOffset, + int64_t rowNumberUpperBound, + BufferPtr deleteBitmapBuffer) { + auto deleteBitmap = deleteBitmapBuffer->asMutable(); + + // Convert the positions in file into positions relative to the start of the + // split. + const int64_t* deletePositions = + deletePositionsVector->as>()->rawValues(); + int64_t rowNumberLowerBound = baseReadOffset + splitOffset_; + + // If the rowNumberLowerBound is greater than the last position in this delete + // rows batch, nothing to delete. + if (rowNumberLowerBound > + deletePositions[deletePositionsVector->size() - 1]) { + return; + } + + // Skip the delete positions in deletePositionsVector until they are in the + // [rowNumberLowerBound, rowNumberUpperBound) range. + while (deletePositionsOffset_ < deletePositionsVector->size() && + deletePositions[deletePositionsOffset_] < rowNumberLowerBound) { + deletePositionsOffset_++; + } + while (deletePositionsOffset_ < deletePositionsVector->size() && + deletePositions[deletePositionsOffset_] < rowNumberUpperBound) { + bits::setBit( + deleteBitmap, + deletePositions[deletePositionsOffset_] - rowNumberLowerBound); + deletePositionsOffset_++; + } + + deleteBitmapBuffer->setSize(std::max( + static_cast(deleteBitmapBuffer->size()), + deletePositionsOffset_ == 0 || + (deletePositionsOffset_ < deletePositionsVector->size() && + deletePositions[deletePositionsOffset_] >= rowNumberUpperBound) + ? 0 + : bits::nbytes( + deletePositions[deletePositionsOffset_ - 1] + 1 - + rowNumberLowerBound))); +} + +bool PositionalDeleteFileReader::readFinishedForBatch( + int64_t rowNumberUpperBound) { + VELOX_CHECK_NOT_NULL(deletePositionsOutput_); + + auto deletePositionsVector = + std::dynamic_pointer_cast(deletePositionsOutput_)->childAt(0); + const int64_t* deletePositions = + deletePositionsVector->as>()->rawValues(); + + // We've read enough of the delete positions from this delete file when 1) it + // reaches the end of the file, or 2) the last read delete position is greater + // than the largest base file row number that is going to be read in this + // batch + if (totalNumRowsScanned_ >= deleteFile_.recordCount || + (deletePositionsVector->size() != 0 && + (deletePositionsOffset_ < deletePositionsVector->size() && + deletePositions[deletePositionsOffset_] >= rowNumberUpperBound))) { + return true; + } + return false; +} + +} // namespace facebook::velox::connector::hive::iceberg diff --git a/velox/connectors/lakehouse/iceberg/PositionalDeleteFileReader.h b/velox/connectors/lakehouse/iceberg/PositionalDeleteFileReader.h new file mode 100644 index 000000000000..211359d7fb93 --- /dev/null +++ b/velox/connectors/lakehouse/iceberg/PositionalDeleteFileReader.h @@ -0,0 +1,94 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#pragma once + +#include +#include + +#include "velox/connectors/Connector.h" +#include "velox/connectors/hive/FileHandle.h" +#include "velox/connectors/hive/HiveConfig.h" +#include "velox/connectors/hive/HiveConnectorSplit.h" +#include "velox/dwio/common/Reader.h" + +namespace facebook::velox::connector::hive::iceberg { + +struct IcebergDeleteFile; +struct IcebergMetadataColumn; + +class PositionalDeleteFileReader { + public: + PositionalDeleteFileReader( + const IcebergDeleteFile& deleteFile, + const std::string& baseFilePath, + FileHandleFactory* fileHandleFactory, + const ConnectorQueryCtx* connectorQueryCtx, + folly::Executor* executor, + const std::shared_ptr& hiveConfig, + const std::shared_ptr& ioStats, + const std::shared_ptr& fsStats, + dwio::common::RuntimeStatistics& runtimeStats, + uint64_t splitOffset, + const std::string& connectorId); + + void readDeletePositions( + uint64_t baseReadOffset, + uint64_t size, + BufferPtr deleteBitmap); + + bool noMoreData(); + + private: + void updateDeleteBitmap( + VectorPtr deletePositionsVector, + uint64_t baseReadOffset, + int64_t rowNumberUpperBound, + BufferPtr deleteBitmapBuffer); + + bool readFinishedForBatch(int64_t rowNumberUpperBound); + + const IcebergDeleteFile& deleteFile_; + const std::string& baseFilePath_; + FileHandleFactory* const fileHandleFactory_; + folly::Executor* const executor_; + const ConnectorQueryCtx* connectorQueryCtx_; + const std::shared_ptr hiveConfig_; + const std::shared_ptr ioStats_; + const std::shared_ptr fsStats_; + const std::shared_ptr fsStats; + memory::MemoryPool* const pool_; + + std::shared_ptr filePathColumn_; + std::shared_ptr posColumn_; + uint64_t splitOffset_; + + std::shared_ptr deleteSplit_; + std::unique_ptr deleteRowReader_; + // The vector to hold the delete positions read from the positional delete + // file. These positions are relative to the start of the whole base data + // file. + VectorPtr deletePositionsOutput_; + // The index of deletePositionsOutput_ that indicates up to where the delete + // positions have been converted into the bitmap + uint64_t deletePositionsOffset_; + // Total number of rows read from this positional delete file reader, + // including the rows filtered out from filters on both filePathColumn_ and + // posColumn_. + uint64_t totalNumRowsScanned_; +}; + +} // namespace facebook::velox::connector::hive::iceberg diff --git a/velox/connectors/lakehouse/iceberg/tests/CMakeLists.txt b/velox/connectors/lakehouse/iceberg/tests/CMakeLists.txt new file mode 100644 index 000000000000..fdd6f53f4125 --- /dev/null +++ b/velox/connectors/lakehouse/iceberg/tests/CMakeLists.txt @@ -0,0 +1,72 @@ +# Copyright (c) Facebook, Inc. and its affiliates. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +add_library(velox_dwio_iceberg_reader_benchmark_lib + IcebergSplitReaderBenchmark.cpp) +target_link_libraries( + velox_dwio_iceberg_reader_benchmark_lib + velox_exec_test_lib + velox_exec + velox_hive_connector + Folly::folly + Folly::follybenchmark + ${TEST_LINK_LIBS}) + +if(VELOX_ENABLE_BENCHMARKS) + add_executable(velox_dwio_iceberg_reader_benchmark + IcebergSplitReaderBenchmarkMain.cpp) + target_link_libraries( + velox_dwio_iceberg_reader_benchmark + velox_dwio_iceberg_reader_benchmark_lib + velox_exec_test_lib + velox_exec + velox_hive_connector + Folly::folly + Folly::follybenchmark + ${TEST_LINK_LIBS}) +endif() + +if(NOT VELOX_DISABLE_GOOGLETEST) + + add_executable( + velox_hive_iceberg_test + IcebergInsertTest.cpp + IcebergReadTest.cpp + IcebergSplitReaderBenchmarkTest.cpp + IcebergTestBase.cpp + Main.cpp) + add_test(velox_hive_iceberg_test velox_hive_iceberg_test) + + target_link_libraries( + velox_hive_iceberg_test + velox_dwio_iceberg_reader_benchmark_lib + velox_hive_connector + velox_hive_iceberg_splitreader + velox_hive_partition_function + velox_dwio_common_exception + velox_dwio_common_test_utils + velox_vector_test_lib + velox_vector_fuzzer + velox_exec + velox_exec_test_lib + Folly::folly + Folly::follybenchmark + GTest::gtest + GTest::gtest_main) + + if(VELOX_ENABLE_PARQUET) + target_link_libraries(velox_hive_iceberg_test velox_dwio_parquet_writer + velox_dwio_parquet_reader) + endif() + +endif() diff --git a/velox/connectors/lakehouse/iceberg/tests/IcebergInsertTest.cpp b/velox/connectors/lakehouse/iceberg/tests/IcebergInsertTest.cpp new file mode 100644 index 000000000000..e4a6caf8045d --- /dev/null +++ b/velox/connectors/lakehouse/iceberg/tests/IcebergInsertTest.cpp @@ -0,0 +1,268 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include + +#include "velox/common/base/tests/GTestUtils.h" +#include "velox/connectors/hive/iceberg/tests/IcebergTestBase.h" +#include "velox/exec/tests/utils/PlanBuilder.h" + +namespace facebook::velox::connector::hive::iceberg::test { +class IcebergInsertTest : public IcebergTestBase { + protected: + void SetUp() override { + IcebergTestBase::SetUp(); + rowType_ = + ROW({"c1", "c2", "c3", "c4", "c5", "c6"}, + {BIGINT(), + INTEGER(), + SMALLINT(), + DECIMAL(18, 5), + BOOLEAN(), + VARCHAR()}); + } +}; + +TEST_F(IcebergInsertTest, testIcebergTableWrite) { + const auto outputDirectory = exec::test::TempDirectoryPath::create(); + const auto dataPath = fmt::format("{}", outputDirectory->getPath()); + constexpr int32_t numBatches = 10; + constexpr int32_t vectorSize = 5'000; + const auto vectors = createTestData(numBatches, vectorSize, 0.5); + auto dataSink = + createIcebergDataSink(rowType_, outputDirectory->getPath(), {}); + + for (const auto& vector : vectors) { + dataSink->appendData(vector); + } + + ASSERT_TRUE(dataSink->finish()); + const auto commitTasks = dataSink->close(); + createDuckDbTable(vectors); + auto splits = createSplitsForDirectory(dataPath); + ASSERT_EQ(splits.size(), commitTasks.size()); + auto plan = exec::test::PlanBuilder().tableScan(rowType_).planNode(); + assertQuery(plan, splits, fmt::format("SELECT * FROM tmp")); +} + +TEST_F(IcebergInsertTest, testSingleColumnAsPartition) { + for (auto colIndex = 0; colIndex < rowType_->size() - 1; colIndex++) { + const auto& colName = rowType_->nameOf(colIndex); + const auto colType = rowType_->childAt(colIndex); + + const bool isDecimal = colType->isDecimal(); + const bool isVarbinary = colType->isVarbinary(); + constexpr int32_t numBatches = 2; + constexpr int32_t vectorSize = 50; + const auto outputDirectory = exec::test::TempDirectoryPath::create(); + + if (isDecimal || isVarbinary) { + const auto vectors = createTestData(numBatches, vectorSize, 0.5); + std::vector partitionTransforms = {colName}; + auto dataSink = createIcebergDataSink( + rowType_, outputDirectory->getPath(), partitionTransforms); + for (const auto& vector : vectors) { + if (isDecimal) { + VELOX_ASSERT_THROW( + dataSink->appendData(vector), + "Partition on decimal column is not supported yet."); + } else if (isVarbinary) { + VELOX_ASSERT_THROW( + dataSink->appendData(vector), + "Partition on varbinary column is not supported yet."); + } + } + continue; + } + const auto dataPath = fmt::format("{}", outputDirectory->getPath()); + const auto vectors = createTestData(numBatches, vectorSize, 0.5); + std::vector partitionTransforms = {colName}; + auto dataSink = createIcebergDataSink( + rowType_, outputDirectory->getPath(), partitionTransforms); + + for (const auto& vector : vectors) { + dataSink->appendData(vector); + } + + ASSERT_TRUE(dataSink->finish()); + const auto commitTasks = dataSink->close(); + createDuckDbTable(vectors); + auto splits = createSplitsForDirectory(dataPath); + + ASSERT_GT(commitTasks.size(), 0); + ASSERT_EQ(splits.size(), commitTasks.size()); + + for (const auto& task : commitTasks) { + auto taskJson = folly::parseJson(task); + ASSERT_TRUE(taskJson.count("partitionDataJson") > 0); + ASSERT_FALSE(taskJson["partitionDataJson"].empty()); + } + + connector::ColumnHandleMap assignments; + for (auto i = 0; i < rowType_->size(); i++) { + const auto& name = rowType_->nameOf(i); + if (i != colIndex) { + assignments.insert( + {name, + std::make_shared( + name, + 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) { + for (auto colIndex = 0; colIndex < rowType_->size() - 1; colIndex++) { + const auto& colName = rowType_->nameOf(colIndex); + const auto colType = rowType_->childAt(colIndex); + const auto outputDirectory = exec::test::TempDirectoryPath::create(); + const auto dataPath = fmt::format("{}", outputDirectory->getPath()); + constexpr int32_t numBatches = 2; + constexpr int32_t vectorSize = 100; + const bool isDecimal = colType->isDecimal(); + const bool isVarbinary = colType->isVarbinary(); + + if (isDecimal || isVarbinary) { + const auto vectors = createTestData(numBatches, vectorSize, 0.5); + std::vector partitionTransforms = {colName}; + auto dataSink = createIcebergDataSink( + rowType_, outputDirectory->getPath(), partitionTransforms); + for (const auto& vector : vectors) { + if (isDecimal) { + VELOX_ASSERT_THROW( + dataSink->appendData(vector), + "Partition on decimal column is not supported yet."); + } else if (isVarbinary) { + VELOX_ASSERT_THROW( + dataSink->appendData(vector), + "Partition on varbinary column is not supported yet."); + } + } + continue; + } + + const auto vectors = createTestData(numBatches, vectorSize, 1.0); + std::vector partitionTransforms = {colName}; + auto dataSink = createIcebergDataSink( + rowType_, outputDirectory->getPath(), partitionTransforms); + + for (const auto& vector : vectors) { + dataSink->appendData(vector); + } + + ASSERT_TRUE(dataSink->finish()); + const auto commitTasks = dataSink->close(); + + auto files = listFiles(dataPath); + ASSERT_EQ(files.size(), 1); + + for (const auto& file : files) { + std::vector pathComponents; + folly::split("/", file, pathComponents); + bool foundPartitionDir = false; + for (const auto& component : pathComponents) { + if (component.find('=') != std::string::npos) { + foundPartitionDir = true; + std::vector parts; + folly::split('=', component, parts); + ASSERT_EQ(parts.size(), 2); + ASSERT_EQ(parts[0], colName); + ASSERT_EQ(parts[1], "null"); + } + } + ASSERT_TRUE(foundPartitionDir) + << "No partition directory found in path: " << file; + } + } +} + +TEST_F(IcebergInsertTest, testColumnCombinationsAsPartition) { + std::vector> columnCombinations = { + {0, 1}, // BIGINT, INTEGER. + {2, 1}, // SMALLINT, INTEGER. + {2, 0}, // SMALLINT, BIGINT. + {0, 2, 1} // BIGINT, SMALLINT, INTEGER. + }; + + for (const auto& combination : columnCombinations) { + const auto outputDirectory = exec::test::TempDirectoryPath::create(); + const auto dataPath = fmt::format("{}", outputDirectory->getPath()); + constexpr int32_t numBatches = 2; + constexpr int32_t vectorSize = 50; + const auto vectors = createTestData(numBatches, vectorSize); + std::vector partitionTransforms; + for (auto colIndex : combination) { + partitionTransforms.push_back(rowType_->nameOf(colIndex)); + } + + auto dataSink = createIcebergDataSink( + rowType_, outputDirectory->getPath(), partitionTransforms); + + for (const auto& vector : vectors) { + dataSink->appendData(vector); + } + + ASSERT_TRUE(dataSink->finish()); + const auto commitTasks = dataSink->close(); + createDuckDbTable(vectors); + auto splits = createSplitsForDirectory(dataPath); + + ASSERT_GT(commitTasks.size(), 0); + ASSERT_EQ(splits.size(), commitTasks.size()); + + connector::ColumnHandleMap assignments; + std::unordered_set partitionColumns( + combination.begin(), combination.end()); + + for (auto i = 0; i < rowType_->size(); i++) { + const auto& name = rowType_->nameOf(i); + auto columnType = partitionColumns.count(i) > 0 + ? 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")); + } +} + +} // namespace facebook::velox::connector::hive::iceberg::test diff --git a/velox/connectors/lakehouse/iceberg/tests/IcebergReadTest.cpp b/velox/connectors/lakehouse/iceberg/tests/IcebergReadTest.cpp new file mode 100644 index 000000000000..46544873bfaf --- /dev/null +++ b/velox/connectors/lakehouse/iceberg/tests/IcebergReadTest.cpp @@ -0,0 +1,1272 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "velox/common/base/tests/GTestUtils.h" +#include "velox/common/file/FileSystems.h" +#include "velox/connectors/hive/HiveConnectorSplit.h" +#include "velox/connectors/hive/iceberg/IcebergDeleteFile.h" +#include "velox/connectors/hive/iceberg/IcebergMetadataColumns.h" +#include "velox/connectors/hive/iceberg/IcebergSplit.h" +#include "velox/exec/PlanNodeStats.h" +#include "velox/exec/tests/utils/HiveConnectorTestBase.h" +#include "velox/exec/tests/utils/PlanBuilder.h" + +#include + +using namespace facebook::velox::exec::test; +using namespace facebook::velox::exec; +using namespace facebook::velox::dwio::common; +using namespace facebook::velox::test; + +namespace facebook::velox::connector::hive::iceberg { + +class HiveIcebergTest : public HiveConnectorTestBase { + public: + HiveIcebergTest() + : config_{std::make_shared()} { + // Make the writers flush per batch so that we can create non-aligned + // RowGroups between the base data files and delete files + flushPolicyFactory_ = []() { + return std::make_unique([]() { return true; }); + }; + } + + /// Create 1 base data file data_file_1 with 2 RowGroups of 10000 rows each. + /// Also create 1 delete file delete_file_1 which contains delete positions + /// for data_file_1. + void assertSingleBaseFileSingleDeleteFile( + const std::vector& deletePositionsVec) { + std::map> rowGroupSizesForFiles = { + {"data_file_1", {10000, 10000}}}; + std::unordered_map< + std::string, + std::multimap>> + deleteFilesForBaseDatafiles = { + {"delete_file_1", {{"data_file_1", deletePositionsVec}}}}; + + assertPositionalDeletes( + rowGroupSizesForFiles, deleteFilesForBaseDatafiles, 0); + } + + /// Create 3 base data files, where the first file data_file_0 has 500 rows, + /// the second file data_file_1 contains 2 RowGroups of 10000 rows each, and + /// the third file data_file_2 contains 500 rows. It creates 1 positional + /// delete file delete_file_1, which contains delete positions for + /// data_file_1. + void assertMultipleBaseFileSingleDeleteFile( + const std::vector& deletePositionsVec) { + int64_t previousFileRowCount = 500; + int64_t afterFileRowCount = 500; + + assertPositionalDeletes( + { + {"data_file_0", {previousFileRowCount}}, + {"data_file_1", {10000, 10000}}, + {"data_file_2", {afterFileRowCount}}, + }, + {{"delete_file_1", {{"data_file_1", deletePositionsVec}}}}, + 0); + } + + /// Create 1 base data file data_file_1 with 2 RowGroups of 10000 rows each. + /// Create multiple delete files with name data_file_1, data_file_2, and so on + void assertSingleBaseFileMultipleDeleteFiles( + const std::vector>& deletePositionsVecs) { + std::map> rowGroupSizesForFiles = { + {"data_file_1", {10000, 10000}}}; + + std::unordered_map< + std::string, + std::multimap>> + deleteFilesForBaseDatafiles; + for (int i = 0; i < deletePositionsVecs.size(); i++) { + std::string deleteFileName = fmt::format("delete_file_{}", i); + deleteFilesForBaseDatafiles[deleteFileName] = { + {"data_file_1", deletePositionsVecs[i]}}; + } + assertPositionalDeletes( + rowGroupSizesForFiles, deleteFilesForBaseDatafiles, 0); + } + + void assertMultipleSplits( + const std::vector& deletePositions, + int32_t fileCount, + int32_t numPrefetchSplits, + int rowCountPerFile = rowCount, + int32_t splitCountPerFile = 1) { + std::map> rowGroupSizesForFiles; + for (int32_t i = 0; i < fileCount; i++) { + std::string dataFileName = fmt::format("data_file_{}", i); + rowGroupSizesForFiles[dataFileName] = {rowCountPerFile}; + } + + std::unordered_map< + std::string, + std::multimap>> + deleteFilesForBaseDatafiles; + for (int i = 0; i < fileCount; i++) { + std::string deleteFileName = fmt::format("delete_file_{}", i); + deleteFilesForBaseDatafiles[deleteFileName] = { + {fmt::format("data_file_{}", i), deletePositions}}; + } + + assertPositionalDeletes( + rowGroupSizesForFiles, + deleteFilesForBaseDatafiles, + numPrefetchSplits, + splitCountPerFile); + } + + std::vector makeRandomIncreasingValues(int64_t begin, int64_t end) { + VELOX_CHECK(begin < end); + + std::mt19937 gen{0}; + std::vector values; + values.reserve(end - begin); + for (int i = begin; i < end; i++) { + if (folly::Random::rand32(0, 10, gen) > 8) { + values.push_back(i); + } + } + return values; + } + + std::vector makeContinuousIncreasingValues( + int64_t begin, + int64_t end) { + std::vector values; + values.resize(end - begin); + std::iota(values.begin(), values.end(), begin); + return values; + } + + /// @rowGroupSizesForFiles The key is the file name, and the value is a vector + /// of RowGroup sizes + /// @deleteFilesForBaseDatafiles The key is the delete file name, and the + /// value contains the information about the content of this delete file. + /// e.g. { + /// "delete_file_1", + /// { + /// {"data_file_1", {1, 2, 3}}, + /// {"data_file_1", {4, 5, 6}}, + /// {"data_file_2", {0, 2, 4}} + /// } + /// } + /// represents one delete file called delete_file_1, which contains delete + /// positions for data_file_1 and data_file_2. THere are 3 RowGroups in this + /// delete file, the first two contain positions for data_file_1, and the last + /// contain positions for data_file_2 + void assertPositionalDeletes( + const std::map>& rowGroupSizesForFiles, + const std::unordered_map< + std::string, + std::multimap>>& + deleteFilesForBaseDatafiles, + int32_t numPrefetchSplits = 0, + int32_t splitCount = 1) { + // Keep the reference to the deleteFilePath, otherwise the corresponding + // file will be deleted. + std::map> dataFilePaths = + writeDataFiles(rowGroupSizesForFiles); + std::unordered_map< + std::string, + std::pair>> + deleteFilePaths = writePositionDeleteFiles( + deleteFilesForBaseDatafiles, dataFilePaths); + + std::vector> splits; + + for (const auto& dataFile : dataFilePaths) { + std::string baseFileName = dataFile.first; + std::string baseFilePath = dataFile.second->getPath(); + + std::vector deleteFiles; + + for (auto const& deleteFile : deleteFilesForBaseDatafiles) { + std::string deleteFileName = deleteFile.first; + std::multimap> deleteFileContent = + deleteFile.second; + + if (deleteFileContent.count(baseFileName) != 0) { + // If this delete file contains rows for the target base file, then + // add it to the split + auto deleteFilePath = + deleteFilePaths[deleteFileName].second->getPath(); + IcebergDeleteFile icebergDeleteFile( + FileContent::kPositionalDeletes, + deleteFilePath, + fileFormat_, + deleteFilePaths[deleteFileName].first, + testing::internal::GetFileSize( + std::fopen(deleteFilePath.c_str(), "r"))); + deleteFiles.push_back(icebergDeleteFile); + } + } + + auto icebergSplits = + makeIcebergSplits(baseFilePath, deleteFiles, {}, splitCount); + splits.insert(splits.end(), icebergSplits.begin(), icebergSplits.end()); + } + + std::string duckdbSql = + getDuckDBQuery(rowGroupSizesForFiles, deleteFilesForBaseDatafiles); + auto plan = tableScanNode(rowType_); + auto task = HiveConnectorTestBase::assertQuery( + plan, splits, duckdbSql, numPrefetchSplits); + + auto planStats = toPlanStats(task->taskStats()); + auto scanNodeId = plan->id(); + auto it = planStats.find(scanNodeId); + ASSERT_TRUE(it != planStats.end()); + ASSERT_TRUE(it->second.peakMemoryBytes > 0); + } + + void assertEqualityDeletes( + const std::unordered_map>>& + equalityDeleteVectorMap, + const std::unordered_map>& + equalityFieldIdsMap, + std::string duckDbSql = "", + std::vector dataVectors = {}) { + VELOX_CHECK_EQ(equalityDeleteVectorMap.size(), equalityFieldIdsMap.size()); + // We will create data vectors with numColumns number of columns that is the + // max field Id in equalityFieldIds + int32_t numDataColumns = 0; + + for (auto it = equalityFieldIdsMap.begin(); it != equalityFieldIdsMap.end(); + ++it) { + auto equalityFieldIds = it->second; + auto currentMax = + *std::max_element(equalityFieldIds.begin(), equalityFieldIds.end()); + numDataColumns = std::max(numDataColumns, currentMax); + } + + VELOX_CHECK_GT(numDataColumns, 0); + VELOX_CHECK_GE(numDataColumns, equalityDeleteVectorMap.size()); + VELOX_CHECK_GT(equalityDeleteVectorMap.size(), 0); + + VELOX_CHECK_LE(equalityFieldIdsMap.size(), numDataColumns); + + std::shared_ptr dataFilePath = + writeDataFiles(rowCount, numDataColumns, 1, dataVectors)[0]; + + std::vector deleteFiles; + std::string predicates = ""; + unsigned long numDeletedValues = 0; + + std::vector> deleteFilePaths; + for (auto it = equalityFieldIdsMap.begin(); + it != equalityFieldIdsMap.end();) { + auto equalityFieldIds = it->second; + auto equalityDeleteVector = equalityDeleteVectorMap.at(it->first); + VELOX_CHECK_GT(equalityDeleteVector.size(), 0); + numDeletedValues = + std::max(numDeletedValues, equalityDeleteVector[0].size()); + deleteFilePaths.push_back(writeEqualityDeleteFile(equalityDeleteVector)); + IcebergDeleteFile deleteFile( + FileContent::kEqualityDeletes, + deleteFilePaths.back()->getPath(), + fileFormat_, + equalityDeleteVector[0].size(), + testing::internal::GetFileSize( + std::fopen(deleteFilePaths.back()->getPath().c_str(), "r")), + equalityFieldIds); + deleteFiles.push_back(deleteFile); + predicates += makePredicates(equalityDeleteVector, equalityFieldIds); + ++it; + if (it != equalityFieldIdsMap.end()) { + predicates += " AND "; + } + } + + // The default split count is 1. + auto icebergSplits = + makeIcebergSplits(dataFilePath->getPath(), deleteFiles); + + // If the caller passed in a query, use that. + if (duckDbSql == "") { + // Select all columns + duckDbSql = "SELECT * FROM tmp "; + if (numDeletedValues > 0) { + duckDbSql += fmt::format("WHERE {}", predicates); + } + } + + assertEqualityDeletes( + icebergSplits.back(), + !dataVectors.empty() ? asRowType(dataVectors[0]->type()) : rowType_, + duckDbSql); + + // Select a column that's not in the filter columns + if (numDataColumns > 1 && + equalityDeleteVectorMap.at(0).size() < numDataColumns) { + std::string duckDbQuery = "SELECT c0 FROM tmp"; + if (numDeletedValues > 0) { + duckDbQuery += fmt::format(" WHERE {}", predicates); + } + + std::vector names({"c0"}); + std::vector types(1, BIGINT()); + assertEqualityDeletes( + icebergSplits.back(), + std::make_shared(std::move(names), std::move(types)), + duckDbQuery); + } + } + + std::vector makeSequenceValues(int32_t numRows, int8_t repeat = 1) { + VELOX_CHECK_GT(repeat, 0); + + auto maxValue = std::ceil((double)numRows / repeat); + std::vector values; + values.reserve(numRows); + for (int32_t i = 0; i < maxValue; i++) { + for (int8_t j = 0; j < repeat; j++) { + values.push_back(i); + } + } + values.resize(numRows); + return values; + } + + std::vector makeRandomDeleteValues(int32_t maxRowNumber) { + std::mt19937 gen{0}; + std::vector deleteRows; + for (int i = 0; i < maxRowNumber; i++) { + if (folly::Random::rand32(0, 10, gen) > 8) { + deleteRows.push_back(i); + } + } + return deleteRows; + } + + const static int rowCount = 20000; + + protected: + std::shared_ptr config_; + std::function()> flushPolicyFactory_; + + std::vector> makeIcebergSplits( + const std::string& dataFilePath, + const std::vector& deleteFiles = {}, + const std::unordered_map>& + partitionKeys = {}, + const uint32_t splitCount = 1) { + std::unordered_map customSplitInfo; + customSplitInfo["table_format"] = "hive-iceberg"; + + auto file = filesystems::getFileSystem(dataFilePath, nullptr) + ->openFileForRead(dataFilePath); + const int64_t fileSize = file->size(); + std::vector> splits; + const uint64_t splitSize = std::floor((fileSize) / splitCount); + + for (int i = 0; i < splitCount; ++i) { + splits.emplace_back(std::make_shared( + kHiveConnectorId, + dataFilePath, + fileFormat_, + i * splitSize, + splitSize, + partitionKeys, + std::nullopt, + customSplitInfo, + nullptr, + /*cacheable=*/true, + deleteFiles)); + } + + return splits; + } + + void assertEqualityDeletes( + std::shared_ptr split, + RowTypePtr outputRowType, + const std::string& duckDbSql) { + auto plan = tableScanNode(outputRowType); + auto task = OperatorTestBase::assertQuery(plan, {split}, duckDbSql); + + auto planStats = toPlanStats(task->taskStats()); + auto scanNodeId = plan->id(); + auto it = planStats.find(scanNodeId); + ASSERT_TRUE(it != planStats.end()); + ASSERT_TRUE(it->second.peakMemoryBytes > 0); + } + + private: + std::map> writeDataFiles( + std::map> rowGroupSizesForFiles) { + std::map> dataFilePaths; + + std::vector dataVectorsJoined; + dataVectorsJoined.reserve(rowGroupSizesForFiles.size()); + + int64_t startingValue = 0; + for (auto& dataFile : rowGroupSizesForFiles) { + dataFilePaths[dataFile.first] = TempFilePath::create(); + + // We make the values are continuously increasing even across base data + // files. This is to make constructing DuckDB queries easier + std::vector dataVectors = + makeVectors(dataFile.second, startingValue); + writeToFile( + dataFilePaths[dataFile.first]->getPath(), + dataVectors, + config_, + flushPolicyFactory_); + + for (int i = 0; i < dataVectors.size(); i++) { + dataVectorsJoined.push_back(dataVectors[i]); + } + } + + createDuckDbTable(dataVectorsJoined); + return dataFilePaths; + } + + /// Input is like <"deleteFile1", <"dataFile1", {pos_RG1, pos_RG2,..}>, + /// <"dataFile2", {pos_RG1, pos_RG2,..}> + std::unordered_map< + std::string, + std::pair>> + writePositionDeleteFiles( + const std::unordered_map< + std::string, // delete file name + std::multimap< + std::string, + std::vector>>& + deleteFilesForBaseDatafiles, // + std::map> baseFilePaths) { + std::unordered_map< + std::string, + std::pair>> + deleteFilePaths; + deleteFilePaths.reserve(deleteFilesForBaseDatafiles.size()); + + for (auto& deleteFile : deleteFilesForBaseDatafiles) { + auto deleteFileName = deleteFile.first; + auto deleteFileContent = deleteFile.second; + auto deleteFilePath = TempFilePath::create(); + + std::vector deleteFileVectors; + int64_t totalPositionsInDeleteFile = 0; + + for (auto& deleteFileRowGroup : deleteFileContent) { + auto baseFileName = deleteFileRowGroup.first; + auto baseFilePath = baseFilePaths[baseFileName]->getPath(); + auto positionsInRowGroup = deleteFileRowGroup.second; + + auto filePathVector = makeFlatVector( + static_cast(positionsInRowGroup.size()), + [&](vector_size_t row) { return baseFilePath; }); + auto deletePosVector = makeFlatVector(positionsInRowGroup); + + RowVectorPtr deleteFileVector = makeRowVector( + {pathColumn_->name, posColumn_->name}, + {filePathVector, deletePosVector}); + + deleteFileVectors.push_back(deleteFileVector); + totalPositionsInDeleteFile += positionsInRowGroup.size(); + } + + writeToFile( + deleteFilePath->getPath(), + deleteFileVectors, + config_, + flushPolicyFactory_); + + deleteFilePaths[deleteFileName] = + std::make_pair(totalPositionsInDeleteFile, deleteFilePath); + } + + return deleteFilePaths; + } + + std::vector makeVectors( + std::vector vectorSizes, + int64_t& startingValue) { + std::vector vectors; + vectors.reserve(vectorSizes.size()); + + vectors.reserve(vectorSizes.size()); + for (int j = 0; j < vectorSizes.size(); j++) { + auto data = makeContinuousIncreasingValues( + startingValue, startingValue + vectorSizes[j]); + VectorPtr c0 = makeFlatVector(data); + vectors.push_back(makeRowVector({"c0"}, {c0})); + startingValue += vectorSizes[j]; + } + + return vectors; + } + + std::string getDuckDBQuery( + const std::map>& rowGroupSizesForFiles, + const std::unordered_map< + std::string, + std::multimap>>& + deleteFilesForBaseDatafiles) { + int64_t totalNumRowsInAllBaseFiles = 0; + std::map baseFileSizes; + for (auto rowGroupSizesInFile : rowGroupSizesForFiles) { + // Sum up the row counts in all RowGroups in each base file + baseFileSizes[rowGroupSizesInFile.first] += std::accumulate( + rowGroupSizesInFile.second.begin(), + rowGroupSizesInFile.second.end(), + 0LL); + totalNumRowsInAllBaseFiles += baseFileSizes[rowGroupSizesInFile.first]; + } + + // Group the delete vectors by baseFileName + std::map>> + deletePosVectorsForAllBaseFiles; + for (auto deleteFile : deleteFilesForBaseDatafiles) { + auto deleteFileContent = deleteFile.second; + for (auto rowGroup : deleteFileContent) { + auto baseFileName = rowGroup.first; + deletePosVectorsForAllBaseFiles[baseFileName].push_back( + rowGroup.second); + } + } + + // Flatten and deduplicate the delete position vectors in + // deletePosVectorsForAllBaseFiles from previous step, and count the total + // number of distinct delete positions for all base files + std::map> + flattenedDeletePosVectorsForAllBaseFiles; + int64_t totalNumDeletePositions = 0; + for (auto deleteVectorsForBaseFile : deletePosVectorsForAllBaseFiles) { + auto baseFileName = deleteVectorsForBaseFile.first; + auto deletePositionVectors = deleteVectorsForBaseFile.second; + std::vector deletePositionVector = + flattenAndDedup(deletePositionVectors, baseFileSizes[baseFileName]); + flattenedDeletePosVectorsForAllBaseFiles[baseFileName] = + deletePositionVector; + totalNumDeletePositions += deletePositionVector.size(); + } + + // Now build the DuckDB queries + if (totalNumDeletePositions == 0) { + return "SELECT * FROM tmp"; + } else if (totalNumDeletePositions >= totalNumRowsInAllBaseFiles) { + return "SELECT * FROM tmp WHERE 1 = 0"; + } else { + // Convert the delete positions in all base files into column values + std::vector allDeleteValues; + + int64_t numRowsInPreviousBaseFiles = 0; + for (auto baseFileSize : baseFileSizes) { + auto deletePositions = + flattenedDeletePosVectorsForAllBaseFiles[baseFileSize.first]; + + if (numRowsInPreviousBaseFiles > 0) { + for (int64_t& deleteValue : deletePositions) { + deleteValue += numRowsInPreviousBaseFiles; + } + } + + allDeleteValues.insert( + allDeleteValues.end(), + deletePositions.begin(), + deletePositions.end()); + + numRowsInPreviousBaseFiles += baseFileSize.second; + } + + return fmt::format( + "SELECT * FROM tmp WHERE c0 NOT IN ({})", + makeNotInList(allDeleteValues)); + } + } + + std::vector flattenAndDedup( + const std::vector>& deletePositionVectors, + int64_t baseFileSize) { + std::vector deletePositionVector; + for (auto vec : deletePositionVectors) { + for (auto pos : vec) { + if (pos >= 0 && pos < baseFileSize) { + deletePositionVector.push_back(pos); + } + } + } + + std::sort(deletePositionVector.begin(), deletePositionVector.end()); + auto last = + std::unique(deletePositionVector.begin(), deletePositionVector.end()); + deletePositionVector.erase(last, deletePositionVector.end()); + + return deletePositionVector; + } + + std::string makeNotInList(const std::vector& deletePositionVector) { + if (deletePositionVector.empty()) { + return ""; + } + + return std::accumulate( + deletePositionVector.begin() + 1, + deletePositionVector.end(), + std::to_string(deletePositionVector[0]), + [](const std::string& a, int64_t b) { + return a + ", " + std::to_string(b); + }); + } + + core::PlanNodePtr tableScanNode(RowTypePtr outputRowType) { + return PlanBuilder(pool_.get()).tableScan(outputRowType).planNode(); + } + + std::string makePredicates( + const std::vector>& equalityDeleteVector, + const std::vector& equalityFieldIds) { + std::string predicates(""); + int32_t numDataColumns = + *std::max_element(equalityFieldIds.begin(), equalityFieldIds.end()); + + VELOX_CHECK_GT(numDataColumns, 0); + VELOX_CHECK_GE(numDataColumns, equalityDeleteVector.size()); + VELOX_CHECK_GT(equalityDeleteVector.size(), 0); + + auto numDeletedValues = equalityDeleteVector[0].size(); + + if (numDeletedValues == 0) { + return predicates; + } + + // If all values for a column are deleted, just return an always-false + // predicate + for (auto i = 0; i < equalityDeleteVector.size(); i++) { + auto equalityFieldId = equalityFieldIds[i]; + auto deleteValues = equalityDeleteVector[i]; + + auto lastIter = std::unique(deleteValues.begin(), deleteValues.end()); + auto numDistinctValues = lastIter - deleteValues.begin(); + auto minValue = 1; + auto maxValue = *std::max_element(deleteValues.begin(), lastIter); + if (maxValue - minValue + 1 == numDistinctValues && + maxValue == (rowCount - 1) / equalityFieldId) { + return "1 = 0"; + } + } + + if (equalityDeleteVector.size() == 1) { + std::string name = fmt::format("c{}", equalityFieldIds[0] - 1); + predicates = fmt::format( + "{} NOT IN ({})", name, makeNotInList({equalityDeleteVector[0]})); + } else { + for (int i = 0; i < numDeletedValues; i++) { + std::string oneRow(""); + for (int j = 0; j < equalityFieldIds.size(); j++) { + std::string name = fmt::format("c{}", equalityFieldIds[j] - 1); + std::string predicate = + fmt::format("({} <> {})", name, equalityDeleteVector[j][i]); + + oneRow = oneRow == "" ? predicate + : fmt::format("({} OR {})", oneRow, predicate); + } + + predicates = predicates == "" + ? oneRow + : fmt::format("{} AND {}", predicates, oneRow); + } + } + return predicates; + } + + std::shared_ptr pathColumn_ = + IcebergMetadataColumn::icebergDeleteFilePathColumn(); + std::shared_ptr posColumn_ = + IcebergMetadataColumn::icebergDeletePosColumn(); + + protected: + RowTypePtr rowType_{ROW({"c0"}, {BIGINT()})}; + dwio::common::FileFormat fileFormat_{dwio::common::FileFormat::DWRF}; + + std::shared_ptr writeEqualityDeleteFile( + const std::vector>& equalityDeleteVector) { + std::vector names; + std::vector vectors; + for (int i = 0; i < equalityDeleteVector.size(); i++) { + names.push_back(fmt::format("c{}", i)); + vectors.push_back(makeFlatVector(equalityDeleteVector[i])); + } + + RowVectorPtr deleteFileVectors = makeRowVector(names, vectors); + + auto deleteFilePath = 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 +/// passed to assertSingleBaseFileSingleDeleteFile is the delete positions. +TEST_F(HiveIcebergTest, singleBaseFileSinglePositionalDeleteFile) { + folly::SingletonVault::singleton()->registrationComplete(); + + assertSingleBaseFileSingleDeleteFile({{0, 1, 2, 3}}); + // Delete the first and last row in each batch (10000 rows per batch) + assertSingleBaseFileSingleDeleteFile({{0, 9999, 10000, 19999}}); + // Delete several rows in the second batch (10000 rows per batch) + assertSingleBaseFileSingleDeleteFile({{10000, 10002, 19999}}); + // Delete random rows + assertSingleBaseFileSingleDeleteFile({makeRandomIncreasingValues(0, 20000)}); + // Delete 0 rows + assertSingleBaseFileSingleDeleteFile({}); + // Delete all rows + assertSingleBaseFileSingleDeleteFile( + {makeContinuousIncreasingValues(0, 20000)}); + // Delete rows that don't exist + assertSingleBaseFileSingleDeleteFile({{20000, 29999}}); +} + +/// This test creates 3 base data files, only the middle one has corresponding +/// delete positions. The parameter passed to +/// assertSingleBaseFileSingleDeleteFile is the delete positions.for the middle +/// base file. +TEST_F(HiveIcebergTest, MultipleBaseFilesSinglePositionalDeleteFile) { + folly::SingletonVault::singleton()->registrationComplete(); + + assertMultipleBaseFileSingleDeleteFile({0, 1, 2, 3}); + assertMultipleBaseFileSingleDeleteFile({0, 9999, 10000, 19999}); + assertMultipleBaseFileSingleDeleteFile({10000, 10002, 19999}); + assertMultipleBaseFileSingleDeleteFile({10000, 10002, 19999}); + assertMultipleBaseFileSingleDeleteFile( + makeRandomIncreasingValues(0, rowCount)); + assertMultipleBaseFileSingleDeleteFile({}); + assertMultipleBaseFileSingleDeleteFile( + makeContinuousIncreasingValues(0, rowCount)); +} + +/// This test creates one base data file/split with multiple delete files. The +/// parameter passed to assertSingleBaseFileMultipleDeleteFiles is the vector of +/// delete files. Each leaf vector represents the delete positions in that +/// delete file. +TEST_F(HiveIcebergTest, singleBaseFileMultiplePositionalDeleteFiles) { + folly::SingletonVault::singleton()->registrationComplete(); + + // Delete row 0, 1, 2, 3 from the first batch out of two. + assertSingleBaseFileMultipleDeleteFiles({{1}, {2}, {3}, {4}}); + // Delete the first and last row in each batch (10000 rows per batch). + assertSingleBaseFileMultipleDeleteFiles({{0}, {9999}, {10000}, {19999}}); + + assertSingleBaseFileMultipleDeleteFiles({{500, 21000}}); + + assertSingleBaseFileMultipleDeleteFiles( + {makeRandomIncreasingValues(0, 10000), + makeRandomIncreasingValues(10000, 20000), + makeRandomIncreasingValues(5000, 15000)}); + + assertSingleBaseFileMultipleDeleteFiles( + {makeContinuousIncreasingValues(0, 10000), + makeContinuousIncreasingValues(10000, 20000)}); + + assertSingleBaseFileMultipleDeleteFiles( + {makeContinuousIncreasingValues(0, 10000), + makeContinuousIncreasingValues(10000, 20000), + makeRandomIncreasingValues(5000, 15000)}); + + assertSingleBaseFileMultipleDeleteFiles( + {makeContinuousIncreasingValues(0, 20000), + makeContinuousIncreasingValues(0, 20000)}); + + assertSingleBaseFileMultipleDeleteFiles( + {makeRandomIncreasingValues(0, 20000), + {}, + makeRandomIncreasingValues(5000, 15000)}); + + assertSingleBaseFileMultipleDeleteFiles({{}, {}}); +} + +/// This test creates 2 base data files, and 1 or 2 delete files, with unaligned +/// RowGroup boundaries +TEST_F(HiveIcebergTest, multipleBaseFileMultiplePositionalDeleteFiles) { + folly::SingletonVault::singleton()->registrationComplete(); + + std::map> rowGroupSizesForFiles; + std::unordered_map< + std::string, + std::multimap>> + deleteFilesForBaseDatafiles; + + // Create two data files, each with two RowGroups + rowGroupSizesForFiles["data_file_1"] = {100, 85}; + rowGroupSizesForFiles["data_file_2"] = {99, 1}; + + // Delete 3 rows from the first RowGroup in data_file_1 + deleteFilesForBaseDatafiles["delete_file_1"] = {{"data_file_1", {0, 1, 99}}}; + assertPositionalDeletes(rowGroupSizesForFiles, deleteFilesForBaseDatafiles); + + // Delete 3 rows from the second RowGroup in data_file_1 + deleteFilesForBaseDatafiles["delete_file_1"] = { + {"data_file_1", {100, 101, 184}}}; + assertPositionalDeletes(rowGroupSizesForFiles, deleteFilesForBaseDatafiles); + + // Delete random rows from the both RowGroups in data_file_1 + deleteFilesForBaseDatafiles["delete_file_1"] = { + {"data_file_1", makeRandomIncreasingValues(0, 185)}}; + assertPositionalDeletes(rowGroupSizesForFiles, deleteFilesForBaseDatafiles); + + // Delete all rows in data_file_1 + deleteFilesForBaseDatafiles["delete_file_1"] = { + {"data_file_1", makeContinuousIncreasingValues(0, 185)}}; + assertPositionalDeletes(rowGroupSizesForFiles, deleteFilesForBaseDatafiles); + // + // Delete non-existent rows from data_file_1 + deleteFilesForBaseDatafiles["delete_file_1"] = { + {"data_file_1", makeRandomIncreasingValues(186, 300)}}; + assertPositionalDeletes(rowGroupSizesForFiles, deleteFilesForBaseDatafiles); + + // Delete several rows from both RowGroups in both data files + deleteFilesForBaseDatafiles.clear(); + deleteFilesForBaseDatafiles["delete_file_1"] = { + {"data_file_1", {0, 100, 102, 184}}, {"data_file_2", {1, 98, 99}}}; + assertPositionalDeletes(rowGroupSizesForFiles, deleteFilesForBaseDatafiles); + + // The delete file delete_file_1 contains 3 RowGroups itself, with the first 3 + // deleting some repeating rows in data_file_1, and the last 2 RowGroups + // deleting some repeating rows in data_file_2 + deleteFilesForBaseDatafiles.clear(); + deleteFilesForBaseDatafiles["delete_file_1"] = { + {"data_file_1", {0, 1, 2, 3}}, + {"data_file_1", {1, 2, 3, 4}}, + {"data_file_1", makeRandomIncreasingValues(0, 185)}, + {"data_file_2", {1, 3, 5, 7}}, + {"data_file_2", makeRandomIncreasingValues(0, 100)}}; + assertPositionalDeletes(rowGroupSizesForFiles, deleteFilesForBaseDatafiles); + + // delete_file_2 contains non-overlapping delete rows for each data files in + // each RowGroup + deleteFilesForBaseDatafiles.clear(); + deleteFilesForBaseDatafiles["delete_file_1"] = { + {"data_file_1", {0, 1, 2, 3}}, {"data_file_2", {1, 3, 5, 7}}}; + deleteFilesForBaseDatafiles["delete_file_2"] = { + {"data_file_1", {1, 2, 3, 4}}, + {"data_file_1", {98, 99, 100, 101, 184}}, + {"data_file_2", {3, 5, 7, 9}}, + {"data_file_2", {98, 99, 100}}}; + assertPositionalDeletes(rowGroupSizesForFiles, deleteFilesForBaseDatafiles); + + // Two delete files each containing overlapping delete rows for both data + // files + deleteFilesForBaseDatafiles.clear(); + deleteFilesForBaseDatafiles["delete_file_1"] = { + {"data_file_1", makeRandomIncreasingValues(0, 185)}, + {"data_file_2", makeRandomIncreasingValues(0, 100)}}; + deleteFilesForBaseDatafiles["delete_file_2"] = { + {"data_file_1", makeRandomIncreasingValues(10, 120)}, + {"data_file_2", makeRandomIncreasingValues(50, 100)}}; + assertPositionalDeletes(rowGroupSizesForFiles, deleteFilesForBaseDatafiles); +} + +TEST_F(HiveIcebergTest, positionalDeletesMultipleSplits) { + folly::SingletonVault::singleton()->registrationComplete(); + + assertMultipleSplits({1, 2, 3, 4}, 10, 5); + assertMultipleSplits({1, 2, 3, 4}, 10, 0); + assertMultipleSplits({1, 2, 3, 4}, 10, 10); + assertMultipleSplits({0, 9999, 10000, 19999}, 10, 3); + assertMultipleSplits(makeRandomIncreasingValues(0, 20000), 10, 3); + assertMultipleSplits(makeContinuousIncreasingValues(0, 20000), 10, 3); + assertMultipleSplits({}, 10, 3); + + assertMultipleSplits({1, 2, 3, 4}, 10, 5, 30000, 3); + assertPositionalDeletes( + { + {"data_file_0", {500}}, + {"data_file_1", {10000, 10000}}, + {"data_file_2", {500}}, + }, + {{"delete_file_1", + {{"data_file_1", makeRandomIncreasingValues(0, 20000)}}}}, + 0, + 3); + + // Include only upper bound(which is exclusive) in delete positions for the + // second 10k batch of rows. + assertMultipleSplits({1000, 9000, 20000}, 1, 0, 20000, 3); +} + +TEST_F(HiveIcebergTest, testPartitionedRead) { + RowTypePtr rowType{ROW({"c0", "ds"}, {BIGINT(), DateType::get()})}; + std::unordered_map> partitionKeys; + // Iceberg API sets partition values for dates to daysSinceEpoch, so + // in velox, we do not need to convert it to days. + // Test query on two partitions ds=17627(2018-04-06), ds=17628(2018-04-07) + std::vector> splits; + std::vector> dataFilePaths; + for (int i = 0; i <= 1; ++i) { + std::vector dataVectors; + int32_t daysSinceEpoch = 17627 + i; + VectorPtr c0 = makeFlatVector((std::vector){i}); + VectorPtr ds = + makeFlatVector((std::vector){daysSinceEpoch}); + dataVectors.push_back(makeRowVector({"c0", "ds"}, {c0, ds})); + + auto dataFilePath = TempFilePath::create(); + dataFilePaths.push_back(dataFilePath); + writeToFile( + dataFilePath->getPath(), dataVectors, config_, flushPolicyFactory_); + partitionKeys["ds"] = std::to_string(daysSinceEpoch); + auto icebergSplits = + makeIcebergSplits(dataFilePath->getPath(), {}, partitionKeys); + splits.insert(splits.end(), icebergSplits.begin(), icebergSplits.end()); + } + + connector::ColumnHandleMap assignments; + assignments.insert( + {"c0", + std::make_shared( + "c0", + HiveColumnHandle::ColumnType::kRegular, + rowType->childAt(0), + rowType->childAt(0))}); + + std::vector requiredSubFields; + HiveColumnHandle::ColumnParseParameters columnParseParameters; + columnParseParameters.partitionDateValueFormat = + HiveColumnHandle::ColumnParseParameters::kDaysSinceEpoch; + assignments.insert( + {"ds", + std::make_shared( + "ds", + HiveColumnHandle::ColumnType::kPartitionKey, + rowType->childAt(1), + rowType->childAt(1), + std::move(requiredSubFields), + columnParseParameters)}); + + auto plan = PlanBuilder(pool_.get()) + .tableScan(rowType, {}, "", nullptr, assignments) + .planNode(); + + HiveConnectorTestBase::assertQuery( + plan, + splits, + "SELECT * FROM (VALUES (0, '2018-04-06'), (1, '2018-04-07'))", + 0); + + // Test filter on non-partitioned non-date column + std::vector nonPartitionFilters = {"c0 = 1"}; + plan = PlanBuilder(pool_.get()) + .tableScan(rowType, nonPartitionFilters, "", nullptr, assignments) + .planNode(); + + HiveConnectorTestBase::assertQuery(plan, splits, "SELECT 1, '2018-04-07'"); + + // Test filter on non-partitioned date column + std::vector filters = {"ds = date'2018-04-06'"}; + plan = PlanBuilder(pool_.get()).tableScan(rowType, filters).planNode(); + + splits.clear(); + for (auto& dataFilePath : dataFilePaths) { + auto icebergSplits = makeIcebergSplits(dataFilePath->getPath()); + splits.insert(splits.end(), icebergSplits.begin(), icebergSplits.end()); + } + + HiveConnectorTestBase::assertQuery(plan, splits, "SELECT 0, '2018-04-06'"); +} + +// Delete values from a single column file +TEST_F(HiveIcebergTest, equalityDeletesSingleFileColumn1) { + folly::SingletonVault::singleton()->registrationComplete(); + + std::unordered_map> equalityFieldIdsMap; + std::unordered_map>> + equalityDeleteVectorMap; + equalityFieldIdsMap.insert({0, {1}}); + + // Delete row 0, 1, 2, 3 from the first batch out of two. + equalityDeleteVectorMap.insert({0, {{0, 1, 2, 3}}}); + assertEqualityDeletes(equalityDeleteVectorMap, equalityFieldIdsMap); + + // Delete the first and last row in each batch (10000 rows per batch) + equalityDeleteVectorMap.clear(); + equalityDeleteVectorMap.insert({0, {{0, 9999, 10000, 19999}}}); + assertEqualityDeletes(equalityDeleteVectorMap, equalityFieldIdsMap); + + // Delete several rows in the second batch (10000 rows per batch) + equalityDeleteVectorMap.clear(); + equalityDeleteVectorMap.insert({0, {{10000, 10002, 19999}}}); + assertEqualityDeletes(equalityDeleteVectorMap, equalityFieldIdsMap); + + // Delete random rows + equalityDeleteVectorMap.clear(); + equalityDeleteVectorMap.insert({0, {makeRandomDeleteValues(rowCount)}}); + assertEqualityDeletes(equalityDeleteVectorMap, equalityFieldIdsMap); + + // Delete 0 rows + equalityDeleteVectorMap.clear(); + equalityDeleteVectorMap.insert({0, {{}}}); + assertEqualityDeletes(equalityDeleteVectorMap, equalityFieldIdsMap); + + // Delete all rows + equalityDeleteVectorMap.insert({0, {makeSequenceValues(rowCount)}}); + assertEqualityDeletes(equalityDeleteVectorMap, equalityFieldIdsMap); + + // Delete rows that don't exist + equalityDeleteVectorMap.clear(); + equalityDeleteVectorMap.insert({0, {{20000, 29999}}}); + assertEqualityDeletes(equalityDeleteVectorMap, equalityFieldIdsMap); +} + +// Delete values from the second column in a 2-column file +// +// c1 c2 +// 0 0 +// 1 0 +// 2 1 +// 3 1 +// 4 2 +// ... ... +// 19999 9999 +TEST_F(HiveIcebergTest, equalityDeletesSingleFileColumn2) { + folly::SingletonVault::singleton()->registrationComplete(); + + std::unordered_map> equalityFieldIdsMap; + std::unordered_map>> + equalityDeleteVectorMap; + equalityFieldIdsMap.insert({0, {2}}); + + // Delete values 0, 1, 2, 3 from the second column + equalityDeleteVectorMap.insert({0, {{0, 1, 2, 3}}}); + assertEqualityDeletes(equalityDeleteVectorMap, equalityFieldIdsMap); + + // Delete the smallest value 0 and the largest value 9999 from the second + // column, which has the range [0, 9999] + equalityDeleteVectorMap.clear(); + equalityDeleteVectorMap.insert({0, {{0, 9999}}}); + assertEqualityDeletes(equalityDeleteVectorMap, equalityFieldIdsMap); + + // Delete non-existent values from the second column + equalityDeleteVectorMap.clear(); + equalityDeleteVectorMap.insert({0, {{10000, 10002, 19999}}}); + assertEqualityDeletes(equalityDeleteVectorMap, equalityFieldIdsMap); + + // Delete random rows from the second column + equalityDeleteVectorMap.clear(); + equalityDeleteVectorMap.insert({0, {makeSequenceValues(rowCount)}}); + assertEqualityDeletes(equalityDeleteVectorMap, equalityFieldIdsMap); + + // Delete 0 values + equalityDeleteVectorMap.clear(); + equalityDeleteVectorMap.insert({0, {{}}}); + assertEqualityDeletes(equalityDeleteVectorMap, equalityFieldIdsMap); + + // Delete all values + equalityDeleteVectorMap.clear(); + equalityDeleteVectorMap.insert({0, {makeSequenceValues(rowCount / 2)}}); + assertEqualityDeletes(equalityDeleteVectorMap, equalityFieldIdsMap); +} + +// Delete values from 2 columns with the following data: +// +// c1 c2 +// 0 0 +// 1 0 +// 2 1 +// 3 1 +// 4 2 +// ... ... +// 19999 9999 +TEST_F(HiveIcebergTest, equalityDeletesSingleFileMultipleColumns) { + folly::SingletonVault::singleton()->registrationComplete(); + + std::unordered_map> equalityFieldIdsMap; + std::unordered_map>> + equalityDeleteVectorMap; + equalityFieldIdsMap.insert({0, {1, 2}}); + + // Delete rows 0, 1 + equalityDeleteVectorMap.insert({0, {{0, 1}, {0, 0}}}); + assertEqualityDeletes(equalityDeleteVectorMap, equalityFieldIdsMap); + + // Delete rows 0, 2, 4, 6 + equalityDeleteVectorMap.clear(); + equalityDeleteVectorMap.insert({0, {{0, 2, 4, 6}, {0, 1, 2, 3}}}); + assertEqualityDeletes(equalityDeleteVectorMap, equalityFieldIdsMap); + + // Delete the last row + equalityDeleteVectorMap.clear(); + equalityDeleteVectorMap.insert({0, {{19999}, {9999}}}); + assertEqualityDeletes(equalityDeleteVectorMap, equalityFieldIdsMap); + + // Delete non-existent values + equalityDeleteVectorMap.clear(); + equalityDeleteVectorMap.insert({0, {{20000, 30000}, {10000, 1500}}}); + assertEqualityDeletes(equalityDeleteVectorMap, equalityFieldIdsMap); + + // Delete 0 values + equalityDeleteVectorMap.clear(); + equalityDeleteVectorMap.insert({0, {{}, {}}}); + assertEqualityDeletes(equalityDeleteVectorMap, equalityFieldIdsMap); + + // Delete all values + equalityDeleteVectorMap.clear(); + equalityDeleteVectorMap.insert( + {0, {makeSequenceValues(rowCount), makeSequenceValues(rowCount, 2)}}); + assertEqualityDeletes( + equalityDeleteVectorMap, + equalityFieldIdsMap, + "SELECT * FROM tmp WHERE 1 = 0"); +} + +TEST_F(HiveIcebergTest, equalityDeletesMultipleFiles) { + folly::SingletonVault::singleton()->registrationComplete(); + + std::unordered_map> equalityFieldIdsMap; + std::unordered_map>> + equalityDeleteVectorMap; + equalityFieldIdsMap.insert({{0, {1}}, {1, {2}}}); + + // Delete rows {0, 1} from c0, {2, 3} from c1, with two equality delete files + equalityDeleteVectorMap.insert({{0, {{0, 1}}}, {1, {{2, 3}}}}); + assertEqualityDeletes(equalityDeleteVectorMap, equalityFieldIdsMap); + + // Delete using 3 equality delete files + equalityFieldIdsMap.insert({{2, {3}}}); + equalityDeleteVectorMap.insert({{0, {{0, 1}}}, {1, {{2, 3}}}, {2, {{4, 5}}}}); + assertEqualityDeletes(equalityDeleteVectorMap, equalityFieldIdsMap); + + // Delete 0 values + equalityDeleteVectorMap.clear(); + equalityDeleteVectorMap.insert({{0, {{}}}, {1, {{}}}, {2, {{}}}}); + assertEqualityDeletes(equalityDeleteVectorMap, equalityFieldIdsMap); + + // Delete all values + equalityDeleteVectorMap.clear(); + equalityDeleteVectorMap.insert( + {{0, {makeSequenceValues(rowCount)}}, + {1, {makeSequenceValues(rowCount)}}, + {2, {makeSequenceValues(rowCount)}}}); + assertEqualityDeletes( + equalityDeleteVectorMap, + equalityFieldIdsMap, + "SELECT * FROM tmp WHERE 1 = 0"); +} + +TEST_F(HiveIcebergTest, TestSubFieldEqualityDelete) { + folly::SingletonVault::singleton()->registrationComplete(); + + // Write the base file + std::shared_ptr dataFilePath = TempFilePath::create(); + std::vector dataVectors = {makeRowVector( + {"c_bigint", "c_row"}, + {makeFlatVector(20, [](auto row) { return row + 1; }), + makeRowVector( + {"c0", "c1", "c2"}, + {makeFlatVector(20, [](auto row) { return row + 1; }), + makeFlatVector(20, [](auto row) { return row + 1; }), + makeFlatVector(20, [](auto row) { return row + 1; })})})}; + int32_t numDataColumns = 1; + dataFilePath = writeDataFiles(rowCount, numDataColumns, 1, dataVectors)[0]; + + // Write the delete file. Equality delete field is c_row.c1 + std::vector deleteFiles; + // Delete rows {0, 1} from c_row.c1, whose schema Id is 4 + std::vector deleteDataVectors = {makeRowVector( + {"c1"}, {makeFlatVector(2, [](auto row) { return row + 1; })})}; + + std::vector> deleteFilePaths; + auto equalityFieldIds = std::vector({4}); + auto deleteFilePath = TempFilePath::create(); + writeToFile(deleteFilePath->getPath(), deleteDataVectors.back()); + deleteFilePaths.push_back(deleteFilePath); + IcebergDeleteFile deleteFile( + FileContent::kEqualityDeletes, + deleteFilePaths.back()->getPath(), + fileFormat_, + 2, + testing::internal::GetFileSize( + std::fopen(deleteFilePaths.back()->getPath().c_str(), "r")), + equalityFieldIds); + deleteFiles.push_back(deleteFile); + + auto icebergSplits = makeIcebergSplits(dataFilePath->getPath(), deleteFiles); + + // Select both c_bigint and c_row column columns + std::string duckDbSql = "SELECT * FROM tmp WHERE c_row.c0 not in (1, 2)"; + assertEqualityDeletes( + icebergSplits.back(), asRowType(dataVectors[0]->type()), duckDbSql); + + // SELECT only c_bigint column + duckDbSql = "SELECT c_bigint FROM tmp WHERE c_row.c0 not in (1, 2)"; + assertEqualityDeletes( + icebergSplits.back(), ROW({"c_bigint"}, {BIGINT()}), duckDbSql); +} +} // namespace facebook::velox::connector::hive::iceberg diff --git a/velox/connectors/lakehouse/iceberg/tests/IcebergSplitReaderBenchmark.cpp b/velox/connectors/lakehouse/iceberg/tests/IcebergSplitReaderBenchmark.cpp new file mode 100644 index 000000000000..d835d7239b71 --- /dev/null +++ b/velox/connectors/lakehouse/iceberg/tests/IcebergSplitReaderBenchmark.cpp @@ -0,0 +1,405 @@ +/* + * 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/tests/IcebergSplitReaderBenchmark.h" + +#include + +#include + +#include "velox/vector/tests/utils/VectorMaker.h" + +using namespace facebook::velox; +using namespace facebook::velox::dwio; +using namespace facebook::velox::dwio::common; +using namespace facebook::velox::dwrf; +using namespace facebook::velox::connector::hive; +using namespace facebook::velox::connector::hive::iceberg; +using namespace facebook::velox::memory; + +namespace facebook::velox::iceberg::reader::test { +void IcebergSplitReaderBenchmark::writeToFile( + const std::vector& batches) { + auto path = fileFolder_->getPath() + "/" + fileName_; + auto localWriteFile = std::make_unique(path, true, false); + auto sink = std::make_unique(std::move(localWriteFile), path); + dwrf::WriterOptions options; + options.memoryPool = rootPool_.get(); + options.schema = batches[0]->type(); + dwrf::Writer dataFilewriter{std::move(sink), options}; + for (auto& batch : batches) { + dataFilewriter.write(batch); + } + dataFilewriter.flush(); + dataFilewriter.close(); +} + +void IcebergSplitReaderBenchmark::writeToPositionDeleteFile( + const std::string& filePath, + const std::vector& vectors) { + auto localPosWriteFile = + std::make_unique(filePath, true, false); + auto posDeletesink = + std::make_unique(std::move(localPosWriteFile), filePath); + dwrf::WriterOptions options; + options.memoryPool = rootPool_.get(); + options.schema = vectors[0]->type(); + dwrf::Writer posDeletewriter{std::move(posDeletesink), options}; + for (size_t i = 0; i < vectors.size(); ++i) { + posDeletewriter.write(vectors[i]); + } + posDeletewriter.close(); +} + +std::vector IcebergSplitReaderBenchmark::makeRandomDeleteRows( + int32_t deleteRowsCount) { + std::random_device rd; + std::mt19937 gen(rd()); + int64_t totalDataRows = kNumBatches * kNumRowsPerBatch; + std::uniform_int_distribution<> dis(0, totalDataRows - 1); + std::set uniqueDeleteRows; + while (uniqueDeleteRows.size() < deleteRowsCount) { + uniqueDeleteRows.insert(dis(gen)); + } + std::vector deleteRows( + uniqueDeleteRows.begin(), uniqueDeleteRows.end()); + return deleteRows; +} + +std::vector IcebergSplitReaderBenchmark::makeSequenceRows( + int32_t maxRowNumber) { + std::vector deleteRows; + deleteRows.resize(maxRowNumber); + std::iota(deleteRows.begin(), deleteRows.end(), 0); + return deleteRows; +} + +std::vector IcebergSplitReaderBenchmark::listFiles( + const std::string& dirPath) { + std::vector files; + for (auto& dirEntry : + std::filesystem::recursive_directory_iterator(dirPath)) { + if (dirEntry.is_regular_file()) { + files.push_back(dirEntry.path().string()); + } + } + return files; +} + +std::shared_ptr +IcebergSplitReaderBenchmark::makeIcebergSplit( + const std::string& dataFilePath, + const std::vector& deleteFiles) { + std::unordered_map> partitionKeys; + std::unordered_map customSplitInfo; + customSplitInfo["table_format"] = "hive-iceberg"; + + auto readFile = std::make_shared(dataFilePath); + const int64_t fileSize = readFile->size(); + + return std::make_shared( + kHiveConnectorId, + dataFilePath, + fileFomat_, + 0, + fileSize, + partitionKeys, + std::nullopt, + customSplitInfo, + nullptr, + /*cacheable=*/true, + deleteFiles); +} + +std::string IcebergSplitReaderBenchmark::writePositionDeleteFile( + const std::string& dataFilePath, + int64_t numDeleteRows) { + facebook::velox::test::VectorMaker vectorMaker{leafPool_.get()}; + auto filePathVector = + vectorMaker.flatVector(numDeleteRows, [&](auto row) { + if (row < numDeleteRows) { + return StringView(dataFilePath); + } else { + return StringView(); + } + }); + + std::vector deleteRowsVec; + deleteRowsVec.reserve(numDeleteRows); + auto deleteRows = makeRandomDeleteRows(numDeleteRows); + deleteRowsVec.insert( + deleteRowsVec.end(), deleteRows.begin(), deleteRows.end()); + + auto deletePositionsVector = vectorMaker.flatVector(deleteRowsVec); + + std::shared_ptr pathColumn = + IcebergMetadataColumn::icebergDeleteFilePathColumn(); + std::shared_ptr posColumn = + IcebergMetadataColumn::icebergDeletePosColumn(); + RowVectorPtr deleteFileVectors = vectorMaker.rowVector( + {pathColumn->name, posColumn->name}, + {filePathVector, deletePositionsVector}); + + auto deleteFilePath = deleteFileFolder_->getPath() + "/" + "posDelete.data"; + writeToPositionDeleteFile(deleteFilePath, std::vector{deleteFileVectors}); + + return deleteFilePath; +} + +std::vector> +IcebergSplitReaderBenchmark::createIcebergSplitsWithPositionalDelete( + int32_t deleteRowsPercentage, + int32_t deleteFilesCount) { + std::vector> splits; + + std::vector deleteFilePaths; + std::vector dataFilePaths = listFiles(fileFolder_->getPath()); + + for (const auto& dataFilePath : dataFilePaths) { + std::vector deleteFiles; + int64_t deleteRowsCount = + kNumBatches * kNumRowsPerBatch * deleteRowsPercentage * 0.01; + deleteFiles.reserve(deleteRowsCount); + for (int i = 0; i < deleteFilesCount; i++) { + std::string deleteFilePath = + writePositionDeleteFile(dataFilePath, deleteRowsCount); + + IcebergDeleteFile deleteFile( + FileContent::kPositionalDeletes, + deleteFilePath, + fileFomat_, + deleteRowsCount, + testing::internal::GetFileSize( + std::fopen(deleteFilePath.c_str(), "r"))); + deleteFilePaths.emplace_back(deleteFilePath); + deleteFiles.emplace_back(deleteFile); + } + splits.emplace_back(makeIcebergSplit(dataFilePath, deleteFiles)); + } + return splits; +} + +FilterSpec IcebergSplitReaderBenchmark::createFilterSpec( + const std::string& columnName, + float startPct, + float selectPct, + const TypePtr& type, + bool isForRowGroupSkip, + bool allowNulls) { + switch (type->childAt(0)->kind()) { + case TypeKind::BIGINT: + case TypeKind::INTEGER: + return FilterSpec( + columnName, + startPct, + selectPct, + FilterKind::kBigintRange, + isForRowGroupSkip, + allowNulls); + default: + VELOX_FAIL("Unsupported Data Type {}", type->childAt(0)->toString()); + } + return FilterSpec(columnName, startPct, selectPct, FilterKind(), false); +} + +std::shared_ptr IcebergSplitReaderBenchmark::createScanSpec( + const std::vector& batches, + RowTypePtr& rowType, + const std::vector& filterSpecs, + std::vector& hitRows, + SubfieldFilters& filters) { + std::unique_ptr filterGenerator = + std::make_unique(rowType, 0); + filters = filterGenerator->makeSubfieldFilters( + filterSpecs, batches, nullptr, hitRows); + auto scanSpec = filterGenerator->makeScanSpec(std::move(filters)); + return scanSpec; +} + +// This method is the place where we do the read operations using +// icebergSplitReader. scanSpec contains the setting of filters. e.g. +// filterRateX100 = 30 means it would filter out 70% of rows and 30% remain. +// deleteRateX100 = 30 means it would delete 30% of overall data rows and 70% +// remain. Return the number of rows after the filter and delete. +int IcebergSplitReaderBenchmark::read( + const RowTypePtr& rowType, + uint32_t nextSize, + std::unique_ptr icebergSplitReader) { + runtimeStats_ = RuntimeStatistics(); + icebergSplitReader->resetFilterCaches(); + int resultSize = 0; + auto result = BaseVector::create(rowType, 0, leafPool_.get()); + while (true) { + bool hasData = icebergSplitReader->next(nextSize, result); + if (!hasData) { + break; + } + auto rowsRemaining = result->size(); + resultSize += rowsRemaining; + } + icebergSplitReader->updateRuntimeStats(runtimeStats_); + return resultSize; +} + +void IcebergSplitReaderBenchmark::readSingleColumn( + const std::string& columnName, + const TypePtr& type, + float startPct, + float selectPct, + float deletePct, + uint32_t nextSize) { + folly::BenchmarkSuspender suspender; + auto rowType = ROW({columnName}, {type}); + + auto batches = + dataSetBuilder_->makeDataset(rowType, kNumBatches, kNumRowsPerBatch) + .withRowGroupSpecificData(kNumRowsPerRowGroup) + .withNullsForField(Subfield(columnName), 0) + .build(); + writeToFile(*batches); + std::vector filterSpecs; + + filterSpecs.emplace_back( + createFilterSpec(columnName, startPct, selectPct, rowType, false, false)); + + std::vector hitRows; + SubfieldFilters filters; + auto scanSpec = + createScanSpec(*batches, rowType, filterSpecs, hitRows, filters); + + std::vector> splits = + createIcebergSplitsWithPositionalDelete(deletePct, 1); + + core::TypedExprPtr remainingFilterExpr; + + std::shared_ptr hiveTableHandle = + std::make_shared( + "kHiveConnectorId", + "tableName", + false, + std::move(filters), + remainingFilterExpr, + rowType); + + std::shared_ptr hiveConfig = + std::make_shared(std::make_shared( + std::unordered_map(), true)); + const RowTypePtr readerOutputType; + const std::shared_ptr ioStats = + std::make_shared(); + const std::shared_ptr fsStats = + std::make_shared(); + + std::shared_ptr root = + memory::memoryManager()->addRootPool( + "IcebergSplitReader", kMaxMemory, MemoryReclaimer::create()); + std::shared_ptr opPool = root->addLeafChild("operator"); + std::shared_ptr connectorPool = + root->addAggregateChild(kHiveConnectorId, MemoryReclaimer::create()); + std::shared_ptr connectorSessionProperties_ = + std::make_shared( + std::unordered_map()); + + std::unique_ptr connectorQueryCtx_ = + std::make_unique( + opPool.get(), + connectorPool.get(), + connectorSessionProperties_.get(), + nullptr, + common::PrefixSortConfig(), + nullptr, + nullptr, + "query.IcebergSplitReader", + "task.IcebergSplitReader", + "planNodeId.IcebergSplitReader", + 0, + ""); + + FileHandleFactory fileHandleFactory( + std::make_unique>( + hiveConfig->numCacheFileHandles()), + std::make_unique(connectorSessionProperties_)); + + suspender.dismiss(); + + auto ioExecutor = std::make_unique(3); + std::shared_ptr remainingFilterExprSet{nullptr}; + std::atomic totalRemainingFilterMs; + + uint64_t resultSize = 0; + for (std::shared_ptr split : splits) { + scanSpec->resetCachedValues(true); + + std::unique_ptr icebergSplitReader = + std::make_unique( + split, + hiveTableHandle, + nullptr, + connectorQueryCtx_.get(), + hiveConfig, + rowType, + ioStats, + fsStats, + &fileHandleFactory, + ioExecutor.get(), + scanSpec, + connectorQueryCtx_->expressionEvaluator(), + totalRemainingFilterMs); + + std::shared_ptr randomSkip; + icebergSplitReader->configureReaderOptions(randomSkip); + icebergSplitReader->prepareSplit(nullptr, runtimeStats_); + + // Filter range is generated from a small sample data of 4096 rows. So the + // upperBound and lowerBound are introduced to estimate the result size. + resultSize += read(rowType, nextSize, std::move(icebergSplitReader)); + } + // Calculate the expected number of rows after the filters. + // Add one to expected to avoid 0 in calculating upperBound and lowerBound. + int expected = kNumBatches * kNumRowsPerBatch * ((double)selectPct / 100) * + (1 - (double)deletePct / 100) + + 1; + + // Make the upperBound and lowerBound large enough to avoid very small + // resultSize and expected size, where the diff ratio is relatively very + // large. + int upperBound = expected * (1 + kFilterErrorMargin) + 1; + int lowerBound = expected * (1 - kFilterErrorMargin) - 1; + upperBound = std::max(16, upperBound); + lowerBound = std::max(0, lowerBound); + + VELOX_CHECK( + resultSize <= upperBound && resultSize >= lowerBound, + "Result Size {} and Expected Size {} Mismatch", + resultSize, + expected); +} + +void run( + uint32_t, + const std::string& columnName, + const TypePtr& type, + float filterRateX100, + float deleteRateX100, + uint32_t nextSize) { + RowTypePtr rowType = ROW({columnName}, {type}); + IcebergSplitReaderBenchmark benchmark; + BIGINT()->toString(); + benchmark.readSingleColumn( + columnName, type, 0, filterRateX100, deleteRateX100, nextSize); +} + +} // namespace facebook::velox::iceberg::reader::test diff --git a/velox/connectors/lakehouse/iceberg/tests/IcebergSplitReaderBenchmark.h b/velox/connectors/lakehouse/iceberg/tests/IcebergSplitReaderBenchmark.h new file mode 100644 index 000000000000..3408fa4ce837 --- /dev/null +++ b/velox/connectors/lakehouse/iceberg/tests/IcebergSplitReaderBenchmark.h @@ -0,0 +1,134 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#pragma once + +#include "velox/common/file/FileSystems.h" +#include "velox/connectors/hive/TableHandle.h" +#include "velox/connectors/hive/iceberg/IcebergDeleteFile.h" +#include "velox/connectors/hive/iceberg/IcebergMetadataColumns.h" +#include "velox/connectors/hive/iceberg/IcebergSplit.h" +#include "velox/connectors/hive/iceberg/IcebergSplitReader.h" +#include "velox/dwio/common/tests/utils/DataSetBuilder.h" +#include "velox/dwio/dwrf/RegisterDwrfReader.h" +#include "velox/dwio/dwrf/writer/Writer.h" +#include "velox/exec/tests/utils/TempDirectoryPath.h" +#include "velox/vector/tests/utils/VectorTestBase.h" + +#include +#include + +namespace facebook::velox::iceberg::reader::test { + +constexpr uint32_t kNumRowsPerBatch = 20000; +constexpr uint32_t kNumBatches = 50; +constexpr uint32_t kNumRowsPerRowGroup = 10000; +constexpr double kFilterErrorMargin = 0.2; + +class IcebergSplitReaderBenchmark { + public: + explicit IcebergSplitReaderBenchmark() { + rootPool_ = + memory::memoryManager()->addRootPool("IcebergSplitReaderBenchmark"); + leafPool_ = rootPool_->addLeafChild("IcebergSplitReaderBenchmark"); + dataSetBuilder_ = + std::make_unique(*leafPool_, 0); + filesystems::registerLocalFileSystem(); + dwrf::registerDwrfReaderFactory(); + } + + ~IcebergSplitReaderBenchmark() {} + + void writeToFile(const std::vector& batches); + + void writeToPositionDeleteFile( + const std::string& filePath, + const std::vector& vectors); + + dwio::common::FilterSpec createFilterSpec( + const std::string& columnName, + float startPct, + float selectPct, + const TypePtr& type, + bool isForRowGroupSkip, + bool allowNulls); + + std::shared_ptr createScanSpec( + const std::vector& batches, + RowTypePtr& rowType, + const std::vector& filterSpecs, + std::vector& hitRows, + common::SubfieldFilters& filters); + + int read( + const RowTypePtr& rowType, + uint32_t nextSize, + std::unique_ptr + icebergSplitReader); + + void readSingleColumn( + const std::string& columnName, + const TypePtr& type, + float startPct, + float selectPct, + float deleteRate, + uint32_t nextSize); + + std::vector> + createIcebergSplitsWithPositionalDelete( + int32_t deleteRowsPercentage, + int32_t deleteFilesCount); + + std::vector listFiles(const std::string& dirPath); + + std::shared_ptr makeIcebergSplit( + const std::string& dataFilePath, + const std::vector& + deleteFiles = {}); + + std::vector makeRandomDeleteRows(int32_t deleteRowsCount); + + std::vector makeSequenceRows(int32_t maxRowNumber); + + std::string writePositionDeleteFile( + const std::string& dataFilePath, + int64_t numDeleteRows); + + private: + const std::string fileName_ = "test.data"; + const std::shared_ptr fileFolder_ = + exec::test::TempDirectoryPath::create(); + const std::shared_ptr deleteFileFolder_ = + exec::test::TempDirectoryPath::create(); + + std::unique_ptr dataSetBuilder_; + std::shared_ptr rootPool_; + std::shared_ptr leafPool_; + std::unique_ptr writer_; + dwio::common::RuntimeStatistics runtimeStats_; + + dwio::common::FileFormat fileFomat_{dwio::common::FileFormat::DWRF}; + const std::string kHiveConnectorId = "hive-iceberg"; +}; + +void run( + uint32_t, + const std::string& columnName, + const TypePtr& type, + float filterRateX100, + float deleteRateX100, + uint32_t nextSize); + +} // namespace facebook::velox::iceberg::reader::test diff --git a/velox/connectors/lakehouse/iceberg/tests/IcebergSplitReaderBenchmarkMain.cpp b/velox/connectors/lakehouse/iceberg/tests/IcebergSplitReaderBenchmarkMain.cpp new file mode 100644 index 000000000000..dd33c1ccf53a --- /dev/null +++ b/velox/connectors/lakehouse/iceberg/tests/IcebergSplitReaderBenchmarkMain.cpp @@ -0,0 +1,67 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "velox/connectors/hive/iceberg/tests/IcebergSplitReaderBenchmark.h" + +using namespace facebook::velox; +using namespace facebook::velox::dwio; +using namespace facebook::velox::dwio::common; +using namespace facebook::velox::dwrf; +using namespace facebook::velox::iceberg::reader::test; +using namespace facebook::velox::test; + +#define PARQUET_BENCHMARKS_FILTER_DELETES(_type_, _name_, _filter_, _deletes_) \ + BENCHMARK_NAMED_PARAM( \ + run, \ + _name_##_Filter_##_filter_##_Delete_##_deletes_##_next_5k, \ + #_name_, \ + _type_, \ + _filter_, \ + _deletes_, \ + 5000); \ + BENCHMARK_NAMED_PARAM( \ + run, \ + _name_##_Filter_##_filter_##_Delete_##_deletes_##_next_10k, \ + #_name_, \ + _type_, \ + _filter_, \ + _deletes_, \ + 10000); \ + BENCHMARK_DRAW_LINE(); + +#define PARQUET_BENCHMARKS_FILTERS(_type_, _name_, _filter_) \ + PARQUET_BENCHMARKS_FILTER_DELETES(_type_, _name_, _filter_, 0) \ + PARQUET_BENCHMARKS_FILTER_DELETES(_type_, _name_, _filter_, 20) \ + PARQUET_BENCHMARKS_FILTER_DELETES(_type_, _name_, _filter_, 50) \ + PARQUET_BENCHMARKS_FILTER_DELETES(_type_, _name_, _filter_, 70) \ + PARQUET_BENCHMARKS_FILTER_DELETES(_type_, _name_, _filter_, 100) + +#define PARQUET_BENCHMARKS(_type_, _name_) \ + PARQUET_BENCHMARKS_FILTERS(_type_, _name_, 0) \ + PARQUET_BENCHMARKS_FILTERS(_type_, _name_, 20) \ + PARQUET_BENCHMARKS_FILTERS(_type_, _name_, 50) \ + PARQUET_BENCHMARKS_FILTERS(_type_, _name_, 70) \ + PARQUET_BENCHMARKS_FILTERS(_type_, _name_, 100) \ + BENCHMARK_DRAW_LINE(); + +PARQUET_BENCHMARKS(BIGINT(), BigInt); + +int main(int argc, char** argv) { + folly::Init init{&argc, &argv}; + memory::MemoryManager::initialize(memory::MemoryManager::Options{}); + folly::runBenchmarks(); + return 0; +} diff --git a/velox/connectors/lakehouse/iceberg/tests/IcebergSplitReaderBenchmarkTest.cpp b/velox/connectors/lakehouse/iceberg/tests/IcebergSplitReaderBenchmarkTest.cpp new file mode 100644 index 000000000000..0811ceb8b5d7 --- /dev/null +++ b/velox/connectors/lakehouse/iceberg/tests/IcebergSplitReaderBenchmarkTest.cpp @@ -0,0 +1,30 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "velox/connectors/hive/iceberg/tests/IcebergSplitReaderBenchmark.h" +#include + +namespace facebook::velox::iceberg::reader::test { +namespace { +TEST(IcebergSplitReaderBenchmarkTest, basic) { + memory::MemoryManager::testingSetInstance(memory::MemoryManager::Options{}); + run(1, "BigInt", BIGINT(), 20, 0, 500); + run(1, "BigInt", BIGINT(), 50, 20, 500); + run(1, "BigInt", BIGINT(), 100, 20, 500); + run(1, "BigInt", BIGINT(), 100, 100, 500); +} +} // namespace +} // namespace facebook::velox::iceberg::reader::test diff --git a/velox/connectors/lakehouse/iceberg/tests/IcebergTestBase.cpp b/velox/connectors/lakehouse/iceberg/tests/IcebergTestBase.cpp new file mode 100644 index 000000000000..2ffd5694d655 --- /dev/null +++ b/velox/connectors/lakehouse/iceberg/tests/IcebergTestBase.cpp @@ -0,0 +1,240 @@ +/* + * 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/tests/IcebergTestBase.h" + +#include + +#include "velox/connectors/hive/iceberg/PartitionSpec.h" + +namespace facebook::velox::connector::hive::iceberg::test { +void IcebergTestBase::SetUp() { + HiveConnectorTestBase::SetUp(); + parquet::registerParquetReaderFactory(); + parquet::registerParquetWriterFactory(); + Type::registerSerDe(); + + // Initialize session properties and config. + connectorSessionProperties_ = std::make_shared( + std::unordered_map(), true); + + connectorConfig_ = + std::make_shared(std::make_shared( + std::unordered_map())); + + setupMemoryPools("IcebergTestBase"); + + // Initialize vector fuzzer for test data generation. + fuzzerOptions_.vectorSize = 100; + fuzzerOptions_.nullRatio = 0.1; + fuzzer_ = std::make_unique(fuzzerOptions_, opPool_.get()); + + vectorMaker_ = + std::make_unique(opPool_.get()); +} + +void IcebergTestBase::TearDown() { + vectorMaker_.reset(); + fuzzer_.reset(); + connectorQueryCtx_.reset(); + connectorPool_.reset(); + opPool_.reset(); + root_.reset(); + HiveConnectorTestBase::TearDown(); +} + +void IcebergTestBase::setupMemoryPools(const std::string& name) { + root_.reset(); + opPool_.reset(); + connectorPool_.reset(); + connectorQueryCtx_.reset(); + + root_ = memory::memoryManager()->addRootPool( + name, 1L << 30, exec::MemoryReclaimer::create()); + opPool_ = root_->addLeafChild("operator"); + connectorPool_ = + root_->addAggregateChild("connector", exec::MemoryReclaimer::create()); + + connectorQueryCtx_ = std::make_unique( + opPool_.get(), + connectorPool_.get(), + connectorSessionProperties_.get(), + nullptr, + common::PrefixSortConfig(), + nullptr, + nullptr, + "query" + name, + "task" + name, + "planNodeId" + name, + 0, + ""); +} + +std::vector IcebergTestBase::createTestData( + int32_t numBatches, + vector_size_t rowsPerBatch, + double nullRatio) { + std::vector vectors; + vectors.reserve(numBatches); + + fuzzerOptions_.nullRatio = nullRatio; + fuzzerOptions_.allowDictionaryVector = false; + fuzzer_->setOptions(fuzzerOptions_); + + for (auto i = 0; i < numBatches; ++i) { + vectors.push_back(fuzzer_->fuzzRow(rowType_, rowsPerBatch, false)); + } + + return vectors; +} + +std::shared_ptr IcebergTestBase::createPartitionSpec( + const std::vector& transformSpecs) { + std::vector fields; + static const std::regex identityRegex(R"(([a-z_][a-z0-9_]*))"); + + for (const auto& spec : transformSpecs) { + TransformType transformType = TransformType::kIdentity; + std::string name; + std::smatch matches; + + if (std::regex_match(spec, matches, identityRegex)) { + transformType = TransformType::kIdentity; + name = matches[1]; + } else { + VELOX_FAIL("Unsupported transform specification: {}", spec); + } + + fields.push_back( + IcebergPartitionSpec::Field(name, transformType, std::nullopt)); + } + + return std::make_shared(1, fields); +} + +std::shared_ptr +IcebergTestBase::createIcebergInsertTableHandle( + const RowTypePtr& rowType, + const std::string& outputDirectoryPath, + const std::vector& partitionTransforms) { + std::vector> columnHandles; + for (auto i = 0; i < rowType->size(); ++i) { + auto columnName = rowType->nameOf(i); + auto columnType = HiveColumnHandle::ColumnType::kRegular; + for (auto transform : partitionTransforms) { + if (columnName == transform) { + columnType = HiveColumnHandle::ColumnType::kPartitionKey; + break; + } + } + columnHandles.push_back(std::make_shared( + columnName, columnType, rowType->childAt(i), rowType->childAt(i))); + } + + auto locationHandle = std::make_shared( + outputDirectoryPath, + outputDirectoryPath, + LocationHandle::TableType::kNew); + + auto partitionSpec = createPartitionSpec(partitionTransforms); + + return std::make_shared( + columnHandles, + locationHandle, + partitionSpec, + fileFormat_, + nullptr, + common::CompressionKind::CompressionKind_ZSTD); +} + +std::shared_ptr IcebergTestBase::createIcebergDataSink( + const RowTypePtr& rowType, + const std::string& outputDirectoryPath, + const std::vector& partitionTransforms) { + auto tableHandle = createIcebergInsertTableHandle( + rowType, outputDirectoryPath, partitionTransforms); + return std::make_shared( + rowType, + tableHandle, + connectorQueryCtx_.get(), + connector::CommitStrategy::kNoCommit, + connectorConfig_); +} + +std::vector IcebergTestBase::listFiles( + const std::string& dirPath) { + std::vector files; + if (!std::filesystem::exists(dirPath)) { + return files; + } + + for (auto& dirEntry : + std::filesystem::recursive_directory_iterator(dirPath)) { + if (dirEntry.is_regular_file()) { + files.push_back(dirEntry.path().string()); + } + } + return files; +} + +std::vector> +IcebergTestBase::createSplitsForDirectory(const std::string& directory) { + std::vector> splits; + std::unordered_map customSplitInfo; + customSplitInfo["table_format"] = "hive-iceberg"; + + auto files = listFiles(directory); + for (const auto& filePath : files) { + std::unordered_map> partitionKeys; + + // Extract partition keys from path if any. + std::vector pathComponents; + folly::split("/", filePath, pathComponents); + for (const auto& component : pathComponents) { + if (component.find('=') != std::string::npos) { + std::vector keys; + folly::split('=', component, keys); + if (keys.size() == 2) { + partitionKeys[keys[0]] = keys[1]; + if (keys[1] == "null") { + partitionKeys[keys[0]] = std::nullopt; + } + } + } + } + + const auto file = filesystems::getFileSystem(filePath, nullptr) + ->openFileForRead(filePath); + const auto fileSize = file->size(); + + splits.push_back(std::make_shared( + kHiveConnectorId, + filePath, + fileFormat_, + 0, + fileSize, + partitionKeys, + std::nullopt, + customSplitInfo, + nullptr, + /*cacheable=*/true, + std::vector())); + } + + return splits; +} + +} // namespace facebook::velox::connector::hive::iceberg::test diff --git a/velox/connectors/lakehouse/iceberg/tests/IcebergTestBase.h b/velox/connectors/lakehouse/iceberg/tests/IcebergTestBase.h new file mode 100644 index 000000000000..61a43f187fcd --- /dev/null +++ b/velox/connectors/lakehouse/iceberg/tests/IcebergTestBase.h @@ -0,0 +1,88 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#pragma once + +#include + +#include "velox/connectors/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" +#ifdef VELOX_ENABLE_PARQUET +#include "velox/dwio/parquet/RegisterParquetWriter.h" +#include "velox/dwio/parquet/reader/ParquetReader.h" +#endif + +namespace facebook::velox::connector::hive::iceberg::test { + +class IcebergTestBase : public exec::test::HiveConnectorTestBase { + protected: + void SetUp() override; + + void TearDown() override; + + std::vector createTestData( + int32_t numBatches, + vector_size_t rowsPerBatch, + double nullRatio = 0.0); + + std::shared_ptr createIcebergDataSink( + const RowTypePtr& rowType, + const std::string& outputDirectoryPath, + const std::vector& partitionTransforms = {}); + + std::vector> createSplitsForDirectory( + const std::string& directory); + + std::vector listFiles(const std::string& dirPath); + + private: + std::shared_ptr createPartitionSpec( + const std::vector& transformSpecs); + + std::shared_ptr createIcebergInsertTableHandle( + const RowTypePtr& rowType, + const std::string& outputDirectoryPath, + const std::vector& partitionTransforms = {}); + + std::vector listPartitionDirectories( + const std::string& dataPath); + + void setupMemoryPools(const std::string& name); + + protected: + RowTypePtr rowType_; + + private: + static constexpr const char* kHiveConnectorId = "test-hive"; + + // The only supported file format is PARQUET. + dwio::common::FileFormat fileFormat_ = dwio::common::FileFormat::PARQUET; + + std::shared_ptr root_; + std::shared_ptr opPool_; + std::shared_ptr connectorPool_; + std::shared_ptr connectorSessionProperties_; + std::shared_ptr connectorConfig_; + std::unique_ptr connectorQueryCtx_; + VectorFuzzer::Options fuzzerOptions_; + std::unique_ptr fuzzer_; + std::unique_ptr vectorMaker_; +}; + +} // namespace facebook::velox::connector::hive::iceberg::test diff --git a/velox/connectors/lakehouse/iceberg/tests/Main.cpp b/velox/connectors/lakehouse/iceberg/tests/Main.cpp new file mode 100644 index 000000000000..3c9dd6615055 --- /dev/null +++ b/velox/connectors/lakehouse/iceberg/tests/Main.cpp @@ -0,0 +1,29 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "velox/common/process/ThreadDebugInfo.h" + +#include +#include + +// This main is needed for some tests on linux. +int main(int argc, char** argv) { + testing::InitGoogleTest(&argc, argv); + // Signal handler required for ThreadDebugInfoTest + facebook::velox::process::addDefaultFatalSignalHandler(); + folly::Init init(&argc, &argv, false); + return RUN_ALL_TESTS(); +} diff --git a/velox/connectors/lakehouse/storage_adapters/CMakeLists.txt b/velox/connectors/lakehouse/storage_adapters/CMakeLists.txt new file mode 100644 index 000000000000..bd7c37f81640 --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/CMakeLists.txt @@ -0,0 +1,18 @@ +# Copyright (c) Facebook, Inc. and its affiliates. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +add_subdirectory(s3fs) +add_subdirectory(hdfs) +add_subdirectory(gcs) +add_subdirectory(abfs) diff --git a/velox/connectors/lakehouse/storage_adapters/abfs/AbfsConfig.cpp b/velox/connectors/lakehouse/storage_adapters/abfs/AbfsConfig.cpp new file mode 100644 index 000000000000..7604e3ef3b69 --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/abfs/AbfsConfig.cpp @@ -0,0 +1,201 @@ +/* + * 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/storage_adapters/abfs/AbfsConfig.h" + +#include "velox/common/config/Config.h" +#include "velox/connectors/hive/storage_adapters/abfs/AbfsUtil.h" + +#include + +namespace facebook::velox::filesystems { + +std::function()> + AbfsConfig::testWriteClientFn_; + +class DataLakeFileClientWrapper final : public AzureDataLakeFileClient { + public: + DataLakeFileClientWrapper(std::unique_ptr client) + : client_(std::move(client)) {} + + void create() override { + client_->Create(); + } + + Azure::Storage::Files::DataLake::Models::PathProperties getProperties() + override { + return client_->GetProperties().Value; + } + + void append(const uint8_t* buffer, size_t size, uint64_t offset) override { + auto bodyStream = Azure::Core::IO::MemoryBodyStream(buffer, size); + client_->Append(bodyStream, offset); + } + + void flush(uint64_t position) override { + client_->Flush(position); + } + + void close() override { + // do nothing. + } + + std::string getUrl() const override { + return client_->GetUrl(); + } + + private: + const std::unique_ptr client_; +}; + +AbfsConfig::AbfsConfig( + std::string_view path, + const config::ConfigBase& config) { + std::string_view file; + isHttps_ = true; + if (path.find(kAbfssScheme) == 0) { + file = path.substr(kAbfssScheme.size()); + } else if (path.find(kAbfsScheme) == 0) { + file = path.substr(kAbfsScheme.size()); + isHttps_ = false; + } else { + VELOX_FAIL("Invalid ABFS Path {}", path); + } + + auto firstAt = file.find_first_of("@"); + fileSystem_ = file.substr(0, firstAt); + auto firstSep = file.find_first_of("/"); + filePath_ = file.substr(firstSep + 1); + accountNameWithSuffix_ = file.substr(firstAt + 1, firstSep - firstAt - 1); + + auto authTypeKey = + fmt::format("{}.{}", kAzureAccountAuthType, accountNameWithSuffix_); + authType_ = kAzureSharedKeyAuthType; + if (config.valueExists(authTypeKey)) { + authType_ = config.get(authTypeKey).value(); + } + if (authType_ == kAzureSharedKeyAuthType) { + auto credKey = + fmt::format("{}.{}", kAzureAccountKey, accountNameWithSuffix_); + VELOX_USER_CHECK( + config.valueExists(credKey), "Config {} not found", credKey); + auto firstDot = accountNameWithSuffix_.find_first_of("."); + auto accountName = accountNameWithSuffix_.substr(0, firstDot); + auto endpointSuffix = accountNameWithSuffix_.substr(firstDot + 5); + std::stringstream ss; + ss << "DefaultEndpointsProtocol=" << (isHttps_ ? "https" : "http"); + ss << ";AccountName=" << accountName; + ss << ";AccountKey=" << config.get(credKey).value(); + ss << ";EndpointSuffix=" << endpointSuffix; + + if (config.valueExists(kAzureBlobEndpoint)) { + ss << ";BlobEndpoint=" + << config.get(kAzureBlobEndpoint).value(); + } + ss << ";"; + connectionString_ = ss.str(); + } else if (authType_ == kAzureOAuthAuthType) { + auto clientIdKey = fmt::format( + "{}.{}", kAzureAccountOAuth2ClientId, accountNameWithSuffix_); + auto clientSecretKey = fmt::format( + "{}.{}", kAzureAccountOAuth2ClientSecret, accountNameWithSuffix_); + auto clientEndpointKey = fmt::format( + "{}.{}", kAzureAccountOAuth2ClientEndpoint, accountNameWithSuffix_); + VELOX_USER_CHECK( + config.valueExists(clientIdKey), "Config {} not found", clientIdKey); + VELOX_USER_CHECK( + config.valueExists(clientSecretKey), + "Config {} not found", + clientSecretKey); + VELOX_USER_CHECK( + config.valueExists(clientEndpointKey), + "Config {} not found", + clientEndpointKey); + auto clientEndpoint = config.get(clientEndpointKey).value(); + auto firstSep = clientEndpoint.find_first_of("/", /* https:// */ 8); + authorityHost_ = clientEndpoint.substr(0, firstSep + 1); + auto sedondSep = clientEndpoint.find_first_of("/", firstSep + 1); + tenentId_ = clientEndpoint.substr(firstSep + 1, sedondSep - firstSep - 1); + Azure::Identity::ClientSecretCredentialOptions options; + options.AuthorityHost = authorityHost_; + tokenCredential_ = + std::make_shared( + tenentId_, + config.get(clientIdKey).value(), + config.get(clientSecretKey).value(), + options); + } else if (authType_ == kAzureSASAuthType) { + auto sasKey = fmt::format("{}.{}", kAzureSASKey, accountNameWithSuffix_); + VELOX_USER_CHECK(config.valueExists(sasKey), "Config {} not found", sasKey); + sas_ = config.get(sasKey).value(); + } else { + VELOX_USER_FAIL( + "Unsupported auth type {}, supported auth types are SharedKey, OAuth and SAS.", + authType_); + } +} + +std::unique_ptr AbfsConfig::getReadFileClient() { + if (authType_ == kAzureSASAuthType) { + auto url = getUrl(true); + return std::make_unique(fmt::format("{}?{}", url, sas_)); + } else if (authType_ == kAzureOAuthAuthType) { + auto url = getUrl(true); + return std::make_unique(url, tokenCredential_); + } else { + return std::make_unique(BlobClient::CreateFromConnectionString( + connectionString_, fileSystem_, filePath_)); + } +} + +std::unique_ptr AbfsConfig::getWriteFileClient() { + if (testWriteClientFn_) { + return testWriteClientFn_(); + } + std::unique_ptr client; + if (authType_ == kAzureSASAuthType) { + auto url = getUrl(false); + client = + std::make_unique(fmt::format("{}?{}", url, sas_)); + } else if (authType_ == kAzureOAuthAuthType) { + auto url = getUrl(false); + client = std::make_unique(url, tokenCredential_); + } else { + client = std::make_unique( + DataLakeFileClient::CreateFromConnectionString( + connectionString_, fileSystem_, filePath_)); + } + return std::make_unique(std::move(client)); +} + +std::string AbfsConfig::getUrl(bool withblobSuffix) { + std::string accountNameWithSuffixForUrl(accountNameWithSuffix_); + if (withblobSuffix) { + // We should use correct suffix for blob client. + size_t start_pos = accountNameWithSuffixForUrl.find("dfs"); + if (start_pos != std::string::npos) { + accountNameWithSuffixForUrl.replace(start_pos, 3, "blob"); + } + } + return fmt::format( + "{}{}/{}/{}", + isHttps_ ? "https://" : "http://", + accountNameWithSuffixForUrl, + fileSystem_, + filePath_); +} + +} // namespace facebook::velox::filesystems diff --git a/velox/connectors/lakehouse/storage_adapters/abfs/AbfsConfig.h b/velox/connectors/lakehouse/storage_adapters/abfs/AbfsConfig.h new file mode 100644 index 000000000000..af4b46708d86 --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/abfs/AbfsConfig.h @@ -0,0 +1,130 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#pragma once + +#include +#include +#include +#include +#include +#include "velox/connectors/hive/storage_adapters/abfs/AzureDataLakeFileClient.h" + +using namespace Azure::Storage::Blobs; +using namespace Azure::Storage::Files::DataLake; + +namespace facebook::velox::config { +class ConfigBase; +} + +namespace facebook::velox::filesystems { + +// This is used to specify the Azurite endpoint in testing. +static constexpr const char* kAzureBlobEndpoint{"fs.azure.blob-endpoint"}; + +// The authentication mechanism is set in `fs.azure.account.auth.type` (or the +// account specific variant). The supported values are SharedKey, OAuth and SAS. +static constexpr const char* kAzureAccountAuthType = + "fs.azure.account.auth.type"; + +static constexpr const char* kAzureAccountKey = "fs.azure.account.key"; + +static constexpr const char* kAzureSASKey = "fs.azure.sas.fixed.token"; + +static constexpr const char* kAzureAccountOAuth2ClientId = + "fs.azure.account.oauth2.client.id"; + +static constexpr const char* kAzureAccountOAuth2ClientSecret = + "fs.azure.account.oauth2.client.secret"; + +// Token end point, this can be found through Azure portal. For example: +// https://login.microsoftonline.com/{TENANTID}/oauth2/token +static constexpr const char* kAzureAccountOAuth2ClientEndpoint = + "fs.azure.account.oauth2.client.endpoint"; + +static constexpr const char* kAzureSharedKeyAuthType = "SharedKey"; + +static constexpr const char* kAzureOAuthAuthType = "OAuth"; + +static constexpr const char* kAzureSASAuthType = "SAS"; + +class AbfsConfig { + public: + explicit AbfsConfig(std::string_view path, const config::ConfigBase& config); + + std::unique_ptr getReadFileClient(); + + std::unique_ptr getWriteFileClient(); + + std::string filePath() const { + return filePath_; + } + + /// Test only. + std::string fileSystem() const { + return fileSystem_; + } + + /// Test only. + std::string connectionString() const { + return connectionString_; + } + + /// Test only. + std::string tenentId() const { + return tenentId_; + } + + /// Test only. + std::string authorityHost() const { + return authorityHost_; + } + + /// Test only. + static void setUpTestWriteClient( + std::function()> testClientFn) { + testWriteClientFn_ = testClientFn; + } + + /// Test only. + static void tearDownTestWriteClient() { + testWriteClientFn_ = nullptr; + } + + private: + std::string getUrl(bool withblobSuffix); + + std::string authType_; + + // Container name is called FileSystem in some Azure API. + std::string fileSystem_; + std::string filePath_; + std::string connectionString_; + + bool isHttps_; + std::string accountNameWithSuffix_; + + std::string sas_; + + std::string tenentId_; + std::string authorityHost_; + std::shared_ptr tokenCredential_; + + static std::function()> + testWriteClientFn_; +}; + +} // namespace facebook::velox::filesystems diff --git a/velox/connectors/lakehouse/storage_adapters/abfs/AbfsFileSystem.cpp b/velox/connectors/lakehouse/storage_adapters/abfs/AbfsFileSystem.cpp new file mode 100644 index 000000000000..18fbd3f2284a --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/abfs/AbfsFileSystem.cpp @@ -0,0 +1,237 @@ +/* + * 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/storage_adapters/abfs/AbfsFileSystem.h" + +#include +#include +#include + +#include "velox/connectors/hive/storage_adapters/abfs/AbfsConfig.h" +#include "velox/connectors/hive/storage_adapters/abfs/AbfsReadFile.h" +#include "velox/connectors/hive/storage_adapters/abfs/AbfsUtil.h" +#include "velox/connectors/hive/storage_adapters/abfs/AbfsWriteFile.h" + +namespace facebook::velox::filesystems { + +class AbfsReadFile::Impl { + constexpr static uint64_t kNaturalReadSize = 4 << 20; // 4M + constexpr static uint64_t kReadConcurrency = 8; + + public: + explicit Impl(std::string_view path, const config::ConfigBase& config) { + auto abfsConfig = AbfsConfig(path, config); + filePath_ = abfsConfig.filePath(); + fileClient_ = abfsConfig.getReadFileClient(); + } + + void initialize(const FileOptions& options) { + if (options.fileSize.has_value()) { + VELOX_CHECK_GE( + options.fileSize.value(), 0, "File size must be non-negative"); + length_ = options.fileSize.value(); + } + + if (length_ != -1) { + return; + } + + try { + auto properties = fileClient_->GetProperties(); + length_ = properties.Value.BlobSize; + } catch (Azure::Storage::StorageException& e) { + throwStorageExceptionWithOperationDetails("GetProperties", filePath_, e); + } + VELOX_CHECK_GE(length_, 0); + } + + std::string_view pread( + uint64_t offset, + uint64_t length, + void* buffer, + File::IoStats* stats) const { + preadInternal(offset, length, static_cast(buffer)); + return {static_cast(buffer), length}; + } + + std::string pread(uint64_t offset, uint64_t length, File::IoStats* stats) + const { + std::string result(length, 0); + preadInternal(offset, length, result.data()); + return result; + } + + uint64_t preadv( + uint64_t offset, + const std::vector>& buffers, + File::IoStats* stats) const { + size_t length = 0; + auto size = buffers.size(); + for (auto& range : buffers) { + length += range.size(); + } + std::string result(length, 0); + preadInternal(offset, length, static_cast(result.data())); + size_t resultOffset = 0; + for (auto range : buffers) { + if (range.data()) { + memcpy(range.data(), &(result.data()[resultOffset]), range.size()); + } + resultOffset += range.size(); + } + + return length; + } + + uint64_t preadv( + folly::Range regions, + folly::Range iobufs, + File::IoStats* stats) const { + size_t length = 0; + VELOX_CHECK_EQ(regions.size(), iobufs.size()); + for (size_t i = 0; i < regions.size(); ++i) { + const auto& region = regions[i]; + auto& output = iobufs[i]; + output = folly::IOBuf(folly::IOBuf::CREATE, region.length); + pread(region.offset, region.length, output.writableData(), stats); + output.append(region.length); + length += region.length; + } + + return length; + } + + uint64_t size() const { + return length_; + } + + uint64_t memoryUsage() const { + return 3 * sizeof(std::string) + sizeof(int64_t); + } + + bool shouldCoalesce() const { + return false; + } + + std::string getName() const { + return filePath_; + } + + uint64_t getNaturalReadSize() const { + return kNaturalReadSize; + } + + private: + void preadInternal(uint64_t offset, uint64_t length, char* position) const { + // Read the desired range of bytes. + Azure::Core::Http::HttpRange range; + range.Offset = offset; + range.Length = length; + + Azure::Storage::Blobs::DownloadBlobOptions blob; + blob.Range = range; + auto response = fileClient_->Download(blob); + response.Value.BodyStream->ReadToCount( + reinterpret_cast(position), length); + } + + std::string filePath_; + std::unique_ptr fileClient_; + int64_t length_ = -1; +}; + +AbfsReadFile::AbfsReadFile( + std::string_view path, + const config::ConfigBase& config) { + impl_ = std::make_shared(path, config); +} + +void AbfsReadFile::initialize(const FileOptions& options) { + return impl_->initialize(options); +} + +std::string_view AbfsReadFile::pread( + uint64_t offset, + uint64_t length, + void* buffer, + File::IoStats* stats) const { + return impl_->pread(offset, length, buffer, stats); +} + +std::string AbfsReadFile::pread( + uint64_t offset, + uint64_t length, + File::IoStats* stats) const { + return impl_->pread(offset, length, stats); +} + +uint64_t AbfsReadFile::preadv( + uint64_t offset, + const std::vector>& buffers, + File::IoStats* stats) const { + return impl_->preadv(offset, buffers, stats); +} + +uint64_t AbfsReadFile::preadv( + folly::Range regions, + folly::Range iobufs, + File::IoStats* stats) const { + return impl_->preadv(regions, iobufs, stats); +} + +uint64_t AbfsReadFile::size() const { + return impl_->size(); +} + +uint64_t AbfsReadFile::memoryUsage() const { + return impl_->memoryUsage(); +} + +bool AbfsReadFile::shouldCoalesce() const { + return false; +} + +std::string AbfsReadFile::getName() const { + return impl_->getName(); +} + +uint64_t AbfsReadFile::getNaturalReadSize() const { + return impl_->getNaturalReadSize(); +} + +AbfsFileSystem::AbfsFileSystem(std::shared_ptr config) + : FileSystem(config) { + VELOX_CHECK_NOT_NULL(config.get()); +} + +std::string AbfsFileSystem::name() const { + return "ABFS"; +} + +std::unique_ptr AbfsFileSystem::openFileForRead( + std::string_view path, + const FileOptions& options) { + auto abfsfile = std::make_unique(path, *config_); + abfsfile->initialize(options); + return abfsfile; +} + +std::unique_ptr AbfsFileSystem::openFileForWrite( + std::string_view path, + const FileOptions& /*unused*/) { + return std::make_unique(path, *config_); +} +} // namespace facebook::velox::filesystems diff --git a/velox/connectors/lakehouse/storage_adapters/abfs/AbfsFileSystem.h b/velox/connectors/lakehouse/storage_adapters/abfs/AbfsFileSystem.h new file mode 100644 index 000000000000..72015de61cd6 --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/abfs/AbfsFileSystem.h @@ -0,0 +1,80 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#pragma once + +#include "velox/common/file/FileSystems.h" + +namespace facebook::velox::filesystems { + +/// Implementation of the ABFS (Azure Blob File Storage) filesystem and file +/// interface. We provide a registration method for reading and writing files so +/// that the appropriate type of file can be constructed based on a filename. +/// The supported schema is `abfs(s)://` to align with the valid scheme +/// identifiers used in the Hadoop Filesystem ABFS driver when integrating with +/// Azure Blob Storage. One key difference here is that the ABFS Hadoop client +/// driver always uses Transport Layer Security (TLS) regardless of the +/// authentication method chosen when using the `abfss` schema, but not mandated +/// when using the `abfs` schema. In our implementation, we always use the HTTPS +/// protocol, regardless of whether the schema is `abfs://` or `abfss://`. The +/// legacy wabs(s):// schema is not supported as it has been deprecated already +/// by Azure Storage team. Reference document - +/// https://learn.microsoft.com/en-us/azure/databricks/storage/azure-storage. +class AbfsFileSystem : public FileSystem { + public: + explicit AbfsFileSystem(std::shared_ptr config); + + std::string name() const override; + + std::unique_ptr openFileForRead( + std::string_view path, + const FileOptions& options = {}) override; + + std::unique_ptr openFileForWrite( + std::string_view path, + const FileOptions& options = {}) override; + + void rename( + std::string_view path, + std::string_view newPath, + bool overWrite = false) override { + VELOX_UNSUPPORTED("rename for abfs not implemented"); + } + + void remove(std::string_view path) override { + VELOX_UNSUPPORTED("remove for abfs not implemented"); + } + + bool exists(std::string_view path) override { + VELOX_UNSUPPORTED("exists for abfs not implemented"); + } + + std::vector list(std::string_view path) override { + VELOX_UNSUPPORTED("list for abfs not implemented"); + } + + void mkdir( + std::string_view path, + const filesystems::DirectoryOptions& options = {}) override { + VELOX_UNSUPPORTED("mkdir for abfs not implemented"); + } + + void rmdir(std::string_view path) override { + VELOX_UNSUPPORTED("rmdir for abfs not implemented"); + } +}; + +void registerAbfsFileSystem(); +} // namespace facebook::velox::filesystems diff --git a/velox/connectors/lakehouse/storage_adapters/abfs/AbfsReadFile.h b/velox/connectors/lakehouse/storage_adapters/abfs/AbfsReadFile.h new file mode 100644 index 000000000000..942439c06c1e --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/abfs/AbfsReadFile.h @@ -0,0 +1,69 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#pragma once + +#include "velox/common/file/File.h" + +namespace facebook::velox::config { +class ConfigBase; +} + +namespace facebook::velox::filesystems { +class AbfsReadFile final : public ReadFile { + public: + explicit AbfsReadFile( + std::string_view path, + const config::ConfigBase& config); + + void initialize(const FileOptions& options); + + std::string_view pread( + uint64_t offset, + uint64_t length, + void* buf, + File::IoStats* stats = nullptr) const final; + + std::string pread( + uint64_t offset, + uint64_t length, + File::IoStats* stats = nullptr) const final; + + uint64_t preadv( + uint64_t offset, + const std::vector>& buffers, + File::IoStats* stats = nullptr) const final; + + uint64_t preadv( + folly::Range regions, + folly::Range iobufs, + File::IoStats* stats = nullptr) const final; + + uint64_t size() const final; + + uint64_t memoryUsage() const final; + + bool shouldCoalesce() const final; + + std::string getName() const final; + + uint64_t getNaturalReadSize() const final; + + protected: + class Impl; + std::shared_ptr impl_; +}; +} // namespace facebook::velox::filesystems diff --git a/velox/connectors/lakehouse/storage_adapters/abfs/AbfsUtil.h b/velox/connectors/lakehouse/storage_adapters/abfs/AbfsUtil.h new file mode 100644 index 000000000000..925c6f91ece9 --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/abfs/AbfsUtil.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 +#include +#include "velox/common/file/File.h" + +namespace facebook::velox::filesystems { +namespace { +constexpr std::string_view kAbfsScheme{"abfs://"}; +constexpr std::string_view kAbfssScheme{"abfss://"}; +} // namespace + +inline bool isAbfsFile(const std::string_view filename) { + return filename.find(kAbfsScheme) == 0 || filename.find(kAbfssScheme) == 0; +} + +inline std::string throwStorageExceptionWithOperationDetails( + std::string operation, + std::string path, + Azure::Storage::StorageException& error) { + const auto errMsg = fmt::format( + "Operation '{}' to path '{}' encountered azure storage exception, Details: '{}'.", + operation, + path, + error.what()); + if (error.StatusCode == Azure::Core::Http::HttpStatusCode::NotFound) { + VELOX_FILE_NOT_FOUND_ERROR(errMsg); + } + VELOX_FAIL(errMsg); +} + +} // namespace facebook::velox::filesystems diff --git a/velox/connectors/lakehouse/storage_adapters/abfs/AbfsWriteFile.cpp b/velox/connectors/lakehouse/storage_adapters/abfs/AbfsWriteFile.cpp new file mode 100644 index 000000000000..2ba015743a0d --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/abfs/AbfsWriteFile.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/storage_adapters/abfs/AbfsWriteFile.h" +#include "velox/connectors/hive/storage_adapters/abfs/AbfsConfig.h" +#include "velox/connectors/hive/storage_adapters/abfs/AbfsUtil.h" + +namespace facebook::velox::filesystems { + +class AbfsWriteFile::Impl { + public: + explicit Impl( + std::string_view path, + std::unique_ptr& client) + : path_(path), client_(std::move(client)) { + // Make it a no-op if invoked twice. + if (position_ != -1) { + return; + } + position_ = 0; + VELOX_CHECK(!checkIfFileExists(), "File already exists"); + client_->create(); + } + + void close() { + if (!closed_) { + flush(); + closed_ = true; + } + } + + void flush() { + if (!closed_) { + client_->flush(position_); + } + } + + void append(std::string_view data) { + VELOX_CHECK(!closed_, "File is not open"); + if (data.size() == 0) { + return; + } + append(data.data(), data.size()); + } + + uint64_t size() const { + return client_->getProperties().FileSize; + } + + void append(const char* buffer, size_t size) { + client_->append(reinterpret_cast(buffer), size, position_); + position_ += size; + } + + private: + bool checkIfFileExists() { + try { + client_->getProperties(); + return true; + } catch (Azure::Storage::StorageException& e) { + if (e.StatusCode != Azure::Core::Http::HttpStatusCode::NotFound) { + throwStorageExceptionWithOperationDetails("GetProperties", path_, e); + } + return false; + } + } + + const std::string path_; + const std::unique_ptr client_; + + uint64_t position_ = -1; + bool closed_ = false; +}; + +AbfsWriteFile::AbfsWriteFile( + std::string_view path, + const config::ConfigBase& config) { + auto abfsConfig = AbfsConfig(path, config); + auto client = abfsConfig.getWriteFileClient(); + impl_ = std::make_unique(path, client); +} + +AbfsWriteFile::AbfsWriteFile( + std::string_view path, + std::unique_ptr& client) { + impl_ = std::make_unique(path, client); +} + +AbfsWriteFile::~AbfsWriteFile() {} + +void AbfsWriteFile::close() { + impl_->close(); +} + +void AbfsWriteFile::flush() { + impl_->flush(); +} + +void AbfsWriteFile::append(std::string_view data) { + impl_->append(data); +} + +uint64_t AbfsWriteFile::size() const { + return impl_->size(); +} + +} // namespace facebook::velox::filesystems diff --git a/velox/connectors/lakehouse/storage_adapters/abfs/AbfsWriteFile.h b/velox/connectors/lakehouse/storage_adapters/abfs/AbfsWriteFile.h new file mode 100644 index 000000000000..6efc3c83a6f5 --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/abfs/AbfsWriteFile.h @@ -0,0 +1,68 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#pragma once + +#include "velox/common/file/File.h" +#include "velox/connectors/hive/storage_adapters/abfs/AzureDataLakeFileClient.h" + +namespace facebook::velox::config { +class ConfigBase; +} + +namespace facebook::velox::filesystems { + +/// We are using the DFS (Data Lake Storage) endpoint for Azure Blob File write +/// operations because the DFS endpoint is designed to be compatible with file +/// operation semantics, such as `Append` to a file and file `Flush` operations. +/// The legacy Blob endpoint can only be used for blob level append and flush +/// operations. When using the Blob endpoint, we would need to manually manage +/// the creation, appending, and committing of file-related blocks. + +/// Implementation of abfs write file. Nothing written to the file should be +/// read back until it is closed. +class AbfsWriteFile : public WriteFile { + public: + constexpr static uint64_t kNaturalWriteSize = 8 << 20; // 8M + + /// @param path The file path to write. + /// @param connectStr The connection string used to auth the storage account. + AbfsWriteFile(std::string_view path, const config::ConfigBase& config); + + /// @param path The file path to write. + /// @param client The AdlsFileClient. + AbfsWriteFile( + std::string_view path, + std::unique_ptr& client); + + ~AbfsWriteFile(); + + /// Get the file size. + uint64_t size() const override; + + /// Flush the data. + void flush() override; + + /// Write the data by append mode. + void append(std::string_view data) override; + + /// Close the file. + void close() override; + + protected: + class Impl; + std::unique_ptr impl_; +}; +} // namespace facebook::velox::filesystems diff --git a/velox/connectors/lakehouse/storage_adapters/abfs/AzureDataLakeFileClient.h b/velox/connectors/lakehouse/storage_adapters/abfs/AzureDataLakeFileClient.h new file mode 100644 index 000000000000..abd607c0d1b2 --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/abfs/AzureDataLakeFileClient.h @@ -0,0 +1,49 @@ +/* + * 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 Azure::Storage::Files::DataLake::Models { +class PathProperties; +} + +namespace facebook::velox::filesystems { + +// Azurite Simulator does not yet support the DFS endpoint. +// (For more information, see https://github.com/Azure/Azurite/issues/553 and +// https://github.com/Azure/Azurite/issues/409). +// You can find a comparison between DFS and Blob endpoints here: +// https://github.com/Azure/Azurite/wiki/ADLS-Gen2-Implementation-Guidance +// To facilitate unit testing of file write scenarios, we define the +// AzureDatalakeFileClient which can be mocked during testing. + +class AzureDataLakeFileClient { + public: + virtual ~AzureDataLakeFileClient() {} + + virtual void create() = 0; + virtual Azure::Storage::Files::DataLake::Models::PathProperties + getProperties() = 0; + virtual void append(const uint8_t* buffer, size_t size, uint64_t offset) = 0; + virtual void flush(uint64_t position) = 0; + virtual void close() = 0; + virtual std::string getUrl() const = 0; +}; +} // namespace facebook::velox::filesystems diff --git a/velox/connectors/lakehouse/storage_adapters/abfs/CMakeLists.txt b/velox/connectors/lakehouse/storage_adapters/abfs/CMakeLists.txt new file mode 100644 index 000000000000..799e93830373 --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/abfs/CMakeLists.txt @@ -0,0 +1,43 @@ +# 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. + +# for generated headers + +velox_add_library(velox_abfs RegisterAbfsFileSystem.cpp) + +if(VELOX_ENABLE_ABFS) + velox_sources( + velox_abfs + PRIVATE + AbfsFileSystem.cpp + AbfsConfig.cpp + AbfsWriteFile.cpp) + + velox_link_libraries( + velox_abfs + PUBLIC velox_file + velox_core + velox_hive_config + velox_dwio_common_exception + Azure::azure-identity + Azure::azure-storage-blobs + Azure::azure-storage-files-datalake + Folly::folly + glog::glog + fmt::fmt) + + if(${VELOX_BUILD_TESTING}) + add_subdirectory(tests) + endif() +endif() diff --git a/velox/connectors/lakehouse/storage_adapters/abfs/RegisterAbfsFileSystem.cpp b/velox/connectors/lakehouse/storage_adapters/abfs/RegisterAbfsFileSystem.cpp new file mode 100644 index 000000000000..112aaf7f87f6 --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/abfs/RegisterAbfsFileSystem.cpp @@ -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. + */ + +#ifdef VELOX_ENABLE_ABFS +#include "velox/common/config/Config.h" +#include "velox/connectors/hive/storage_adapters/abfs/AbfsFileSystem.h" // @manual +#include "velox/connectors/hive/storage_adapters/abfs/AbfsUtil.h" // @manual +#include "velox/dwio/common/FileSink.h" +#endif + +namespace facebook::velox::filesystems { + +#ifdef VELOX_ENABLE_ABFS +folly::once_flag abfsInitiationFlag; + +std::shared_ptr abfsFileSystemGenerator( + std::shared_ptr properties, + std::string_view filePath) { + static std::shared_ptr filesystem; + folly::call_once(abfsInitiationFlag, [&properties]() { + filesystem = std::make_shared(properties); + }); + return filesystem; +} + +std::unique_ptr abfsWriteFileSinkGenerator( + const std::string& fileURI, + const velox::dwio::common::FileSink::Options& options) { + if (isAbfsFile(fileURI)) { + auto fileSystem = + filesystems::getFileSystem(fileURI, options.connectorProperties); + return std::make_unique( + fileSystem->openFileForWrite(fileURI), + fileURI, + options.metricLogger, + options.stats); + } + return nullptr; +} +#endif + +void registerAbfsFileSystem() { +#ifdef VELOX_ENABLE_ABFS + registerFileSystem(isAbfsFile, std::function(abfsFileSystemGenerator)); + dwio::common::FileSink::registerFactory( + std::function(abfsWriteFileSinkGenerator)); +#endif +} + +} // namespace facebook::velox::filesystems diff --git a/velox/connectors/lakehouse/storage_adapters/abfs/RegisterAbfsFileSystem.h b/velox/connectors/lakehouse/storage_adapters/abfs/RegisterAbfsFileSystem.h new file mode 100644 index 000000000000..cbe6758808a0 --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/abfs/RegisterAbfsFileSystem.h @@ -0,0 +1,24 @@ +/* + * 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 + +namespace facebook::velox::filesystems { + +// Register the ABFS filesystem. +void registerAbfsFileSystem(); + +} // namespace facebook::velox::filesystems diff --git a/velox/connectors/lakehouse/storage_adapters/abfs/tests/AbfsCommonTest.cpp b/velox/connectors/lakehouse/storage_adapters/abfs/tests/AbfsCommonTest.cpp new file mode 100644 index 000000000000..53fd09323b41 --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/abfs/tests/AbfsCommonTest.cpp @@ -0,0 +1,154 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "velox/common/base/tests/GTestUtils.h" +#include "velox/common/config/Config.h" +#include "velox/connectors/hive/storage_adapters/abfs/AbfsConfig.h" +#include "velox/connectors/hive/storage_adapters/abfs/AbfsUtil.h" + +#include "gtest/gtest.h" + +using namespace facebook::velox::filesystems; +using namespace facebook::velox; + +TEST(AbfsUtilsTest, isAbfsFile) { + EXPECT_FALSE(isAbfsFile("abfs:")); + EXPECT_FALSE(isAbfsFile("abfss:")); + EXPECT_FALSE(isAbfsFile("abfs:/")); + EXPECT_FALSE(isAbfsFile("abfss:/")); + EXPECT_TRUE(isAbfsFile("abfs://test@test.dfs.core.windows.net/test")); + EXPECT_TRUE(isAbfsFile("abfss://test@test.dfs.core.windows.net/test")); +} + +TEST(AbfsConfigTest, authType) { + const config::ConfigBase config( + {{"fs.azure.account.auth.type.efg.dfs.core.windows.net", "Custom"}, + {"fs.azure.account.key.efg.dfs.core.windows.net", "456"}}, + false); + VELOX_ASSERT_USER_THROW( + std::make_unique( + "abfss://foo@efg.dfs.core.windows.net/test.txt", config), + "Unsupported auth type Custom, supported auth types are SharedKey, OAuth and SAS."); +} + +TEST(AbfsConfigTest, clientSecretOAuth) { + const config::ConfigBase config( + {{"fs.azure.account.auth.type.efg.dfs.core.windows.net", "OAuth"}, + {"fs.azure.account.auth.type.bar1.dfs.core.windows.net", "OAuth"}, + {"fs.azure.account.auth.type.bar2.dfs.core.windows.net", "OAuth"}, + {"fs.azure.account.auth.type.bar3.dfs.core.windows.net", "OAuth"}, + {"fs.azure.account.oauth2.client.id.efg.dfs.core.windows.net", "test"}, + {"fs.azure.account.oauth2.client.secret.efg.dfs.core.windows.net", + "test"}, + {"fs.azure.account.oauth2.client.endpoint.efg.dfs.core.windows.net", + "https://login.microsoftonline.com/{TENANTID}/oauth2/token"}, + {"fs.azure.account.oauth2.client.id.bar2.dfs.core.windows.net", "test"}, + {"fs.azure.account.oauth2.client.id.bar3.dfs.core.windows.net", "test"}, + {"fs.azure.account.oauth2.client.secret.bar3.dfs.core.windows.net", + "test"}}, + false); + VELOX_ASSERT_USER_THROW( + std::make_unique( + "abfss://foo@bar1.dfs.core.windows.net/test.txt", config), + "Config fs.azure.account.oauth2.client.id.bar1.dfs.core.windows.net not found"); + VELOX_ASSERT_USER_THROW( + std::make_unique( + "abfss://foo@bar2.dfs.core.windows.net/test.txt", config), + "Config fs.azure.account.oauth2.client.secret.bar2.dfs.core.windows.net not found"); + VELOX_ASSERT_USER_THROW( + std::make_unique( + "abfss://foo@bar3.dfs.core.windows.net/test.txt", config), + "Config fs.azure.account.oauth2.client.endpoint.bar3.dfs.core.windows.net not found"); + auto abfsConfig = + AbfsConfig("abfss://abc@efg.dfs.core.windows.net/file/test.txt", config); + EXPECT_EQ(abfsConfig.tenentId(), "{TENANTID}"); + EXPECT_EQ(abfsConfig.authorityHost(), "https://login.microsoftonline.com/"); + auto readClient = abfsConfig.getReadFileClient(); + EXPECT_EQ( + readClient->GetUrl(), + "https://efg.blob.core.windows.net/abc/file/test.txt"); + auto writeClient = abfsConfig.getWriteFileClient(); + // GetUrl retrieves the value from the internal blob client, which represents + // the blob's path as well. + EXPECT_EQ( + writeClient->getUrl(), + "https://efg.blob.core.windows.net/abc/file/test.txt"); +} + +TEST(AbfsConfigTest, sasToken) { + const config::ConfigBase config( + {{"fs.azure.account.auth.type.efg.dfs.core.windows.net", "SAS"}, + {"fs.azure.account.auth.type.bar.dfs.core.windows.net", "SAS"}, + {"fs.azure.sas.fixed.token.bar.dfs.core.windows.net", "sas=test"}}, + false); + VELOX_ASSERT_USER_THROW( + std::make_unique( + "abfss://foo@efg.dfs.core.windows.net/test.txt", config), + "Config fs.azure.sas.fixed.token.efg.dfs.core.windows.net not found"); + auto abfsConfig = + AbfsConfig("abfs://abc@bar.dfs.core.windows.net/file", config); + auto readClient = abfsConfig.getReadFileClient(); + EXPECT_EQ( + readClient->GetUrl(), + "http://bar.blob.core.windows.net/abc/file?sas=test"); + auto writeClient = abfsConfig.getWriteFileClient(); + // GetUrl retrieves the value from the internal blob client, which represents + // the blob's path as well. + EXPECT_EQ( + writeClient->getUrl(), + "http://bar.blob.core.windows.net/abc/file?sas=test"); +} + +TEST(AbfsConfigTest, sharedKey) { + const config::ConfigBase config( + {{"fs.azure.account.key.efg.dfs.core.windows.net", "123"}, + {"fs.azure.account.auth.type.efg.dfs.core.windows.net", "SharedKey"}, + {"fs.azure.account.key.foobar.dfs.core.windows.net", "456"}, + {"fs.azure.account.key.bar.dfs.core.windows.net", "789"}}, + false); + + auto abfsConfig = + AbfsConfig("abfs://abc@efg.dfs.core.windows.net/file", config); + EXPECT_EQ(abfsConfig.fileSystem(), "abc"); + EXPECT_EQ(abfsConfig.filePath(), "file"); + EXPECT_EQ( + abfsConfig.connectionString(), + "DefaultEndpointsProtocol=http;AccountName=efg;AccountKey=123;EndpointSuffix=core.windows.net;"); + + auto abfssConfig = AbfsConfig( + "abfss://abc@foobar.dfs.core.windows.net/sf_1/store_sales/ss_sold_date_sk=2450816/part-00002-a29c25f1-4638-494e-8428-a84f51dcea41.c000.snappy.parquet", + config); + EXPECT_EQ(abfssConfig.fileSystem(), "abc"); + EXPECT_EQ( + abfssConfig.filePath(), + "sf_1/store_sales/ss_sold_date_sk=2450816/part-00002-a29c25f1-4638-494e-8428-a84f51dcea41.c000.snappy.parquet"); + EXPECT_EQ( + abfssConfig.connectionString(), + "DefaultEndpointsProtocol=https;AccountName=foobar;AccountKey=456;EndpointSuffix=core.windows.net;"); + + // Test with special character space. + auto abfssConfigWithSpecialCharacters = AbfsConfig( + "abfss://foo@bar.dfs.core.windows.net/main@dir/sub dir/test.txt", config); + + EXPECT_EQ(abfssConfigWithSpecialCharacters.fileSystem(), "foo"); + EXPECT_EQ( + abfssConfigWithSpecialCharacters.filePath(), "main@dir/sub dir/test.txt"); + + VELOX_ASSERT_USER_THROW( + std::make_unique( + "abfss://foo@otheraccount.dfs.core.windows.net/test.txt", config), + "Config fs.azure.account.key.otheraccount.dfs.core.windows.net not found"); +} diff --git a/velox/connectors/lakehouse/storage_adapters/abfs/tests/AbfsFileSystemTest.cpp b/velox/connectors/lakehouse/storage_adapters/abfs/tests/AbfsFileSystemTest.cpp new file mode 100644 index 000000000000..0aaf44713384 --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/abfs/tests/AbfsFileSystemTest.cpp @@ -0,0 +1,291 @@ +/* + * 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 +#include +#include + +#include "velox/common/base/tests/GTestUtils.h" +#include "velox/common/config/Config.h" +#include "velox/common/file/File.h" +#include "velox/common/file/FileSystems.h" +#include "velox/connectors/hive/FileHandle.h" +#include "velox/connectors/hive/storage_adapters/abfs/AbfsConfig.h" +#include "velox/connectors/hive/storage_adapters/abfs/AbfsFileSystem.h" +#include "velox/connectors/hive/storage_adapters/abfs/AbfsReadFile.h" +#include "velox/connectors/hive/storage_adapters/abfs/AbfsWriteFile.h" +#include "velox/connectors/hive/storage_adapters/abfs/tests/AzuriteServer.h" +#include "velox/connectors/hive/storage_adapters/abfs/tests/MockDataLakeFileClient.h" +#include "velox/dwio/common/FileSink.h" +#include "velox/exec/tests/utils/PortUtil.h" +#include "velox/exec/tests/utils/TempFilePath.h" + +using namespace facebook::velox; +using namespace facebook::velox::filesystems; +using ::facebook::velox::common::Region; + +constexpr int kOneMB = 1 << 20; + +class AbfsFileSystemTest : public testing::Test { + public: + std::shared_ptr azuriteServer_; + std::unique_ptr abfs_; + + static void SetUpTestCase() { + registerAbfsFileSystem(); + AbfsConfig::setUpTestWriteClient( + []() { return std::make_unique(); }); + } + + static void TearDownTestSuite() { + AbfsConfig::tearDownTestWriteClient(); + } + + void SetUp() override { + auto port = facebook::velox::exec::test::getFreePort(); + azuriteServer_ = std::make_shared(port); + azuriteServer_->start(); + auto tempFile = createFile(); + azuriteServer_->addFile(tempFile->getPath()); + abfs_ = std::make_unique(azuriteServer_->hiveConfig()); + } + + void TearDown() override { + azuriteServer_->stop(); + } + + static std::string generateRandomData(int size) { + static const char charset[] = + "abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWXYZ"; + + std::string data(size, ' '); + + for (int i = 0; i < size; ++i) { + int index = rand() % (sizeof(charset) - 1); + data[i] = charset[index]; + } + + return data; + } + + private: + static std::shared_ptr<::exec::test::TempFilePath> createFile() { + auto tempFile = exec::test::TempFilePath::create(); + tempFile->append("aaaaa"); + tempFile->append("bbbbb"); + tempFile->append(std::string(kOneMB, 'c')); + tempFile->append("ddddd"); + return tempFile; + } +}; + +void readData(ReadFile* readFile) { + ASSERT_EQ(readFile->size(), 15 + kOneMB); + char buffer1[5]; + ASSERT_EQ(readFile->pread(10 + kOneMB, 5, &buffer1), "ddddd"); + char buffer2[10]; + ASSERT_EQ(readFile->pread(0, 10, &buffer2), "aaaaabbbbb"); + char buffer3[kOneMB]; + ASSERT_EQ(readFile->pread(10, kOneMB, buffer3), std::string(kOneMB, 'c')); + ASSERT_EQ(readFile->size(), 15 + kOneMB); + char buffer4[10]; + const std::string_view arf = readFile->pread(5, 10, &buffer4); + const std::string zarf = readFile->pread(kOneMB, 15); + auto buf = std::make_unique(8); + const std::string_view warf = readFile->pread(4, 8, buf.get()); + const std::string_view warfFromBuf(buf.get(), 8); + ASSERT_EQ(arf, "bbbbbccccc"); + ASSERT_EQ(zarf, "ccccccccccddddd"); + ASSERT_EQ(warf, "abbbbbcc"); + ASSERT_EQ(warfFromBuf, "abbbbbcc"); + + char buff1[10]; + char buff2[10]; + std::vector> buffers = { + folly::Range(buff1, 10), + folly::Range(nullptr, kOneMB - 5), + folly::Range(buff2, 10)}; + ASSERT_EQ(10 + kOneMB - 5 + 10, readFile->preadv(0, buffers)); + ASSERT_EQ(std::string_view(buff1, sizeof(buff1)), "aaaaabbbbb"); + ASSERT_EQ(std::string_view(buff2, sizeof(buff2)), "cccccddddd"); + + std::vector iobufs(2); + std::vector regions = {{0, 10}, {10, 5}}; + ASSERT_EQ( + 10 + 5, + readFile->preadv( + {regions.data(), regions.size()}, {iobufs.data(), iobufs.size()})); + ASSERT_EQ( + std::string_view( + reinterpret_cast(iobufs[0].writableData()), + iobufs[0].length()), + "aaaaabbbbb"); + ASSERT_EQ( + std::string_view( + reinterpret_cast(iobufs[1].writableData()), + iobufs[1].length()), + "ccccc"); +} + +TEST_F(AbfsFileSystemTest, readFile) { + auto readFile = abfs_->openFileForRead(azuriteServer_->fileURI()); + readData(readFile.get()); +} + +TEST_F(AbfsFileSystemTest, openFileForReadWithOptions) { + FileOptions options; + options.fileSize = 15 + kOneMB; + auto readFile = abfs_->openFileForRead(azuriteServer_->fileURI(), options); + readData(readFile.get()); +} + +TEST_F(AbfsFileSystemTest, openFileForReadWithInvalidOptions) { + FileOptions options; + options.fileSize = -kOneMB; + VELOX_ASSERT_THROW( + abfs_->openFileForRead(azuriteServer_->fileURI(), options), + "File size must be non-negative"); +} + +TEST_F(AbfsFileSystemTest, fileHandleWithProperties) { + FileHandleFactory factory( + std::make_unique>(1), + std::make_unique(azuriteServer_->hiveConfig())); + FileProperties properties = {15 + kOneMB, 1}; + FileHandleKey key{azuriteServer_->fileURI()}; + auto fileHandleProperties = factory.generate(key, &properties); + readData(fileHandleProperties->file.get()); + + auto fileHandleWithoutProperties = factory.generate(key); + readData(fileHandleWithoutProperties->file.get()); +} + +TEST_F(AbfsFileSystemTest, multipleThreadsWithReadFile) { + std::atomic startThreads = false; + std::vector threads; + std::mt19937 generator(std::random_device{}()); + std::vector sleepTimesInMicroseconds = {0, 500, 5000}; + std::uniform_int_distribution distribution( + 0, sleepTimesInMicroseconds.size() - 1); + for (int i = 0; i < 10; i++) { + auto thread = std::thread([&] { + int index = distribution(generator); + while (!startThreads) { + std::this_thread::yield(); + } + std::this_thread::sleep_for( + std::chrono::microseconds(sleepTimesInMicroseconds[index])); + auto readFile = abfs_->openFileForRead(azuriteServer_->fileURI()); + readData(readFile.get()); + }); + threads.emplace_back(std::move(thread)); + } + startThreads = true; + for (auto& thread : threads) { + thread.join(); + } +} + +TEST_F(AbfsFileSystemTest, missingFile) { + const std::string abfsFile = azuriteServer_->URI() + "test.txt"; + VELOX_ASSERT_RUNTIME_THROW_CODE( + abfs_->openFileForRead(abfsFile), error_code::kFileNotFound, "404"); +} + +TEST(AbfsWriteFileTest, openFileForWriteTest) { + std::string_view kAbfsFile = + "abfs://test@test.dfs.core.windows.net/test/writetest.txt"; + std::unique_ptr mockClient = + std::make_unique(); + auto mockClientPath = + reinterpret_cast(mockClient.get())->path(); + AbfsWriteFile abfsWriteFile(kAbfsFile, mockClient); + EXPECT_EQ(abfsWriteFile.size(), 0); + std::string dataContent = ""; + uint64_t totalSize = 0; + std::string randomData = + AbfsFileSystemTest::generateRandomData(1 * 1024 * 1024); + for (int i = 0; i < 8; ++i) { + abfsWriteFile.append(randomData); + dataContent += randomData; + } + totalSize = randomData.size() * 8; + abfsWriteFile.flush(); + EXPECT_EQ(abfsWriteFile.size(), totalSize); + + randomData = AbfsFileSystemTest::generateRandomData(9 * 1024 * 1024); + dataContent += randomData; + abfsWriteFile.append(randomData); + totalSize += randomData.size(); + randomData = AbfsFileSystemTest::generateRandomData(2 * 1024 * 1024); + dataContent += randomData; + totalSize += randomData.size(); + abfsWriteFile.append(randomData); + abfsWriteFile.flush(); + EXPECT_EQ(abfsWriteFile.size(), totalSize); + abfsWriteFile.flush(); + abfsWriteFile.close(); + VELOX_ASSERT_THROW(abfsWriteFile.append("abc"), "File is not open"); + + std::unique_ptr mockClientCopy = + std::make_unique(mockClientPath); + VELOX_ASSERT_THROW( + AbfsWriteFile(kAbfsFile, mockClientCopy), "File already exists"); + MockDataLakeFileClient readClient(mockClientPath); + auto fileContent = readClient.readContent(); + ASSERT_EQ(fileContent.size(), dataContent.size()); + ASSERT_EQ(fileContent, dataContent); +} + +TEST_F(AbfsFileSystemTest, renameNotImplemented) { + VELOX_ASSERT_THROW( + abfs_->rename("text", "text2"), "rename for abfs not implemented"); +} + +TEST_F(AbfsFileSystemTest, notImplemented) { + VELOX_ASSERT_THROW(abfs_->remove("text"), "remove for abfs not implemented"); + VELOX_ASSERT_THROW(abfs_->exists("text"), "exists for abfs not implemented"); + VELOX_ASSERT_THROW(abfs_->list("dir"), "list for abfs not implemented"); + VELOX_ASSERT_THROW(abfs_->mkdir("dir"), "mkdir for abfs not implemented"); + VELOX_ASSERT_THROW(abfs_->rmdir("dir"), "rmdir for abfs not implemented"); +} + +TEST_F(AbfsFileSystemTest, credNotFOund) { + const std::string abfsFile = + std::string("abfs://test@test1.dfs.core.windows.net/test"); + VELOX_ASSERT_THROW( + abfs_->openFileForRead(abfsFile), + "Config fs.azure.account.key.test1.dfs.core.windows.net not found"); +} + +TEST_F(AbfsFileSystemTest, registerAbfsFileSink) { + static const std::vector paths = { + "abfs://test@test.dfs.core.windows.net/test", + "abfss://test@test.dfs.core.windows.net/test"}; + std::unordered_map config( + {{"fs.azure.account.key.test.dfs.core.windows.net", "NDU2"}}); + auto hiveConfig = + std::make_shared(std::move(config)); + for (const auto& path : paths) { + auto sink = dwio::common::FileSink::create( + path, {.connectorProperties = hiveConfig}); + auto writeFileSink = dynamic_cast(sink.get()); + auto writeFile = writeFileSink->toWriteFile(); + auto abfsWriteFile = dynamic_cast(writeFile.get()); + ASSERT_TRUE(abfsWriteFile != nullptr); + } +} diff --git a/velox/connectors/lakehouse/storage_adapters/abfs/tests/AzuriteServer.cpp b/velox/connectors/lakehouse/storage_adapters/abfs/tests/AzuriteServer.cpp new file mode 100644 index 000000000000..726c2c531be8 --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/abfs/tests/AzuriteServer.cpp @@ -0,0 +1,122 @@ +/* + * 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/storage_adapters/abfs/tests/AzuriteServer.h" +#include "velox/connectors/hive/storage_adapters/abfs/AbfsConfig.h" + +namespace facebook::velox::filesystems { + +std::string AzuriteServer::URI() const { + return fmt::format( + "abfs://{}@{}.dfs.core.windows.net/", container_, account_); +} + +std::string AzuriteServer::fileURI() const { + return fmt::format( + "abfs://{}@{}.dfs.core.windows.net/{}", container_, account_, file_); +} + +// Return the hiveConfig for the Azurite instance. +// Specify configOverride map to update the default config map. +std::shared_ptr AzuriteServer::hiveConfig( + const std::unordered_map configOverride) const { + auto endpoint = fmt::format("http://127.0.0.1:{}/{}", port_, account_); + std::unordered_map config( + {{"fs.azure.account.key.test.dfs.core.windows.net", key_}, + {kAzureBlobEndpoint, endpoint}}); + + for (const auto& [key, value] : configOverride) { + config[key] = value; + } + + return std::make_shared(std::move(config)); +} + +void AzuriteServer::start() { + try { + serverProcess_ = std::make_unique( + env_, exePath_, commandOptions_); + serverProcess_->wait_for(std::chrono::duration(5000)); + VELOX_CHECK_EQ( + serverProcess_->exit_code(), + 383, + "AzuriteServer process exited, code: ", + serverProcess_->exit_code()); + } catch (const std::exception& e) { + VELOX_FAIL("Failed to launch Azurite server: {}", e.what()); + } +} + +void AzuriteServer::stop() { + if (serverProcess_ && serverProcess_->valid()) { + serverProcess_->terminate(); + serverProcess_->wait(); + serverProcess_.reset(); + } +} + +bool AzuriteServer::isRunning() { + if (serverProcess_) { + return true; + } + return false; +} + +// requires azurite executable to be on the PATH +AzuriteServer::AzuriteServer(int64_t port) : port_(port) { + std::string dataLocation = fmt::format("/tmp/azurite_{}", port); + std::string logFilePath = fmt::format("/tmp/azurite/azurite_{}.log", port); + std::printf( + "Launch azurite instance with port - %s, data location - %s, log file path - %s\n", + std::to_string(port).c_str(), + dataLocation.c_str(), + logFilePath.c_str()); + commandOptions_ = { + "--silent", + "--blobPort", + std::to_string(port), + "--location", + dataLocation, + "--debug", + logFilePath, + }; + env_ = (boost::process::environment)boost::this_process::environment(); + env_["PATH"] = env_["PATH"].to_string() + std::string(kAzuriteSearchPath); + env_["AZURITE_ACCOUNTS"] = fmt::format("{}:{}", account_, key_); + auto path = env_["PATH"].to_vector(); + exePath_ = boost::process::search_path( + kAzuriteServerExecutableName, + std::vector(path.begin(), path.end())); + std::printf("AzuriteServer executable path: %s\n", exePath_.c_str()); + if (exePath_.empty()) { + VELOX_FAIL( + "Failed to find azurite executable {}'", kAzuriteServerExecutableName); + } +} + +void AzuriteServer::addFile(std::string source) { + AbfsConfig conf(fileURI(), *hiveConfig()); + auto containerClient = BlobContainerClient::CreateFromConnectionString( + conf.connectionString(), container_); + containerClient.CreateIfNotExists(); + auto blobClient = containerClient.GetBlockBlobClient(file_); + blobClient.UploadFrom(source); +} + +AzuriteServer::~AzuriteServer() { + // stop(); +} +} // namespace facebook::velox::filesystems diff --git a/velox/connectors/lakehouse/storage_adapters/abfs/tests/AzuriteServer.h b/velox/connectors/lakehouse/storage_adapters/abfs/tests/AzuriteServer.h new file mode 100644 index 000000000000..037c4d9f38d4 --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/abfs/tests/AzuriteServer.h @@ -0,0 +1,78 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "velox/common/config/Config.h" +#include "velox/exec/tests/utils/TempDirectoryPath.h" + +#include +#include +#include +#include +#include +#include +#include "boost/process.hpp" + +namespace facebook::velox::filesystems { + +using namespace Azure::Storage::Blobs; +static std::string_view kAzuriteServerExecutableName{"azurite-blob"}; +static std::string_view kAzuriteSearchPath{":/usr/bin/azurite"}; + +class AzuriteServer { + public: + AzuriteServer(int64_t port); + + const std::string connectionStr() const; + + void start(); + + std::string URI() const; + + std::string fileURI() const; + + std::string container() const { + return container_; + } + + std::string file() const { + return file_; + } + + std::shared_ptr hiveConfig( + const std::unordered_map configOverride = {}) + const; + + void stop(); + + bool isRunning(); + + void addFile(std::string source); + + virtual ~AzuriteServer(); + + private: + int64_t port_; + const std::string account_{"test"}; + const std::string container_{"test"}; + const std::string file_{"test_file.txt"}; + const std::string key_{ + "Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw=="}; + std::vector commandOptions_; + std::unique_ptr<::boost::process::child> serverProcess_; + boost::filesystem::path exePath_; + boost::process::environment env_; +}; +} // namespace facebook::velox::filesystems diff --git a/velox/connectors/lakehouse/storage_adapters/abfs/tests/CMakeLists.txt b/velox/connectors/lakehouse/storage_adapters/abfs/tests/CMakeLists.txt new file mode 100644 index 000000000000..2246dadd385f --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/abfs/tests/CMakeLists.txt @@ -0,0 +1,23 @@ +# Copyright (c) Facebook, Inc. and its affiliates. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +add_executable(velox_abfs_test AbfsFileSystemTest.cpp AbfsCommonTest.cpp + AzuriteServer.cpp MockDataLakeFileClient.cpp) + +add_test(velox_abfs_test velox_abfs_test) +target_link_libraries( + velox_abfs_test + PRIVATE velox_abfs velox_exec_test_lib GTest::gtest GTest::gtest_main) + +target_compile_options(velox_abfs_test PRIVATE -Wno-deprecated-declarations) diff --git a/velox/connectors/lakehouse/storage_adapters/abfs/tests/MockDataLakeFileClient.cpp b/velox/connectors/lakehouse/storage_adapters/abfs/tests/MockDataLakeFileClient.cpp new file mode 100644 index 000000000000..99c4e9e9a8f3 --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/abfs/tests/MockDataLakeFileClient.cpp @@ -0,0 +1,60 @@ +/* + * 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/storage_adapters/abfs/tests/MockDataLakeFileClient.h" + +#include + +#include + +namespace facebook::velox::filesystems { + +void MockDataLakeFileClient::create() { + fileStream_ = std::ofstream( + filePath_, + std::ios_base::out | std::ios_base::binary | std::ios_base::app); +} + +PathProperties MockDataLakeFileClient::getProperties() { + if (!std::filesystem::exists(filePath_)) { + Azure::Storage::StorageException exp(filePath_ + "doesn't exists"); + exp.StatusCode = Azure::Core::Http::HttpStatusCode::NotFound; + throw exp; + } + std::ifstream file(filePath_, std::ios::binary | std::ios::ate); + uint64_t size = static_cast(file.tellg()); + PathProperties ret; + ret.FileSize = size; + return ret; +} + +void MockDataLakeFileClient::append( + const uint8_t* buffer, + size_t size, + uint64_t offset) { + fileStream_.seekp(offset); + fileStream_.write(reinterpret_cast(buffer), size); +} + +void MockDataLakeFileClient::flush(uint64_t position) { + fileStream_.flush(); +} + +void MockDataLakeFileClient::close() { + fileStream_.flush(); + fileStream_.close(); +} +} // namespace facebook::velox::filesystems diff --git a/velox/connectors/lakehouse/storage_adapters/abfs/tests/MockDataLakeFileClient.h b/velox/connectors/lakehouse/storage_adapters/abfs/tests/MockDataLakeFileClient.h new file mode 100644 index 000000000000..5b874fff56b5 --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/abfs/tests/MockDataLakeFileClient.h @@ -0,0 +1,70 @@ +/* + * 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/exec/tests/utils/TempFilePath.h" + +#include "velox/connectors/hive/storage_adapters/abfs/AzureDataLakeFileClient.h" + +using namespace Azure::Storage::Files::DataLake::Models; + +namespace facebook::velox::filesystems { + +// A mock AzureDataLakeFileClient backend with local file store. +class MockDataLakeFileClient : public AzureDataLakeFileClient { + public: + MockDataLakeFileClient() { + auto tempFile = velox::exec::test::TempFilePath::create(); + filePath_ = tempFile->getPath(); + } + + MockDataLakeFileClient(std::string_view filePath) : filePath_(filePath) {} + + std::string_view path() const { + return filePath_; + } + + void create() override; + + PathProperties getProperties() override; + + void append(const uint8_t* buffer, size_t size, uint64_t offset) override; + + void flush(uint64_t position) override; + + void close() override; + + std::string getUrl() const override { + return "testUrl"; + } + + // for testing purpose to verify the written content if correct. + std::string readContent() { + std::ifstream inputFile(filePath_); + std::string content; + inputFile.seekg(0, std::ios::end); + std::streamsize fileSize = inputFile.tellg(); + inputFile.seekg(0, std::ios::beg); + content.resize(fileSize); + inputFile.read(&content[0], fileSize); + inputFile.close(); + return content; + } + + private: + std::string filePath_; + std::ofstream fileStream_; +}; +} // namespace facebook::velox::filesystems diff --git a/velox/connectors/lakehouse/storage_adapters/gcs/CMakeLists.txt b/velox/connectors/lakehouse/storage_adapters/gcs/CMakeLists.txt new file mode 100644 index 000000000000..6633196c59ab --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/gcs/CMakeLists.txt @@ -0,0 +1,37 @@ +# 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. + +# for generated headers + +velox_add_library(velox_gcs RegisterGcsFileSystem.cpp) + +if(VELOX_ENABLE_GCS) + velox_sources( + velox_gcs + PRIVATE + GcsFileSystem.cpp + GcsUtil.cpp + GcsWriteFile.cpp + GcsReadFile.cpp) + velox_link_libraries(velox_gcs velox_dwio_common Folly::folly + google-cloud-cpp::storage) + + if(${VELOX_BUILD_TESTING}) + add_subdirectory(tests) + endif() + + if(${VELOX_ENABLE_EXAMPLES}) + add_subdirectory(examples) + endif() +endif() diff --git a/velox/connectors/lakehouse/storage_adapters/gcs/GcsFileSystem.cpp b/velox/connectors/lakehouse/storage_adapters/gcs/GcsFileSystem.cpp new file mode 100644 index 000000000000..9c03b48e0027 --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/gcs/GcsFileSystem.cpp @@ -0,0 +1,329 @@ +/* + * 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/storage_adapters/gcs/GcsFileSystem.h" +#include "velox/common/base/Exceptions.h" +#include "velox/common/config/Config.h" +#include "velox/connectors/hive/HiveConfig.h" +#include "velox/connectors/hive/storage_adapters/gcs/GcsReadFile.h" +#include "velox/connectors/hive/storage_adapters/gcs/GcsUtil.h" +#include "velox/connectors/hive/storage_adapters/gcs/GcsWriteFile.h" +#include "velox/core/QueryConfig.h" + +#include +#include +#include +#include + +#include + +namespace facebook::velox { +namespace filesystems { +using namespace connector::hive; +namespace gcs = ::google::cloud::storage; +namespace gc = ::google::cloud; + +auto constexpr kGcsInvalidPath = "File {} is not a valid gcs file"; + +class GcsFileSystem::Impl { + public: + Impl(const config::ConfigBase* config) + : hiveConfig_(std::make_shared( + std::make_shared(config->rawConfigsCopy()))) {} + + ~Impl() = default; + + // Use the input Config parameters and initialize the GcsClient. + void initializeClient() { + constexpr std::string_view kHttpsScheme{"https://"}; + auto options = gc::Options{}; + auto endpointOverride = hiveConfig_->gcsEndpoint(); + // Use secure credentials by default. + if (!endpointOverride.empty()) { + options.set(endpointOverride); + // Use Google default credentials if endpoint has https scheme. + if (endpointOverride.find(kHttpsScheme) == 0) { + options.set( + gc::MakeGoogleDefaultCredentials()); + } else { + options.set( + gc::MakeInsecureCredentials()); + } + } else { + options.set( + gc::MakeGoogleDefaultCredentials()); + } + options.set(kUploadBufferSize); + + auto max_retry_count = hiveConfig_->gcsMaxRetryCount(); + if (max_retry_count) { + options.set( + gcs::LimitedErrorCountRetryPolicy(max_retry_count.value()).clone()); + } + + auto max_retry_time = hiveConfig_->gcsMaxRetryTime(); + if (max_retry_time) { + auto retry_time = std::chrono::duration_cast( + facebook::velox::config::toDuration(max_retry_time.value())); + options.set( + gcs::LimitedTimeRetryPolicy(retry_time).clone()); + } + + auto credFile = hiveConfig_->gcsCredentialsPath(); + if (!credFile.empty() && std::filesystem::exists(credFile)) { + std::ifstream jsonFile(credFile, std::ios::in); + if (!jsonFile.is_open()) { + LOG(WARNING) << "Error opening file " << credFile; + } else { + std::stringstream credsBuffer; + credsBuffer << jsonFile.rdbuf(); + auto creds = credsBuffer.str(); + auto credentials = gc::MakeServiceAccountCredentials(std::move(creds)); + options.set(credentials); + } + } else { + LOG(WARNING) + << "Config hive.gcs.json-key-file-path is empty or key file path not found"; + } + + client_ = std::make_shared(options); + } + + std::shared_ptr getClient() const { + return client_; + } + + private: + const std::shared_ptr hiveConfig_; + std::shared_ptr client_; +}; + +GcsFileSystem::GcsFileSystem(std::shared_ptr config) + : FileSystem(config) { + impl_ = std::make_shared(config.get()); +} + +void GcsFileSystem::initializeClient() { + impl_->initializeClient(); +} + +std::unique_ptr GcsFileSystem::openFileForRead( + std::string_view path, + const FileOptions& options) { + const auto gcspath = gcsPath(path); + auto gcsfile = std::make_unique(gcspath, impl_->getClient()); + gcsfile->initialize(options); + return gcsfile; +} + +std::unique_ptr GcsFileSystem::openFileForWrite( + std::string_view path, + const FileOptions& /*unused*/) { + const auto gcspath = gcsPath(path); + auto gcsfile = std::make_unique(gcspath, impl_->getClient()); + gcsfile->initialize(); + return gcsfile; +} + +void GcsFileSystem::remove(std::string_view path) { + if (!isGcsFile(path)) { + VELOX_FAIL(kGcsInvalidPath, path); + } + + // We assume 'path' is well-formed here. + std::string bucket; + std::string object; + const auto file = gcsPath(path); + setBucketAndKeyFromGcsPath(file, bucket, object); + + if (!object.empty()) { + auto stat = impl_->getClient()->GetObjectMetadata(bucket, object); + if (!stat.ok()) { + checkGcsStatus( + stat.status(), + "Failed to get metadata for GCS object", + bucket, + object); + } + } + auto ret = impl_->getClient()->DeleteObject(bucket, object); + if (!ret.ok()) { + checkGcsStatus( + ret, "Failed to get metadata for GCS object", bucket, object); + } +} + +bool GcsFileSystem::exists(std::string_view path) { + std::vector result; + if (!isGcsFile(path)) + VELOX_FAIL(kGcsInvalidPath, path); + + // We assume 'path' is well-formed here. + const auto file = gcsPath(path); + std::string bucket; + std::string object; + setBucketAndKeyFromGcsPath(file, bucket, object); + using ::google::cloud::StatusOr; + StatusOr metadata = + impl_->getClient()->GetBucketMetadata(bucket); + + return metadata.ok(); +} + +std::vector GcsFileSystem::list(std::string_view path) { + std::vector result; + if (!isGcsFile(path)) + VELOX_FAIL(kGcsInvalidPath, path); + + // We assume 'path' is well-formed here. + const auto file = gcsPath(path); + std::string bucket; + std::string object; + setBucketAndKeyFromGcsPath(file, bucket, object); + for (auto&& metadata : impl_->getClient()->ListObjects(bucket)) { + if (!metadata.ok()) { + checkGcsStatus( + metadata.status(), + "Failed to get metadata for GCS object", + bucket, + object); + } + result.push_back(metadata->name()); + } + + return result; +} + +std::string GcsFileSystem::name() const { + return "GCS"; +} + +void GcsFileSystem::rename( + std::string_view originPath, + std::string_view newPath, + bool overwrite) { + if (!isGcsFile(originPath)) { + VELOX_FAIL(kGcsInvalidPath, originPath); + } + + if (!isGcsFile(newPath)) { + VELOX_FAIL(kGcsInvalidPath, newPath); + } + + std::string originBucket; + std::string originObject; + const auto originFile = gcsPath(originPath); + setBucketAndKeyFromGcsPath(originFile, originBucket, originObject); + + std::string newBucket; + std::string newObject; + const auto newFile = gcsPath(newPath); + setBucketAndKeyFromGcsPath(newFile, newBucket, newObject); + + if (!overwrite) { + auto objects = list(newPath); + if (std::find(objects.begin(), objects.end(), newObject) != objects.end()) { + VELOX_USER_FAIL( + "Failed to rename object {} to {} with as {} exists.", + originObject, + newObject, + newObject); + return; + } + } + + // Copy the object to the new name. + auto copyStats = impl_->getClient()->CopyObject( + originBucket, originObject, newBucket, newObject); + if (!copyStats.ok()) { + checkGcsStatus( + copyStats.status(), + fmt::format( + "Failed to rename for GCS object {}/{}", + originBucket, + originObject), + originBucket, + originObject); + } + + // Delete the original object. + auto delStatus = impl_->getClient()->DeleteObject(originBucket, originObject); + if (!delStatus.ok()) { + checkGcsStatus( + delStatus, + fmt::format( + "Failed to delete for GCS object {}/{} after copy when renaming. And the copied object is at {}/{}", + originBucket, + originObject, + newBucket, + newObject), + originBucket, + originObject); + } +} + +void GcsFileSystem::mkdir( + std::string_view path, + const DirectoryOptions& options) { + if (!isGcsFile(path)) { + VELOX_FAIL(kGcsInvalidPath, path); + } + + std::string bucket; + std::string object; + const auto file = gcsPath(path); + setBucketAndKeyFromGcsPath(file, bucket, object); + + // Create an empty object to represent the directory. + auto status = impl_->getClient()->InsertObject(bucket, object, ""); + + checkGcsStatus( + status.status(), + fmt::format("Failed to mkdir for GCS object {}/{}", bucket, object), + bucket, + object); +} + +void GcsFileSystem::rmdir(std::string_view path) { + if (!isGcsFile(path)) { + VELOX_FAIL(kGcsInvalidPath, path); + } + + const auto file = gcsPath(path); + std::string bucket; + std::string object; + setBucketAndKeyFromGcsPath(file, bucket, object); + for (auto&& metadata : impl_->getClient()->ListObjects(bucket)) { + checkGcsStatus( + metadata.status(), + fmt::format("Failed to rmdir for GCS object {}/{}", bucket, object), + bucket, + object); + + auto status = impl_->getClient()->DeleteObject(bucket, metadata->name()); + checkGcsStatus( + metadata.status(), + fmt::format( + "Failed to delete for GCS object {}/{} when rmdir.", + bucket, + metadata->name()), + bucket, + metadata->name()); + } +} + +} // namespace filesystems +} // namespace facebook::velox diff --git a/velox/connectors/lakehouse/storage_adapters/gcs/GcsFileSystem.h b/velox/connectors/lakehouse/storage_adapters/gcs/GcsFileSystem.h new file mode 100644 index 000000000000..04d4c1630a16 --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/gcs/GcsFileSystem.h @@ -0,0 +1,107 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#pragma once + +#include "velox/common/file/FileSystems.h" + +namespace facebook::velox::filesystems { + +/// Implementation of GCS filesystem and file interface. +/// We provide a registration method for read and write files so the appropriate +/// type of file can be constructed based on a filename. See the +/// (register|generate)ReadFile and (register|generate)WriteFile functions. +class GcsFileSystem : public FileSystem { + public: + explicit GcsFileSystem(std::shared_ptr config); + + /// Initialize the google::cloud::storage::Client from the input Config + /// parameters. + void initializeClient(); + + /// Initialize a ReadFile + /// First the method google::cloud::storage::Client::GetObjectMetadata + /// is used to validate + /// [[https://cloud.google.com/storage/docs/samples/storage-get-metadata]] + /// then the method google::cloud::storage::Client::ReadObject + /// is used to read sequentially + /// [[https://cloud.google.com/storage/docs/samples/storage-stream-file-download]]. + std::unique_ptr openFileForRead( + std::string_view path, + const FileOptions& options = {}) override; + + /// Initialize a WriteFile + /// First the method google::cloud::storage::Client::GetObjectMetadata + /// is used to validate + /// [[https://cloud.google.com/storage/docs/samples/storage-get-metadata]] + /// then the method google::cloud::storage::Client::WriteObject + /// is used to append sequentially + /// [[https://cloud.google.com/storage/docs/samples/storage-stream-file-upload]]. + /// The default buffer size is currently 8 MiB + /// but this default value can change. + /// [[https://cloud.google.com/storage/docs/resumable-uploads]]. + /// The in-memory buffer is kept until the instance is closed or there is an + /// excess of data. If any previously buffered data and the data to append are + /// larger than the maximum size of the internal buffer then the largest + /// amount of data that is a multiple of the upload quantum (256KiB) is + /// flushed. Any data in excess of a multiple of the upload quantum are + /// buffered for the next upload. + std::unique_ptr openFileForWrite( + std::string_view path, + const FileOptions& options = {}) override; + + /// Returns the name of the adapter (GCS) + std::string name() const override; + + /// Removes the objects associated to a path by using + /// google::cloud::storage::Client::DeleteObject. + void remove(std::string_view path) override; + + /// Check that the path exists by using + /// google::cloud::storage::Client::GetObjectMetadata + bool exists(std::string_view path) override; + + /// List the objects associated to a path using + /// google::cloud::storage::Client::ListObjects + std::vector list(std::string_view path) override; + + /// Renames the original object to the new object using + /// google::cloud::storage::Client::CopyObject and + /// google::cloud::storage::Client::DeleteObject. + /// Note that this process involves separate copy and delete operations, which + /// may lead to temporary inconsistencies if either operation fails or if + /// there is a delay between them. + void rename( + std::string_view originPath, + std::string_view newPath, + bool overwrite) override; + + /// Supports mkdir operation by using + /// google::cloud::storage::Client::InsertObject + void mkdir(std::string_view path, const DirectoryOptions& options = {}) + override; + + /// Deletes the objects associated to a path using + /// google::cloud::storage::Client::ListObjects and + /// google::cloud::storage::Client::DeleteObjects + void rmdir(std::string_view path) override; + + protected: + class Impl; + std::shared_ptr impl_; +}; + +} // namespace facebook::velox::filesystems diff --git a/velox/connectors/lakehouse/storage_adapters/gcs/GcsReadFile.cpp b/velox/connectors/lakehouse/storage_adapters/gcs/GcsReadFile.cpp new file mode 100644 index 000000000000..072a4f7a37f2 --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/gcs/GcsReadFile.cpp @@ -0,0 +1,194 @@ +/* + * 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/storage_adapters/gcs/GcsReadFile.h" +#include "velox/connectors/hive/storage_adapters/gcs/GcsUtil.h" + +namespace facebook::velox::filesystems { + +namespace gcs = ::google::cloud::storage; + +class GcsReadFile::Impl { + public: + Impl(const std::string& path, std::shared_ptr client) + : client_(client) { + setBucketAndKeyFromGcsPath(path, bucket_, key_); + } + + // Gets the length of the file. + // Checks if there are any issues reading the file. + void initialize(const filesystems::FileOptions& options) { + if (options.fileSize.has_value()) { + VELOX_CHECK_GE( + options.fileSize.value(), 0, "File size must be non-negative"); + length_ = options.fileSize.value(); + } + + // Make it a no-op if invoked twice. + if (length_ != -1) { + return; + } + // get metadata and initialize length + auto metadata = client_->GetObjectMetadata(bucket_, key_); + if (!metadata.ok()) { + checkGcsStatus( + metadata.status(), + "Failed to get metadata for GCS object", + bucket_, + key_); + } + length_ = (*metadata).size(); + VELOX_CHECK_GE(length_, 0); + } + + std::string_view pread( + uint64_t offset, + uint64_t length, + void* buffer, + std::atomic& bytesRead, + filesystems::File::IoStats* stats = nullptr) const { + preadInternal(offset, length, static_cast(buffer), bytesRead); + return {static_cast(buffer), length}; + } + + std::string pread( + uint64_t offset, + uint64_t length, + std::atomic& bytesRead, + filesystems::File::IoStats* stats = nullptr) const { + std::string result(length, 0); + char* position = result.data(); + preadInternal(offset, length, position, bytesRead); + return result; + } + + uint64_t preadv( + uint64_t offset, + const std::vector>& buffers, + std::atomic& bytesRead, + filesystems::File::IoStats* stats = nullptr) const { + // 'buffers' contains Ranges(data, size) with some gaps (data = nullptr) in + // between. This call must populate the ranges (except gap ranges) + // sequentially starting from 'offset'. If a range pointer is nullptr, the + // data from stream of size range.size() will be skipped. + size_t length = 0; + for (const auto range : buffers) { + length += range.size(); + } + std::string result(length, 0); + preadInternal(offset, length, static_cast(result.data()), bytesRead); + size_t resultOffset = 0; + for (auto range : buffers) { + if (range.data()) { + memcpy(range.data(), &(result.data()[resultOffset]), range.size()); + } + resultOffset += range.size(); + } + return length; + } + + uint64_t size() const { + return length_; + } + + uint64_t memoryUsage() const { + return sizeof(GcsReadFile) // this class + + sizeof(gcs::Client) // pointee + + kUploadBufferSize; // buffer size + } + + std::string getName() const { + return key_; + } + + private: + // The assumption here is that "position" has space for at least "length" + // bytes. + void preadInternal( + uint64_t offset, + uint64_t length, + char* position, + std::atomic& bytesRead_) const { + gcs::ObjectReadStream stream = client_->ReadObject( + bucket_, key_, gcs::ReadRange(offset, offset + length)); + if (!stream) { + checkGcsStatus( + stream.status(), "Failed to get GCS object", bucket_, key_); + } + + stream.read(position, length); + if (!stream) { + checkGcsStatus( + stream.status(), "Failed to get read object", bucket_, key_); + } + bytesRead_ += length; + } + + std::shared_ptr client_; + std::string bucket_; + std::string key_; + std::atomic length_ = -1; +}; + +GcsReadFile::GcsReadFile( + const std::string& path, + std::shared_ptr client) + : impl_(std::make_unique(path, client)) {} + +GcsReadFile::~GcsReadFile() = default; + +void GcsReadFile::initialize(const filesystems::FileOptions& options) { + impl_->initialize(options); +} + +std::string_view GcsReadFile::pread( + uint64_t offset, + uint64_t length, + void* buffer, + filesystems::File::IoStats* stats) const { + return impl_->pread(offset, length, buffer, bytesRead_, stats); +} + +std::string GcsReadFile::pread( + uint64_t offset, + uint64_t length, + filesystems::File::IoStats* stats) const { + return impl_->pread(offset, length, bytesRead_, stats); +} +uint64_t GcsReadFile::preadv( + uint64_t offset, + const std::vector>& buffers, + filesystems::File::IoStats* stats) const { + return impl_->preadv(offset, buffers, bytesRead_, stats); +} + +uint64_t GcsReadFile::size() const { + return impl_->size(); +} + +uint64_t GcsReadFile::memoryUsage() const { + return impl_->memoryUsage(); +} + +std::string GcsReadFile::getName() const { + return impl_->getName(); +} + +uint64_t GcsReadFile::getNaturalReadSize() const { + return kUploadBufferSize; +} + +} // namespace facebook::velox::filesystems diff --git a/velox/connectors/lakehouse/storage_adapters/gcs/GcsReadFile.h b/velox/connectors/lakehouse/storage_adapters/gcs/GcsReadFile.h new file mode 100644 index 000000000000..a3d328996ece --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/gcs/GcsReadFile.h @@ -0,0 +1,70 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#pragma once + +#include +#include "velox/common/file/File.h" + +namespace facebook::velox::filesystems { + +/** + * Implementation of gcs read file. + */ +class GcsReadFile : public ReadFile { + public: + GcsReadFile( + const std::string& path, + std::shared_ptr<::google::cloud::storage::Client> client); + + ~GcsReadFile() override; + + void initialize(const filesystems::FileOptions& options); + + std::string_view pread( + uint64_t offset, + uint64_t length, + void* buffer, + filesystems::File::IoStats* stats = nullptr) const override; + + std::string pread( + uint64_t offset, + uint64_t length, + filesystems::File::IoStats* stats = nullptr) const override; + + uint64_t preadv( + uint64_t offset, + const std::vector>& buffers, + filesystems::File::IoStats* stats = nullptr) const override; + + uint64_t size() const override; + + uint64_t memoryUsage() const override; + + bool shouldCoalesce() const final { + return false; + } + + std::string getName() const override; + + uint64_t getNaturalReadSize() const override; + + protected: + class Impl; + std::shared_ptr impl_; +}; + +} // namespace facebook::velox::filesystems diff --git a/velox/connectors/lakehouse/storage_adapters/gcs/GcsUtil.cpp b/velox/connectors/lakehouse/storage_adapters/gcs/GcsUtil.cpp new file mode 100644 index 000000000000..b9fffb3fa1d4 --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/gcs/GcsUtil.cpp @@ -0,0 +1,57 @@ +/* + * 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/storage_adapters/gcs/GcsUtil.h" + +namespace facebook::velox { + +std::string getErrorStringFromGcsError(const google::cloud::StatusCode& code) { + using ::google::cloud::StatusCode; + + switch (code) { + case StatusCode::kNotFound: + return "Resource not found"; + case StatusCode::kPermissionDenied: + return "Access denied"; + case StatusCode::kUnavailable: + return "Service unavailable"; + + default: + return "Unknown error"; + } +} + +void checkGcsStatus( + const google::cloud::Status outcome, + const std::string_view& errorMsgPrefix, + const std::string& bucket, + const std::string& key) { + if (!outcome.ok()) { + const auto errMsg = fmt::format( + "{} due to: Path:'{}', SDK Error Type:{}, GCS Status Code:{}, Message:'{}'", + errorMsgPrefix, + gcsURI(bucket, key), + outcome.error_info().domain(), + getErrorStringFromGcsError(outcome.code()), + outcome.message()); + if (outcome.code() == google::cloud::StatusCode::kNotFound) { + VELOX_FILE_NOT_FOUND_ERROR(errMsg); + } + VELOX_FAIL(errMsg); + } +} + +} // namespace facebook::velox diff --git a/velox/connectors/lakehouse/storage_adapters/gcs/GcsUtil.h b/velox/connectors/lakehouse/storage_adapters/gcs/GcsUtil.h new file mode 100644 index 000000000000..ac02793ad111 --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/gcs/GcsUtil.h @@ -0,0 +1,79 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#pragma once +#include +#include "velox/common/base/Exceptions.h" + +namespace facebook::velox { + +// Reference: https://github.com/apache/arrow/issues/29916 +// Change the default upload buffer size. In general, sending larger buffers is +// more efficient with GCS, as each buffer requires a roundtrip to the service. +// With formatted output (when using `operator<<`), keeping a larger buffer in +// memory before uploading makes sense. With unformatted output (the only +// choice given gcs::io::OutputStream's API) it is better to let the caller +// provide as large a buffer as they want. The GCS C++ client library will +// upload this buffer with zero copies if possible. +auto constexpr kUploadBufferSize = 256 * 1024; + +constexpr const char* kGcsDefaultCacheKeyPrefix = "gcs-default-key"; + +namespace { +constexpr const char* kSep{"/"}; +constexpr std::string_view kGcsScheme{"gs://"}; + +} // namespace + +std::string getErrorStringFromGcsError(const google::cloud::StatusCode& error); + +inline bool isGcsFile(const std::string_view filename) { + return (filename.substr(0, kGcsScheme.size()) == kGcsScheme); +} + +inline void setBucketAndKeyFromGcsPath( + const std::string& path, + std::string& bucket, + std::string& key) { + auto firstSep = path.find_first_of(kSep); + bucket = path.substr(0, firstSep); + key = path.substr(firstSep + 1); +} + +inline std::string gcsURI(std::string_view bucket) { + std::stringstream ss; + ss << kGcsScheme << bucket; + return ss.str(); +} + +inline std::string gcsURI(std::string_view bucket, std::string_view key) { + std::stringstream ss; + ss << kGcsScheme << bucket << kSep << key; + return ss.str(); +} + +inline std::string gcsPath(const std::string_view& path) { + // Remove the prefix gcs:// from the given path + return std::string(path.substr(kGcsScheme.length())); +} + +void checkGcsStatus( + const google::cloud::Status outcome, + const std::string_view& errorMsgPrefix, + const std::string& bucket, + const std::string& key); + +} // namespace facebook::velox diff --git a/velox/connectors/lakehouse/storage_adapters/gcs/GcsWriteFile.cpp b/velox/connectors/lakehouse/storage_adapters/gcs/GcsWriteFile.cpp new file mode 100644 index 000000000000..d47bc64db69c --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/gcs/GcsWriteFile.cpp @@ -0,0 +1,119 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "velox/connectors/hive/storage_adapters/gcs/GcsWriteFile.h" +#include "velox/connectors/hive/storage_adapters/gcs/GcsUtil.h" + +namespace facebook::velox::filesystems { + +namespace gcs = ::google::cloud::storage; + +class GcsWriteFile::Impl { + public: + Impl(const std::string& path, std::shared_ptr client) + : client_(client) { + setBucketAndKeyFromGcsPath(path, bucket_, key_); + } + + ~Impl() { + close(); + } + + void initialize() { + // Make it a no-op if invoked twice. + if (size_ != -1) { + return; + } + + // Check that it doesn't exist, if it does throw an error + auto object_metadata = client_->GetObjectMetadata(bucket_, key_); + VELOX_CHECK(!object_metadata.ok(), "File already exists"); + + auto stream = client_->WriteObject(bucket_, key_); + checkGcsStatus( + stream.last_status(), + "Failed to open GCS object for writing", + bucket_, + key_); + stream_ = std::move(stream); + size_ = 0; + } + + void append(const std::string_view data) { + VELOX_CHECK(isFileOpen(), "File is not open"); + stream_ << data; + size_ += data.size(); + } + + void flush() { + if (isFileOpen()) { + stream_.flush(); + } + } + + void close() { + if (isFileOpen()) { + stream_.flush(); + stream_.Close(); + closed_ = true; + } + } + + uint64_t size() const { + return size_; + } + + private: + inline bool isFileOpen() { + return (!closed_ && stream_.IsOpen()); + } + + gcs::ObjectWriteStream stream_; + std::shared_ptr client_; + std::string bucket_; + std::string key_; + std::atomic size_{-1}; + std::atomic closed_{false}; +}; + +GcsWriteFile::GcsWriteFile( + const std::string& path, + std::shared_ptr client) + : impl_(std::make_unique(path, client)) {} + +GcsWriteFile::~GcsWriteFile() = default; + +void GcsWriteFile::initialize() { + impl_->initialize(); +} + +void GcsWriteFile::append(const std::string_view data) { + impl_->append(data); +} + +void GcsWriteFile::flush() { + impl_->flush(); +} + +void GcsWriteFile::close() { + impl_->close(); +} + +uint64_t GcsWriteFile::size() const { + return impl_->size(); +} + +} // namespace facebook::velox::filesystems diff --git a/velox/connectors/lakehouse/storage_adapters/gcs/GcsWriteFile.h b/velox/connectors/lakehouse/storage_adapters/gcs/GcsWriteFile.h new file mode 100644 index 000000000000..3e6527c3cf6b --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/gcs/GcsWriteFile.h @@ -0,0 +1,54 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#pragma once + +#include +#include "velox/common/file/File.h" + +namespace facebook::velox::filesystems { + +/** + * Implementation of gcs write file. + */ +class GcsWriteFile : public WriteFile { + public: + GcsWriteFile( + const std::string& path, + std::shared_ptr<::google::cloud::storage::Client> client); + + ~GcsWriteFile() override; + + void initialize(); + + /// Writes the data by append mode. + void append(std::string_view data) override; + + /// Flushs the data. + void flush() override; + + /// Closes the file. + void close() override; + + /// Gets the file size. + uint64_t size() const override; + + protected: + class Impl; + std::shared_ptr impl_; +}; + +} // namespace facebook::velox::filesystems diff --git a/velox/connectors/lakehouse/storage_adapters/gcs/RegisterGcsFileSystem.cpp b/velox/connectors/lakehouse/storage_adapters/gcs/RegisterGcsFileSystem.cpp new file mode 100644 index 000000000000..e064e66298ee --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/gcs/RegisterGcsFileSystem.cpp @@ -0,0 +1,117 @@ +/* + * 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. + */ + +#ifdef VELOX_ENABLE_GCS +#include "velox/common/config/Config.h" +#include "velox/connectors/hive/HiveConfig.h" +#include "velox/connectors/hive/storage_adapters/gcs/GcsFileSystem.h" // @manual +#include "velox/connectors/hive/storage_adapters/gcs/GcsUtil.h" // @manual +#include "velox/dwio/common/FileSink.h" +#endif + +namespace facebook::velox::filesystems { + +#ifdef VELOX_ENABLE_GCS + +using FileSystemMap = folly::Synchronized< + std::unordered_map>>; + +/// Multiple GCS filesystems are supported. +FileSystemMap& gcsFileSystems() { + static FileSystemMap instances; + return instances; +} + +std::function(std::shared_ptr, std::string_view)> +gcsFileSystemGenerator() { + static auto filesystemGenerator = + [](std::shared_ptr properties, + std::string_view filePath) { + const auto file = gcsPath(filePath); + std::string bucket; + std::string object; + setBucketAndKeyFromGcsPath(file, bucket, object); + auto cacheKey = fmt::format( + "{}-{}", + properties->get( + connector::hive::HiveConfig::kGcsEndpoint, + kGcsDefaultCacheKeyPrefix), + bucket); + + // Check if an instance exists with a read lock (shared). + auto fs = gcsFileSystems().withRLock( + [&](auto& instanceMap) -> std::shared_ptr { + auto iterator = instanceMap.find(cacheKey); + if (iterator != instanceMap.end()) { + return iterator->second; + } + return nullptr; + }); + if (fs != nullptr) { + return fs; + } + + return gcsFileSystems().withWLock( + [&](auto& instanceMap) -> std::shared_ptr { + // Repeat the checks with a write lock. + auto iterator = instanceMap.find(cacheKey); + if (iterator != instanceMap.end()) { + return iterator->second; + } + + std::shared_ptr fs; + if (properties != nullptr) { + fs = std::make_shared(properties); + } else { + fs = std::make_shared( + std::make_shared( + std::unordered_map())); + } + fs->initializeClient(); + + instanceMap.insert({cacheKey, fs}); + return fs; + }); + }; + return filesystemGenerator; +} + +std::unique_ptr gcsWriteFileSinkGenerator( + const std::string& fileURI, + const velox::dwio::common::FileSink::Options& options) { + if (isGcsFile(fileURI)) { + auto fileSystem = + filesystems::getFileSystem(fileURI, options.connectorProperties); + return std::make_unique( + fileSystem->openFileForWrite(fileURI, {{}, options.pool, std::nullopt}), + fileURI, + options.metricLogger, + options.stats); + } + return nullptr; +} +#endif + +void registerGcsFileSystem() { +#ifdef VELOX_ENABLE_GCS + registerFileSystem(isGcsFile, gcsFileSystemGenerator()); + dwio::common::FileSink::registerFactory( + std::function(gcsWriteFileSinkGenerator)); +#endif +} + +} // namespace facebook::velox::filesystems diff --git a/velox/connectors/lakehouse/storage_adapters/gcs/RegisterGcsFileSystem.h b/velox/connectors/lakehouse/storage_adapters/gcs/RegisterGcsFileSystem.h new file mode 100644 index 000000000000..b0f668d6f413 --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/gcs/RegisterGcsFileSystem.h @@ -0,0 +1,24 @@ +/* + * 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 + +namespace facebook::velox::filesystems { + +// Register the GCS filesystem. +void registerGcsFileSystem(); + +} // namespace facebook::velox::filesystems diff --git a/velox/connectors/lakehouse/storage_adapters/gcs/examples/CMakeLists.txt b/velox/connectors/lakehouse/storage_adapters/gcs/examples/CMakeLists.txt new file mode 100644 index 000000000000..1363d688da44 --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/gcs/examples/CMakeLists.txt @@ -0,0 +1,24 @@ +# Copyright (c) Facebook, Inc. and its affiliates. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +add_executable(velox_gcsfile_example GcsFileSystemExample.cpp) +target_link_libraries( + velox_gcsfile_example + Folly::folly + velox_file + velox_gcs + velox_core + velox_hive_connector + velox_dwio_common_exception + velox_exec) diff --git a/velox/connectors/lakehouse/storage_adapters/gcs/examples/GcsFileSystemExample.cpp b/velox/connectors/lakehouse/storage_adapters/gcs/examples/GcsFileSystemExample.cpp new file mode 100644 index 000000000000..08d9b83a0152 --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/gcs/examples/GcsFileSystemExample.cpp @@ -0,0 +1,57 @@ +/* + * 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/config/Config.h" +#include "velox/common/file/File.h" +#include "velox/connectors/hive/storage_adapters/gcs/GcsFileSystem.h" + +#include +#include +#include + +DEFINE_string(gcs_path, "", "Path of GCS bucket"); +DEFINE_string(gcs_max_retry_count, "", "Max retry count"); +DEFINE_string(gcs_max_retry_time, "", "Max retry time"); + +auto newConfiguration() { + using namespace facebook::velox; + std::unordered_map configOverride = {}; + if (!FLAGS_gcs_max_retry_count.empty()) { + configOverride.emplace( + "hive.gcs.max-retry-count", FLAGS_gcs_max_retry_count); + } + if (!FLAGS_gcs_max_retry_time.empty()) { + configOverride.emplace("hive.gcs.max-retry-time", FLAGS_gcs_max_retry_time); + } + return std::make_shared(std::move(configOverride)); +} + +int main(int argc, char** argv) { + using namespace facebook::velox; + gflags::ParseCommandLineFlags(&argc, &argv, false); + if (FLAGS_gcs_path.empty()) { + gflags::ShowUsageWithFlags(argv[0]); + return 1; + } + filesystems::GcsFileSystem gcfs(newConfiguration()); + gcfs.initializeClient(); + std::cout << "Opening file for read " << FLAGS_gcs_path << std::endl; + std::unique_ptr file_read = gcfs.openFileForRead(FLAGS_gcs_path); + std::size_t file_size = file_read->size(); + std::cout << "File size = " << file_size << std::endl; + std::string buffer(file_size + 1, '\0'); + file_read->pread(0 /*offset*/, file_size /*lenght*/, buffer.data()); + std::cout << "File Content = " << buffer << std::endl; +} diff --git a/velox/connectors/lakehouse/storage_adapters/gcs/tests/CMakeLists.txt b/velox/connectors/lakehouse/storage_adapters/gcs/tests/CMakeLists.txt new file mode 100644 index 000000000000..c56a201a9b92 --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/gcs/tests/CMakeLists.txt @@ -0,0 +1,61 @@ +# Copyright (c) Facebook, Inc. and its affiliates. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +add_executable(velox_gcs_file_test GcsUtilTest.cpp GcsFileSystemTest.cpp) +add_test(velox_gcs_file_test velox_gcs_file_test) +target_link_libraries( + velox_gcs_file_test + velox_core + velox_dwio_common_exception + velox_exec + velox_exec_test_lib + velox_file + velox_gcs + velox_hive_connector + velox_temp_path + GTest::gmock + GTest::gtest + GTest::gtest_main) + +add_executable(velox_gcs_insert_test GcsInsertTest.cpp) +add_test(velox_gcs_insert_test velox_gcs_insert_test) +target_link_libraries( + velox_gcs_insert_test + velox_file + velox_gcs + velox_hive_config + velox_core + velox_exec_test_lib + velox_dwio_common_exception + velox_exec + GTest::gmock + GTest::gtest + GTest::gtest_main) + +add_executable(velox_gcs_multiendpoints_test GcsMultipleEndpointsTest.cpp) +add_test(velox_gcs_multiendpoints_test velox_gcs_multiendpoints_test) +target_link_libraries( + velox_gcs_multiendpoints_test + velox_file + velox_gcs + velox_hive_config + velox_core + velox_exec_test_lib + velox_dwio_parquet_reader + velox_dwio_parquet_writer + velox_dwio_common_exception + velox_exec + GTest::gmock + GTest::gtest + GTest::gtest_main) diff --git a/velox/connectors/lakehouse/storage_adapters/gcs/tests/GcsEmulator.h b/velox/connectors/lakehouse/storage_adapters/gcs/tests/GcsEmulator.h new file mode 100644 index 000000000000..cef4b83c817c --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/gcs/tests/GcsEmulator.h @@ -0,0 +1,145 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#pragma once + +#include +#include +#include +#include +#include "gtest/gtest.h" + +#include "velox/common/config/Config.h" +#include "velox/connectors/hive/HiveConfig.h" +#include "velox/connectors/hive/storage_adapters/gcs/GcsUtil.h" +#include "velox/exec/tests/utils/PortUtil.h" + +namespace bp = boost::process; +namespace gc = google::cloud; +namespace gcs = google::cloud::storage; + +namespace facebook::velox::filesystems { + +static std::string_view kLoremIpsum = + "Lorem ipsum dolor sit amet, consectetur adipiscing elit, sed do eiusmod tempor" + "incididunt ut labore et dolore magna aliqua. Ut enim ad minim veniam, quis " + "nostrud exercitation ullamco laboris nisi ut aliquip ex ea commodo consequat." + "Duis aute irure dolor in reprehenderit in voluptate velit esse cillum dolore eu" + "fugiat nulla pariatur. Excepteur sint occaecat cupidatat non proident, sunt in" + "culpa qui officia deserunt mollit anim id est laborum."; + +class GcsEmulator : public testing::Environment { + public: + GcsEmulator() { + auto port = std::to_string(exec::test::getFreePort()); + endpoint_ = "http://localhost:" + port; + std::vector names{"python3", "python"}; + // If the build script or application developer provides a value in the + // PYTHON environment variable, then just use that. + if (const auto* env = std::getenv("PYTHON")) { + names = {env}; + } + std::stringstream error; + error << R"""({>>"Coud not start GCS emulator." + " The following list of python interpreter names were used:"})"""; + for (const auto& interpreter : names) { + auto exe_path = bp::search_path(interpreter); + error << " " << interpreter; + if (exe_path.empty()) { + error << " (exe not found)"; + continue; + } + + serverProcess_ = bp::child( + boost::this_process::environment(), + exe_path, + "-m", + "testbench", + "--port", + port, + group_); + if (serverProcess_.valid()) { + return; + } + error << " (failed to start)"; + serverProcess_.terminate(); + serverProcess_.wait(); + } + VELOX_FAIL(error.str()); + } + + ~GcsEmulator() override { + // Brutal shutdown, kill the full process group because the GCS emulator + // may launch additional children. + group_.terminate(); + if (serverProcess_.valid()) { + serverProcess_.wait(); + } + } + + std::shared_ptr hiveConfig( + const std::unordered_map configOverride = {}) + const { + std::unordered_map config( + {{connector::hive::HiveConfig::kGcsEndpoint, endpoint_}}); + + // Update the default config map with the supplied configOverride map + for (const auto& [configName, configValue] : configOverride) { + config[configName] = configValue; + } + + return std::make_shared(std::move(config)); + } + + std::string_view preexistingBucketName() { + return bucketName_; + } + + std::string_view preexistingObjectName() { + return objectName_; + } + + void bootstrap() { + ASSERT_THAT(this, ::testing::NotNull()); + + // Create a bucket and a small file in the testbench. This makes it easier + // to bootstrap GcsFileSystem and its tests. + auto client = gcs::Client( + google::cloud::Options{} + .set(this->endpoint_) + .set(gc::MakeInsecureCredentials())); + + auto bucket = client.CreateBucketForProject( + bucketName_, "ignored-by-testbench", gcs::BucketMetadata{}); + ASSERT_TRUE(bucket.ok()) << "Failed to create bucket <" << bucketName_ + << ">, status=" << bucket.status(); + + auto object = client.InsertObject(bucketName_, objectName_, kLoremIpsum); + ASSERT_TRUE(object.ok()) << "Failed to create object <" << objectName_ + << ">, status=" << object.status(); + } + + private: + std::string endpoint_; + bp::child serverProcess_; + bp::group group_; + static std::string bucketName_; + static std::string objectName_; +}; + +std::string GcsEmulator::bucketName_{"test1-gcs"}; +std::string GcsEmulator::objectName_{"test-object-name"}; + +} // namespace facebook::velox::filesystems diff --git a/velox/connectors/lakehouse/storage_adapters/gcs/tests/GcsFileSystemTest.cpp b/velox/connectors/lakehouse/storage_adapters/gcs/tests/GcsFileSystemTest.cpp new file mode 100644 index 000000000000..0f56079d8ba8 --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/gcs/tests/GcsFileSystemTest.cpp @@ -0,0 +1,276 @@ +/* + * 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/storage_adapters/gcs/GcsFileSystem.h" +#include "velox/common/base/tests/GTestUtils.h" +#include "velox/common/file/File.h" +#include "velox/connectors/hive/storage_adapters/gcs/GcsUtil.h" +#include "velox/connectors/hive/storage_adapters/gcs/RegisterGcsFileSystem.h" +#include "velox/connectors/hive/storage_adapters/gcs/tests/GcsEmulator.h" +#include "velox/exec/tests/utils/TempFilePath.h" + +#include "gtest/gtest.h" + +namespace facebook::velox::filesystems { +namespace { + +class GcsFileSystemTest : public testing::Test { + public: + void SetUp() { + emulator_ = std::make_shared(); + emulator_->bootstrap(); + } + + std::shared_ptr emulator_; +}; + +TEST_F(GcsFileSystemTest, readFile) { + const auto gcsFile = gcsURI( + emulator_->preexistingBucketName(), emulator_->preexistingObjectName()); + + filesystems::GcsFileSystem gcfs(emulator_->hiveConfig()); + gcfs.initializeClient(); + auto readFile = gcfs.openFileForRead(gcsFile); + std::int64_t size = readFile->size(); + std::int64_t ref_size = kLoremIpsum.length(); + EXPECT_EQ(size, ref_size); + EXPECT_EQ(readFile->pread(0, size), kLoremIpsum); + + char buffer1[size]; + ASSERT_EQ(readFile->pread(0, size, &buffer1), kLoremIpsum); + ASSERT_EQ(readFile->size(), ref_size); + + char buffer2[50]; + ASSERT_EQ(readFile->pread(10, 50, &buffer2), kLoremIpsum.substr(10, 50)); + ASSERT_EQ(readFile->size(), ref_size); + + EXPECT_EQ(readFile->pread(10, size - 10), kLoremIpsum.substr(10)); + + char buff1[10]; + char buff2[20]; + char buff3[30]; + std::vector> buffers = { + folly::Range(buff1, 10), + folly::Range(nullptr, 20), + folly::Range(buff2, 20), + folly::Range(nullptr, 30), + folly::Range(buff3, 30)}; + ASSERT_EQ(10 + 20 + 20 + 30 + 30, readFile->preadv(0, buffers)); + ASSERT_EQ(std::string_view(buff1, sizeof(buff1)), kLoremIpsum.substr(0, 10)); + ASSERT_EQ(std::string_view(buff2, sizeof(buff2)), kLoremIpsum.substr(30, 20)); + ASSERT_EQ(std::string_view(buff3, sizeof(buff3)), kLoremIpsum.substr(80, 30)); +} + +TEST_F(GcsFileSystemTest, writeAndReadFile) { + const std::string_view newFile = "readWriteFile.txt"; + const auto gcsFile = gcsURI(emulator_->preexistingBucketName(), newFile); + + filesystems::GcsFileSystem gcfs(emulator_->hiveConfig()); + gcfs.initializeClient(); + auto writeFile = gcfs.openFileForWrite(gcsFile); + std::string_view kDataContent = + "Dance me to your beauty with a burning violin" + "Dance me through the panic till I'm gathered safely in" + "Lift me like an olive branch and be my homeward dove" + "Dance me to the end of love"; + + EXPECT_EQ(writeFile->size(), 0); + std::int64_t contentSize = kDataContent.length(); + writeFile->append(kDataContent.substr(0, 10)); + EXPECT_EQ(writeFile->size(), 10); + writeFile->append(kDataContent.substr(10, contentSize - 10)); + EXPECT_EQ(writeFile->size(), contentSize); + writeFile->flush(); + writeFile->close(); + VELOX_ASSERT_THROW( + writeFile->append(kDataContent.substr(0, 10)), "File is not open"); + + auto readFile = gcfs.openFileForRead(gcsFile); + std::int64_t size = readFile->size(); + EXPECT_EQ(readFile->size(), contentSize); + EXPECT_EQ(readFile->pread(0, size), kDataContent); + + // Opening an existing file for write must be an error. + filesystems::GcsFileSystem newGcfs(emulator_->hiveConfig()); + newGcfs.initializeClient(); + VELOX_ASSERT_THROW(newGcfs.openFileForWrite(gcsFile), "File already exists"); +} + +TEST_F(GcsFileSystemTest, rename) { + filesystems::GcsFileSystem gcfs(emulator_->hiveConfig()); + gcfs.initializeClient(); + + const std::string_view oldFile = "oldTest.txt"; + const std::string_view newFile = "newTest.txt"; + + const auto gcsExistingFile = + gcsURI(emulator_->preexistingBucketName(), oldFile); + auto writeFile = gcfs.openFileForWrite(gcsExistingFile); + std::string_view kDataContent = "GcsFileSystemTest rename operation test"; + writeFile->append(kDataContent.substr(0, 10)); + writeFile->flush(); + writeFile->close(); + + const auto gcsNewFile = gcsURI(emulator_->preexistingBucketName(), newFile); + + VELOX_ASSERT_THROW( + gcfs.rename(gcsExistingFile, gcsExistingFile, false), + fmt::format( + "Failed to rename object {} to {} with as {} exists.", + oldFile, + oldFile, + oldFile)); + + gcfs.rename(gcsExistingFile, gcsNewFile, true); + + auto results = gcfs.list(gcsNewFile); + ASSERT_TRUE( + std::find(results.begin(), results.end(), oldFile) == results.end()); + ASSERT_TRUE( + std::find(results.begin(), results.end(), newFile) != results.end()); +} + +TEST_F(GcsFileSystemTest, mkdir) { + const std::string_view dir = "newDirectory"; + const auto gcsNewDirectory = gcsURI(emulator_->preexistingBucketName(), dir); + filesystems::GcsFileSystem gcfs(emulator_->hiveConfig()); + gcfs.initializeClient(); + gcfs.mkdir(gcsNewDirectory); + const auto& results = gcfs.list(gcsNewDirectory); + ASSERT_TRUE(std::find(results.begin(), results.end(), dir) != results.end()); +} + +TEST_F(GcsFileSystemTest, rmdir) { + const std::string_view dir = "Directory"; + const auto gcsDirectory = gcsURI(emulator_->preexistingBucketName(), dir); + filesystems::GcsFileSystem gcfs(emulator_->hiveConfig()); + gcfs.initializeClient(); + + auto writeFile = gcfs.openFileForWrite(gcsDirectory); + std::string_view kDataContent = "GcsFileSystemTest rename operation test"; + writeFile->append(kDataContent.substr(0, 10)); + writeFile->flush(); + writeFile->close(); + + auto results = gcfs.list(gcsDirectory); + ASSERT_TRUE(std::find(results.begin(), results.end(), dir) != results.end()); + gcfs.rmdir(gcsDirectory); + + results = gcfs.list(gcsDirectory); + ASSERT_TRUE(std::find(results.begin(), results.end(), dir) == results.end()); +} + +TEST_F(GcsFileSystemTest, missingFile) { + const std::string_view file = "newTest.txt"; + const auto gcsFile = gcsURI(emulator_->preexistingBucketName(), file); + filesystems::GcsFileSystem gcfs(emulator_->hiveConfig()); + gcfs.initializeClient(); + VELOX_ASSERT_RUNTIME_THROW_CODE( + gcfs.openFileForRead(gcsFile), + error_code::kFileNotFound, + "\\\"message\\\": \\\"Live version of object test1-gcs/newTest.txt does not exist.\\\""); +} + +TEST_F(GcsFileSystemTest, missingBucket) { + filesystems::GcsFileSystem gcfs(emulator_->hiveConfig()); + gcfs.initializeClient(); + const std::string_view gcsFile = "gs://dummy/foo.txt"; + VELOX_ASSERT_RUNTIME_THROW_CODE( + gcfs.openFileForRead(gcsFile), + error_code::kFileNotFound, + "\\\"message\\\": \\\"Bucket dummy does not exist.\\\""); +} + +TEST_F(GcsFileSystemTest, credentialsConfig) { + // credentials from arrow gcsfs test case + // While this service account key has the correct format, it cannot be used + // for authentication because the key has been deactivated on the server-side, + // *and* the account(s) involved are deleted *and* they are not the accounts + // or projects do not match its contents. + const std::string_view kCreds = R"""({ + "type": "service_account", + "project_id": "foo-project", + "private_key_id": "a1a111aa1111a11a11a11aa111a111a1a1111111", + "private_key": "-----BEGIN PRIVATE KEY-----\nMIIEvQIBADANBgkqhkiG9w0BAQEFA" + "ASCBKcwggSjAgEAAoIBAQCltiF2oP3KJJ+S\ntTc1McylY+TuAi3AdohX7mmqIjd8a3eBYDHs7" + "FlnUrFC4CRijCr0rUqYfg2pmk4a\n6TaKbQRAhWDJ7XD931g7EBvCtd8+JQBNWVKnP9ByJUaO0h" + "WVniM50KTsWtyX3up/\nfS0W2R8Cyx4yvasE8QHH8gnNGtr94iiORDC7De2BwHi/iU8FxMVJAIyD" + "LNfyk0hN\neheYKfIDBgJV2v6VaCOGWaZyEuD0FJ6wFeLybFBwibrLIBE5Y/StCrZoVZ5LocFP\n" + "T4o8kT7bU6yonudSCyNMedYmqHj/iF8B2UN1WrYx8zvoDqZk0nxIglmEYKn/6U7U\ngyETGcW9Ag" + "MBAAECggEAC231vmkpwA7JG9UYbviVmSW79UecsLzsOAZnbtbn1VLT\nPg7sup7tprD/LXHoyIxK7S" + "/jqINvPU65iuUhgCg3Rhz8+UiBhd0pCH/arlIdiPuD\n2xHpX8RIxAq6pGCsoPJ0kwkHSw8UTnxPV8Z" + "CPSRyHV71oQHQgSl/WjNhRi6PQroB\nSqc/pS1m09cTwyKQIopBBVayRzmI2BtBxyhQp9I8t5b7PYkE" + "ZDQlbdq0j5Xipoov\n9EW0+Zvkh1FGNig8IJ9Wp+SZi3rd7KLpkyKPY7BK/g0nXBkDxn019cET0SdJOH" + "QG\nDiHiv4yTRsDCHZhtEbAMKZEpku4WxtQ+JjR31l8ueQKBgQDkO2oC8gi6vQDcx/CX\nZ23x2ZUyar" + "6i0BQ8eJFAEN+IiUapEeCVazuxJSt4RjYfwSa/p117jdZGEWD0GxMC\n+iAXlc5LlrrWs4MWUc0AHTgX" + "na28/vii3ltcsI0AjWMqaybhBTTNbMFa2/fV2OX2\nUimuFyBWbzVc3Zb9KAG4Y7OmJQKBgQC5324IjX" + "Pq5oH8UWZTdJPuO2cgRsvKmR/r\n9zl4loRjkS7FiOMfzAgUiXfH9XCnvwXMqJpuMw2PEUjUT+OyWjJO" + "NEK4qGFJkbN5\n3ykc7p5V7iPPc7Zxj4mFvJ1xjkcj+i5LY8Me+gL5mGIrJ2j8hbuv7f+PWIauyjnp\n" + "Nx/0GVFRuQKBgGNT4D1L7LSokPmFIpYh811wHliE0Fa3TDdNGZnSPhaD9/aYyy78\nLkxYKuT7WY7UVv" + "LN+gdNoVV5NsLGDa4cAV+CWPfYr5PFKGXMT/Wewcy1WOmJ5des\nAgMC6zq0TdYmMBN6WpKUpEnQtbmh" + "3eMnuvADLJWxbH3wCkg+4xDGg2bpAoGAYRNk\nMGtQQzqoYNNSkfus1xuHPMA8508Z8O9pwKU795R3zQ" + "s1NAInpjI1sOVrNPD7Ymwc\nW7mmNzZbxycCUL/yzg1VW4P1a6sBBYGbw1SMtWxun4ZbnuvMc2CTCh+43" + "/1l+FHe\nMmt46kq/2rH2jwx5feTbOE6P6PINVNRJh/9BDWECgYEAsCWcH9D3cI/QDeLG1ao7\nrE2Nckn" + "P8N783edM07Z/zxWsIsXhBPY3gjHVz2LDl+QHgPWhGML62M0ja/6SsJW3\nYvLLIc82V7eqcVJTZtaFkuh" + "t68qu/Jn1ezbzJMJ4YXDYo1+KFi+2CAGR06QILb+I\nlUtj+/nH3HDQjM4ltYfTPUg=\n" + "-----END PRIVATE KEY-----\n", + "client_email": "foo-email@foo-project.iam.gserviceaccount.com", + "client_id": "100000000000000000001", + "auth_uri": "https://accounts.google.com/o/oauth2/auth", + "token_uri": "https://oauth2.googleapis.com/token", + "auth_provider_x509_cert_url": "https://www.googleapis.com/oauth2/v1/certs", + "client_x509_cert_url": "https://www.googleapis.com/robot/v1/metadata/x509/foo-email%40foo-project.iam.gserviceaccount.com" + })"""; + auto jsonFile = exec::test::TempFilePath::create(); + std::ofstream credsOut(jsonFile->getPath()); + credsOut << kCreds; + credsOut.close(); + + std::unordered_map configOverride = { + {"hive.gcs.json-key-file-path", jsonFile->getPath()}}; + auto hiveConfig = emulator_->hiveConfig(configOverride); + + filesystems::GcsFileSystem gcfs(hiveConfig); + gcfs.initializeClient(); + const auto gcsFile = gcsURI( + emulator_->preexistingBucketName(), emulator_->preexistingObjectName()); + VELOX_ASSERT_THROW( + gcfs.openFileForRead(gcsFile), "Invalid ServiceAccountCredentials"); +} + +TEST_F(GcsFileSystemTest, defaultCacheKey) { + registerGcsFileSystem(); + std::unordered_map configWithoutEndpoint = {}; + auto hiveConfigDefault = std::make_shared( + std::move(configWithoutEndpoint)); + const auto gcsFile1 = gcsURI( + emulator_->preexistingBucketName(), emulator_->preexistingObjectName()); + // FileSystem should be cached by the default key. + auto defaultGcs = filesystems::getFileSystem(gcsFile1, hiveConfigDefault); + + std::unordered_map configWithEndpoint = { + {connector::hive::HiveConfig::kGcsEndpoint, kGcsDefaultCacheKeyPrefix}}; + auto hiveConfigCustom = + std::make_shared(std::move(configWithEndpoint)); + const auto gcsFile2 = gcsURI(emulator_->preexistingBucketName(), "dummy.txt"); + auto customGcs = filesystems::getFileSystem(gcsFile2, hiveConfigCustom); + // The same FileSystem should be cached by the value of key + // kGcsDefaultCacheKeyPrefix. + ASSERT_EQ(customGcs, defaultGcs); +} + +} // namespace +} // namespace facebook::velox::filesystems diff --git a/velox/connectors/lakehouse/storage_adapters/gcs/tests/GcsInsertTest.cpp b/velox/connectors/lakehouse/storage_adapters/gcs/tests/GcsInsertTest.cpp new file mode 100644 index 000000000000..03b6b8adbd8c --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/gcs/tests/GcsInsertTest.cpp @@ -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. + */ + +#include +#include + +#include "velox/connectors/hive/storage_adapters/gcs/RegisterGcsFileSystem.h" +#include "velox/connectors/hive/storage_adapters/gcs/tests/GcsEmulator.h" +#include "velox/connectors/hive/storage_adapters/test_common/InsertTest.h" + +using namespace facebook::velox::exec::test; + +namespace facebook::velox::filesystems { +namespace { + +class GcsInsertTest : public testing::Test, public test::InsertTest { + protected: + static void SetUpTestSuite() { + registerGcsFileSystem(); + memory::MemoryManager::testingSetInstance(memory::MemoryManager::Options{}); + } + + void SetUp() override { + emulator_ = std::make_shared(); + emulator_->bootstrap(); + InsertTest::SetUp(emulator_->hiveConfig(), ioExecutor_.get()); + ioExecutor_ = std::make_unique(3); + } + + void TearDown() override { + InsertTest::TearDown(); + } + + std::shared_ptr emulator_; + std::unique_ptr ioExecutor_; +}; +} // namespace + +TEST_F(GcsInsertTest, gcsInsertTest) { + const int64_t kExpectedRows = 1'000; + const auto gcsBucket = gcsURI(emulator_->preexistingBucketName(), ""); + runInsertTest(gcsBucket, kExpectedRows, pool()); +} +} // namespace facebook::velox::filesystems + +int main(int argc, char** argv) { + testing::InitGoogleTest(&argc, argv); + folly::Init init{&argc, &argv, false}; + return RUN_ALL_TESTS(); +} diff --git a/velox/connectors/lakehouse/storage_adapters/gcs/tests/GcsMultipleEndpointsTest.cpp b/velox/connectors/lakehouse/storage_adapters/gcs/tests/GcsMultipleEndpointsTest.cpp new file mode 100644 index 000000000000..a2ec3a61e9d7 --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/gcs/tests/GcsMultipleEndpointsTest.cpp @@ -0,0 +1,213 @@ +/* + * 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 + +#include "gtest/gtest.h" +#include "velox/connectors/hive/HiveConnector.h" +#include "velox/connectors/hive/HiveConnectorSplit.h" +#include "velox/connectors/hive/storage_adapters/gcs/RegisterGcsFileSystem.h" +#include "velox/connectors/hive/storage_adapters/gcs/tests/GcsEmulator.h" +#include "velox/dwio/parquet/RegisterParquetReader.h" +#include "velox/dwio/parquet/RegisterParquetWriter.h" +#include "velox/exec/TableWriter.h" +#include "velox/exec/tests/utils/AssertQueryBuilder.h" +#include "velox/exec/tests/utils/PlanBuilder.h" +#include "velox/vector/tests/utils/VectorTestBase.h" + +static const std::string_view kConnectorId1 = "test-hive1"; +static const std::string_view kConnectorId2 = "test-hive2"; +static const std::string_view kBucketName = "writedata"; + +using namespace facebook::velox::exec::test; + +namespace facebook::velox::filesystems { +namespace { + +class GcsMultipleEndpointsTest : public testing::Test, + public velox::test::VectorTestBase { + public: + static void SetUpTestCase() { + registerGcsFileSystem(); + memory::MemoryManager::testingSetInstance(memory::MemoryManager::Options{}); + } + + void SetUp() override { + gcsEmulatorOne_ = std::make_unique(); + gcsEmulatorOne_->bootstrap(); + gcsEmulatorTwo_ = std::make_unique(); + gcsEmulatorTwo_->bootstrap(); + + connector::registerConnectorFactory( + std::make_shared()); + parquet::registerParquetReaderFactory(); + parquet::registerParquetWriterFactory(); + } + + void registerConnectors( + std::string_view connectorId1, + std::string_view connectorId2, + const std::unordered_map config1Override = {}, + const std::unordered_map config2Override = {}) { + auto hiveConnector1 = + connector::getConnectorFactory( + connector::hive::HiveConnectorFactory::kHiveConnectorName) + ->newConnector( + std::string(connectorId1), + gcsEmulatorOne_->hiveConfig(config1Override), + ioExecutor_.get()); + auto hiveConnector2 = + connector::getConnectorFactory( + connector::hive::HiveConnectorFactory::kHiveConnectorName) + ->newConnector( + std::string(connectorId2), + gcsEmulatorTwo_->hiveConfig(config2Override), + ioExecutor_.get()); + connector::registerConnector(hiveConnector1); + connector::registerConnector(hiveConnector2); + } + + void TearDown() override { + parquet::unregisterParquetReaderFactory(); + parquet::unregisterParquetWriterFactory(); + connector::unregisterConnectorFactory( + connector::hive::HiveConnectorFactory::kHiveConnectorName); + } + + folly::dynamic writeData( + const RowVectorPtr input, + const std::string& outputDirectory, + const std::string& connectorId) { + auto plan = PlanBuilder() + .values({input}) + .tableWrite( + outputDirectory.data(), + {}, + 0, + {}, + {}, + dwio::common::FileFormat::PARQUET, + {}, + connectorId) + .planNode(); + // Execute the write plan. + auto results = AssertQueryBuilder(plan).copyResults(pool()); + // Second column contains details about written files. + auto details = results->childAt(exec::TableWriteTraits::kFragmentChannel) + ->as>(); + folly::dynamic obj = folly::parseJson(details->valueAt(1)); + return obj["fileWriteInfos"]; + } + + std::shared_ptr createSplit( + folly::dynamic tableWriteInfo, + std::string outputDirectory, + std::string connectorId) { + auto writeFileName = tableWriteInfo[0]["writeFileName"].asString(); + auto filePath = fmt::format("{}{}", outputDirectory, writeFileName); + const int64_t fileSize = tableWriteInfo[0]["fileSize"].asInt(); + + return connector::hive::HiveConnectorSplitBuilder(filePath) + .connectorId(connectorId) + .fileFormat(dwio::common::FileFormat::PARQUET) + .length(fileSize) + .build(); + } + + void testJoin( + int numRows, + std::string_view outputDirectory, + std::string_view connectorId1, + std::string_view connectorId2) { + auto rowType1 = ROW( + {"a0", "a1", "a2", "a3"}, {BIGINT(), INTEGER(), SMALLINT(), DOUBLE()}); + auto rowType2 = ROW( + {"b0", "b1", "b2", "b3"}, {BIGINT(), INTEGER(), SMALLINT(), DOUBLE()}); + + auto input1 = makeRowVector( + rowType1->names(), + {makeFlatVector(numRows, [](auto row) { return row; }), + makeFlatVector(numRows, [](auto row) { return row; }), + makeFlatVector(numRows, [](auto row) { return row; }), + makeFlatVector(numRows, [](auto row) { return row; })}); + auto input2 = makeRowVector(rowType2->names(), input1->children()); + + // Insert input data into both tables. + auto table1WriteInfo = + writeData(input1, outputDirectory.data(), std::string(connectorId1)); + auto table2WriteInfo = + writeData(input2, outputDirectory.data(), std::string(connectorId2)); + + // Inner Join both the tables. + core::PlanNodeId scan1, scan2; + auto planNodeIdGenerator = std::make_shared(); + auto table1Scan = PlanBuilder(planNodeIdGenerator, pool()) + .startTableScan() + .tableName("hive_table1") + .outputType(rowType1) + .connectorId(std::string(connectorId1)) + .endTableScan() + .capturePlanNodeId(scan1) + .planNode(); + auto join = + PlanBuilder(planNodeIdGenerator, pool()) + .startTableScan() + .tableName("hive_table1") + .outputType(rowType2) + .connectorId(std::string(connectorId2)) + .endTableScan() + .capturePlanNodeId(scan2) + .hashJoin({"b0"}, {"a0"}, table1Scan, "", {"a0", "a1", "a2", "a3"}) + .planNode(); + + auto split1 = createSplit( + table1WriteInfo, outputDirectory.data(), std::string(connectorId1)); + auto split2 = createSplit( + table2WriteInfo, outputDirectory.data(), std::string(connectorId2)); + auto results = AssertQueryBuilder(join) + .split(scan1, split1) + .split(scan2, split2) + .copyResults(pool()); + assertEqualResults({input1}, {results}); + } + + std::unique_ptr gcsEmulatorOne_; + std::unique_ptr gcsEmulatorTwo_; + std::unique_ptr ioExecutor_; +}; +} // namespace + +TEST_F(GcsMultipleEndpointsTest, baseEndpoints) { + const int64_t kExpectedRows = 1'000; + + const auto gcsBucket = gcsURI(gcsEmulatorOne_->preexistingBucketName(), ""); + + registerConnectors(kConnectorId1, kConnectorId2); + + testJoin(kExpectedRows, gcsBucket, kConnectorId1, kConnectorId2); + + connector::unregisterConnector(std::string(kConnectorId1)); + connector::unregisterConnector(std::string(kConnectorId2)); +} + +} // namespace facebook::velox::filesystems + +int main(int argc, char** argv) { + testing::InitGoogleTest(&argc, argv); + folly::Init init{&argc, &argv, false}; + return RUN_ALL_TESTS(); +} diff --git a/velox/connectors/lakehouse/storage_adapters/gcs/tests/GcsUtilTest.cpp b/velox/connectors/lakehouse/storage_adapters/gcs/tests/GcsUtilTest.cpp new file mode 100644 index 000000000000..fd31b692e4e8 --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/gcs/tests/GcsUtilTest.cpp @@ -0,0 +1,36 @@ +/* + * 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/storage_adapters/gcs/GcsUtil.h" + +#include "gtest/gtest.h" + +using namespace facebook::velox; + +TEST(GcsUtilTest, isGcsFile) { + EXPECT_FALSE(isGcsFile("gs:")); + EXPECT_FALSE(isGcsFile("gs::/bucket")); + EXPECT_FALSE(isGcsFile("gs:/bucket")); + EXPECT_TRUE(isGcsFile("gs://bucket/file.txt")); +} + +TEST(GcsUtilTest, setBucketAndKeyFromGcsPath) { + std::string bucket, key; + auto path = "bucket/file.txt"; + setBucketAndKeyFromGcsPath(path, bucket, key); + EXPECT_EQ(bucket, "bucket"); + EXPECT_EQ(key, "file.txt"); +} diff --git a/velox/connectors/lakehouse/storage_adapters/hdfs/CMakeLists.txt b/velox/connectors/lakehouse/storage_adapters/hdfs/CMakeLists.txt new file mode 100644 index 000000000000..44aa7be3489c --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/hdfs/CMakeLists.txt @@ -0,0 +1,36 @@ +# 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. + +# for generated headers + +velox_add_library(velox_hdfs RegisterHdfsFileSystem.cpp) + +if(VELOX_ENABLE_HDFS) + velox_sources( + velox_hdfs + PRIVATE + HdfsFileSystem.cpp + HdfsReadFile.cpp + HdfsWriteFile.cpp) + velox_link_libraries( + velox_hdfs + velox_external_hdfs + velox_dwio_common + Folly::folly + xsimd) + + if(${VELOX_BUILD_TESTING}) + add_subdirectory(tests) + endif() +endif() diff --git a/velox/connectors/lakehouse/storage_adapters/hdfs/HdfsFileSystem.cpp b/velox/connectors/lakehouse/storage_adapters/hdfs/HdfsFileSystem.cpp new file mode 100644 index 000000000000..e38bac2b2c13 --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/hdfs/HdfsFileSystem.cpp @@ -0,0 +1,282 @@ +/* + * 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/storage_adapters/hdfs/HdfsFileSystem.h" +#include "velox/common/config/Config.h" +#include "velox/connectors/hive/storage_adapters/hdfs/HdfsReadFile.h" +#include "velox/connectors/hive/storage_adapters/hdfs/HdfsWriteFile.h" +#include "velox/external/hdfs/ArrowHdfsInternal.h" + +namespace facebook::velox::filesystems { +std::string_view HdfsFileSystem::kScheme("hdfs://"); + +std::string_view HdfsFileSystem::kViewfsScheme("viewfs://"); + +class HdfsFileSystem::Impl { + public: + // Keep config here for possible use in the future. + explicit Impl( + const config::ConfigBase* config, + const HdfsServiceEndpoint& endpoint) { + auto status = filesystems::arrow::io::internal::ConnectLibHdfs(&driver_); + if (!status.ok()) { + LOG(ERROR) << "ConnectLibHdfs failed due to: " << status.ToString(); + } + + // connect to HDFS with the builder object + hdfsBuilder* builder = driver_->NewBuilder(); + if (endpoint.isViewfs) { + // The default NameNode configuration will be used (from the XML + // configuration files). See: + // https://github.com/facebookincubator/velox/blob/main/velox/external/hdfs/hdfs.h#L289 + driver_->BuilderSetNameNode(builder, "default"); + } else { + driver_->BuilderSetNameNode(builder, endpoint.host.c_str()); + driver_->BuilderSetNameNodePort(builder, atoi(endpoint.port.data())); + } + driver_->BuilderSetForceNewInstance(builder); + hdfsClient_ = driver_->BuilderConnect(builder); + VELOX_CHECK_NOT_NULL( + hdfsClient_, + "Unable to connect to HDFS: {}, got error: {}.", + endpoint.identity(), + driver_->GetLastExceptionRootCause()); + } + + ~Impl() { + LOG(INFO) << "Disconnecting HDFS file system"; + int disconnectResult = driver_->Disconnect(hdfsClient_); + if (disconnectResult != 0) { + LOG(WARNING) << "hdfs disconnect failure in HdfsReadFile close: " + << errno; + } + } + + hdfsFS hdfsClient() { + return hdfsClient_; + } + + filesystems::arrow::io::internal::LibHdfsShim* hdfsShim() { + return driver_; + } + + private: + hdfsFS hdfsClient_; + filesystems::arrow::io::internal::LibHdfsShim* driver_; +}; + +HdfsFileSystem::HdfsFileSystem( + const std::shared_ptr& config, + const HdfsServiceEndpoint& endpoint) + : FileSystem(config) { + impl_ = std::make_shared(config.get(), endpoint); +} + +std::string HdfsFileSystem::name() const { + return "HDFS"; +} + +std::unique_ptr HdfsFileSystem::openFileForRead( + std::string_view path, + const FileOptions& /*unused*/) { + // Only remove the scheme for hdfs path. + if (path.find(kScheme) == 0) { + path.remove_prefix(kScheme.length()); + if (auto index = path.find('/')) { + path.remove_prefix(index); + } + } + return std::make_unique( + impl_->hdfsShim(), impl_->hdfsClient(), path); +} + +std::unique_ptr HdfsFileSystem::openFileForWrite( + std::string_view path, + const FileOptions& /*unused*/) { + return std::make_unique( + impl_->hdfsShim(), impl_->hdfsClient(), path); +} + +bool HdfsFileSystem::isHdfsFile(const std::string_view filePath) { + return (filePath.find(kScheme) == 0) || (filePath.find(kViewfsScheme) == 0); +} + +/// Gets hdfs endpoint from a given file path. If not found, fall back to get a +/// fixed one from configuration. +HdfsServiceEndpoint HdfsFileSystem::getServiceEndpoint( + const std::string_view filePath, + const config::ConfigBase* config) { + if (filePath.find(kViewfsScheme) == 0) { + return HdfsServiceEndpoint{"viewfs", "", true}; + } + + auto endOfIdentityInfo = filePath.find('/', kScheme.size()); + std::string hdfsIdentity{ + filePath.data(), kScheme.size(), endOfIdentityInfo - kScheme.size()}; + if (hdfsIdentity.empty()) { + // Fall back to get a fixed endpoint from config. + auto hdfsHost = config->get("hive.hdfs.host"); + VELOX_CHECK( + hdfsHost.hasValue(), + "hdfsHost is empty, configuration missing for hdfs host"); + auto hdfsPort = config->get("hive.hdfs.port"); + VELOX_CHECK( + hdfsPort.hasValue(), + "hdfsPort is empty, configuration missing for hdfs port"); + return HdfsServiceEndpoint{*hdfsHost, *hdfsPort}; + } + + auto hostAndPortSeparator = hdfsIdentity.find(':', 0); + // In HDFS HA mode, the hdfsIdentity is a nameservice ID with no port. + if (hostAndPortSeparator == std::string::npos) { + return HdfsServiceEndpoint{hdfsIdentity, ""}; + } + std::string host{hdfsIdentity.data(), 0, hostAndPortSeparator}; + std::string port{ + hdfsIdentity.data(), + hostAndPortSeparator + 1, + hdfsIdentity.size() - hostAndPortSeparator - 1}; + return HdfsServiceEndpoint{host, port}; +} + +void HdfsFileSystem::remove(std::string_view path) { + // Only remove the scheme for hdfs path. + if (path.find(kScheme) == 0) { + path.remove_prefix(kScheme.length()); + if (auto index = path.find('/')) { + path.remove_prefix(index); + } + } + + VELOX_CHECK_EQ( + impl_->hdfsShim()->Delete(impl_->hdfsClient(), path.data(), 0), + 0, + "Cannot delete file : {} in HDFS, error is : {}", + path, + impl_->hdfsShim()->GetLastExceptionRootCause()); +} + +std::vector HdfsFileSystem::list(std::string_view path) { + // Only remove the scheme for hdfs path. + if (path.find(kScheme) == 0) { + path.remove_prefix(kScheme.length()); + if (auto index = path.find('/')) { + path.remove_prefix(index); + } + } + + std::vector result; + int numEntries; + + auto fileInfo = impl_->hdfsShim()->ListDirectory( + impl_->hdfsClient(), path.data(), &numEntries); + + VELOX_CHECK_NOT_NULL( + fileInfo, + "Unable to list the files in path {}. got error: {}", + path, + impl_->hdfsShim()->GetLastExceptionRootCause()); + + for (auto i = 0; i < numEntries; i++) { + result.emplace_back(fileInfo[i].mName); + } + + impl_->hdfsShim()->FreeFileInfo(fileInfo, numEntries); + + return result; +} + +bool HdfsFileSystem::exists(std::string_view path) { + // Only remove the scheme for hdfs path. + if (path.find(kScheme) == 0) { + path.remove_prefix(kScheme.length()); + if (auto index = path.find('/')) { + path.remove_prefix(index); + } + } + + return impl_->hdfsShim()->Exists(impl_->hdfsClient(), path.data()) == 0; +} + +void HdfsFileSystem::mkdir( + std::string_view path, + const DirectoryOptions& options) { + // Only remove the scheme for hdfs path. + if (path.find(kScheme) == 0) { + path.remove_prefix(kScheme.length()); + if (auto index = path.find('/')) { + path.remove_prefix(index); + } + } + + VELOX_CHECK_EQ( + impl_->hdfsShim()->MakeDirectory(impl_->hdfsClient(), path.data()), + 0, + "Cannot mkdir {} in HDFS, error is : {}", + path, + impl_->hdfsShim()->GetLastExceptionRootCause()); +} + +void HdfsFileSystem::rename( + std::string_view path, + std::string_view newPath, + bool overWrite) { + VELOX_CHECK_EQ( + overWrite, false, "HdfsFileSystem::rename doesn't support overwrite"); + // Only remove the scheme for hdfs path. + if (path.find(kScheme) == 0) { + path.remove_prefix(kScheme.length()); + if (auto index = path.find('/')) { + path.remove_prefix(index); + } + } + + // Only remove the scheme for hdfs path. + if (newPath.find(kScheme) == 0) { + newPath.remove_prefix(kScheme.length()); + if (auto index = newPath.find('/')) { + newPath.remove_prefix(index); + } + } + + VELOX_CHECK_EQ( + impl_->hdfsShim()->Rename( + impl_->hdfsClient(), path.data(), newPath.data()), + 0, + "Cannot rename file from {} to {} in HDFS, error is : {}", + path, + newPath, + impl_->hdfsShim()->GetLastExceptionRootCause()); +} + +void HdfsFileSystem::rmdir(std::string_view path) { + // Only remove the scheme for hdfs path. + if (path.find(kScheme) == 0) { + path.remove_prefix(kScheme.length()); + if (auto index = path.find('/')) { + path.remove_prefix(index); + } + } + + VELOX_CHECK_EQ( + impl_->hdfsShim()->Delete( + impl_->hdfsClient(), path.data(), /*recursive=*/true), + 0, + "Cannot remove directory {} recursively in HDFS, error is : {}", + path, + impl_->hdfsShim()->GetLastExceptionRootCause()); +} + +} // namespace facebook::velox::filesystems diff --git a/velox/connectors/lakehouse/storage_adapters/hdfs/HdfsFileSystem.h b/velox/connectors/lakehouse/storage_adapters/hdfs/HdfsFileSystem.h new file mode 100644 index 000000000000..cebe40aa890d --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/hdfs/HdfsFileSystem.h @@ -0,0 +1,100 @@ +/* + * 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/file/FileSystems.h" + +namespace facebook::velox::filesystems { + +struct HdfsServiceEndpoint { + HdfsServiceEndpoint( + const std::string& hdfsHost, + const std::string& hdfsPort, + bool isViewfs = false) + : host(hdfsHost), port(hdfsPort), isViewfs(isViewfs) {} + + /// In HDFS HA mode, the identity is a nameservice ID with no port, e.g., + /// the identity is nameservice_id for + /// hdfs://nameservice_id/file/path/in/hdfs. Otherwise, a port must be + /// contained, e.g., the identity is hdfs_namenode:9000 for + /// hdfs://hdfs_namenode:9000/file/path/in/hdfs. + std::string identity() const { + return host + (port.empty() ? "" : ":" + port); + } + + const std::string host; + const std::string port; + bool isViewfs; +}; + +/** + * You can configure hdfs settings (timeouts etc) using configure file + * which is given by environment parameter LIBHDFS3_CONF + * or "hdfs-client.xml" in working directory. + * + * Internally you can use hdfsBuilderConfSetStr to configure the client + */ +class HdfsFileSystem : public FileSystem { + public: + explicit HdfsFileSystem( + const std::shared_ptr& config, + const HdfsServiceEndpoint& endpoint); + + std::string name() const override; + + std::unique_ptr openFileForRead( + std::string_view path, + const FileOptions& options = {}) override; + + std::unique_ptr openFileForWrite( + std::string_view path, + const FileOptions& options = {}) override; + + // Deletes the hdfs files. + void remove(std::string_view path) override; + + void rename( + std::string_view path, + std::string_view newPath, + bool overWrite = false) override; + + bool exists(std::string_view path) override; + + /// List the objects associated to a path. + std::vector list(std::string_view path) override; + + void mkdir(std::string_view path, const DirectoryOptions& options = {}) + override; + + void rmdir(std::string_view path) override; + + static bool isHdfsFile(std::string_view filename); + + /// The given filePath is used to infer hdfs endpoint. If hdfs identity is + /// missing from filePath, the configured "hive.hdfs.host" & "hive.hdfs.port" + /// will be used. + static HdfsServiceEndpoint getServiceEndpoint( + const std::string_view filePath, + const config::ConfigBase* config); + + static std::string_view kScheme; + + static std::string_view kViewfsScheme; + + protected: + class Impl; + std::shared_ptr impl_; +}; + +} // namespace facebook::velox::filesystems diff --git a/velox/connectors/lakehouse/storage_adapters/hdfs/HdfsReadFile.cpp b/velox/connectors/lakehouse/storage_adapters/hdfs/HdfsReadFile.cpp new file mode 100644 index 000000000000..affc1dfd2ede --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/hdfs/HdfsReadFile.cpp @@ -0,0 +1,198 @@ +/* + * 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 "HdfsReadFile.h" +#include "velox/external/hdfs/ArrowHdfsInternal.h" + +namespace facebook::velox { + +struct HdfsFile { + filesystems::arrow::io::internal::LibHdfsShim* driver_; + hdfsFS client_; + hdfsFile handle_; + + HdfsFile() : driver_(nullptr), client_(nullptr), handle_(nullptr) {} + ~HdfsFile() { + if (handle_ && driver_->CloseFile(client_, handle_) == -1) { + LOG(ERROR) << "Unable to close file, errno: " << errno; + } + } + + void open( + filesystems::arrow::io::internal::LibHdfsShim* driver, + hdfsFS client, + const std::string& path) { + driver_ = driver; + client_ = client; + handle_ = driver->OpenFile(client, path.data(), O_RDONLY, 0, 0, 0); + VELOX_CHECK_NOT_NULL( + handle_, + "Unable to open file {}. got error: {}", + path, + driver_->GetLastExceptionRootCause()); + } + + void seek(uint64_t offset) const { + VELOX_CHECK_EQ( + driver_->Seek(client_, handle_, offset), + 0, + "Cannot seek through HDFS file, error is : {}", + driver_->GetLastExceptionRootCause()); + } + + int32_t read(char* pos, uint64_t length) const { + auto bytesRead = driver_->Read(client_, handle_, pos, length); + VELOX_CHECK(bytesRead >= 0, "Read failure in HDFSReadFile::preadInternal."); + return bytesRead; + } +}; + +class HdfsReadFile::Impl { + public: + Impl( + filesystems::arrow::io::internal::LibHdfsShim* driver, + hdfsFS hdfs, + const std::string_view path) + : driver_(driver), hdfsClient_(hdfs), filePath_(path) { + fileInfo_ = driver_->GetPathInfo(hdfsClient_, filePath_.data()); + if (fileInfo_ == nullptr) { + auto error = fmt::format( + "FileNotFoundException: Path {} does not exist.", filePath_); + auto errMsg = fmt::format( + "Unable to get file path info for file: {}. got error: {}", + filePath_, + error); + if (error.find("FileNotFoundException") != std::string::npos) { + VELOX_FILE_NOT_FOUND_ERROR(errMsg); + } + VELOX_FAIL(errMsg); + } + } + + ~Impl() { + // Should call hdfsFreeFileInfo to avoid memory leak + if (fileInfo_) { + driver_->FreeFileInfo(fileInfo_, 1); + } + } + + void preadInternal(uint64_t offset, uint64_t length, char* pos) const { + checkFileReadParameters(offset, length); + if (!file_->handle_) { + file_->open(driver_, hdfsClient_, filePath_); + } + file_->seek(offset); + uint64_t totalBytesRead = 0; + while (totalBytesRead < length) { + auto bytesRead = file_->read(pos, length - totalBytesRead); + totalBytesRead += bytesRead; + pos += bytesRead; + } + } + + std::string_view pread(uint64_t offset, uint64_t length, void* buf) const { + preadInternal(offset, length, static_cast(buf)); + return {static_cast(buf), length}; + } + + std::string pread(uint64_t offset, uint64_t length) const { + std::string result(length, 0); + char* pos = result.data(); + preadInternal(offset, length, pos); + return result; + } + + uint64_t size() const { + return fileInfo_->mSize; + } + + uint64_t memoryUsage() const { + return fileInfo_->mBlockSize; + } + + bool shouldCoalesce() const { + return false; + } + + std::string getName() const { + return filePath_; + } + + void checkFileReadParameters(uint64_t offset, uint64_t length) const { + auto fileSize = size(); + auto endPoint = offset + length; + VELOX_CHECK_GE( + fileSize, + endPoint, + "Cannot read HDFS file beyond its size: {}, offset: {}, end point: {}", + fileSize, + offset, + endPoint); + } + + private: + filesystems::arrow::io::internal::LibHdfsShim* driver_; + hdfsFS hdfsClient_; + std::string filePath_; + hdfsFileInfo* fileInfo_; + folly::ThreadLocal file_; +}; + +HdfsReadFile::HdfsReadFile( + filesystems::arrow::io::internal::LibHdfsShim* driver, + hdfsFS hdfs, + const std::string_view path) + : pImpl(std::make_unique(driver, hdfs, path)) {} + +HdfsReadFile::~HdfsReadFile() = default; + +std::string_view HdfsReadFile::pread( + uint64_t offset, + uint64_t length, + void* buf, + filesystems::File::IoStats* stats) const { + return pImpl->pread(offset, length, buf); +} + +std::string HdfsReadFile::pread( + uint64_t offset, + uint64_t length, + filesystems::File::IoStats* stats) const { + return pImpl->pread(offset, length); +} + +uint64_t HdfsReadFile::size() const { + return pImpl->size(); +} + +uint64_t HdfsReadFile::memoryUsage() const { + return pImpl->memoryUsage(); +} + +bool HdfsReadFile::shouldCoalesce() const { + return pImpl->shouldCoalesce(); +} + +std::string HdfsReadFile::getName() const { + return pImpl->getName(); +} + +void HdfsReadFile::checkFileReadParameters(uint64_t offset, uint64_t length) + const { + pImpl->checkFileReadParameters(offset, length); +} + +} // namespace facebook::velox diff --git a/velox/connectors/lakehouse/storage_adapters/hdfs/HdfsReadFile.h b/velox/connectors/lakehouse/storage_adapters/hdfs/HdfsReadFile.h new file mode 100644 index 000000000000..ddd35e511a71 --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/hdfs/HdfsReadFile.h @@ -0,0 +1,67 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "velox/common/file/File.h" +#include "velox/external/hdfs/hdfs.h" + +namespace facebook::velox { + +namespace filesystems::arrow::io::internal { +class LibHdfsShim; +} + +/** + * Implementation of hdfs read file. + */ +class HdfsReadFile final : public ReadFile { + public: + explicit HdfsReadFile( + filesystems::arrow::io::internal::LibHdfsShim* driver, + hdfsFS hdfs, + std::string_view path); + ~HdfsReadFile() override; + + std::string_view pread( + uint64_t offset, + uint64_t length, + void* buf, + filesystems::File::IoStats* stats = nullptr) const final; + + std::string pread( + uint64_t offset, + uint64_t length, + filesystems::File::IoStats* stats = nullptr) const final; + + uint64_t size() const final; + + uint64_t memoryUsage() const final; + + bool shouldCoalesce() const final; + + std::string getName() const final; + + uint64_t getNaturalReadSize() const final { + return 72 << 20; + } + + private: + void checkFileReadParameters(uint64_t offset, uint64_t length) const; + + class Impl; + std::unique_ptr pImpl; +}; + +} // namespace facebook::velox diff --git a/velox/connectors/lakehouse/storage_adapters/hdfs/HdfsUtil.h b/velox/connectors/lakehouse/storage_adapters/hdfs/HdfsUtil.h new file mode 100644 index 000000000000..ac07d929ae48 --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/hdfs/HdfsUtil.h @@ -0,0 +1,34 @@ +/* + * 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::filesystems { +inline std::string getHdfsPath( + const std::string& filePath, + const std::string_view& kScheme) { + auto endOfAuthority = filePath.find('/', kScheme.size()); + std::string hdfsAuthority{ + filePath, kScheme.size(), endOfAuthority - kScheme.size()}; + if (hdfsAuthority.empty()) { + return std::string(filePath.substr(kScheme.size())); + } + + return std::string(filePath.substr(endOfAuthority)); +} +} // namespace facebook::velox::filesystems diff --git a/velox/connectors/lakehouse/storage_adapters/hdfs/HdfsWriteFile.cpp b/velox/connectors/lakehouse/storage_adapters/hdfs/HdfsWriteFile.cpp new file mode 100644 index 000000000000..be668a3133e1 --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/hdfs/HdfsWriteFile.cpp @@ -0,0 +1,100 @@ +/* + * 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/storage_adapters/hdfs/HdfsWriteFile.h" +#include "velox/external/hdfs/ArrowHdfsInternal.h" + +namespace facebook::velox { +HdfsWriteFile::HdfsWriteFile( + filesystems::arrow::io::internal::LibHdfsShim* driver, + hdfsFS hdfsClient, + std::string_view path, + int bufferSize, + short replication, + int blockSize) + : driver_(driver), hdfsClient_(hdfsClient), filePath_(path) { + auto pos = filePath_.rfind("/"); + auto parentDir = filePath_.substr(0, pos + 1); + if (driver_->Exists(hdfsClient_, parentDir.c_str()) == -1) { + driver_->MakeDirectory(hdfsClient_, parentDir.c_str()); + } + + hdfsFile_ = driver_->OpenFile( + hdfsClient_, + filePath_.c_str(), + O_WRONLY, + bufferSize, + replication, + blockSize); + VELOX_CHECK_NOT_NULL( + hdfsFile_, + "Failed to open hdfs file: {}, with error: {}", + filePath_, + driver_->GetLastExceptionRootCause()); +} + +HdfsWriteFile::~HdfsWriteFile() { + if (hdfsFile_) { + close(); + } +} + +void HdfsWriteFile::close() { + int success = driver_->CloseFile(hdfsClient_, hdfsFile_); + VELOX_CHECK_EQ( + success, + 0, + "Failed to close hdfs file: {}", + driver_->GetLastExceptionRootCause()); + hdfsFile_ = nullptr; +} + +void HdfsWriteFile::flush() { + VELOX_CHECK_NOT_NULL( + hdfsFile_, + "Cannot flush HDFS file because file handle is null, file path: {}", + filePath_); + int success = driver_->Flush(hdfsClient_, hdfsFile_); + VELOX_CHECK_EQ( + success, 0, "Hdfs flush error: {}", driver_->GetLastExceptionRootCause()); +} + +void HdfsWriteFile::append(std::string_view data) { + if (data.size() == 0) { + return; + } + VELOX_CHECK_NOT_NULL( + hdfsFile_, + "Cannot append to HDFS file because file handle is null, file path: {}", + filePath_); + int64_t totalWrittenBytes = driver_->Write( + hdfsClient_, hdfsFile_, std::string(data).c_str(), data.size()); + VELOX_CHECK_EQ( + totalWrittenBytes, + data.size(), + "Write failure in HDFSWriteFile::append {}", + driver_->GetLastExceptionRootCause()); +} + +uint64_t HdfsWriteFile::size() const { + auto fileInfo = driver_->GetPathInfo(hdfsClient_, filePath_.c_str()); + uint64_t size = fileInfo->mSize; + // should call hdfsFreeFileInfo to avoid memory leak + driver_->FreeFileInfo(fileInfo, 1); + return size; +} + +} // namespace facebook::velox diff --git a/velox/connectors/lakehouse/storage_adapters/hdfs/HdfsWriteFile.h b/velox/connectors/lakehouse/storage_adapters/hdfs/HdfsWriteFile.h new file mode 100644 index 000000000000..fb311b1a6c3d --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/hdfs/HdfsWriteFile.h @@ -0,0 +1,71 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#pragma once + +#include "velox/common/file/File.h" +#include "velox/external/hdfs/hdfs.h" + +namespace facebook::velox { + +namespace filesystems::arrow::io::internal { +class LibHdfsShim; +} + +/// Implementation of hdfs write file. Nothing written to the file should be +/// read back until it is closed. +class HdfsWriteFile : public WriteFile { + public: + /// The constructor. + /// @param hdfsClient The configured hdfs filesystem handle. + /// @param path The file path to write. + /// @param bufferSize Size of buffer for write - pass 0 if you want + /// to use the default configured values. + /// @param replication Block replication - pass 0 if you want to use + /// the default configured values. + /// @param blockSize Size of block - pass 0 if you want to use the + /// default configured values. + HdfsWriteFile( + filesystems::arrow::io::internal::LibHdfsShim* driver, + hdfsFS hdfsClient, + std::string_view path, + int bufferSize = 0, + short replication = 0, + int blockSize = 0); + + ~HdfsWriteFile() override; + + /// Get the file size. + uint64_t size() const override; + + /// Flush the data. + void flush() override; + + /// Write the data by append mode. + void append(std::string_view data) override; + + /// Close the file. + void close() override; + + private: + filesystems::arrow::io::internal::LibHdfsShim* driver_; + /// The configured hdfs filesystem handle. + hdfsFS hdfsClient_; + /// The hdfs file handle for write. + hdfsFile hdfsFile_; + /// The hdfs file path. + const std::string filePath_; +}; +} // namespace facebook::velox diff --git a/velox/connectors/lakehouse/storage_adapters/hdfs/RegisterHdfsFileSystem.cpp b/velox/connectors/lakehouse/storage_adapters/hdfs/RegisterHdfsFileSystem.cpp new file mode 100644 index 000000000000..1f23179f0a72 --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/hdfs/RegisterHdfsFileSystem.cpp @@ -0,0 +1,105 @@ +/* + * 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. + */ + +#ifdef VELOX_ENABLE_HDFS +#include "folly/concurrency/ConcurrentHashMap.h" + +#include "velox/common/config/Config.h" +#include "velox/connectors/hive/storage_adapters/hdfs/HdfsFileSystem.h" // @manual +#include "velox/connectors/hive/storage_adapters/hdfs/HdfsUtil.h" // @manual +#include "velox/dwio/common/FileSink.h" +#endif + +namespace facebook::velox::filesystems { + +#ifdef VELOX_ENABLE_HDFS +std::mutex mtx; + +std::function(std::shared_ptr, std::string_view)> +hdfsFileSystemGenerator() { + static auto filesystemGenerator = [](std::shared_ptr + properties, + std::string_view filePath) { + static folly::ConcurrentHashMap> + filesystems; + static folly:: + ConcurrentHashMap> + hdfsInitiationFlags; + HdfsServiceEndpoint endpoint = + HdfsFileSystem::getServiceEndpoint(filePath, properties.get()); + std::string hdfsIdentity = endpoint.identity(); + if (filesystems.find(hdfsIdentity) != filesystems.end()) { + return filesystems[hdfsIdentity]; + } + std::unique_lock lk(mtx, std::defer_lock); + /// If the init flag for a given hdfs identity is not found, + /// create one for init use. It's a singleton. + if (hdfsInitiationFlags.find(hdfsIdentity) == hdfsInitiationFlags.end()) { + lk.lock(); + if (hdfsInitiationFlags.find(hdfsIdentity) == hdfsInitiationFlags.end()) { + std::shared_ptr initiationFlagPtr = + std::make_shared(); + hdfsInitiationFlags.insert(hdfsIdentity, initiationFlagPtr); + } + lk.unlock(); + } + folly::call_once( + *hdfsInitiationFlags[hdfsIdentity].get(), + [&properties, endpoint, hdfsIdentity]() { + auto filesystem = + std::make_shared(properties, endpoint); + filesystems.insert(hdfsIdentity, filesystem); + }); + return filesystems[hdfsIdentity]; + }; + return filesystemGenerator; +} + +std::function( + const std::string&, + const velox::dwio::common::FileSink::Options& options)> +hdfsWriteFileSinkGenerator() { + static auto hdfsWriteFileSink = + [](const std::string& fileURI, + const velox::dwio::common::FileSink::Options& options) { + if (HdfsFileSystem::isHdfsFile(fileURI)) { + std::string pathSuffix = + getHdfsPath(fileURI, HdfsFileSystem::kScheme); + auto fileSystem = + filesystems::getFileSystem(fileURI, options.connectorProperties); + return std::make_unique( + fileSystem->openFileForWrite(pathSuffix), + fileURI, + options.metricLogger, + options.stats); + } + return static_cast>( + nullptr); + }; + + return hdfsWriteFileSink; +} +#endif + +void registerHdfsFileSystem() { +#ifdef VELOX_ENABLE_HDFS + registerFileSystem(HdfsFileSystem::isHdfsFile, hdfsFileSystemGenerator()); + dwio::common::FileSink::registerFactory(hdfsWriteFileSinkGenerator()); +#endif +} + +} // namespace facebook::velox::filesystems diff --git a/velox/connectors/lakehouse/storage_adapters/hdfs/RegisterHdfsFileSystem.h b/velox/connectors/lakehouse/storage_adapters/hdfs/RegisterHdfsFileSystem.h new file mode 100644 index 000000000000..6f6f0c032bd7 --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/hdfs/RegisterHdfsFileSystem.h @@ -0,0 +1,24 @@ +/* + * 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 + +namespace facebook::velox::filesystems { + +// Register the HDFS. +void registerHdfsFileSystem(); + +} // namespace facebook::velox::filesystems diff --git a/velox/connectors/lakehouse/storage_adapters/hdfs/tests/CMakeLists.txt b/velox/connectors/lakehouse/storage_adapters/hdfs/tests/CMakeLists.txt new file mode 100644 index 000000000000..3e3b8b2d5a24 --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/hdfs/tests/CMakeLists.txt @@ -0,0 +1,58 @@ +# Copyright (c) Facebook, Inc. and its affiliates. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +add_executable(velox_hdfs_file_test HdfsFileSystemTest.cpp HdfsMiniCluster.cpp + HdfsUtilTest.cpp) + +add_test(velox_hdfs_file_test velox_hdfs_file_test) +target_link_libraries( + velox_hdfs_file_test + velox_file + velox_hdfs + velox_core + velox_exec_test_lib + velox_hive_connector + velox_dwio_common_exception + velox_exec + GTest::gtest + GTest::gtest_main + GTest::gmock) + +target_compile_options(velox_hdfs_file_test + PRIVATE -Wno-deprecated-declarations) + +add_executable(velox_hdfs_insert_test HdfsInsertTest.cpp HdfsMiniCluster.cpp + HdfsUtilTest.cpp) + +add_test(velox_hdfs_insert_test velox_hdfs_insert_test) + +target_link_libraries( + velox_hdfs_insert_test + velox_exec_test_lib + velox_exec + GTest::gtest + GTest::gtest_main + GTest::gmock) + +target_compile_options(velox_hdfs_insert_test + PRIVATE -Wno-deprecated-declarations) + +# velox_hdfs_insert_test and velox_hdfs_file_test two tests can't run in +# parallel due to the port conflict in Hadoop NameNode and DataNode. The +# namenode port conflict can be resolved using the -nnport configuration in +# hadoop-mapreduce-client-jobclient-3.3.0-tests.jar. However the data node port +# cannot be configured. Therefore, we need to make sure that +# velox_hdfs_file_test runs only after velox_hdfs_insert_test has finished. +set_tests_properties(velox_hdfs_insert_test PROPERTIES DEPENDS + velox_hdfs_file_test) diff --git a/velox/connectors/lakehouse/storage_adapters/hdfs/tests/HdfsFileSystemTest.cpp b/velox/connectors/lakehouse/storage_adapters/hdfs/tests/HdfsFileSystemTest.cpp new file mode 100644 index 000000000000..e5c0883284b9 --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/hdfs/tests/HdfsFileSystemTest.cpp @@ -0,0 +1,522 @@ +/* + * 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/storage_adapters/hdfs/HdfsFileSystem.h" +#include +#include +#include +#include +#include "gtest/gtest.h" +#include "velox/common/base/Exceptions.h" +#include "velox/common/base/tests/GTestUtils.h" +#include "velox/connectors/hive/storage_adapters/hdfs/HdfsReadFile.h" +#include "velox/connectors/hive/storage_adapters/hdfs/RegisterHdfsFileSystem.h" +#include "velox/connectors/hive/storage_adapters/hdfs/tests/HdfsMiniCluster.h" +#include "velox/core/QueryConfig.h" +#include "velox/exec/tests/utils/TempFilePath.h" +#include "velox/external/hdfs/ArrowHdfsInternal.h" + +#include + +using namespace facebook::velox; + +using filesystems::arrow::io::internal::LibHdfsShim; + +constexpr int kOneMB = 1 << 20; +static const std::string kDestinationPath = "/test_file.txt"; +static const std::string kRenamePath = "/rename_file.txt"; +static const std::string kRenameNewPath = "/rename_new_file.txt"; +static const std::string kDeletedPath = "/delete_file.txt"; +static const std::string kSimpleDestinationPath = "hdfs://" + kDestinationPath; +static const std::string kViewfsDestinationPath = + "viewfs://" + kDestinationPath; +std::unordered_map configurationValues; + +class HdfsFileSystemTest : public testing::Test { + public: + static void SetUpTestSuite() { + filesystems::registerHdfsFileSystem(); + if (miniCluster == nullptr) { + miniCluster = std::make_shared(); + miniCluster->start(); + auto tempFile = createFile(); + miniCluster->addFile(tempFile->getPath(), kDestinationPath); + miniCluster->addFile(tempFile->getPath(), kRenamePath); + miniCluster->addFile(tempFile->getPath(), kDeletedPath); + } + configurationValues.insert( + {"hive.hdfs.host", std::string(miniCluster->host())}); + configurationValues.insert( + {"hive.hdfs.port", std::string(miniCluster->nameNodePort())}); + fullDestinationPath_ = + fmt::format("{}{}", miniCluster->url(), kDestinationPath); + } + + void SetUp() override { + if (!miniCluster->isRunning()) { + miniCluster->start(); + } + filesystems::registerHdfsFileSystem(); + } + + static void TearDownTestSuite() { + miniCluster->stop(); + } + + static std::unique_ptr openFileForWrite(std::string_view path) { + auto config = std::make_shared( + std::unordered_map(configurationValues)); + auto hdfsFilePath = fmt::format("{}{}", miniCluster->url(), path); + auto hdfsFileSystem = filesystems::getFileSystem(hdfsFilePath, config); + return hdfsFileSystem->openFileForWrite(path); + } + + static std::atomic startThreads; + static std::shared_ptr miniCluster; + static std::string fullDestinationPath_; + + private: + static std::shared_ptr<::exec::test::TempFilePath> createFile() { + auto tempFile = exec::test::TempFilePath::create(); + tempFile->append("aaaaa"); + tempFile->append("bbbbb"); + tempFile->append(std::string(kOneMB, 'c')); + tempFile->append("ddddd"); + return tempFile; + } +}; + +std::shared_ptr + HdfsFileSystemTest::miniCluster = nullptr; +std::atomic HdfsFileSystemTest::startThreads = false; +std::string HdfsFileSystemTest::fullDestinationPath_; + +void readData(ReadFile* readFile) { + ASSERT_EQ(readFile->size(), 15 + kOneMB); + char buffer1[5]; + ASSERT_EQ(readFile->pread(10 + kOneMB, 5, &buffer1), "ddddd"); + char buffer2[10]; + ASSERT_EQ(readFile->pread(0, 10, &buffer2), "aaaaabbbbb"); + auto buffer3 = new char[kOneMB]; + ASSERT_EQ(readFile->pread(10, kOneMB, buffer3), std::string(kOneMB, 'c')); + delete[] buffer3; + ASSERT_EQ(readFile->size(), 15 + kOneMB); + char buffer4[10]; + auto arf = readFile->pread(5, 10, &buffer4); + auto zarf = readFile->pread(kOneMB, 15); + auto buf = std::make_unique(8); + auto warf = readFile->pread(4, 8, buf.get()); + const std::string_view warfFromBuf(buf.get(), 8); + ASSERT_EQ(arf, "bbbbbccccc"); + ASSERT_EQ(zarf, "ccccccccccddddd"); + ASSERT_EQ(warf, "abbbbbcc"); + ASSERT_EQ(warfFromBuf, "abbbbbcc"); +} + +void checkReadErrorMessages( + ReadFile* readFile, + std::string errorMessage, + int endpoint) { + VELOX_ASSERT_THROW(readFile->pread(10 + kOneMB, endpoint), errorMessage); + + auto buf = std::make_unique(8); + VELOX_ASSERT_THROW( + readFile->pread(10 + kOneMB, endpoint, buf.get()), errorMessage); +} + +bool checkMiniClusterStop(ReadFile* readFile, const std::string& errorMessage) { + try { + readFile->pread(0, 1); + return false; + } catch (const VeloxException& error) { + return error.message().find(errorMessage) != std::string::npos; + } +} + +void verifyFailures(LibHdfsShim* driver, hdfsFS hdfs) { + HdfsReadFile readFile(driver, hdfs, kDestinationPath); + HdfsReadFile readFile2(driver, hdfs, kDestinationPath); + auto startPoint = 10 + kOneMB; + auto size = 15 + kOneMB; + auto endpoint = 10 + 2 * kOneMB; + auto offsetErrorMessage = + (boost::format( + "(%d vs. %d) Cannot read HDFS file beyond its size: %d, offset: %d, end point: %d") % + size % endpoint % size % startPoint % endpoint) + .str(); + + auto readFailErrorMessage = + (boost::format( + "Unable to open file %s. got error: ConnectException: Connection refused") % + kDestinationPath) + .str(); + + checkReadErrorMessages(&readFile, offsetErrorMessage, kOneMB); + HdfsFileSystemTest::miniCluster->stop(); + + constexpr auto kMaxRetries = 10; + int retries = 0; + while (true) { + if (checkMiniClusterStop(&readFile2, readFailErrorMessage)) { + checkReadErrorMessages(&readFile2, readFailErrorMessage, 1); + break; + } else { + if (retries >= kMaxRetries) { + FAIL() << "MiniCluster doesn't stop after kMaxRetries try"; + } else { + sleep(1); + retries++; + } + } + } +} + +hdfsFS connectHdfsDriver( + filesystems::arrow::io::internal::LibHdfsShim** driver, + const std::string host, + const std::string port) { + filesystems::arrow::io::internal::LibHdfsShim* libhdfs_shim; + auto status = filesystems::arrow::io::internal::ConnectLibHdfs(&libhdfs_shim); + VELOX_CHECK(status.ok(), "ConnectLibHdfs failed."); + + // Connect to HDFS with the builder object + hdfsBuilder* builder = libhdfs_shim->NewBuilder(); + libhdfs_shim->BuilderSetNameNode(builder, host.c_str()); + libhdfs_shim->BuilderSetNameNodePort(builder, std::stoi(port)); + libhdfs_shim->BuilderSetForceNewInstance(builder); + + auto hdfs = libhdfs_shim->BuilderConnect(builder); + VELOX_CHECK_NOT_NULL( + hdfs, + "Unable to connect to HDFS at {}:{}, got error", + host.c_str(), + port); + *driver = libhdfs_shim; + return hdfs; +} + +TEST_F(HdfsFileSystemTest, read) { + filesystems::arrow::io::internal::LibHdfsShim* driver; + auto hdfs = connectHdfsDriver( + &driver, + std::string(miniCluster->host()), + std::string(miniCluster->nameNodePort())); + HdfsReadFile readFile(driver, hdfs, kDestinationPath); + readData(&readFile); +} + +TEST_F(HdfsFileSystemTest, rename) { + auto config = std::make_shared( + std::unordered_map(configurationValues)); + auto hdfsFileSystem = + filesystems::getFileSystem(fullDestinationPath_, config); + + ASSERT_TRUE(hdfsFileSystem->exists(kRenamePath)); + hdfsFileSystem->rename(kRenamePath, kRenameNewPath); + ASSERT_FALSE(hdfsFileSystem->exists(kRenamePath)); + ASSERT_TRUE(hdfsFileSystem->exists(kRenameNewPath)); +} + +TEST_F(HdfsFileSystemTest, delete) { + auto config = std::make_shared( + std::unordered_map(configurationValues)); + auto hdfsFileSystem = + filesystems::getFileSystem(fullDestinationPath_, config); + + ASSERT_TRUE(hdfsFileSystem->exists(kDeletedPath)); + hdfsFileSystem->remove(kDeletedPath); + ASSERT_FALSE(hdfsFileSystem->exists(kDeletedPath)); +} + +TEST_F(HdfsFileSystemTest, viaFileSystem) { + auto config = std::make_shared( + std::unordered_map(configurationValues)); + auto hdfsFileSystem = + filesystems::getFileSystem(fullDestinationPath_, config); + auto readFile = hdfsFileSystem->openFileForRead(fullDestinationPath_); + readData(readFile.get()); +} + +TEST_F(HdfsFileSystemTest, exists) { + auto config = std::make_shared( + std::unordered_map(configurationValues)); + auto hdfsFileSystem = + filesystems::getFileSystem(fullDestinationPath_, config); + ASSERT_TRUE(hdfsFileSystem->exists(fullDestinationPath_)); + + const std::string_view notExistFilePath = + "hdfs://localhost:7777//path/that/does/not/exist"; + ASSERT_FALSE(hdfsFileSystem->exists(notExistFilePath)); +} + +TEST_F(HdfsFileSystemTest, mkdirAndRmdir) { + auto config = std::make_shared( + std::unordered_map(configurationValues)); + auto hdfsFileSystem = + filesystems::getFileSystem(fullDestinationPath_, config); + const std::string newDir = "/new_directory"; + ASSERT_FALSE(hdfsFileSystem->exists(newDir)); + hdfsFileSystem->mkdir(newDir); + ASSERT_TRUE(hdfsFileSystem->exists(newDir)); + hdfsFileSystem->rmdir(newDir); + ASSERT_FALSE(hdfsFileSystem->exists(newDir)); +} + +TEST_F(HdfsFileSystemTest, initializeFsWithEndpointInfoInFilePath) { + // Without host/port configured. + auto config = std::make_shared( + std::unordered_map()); + auto hdfsFileSystem = + filesystems::getFileSystem(fullDestinationPath_, config); + auto readFile = hdfsFileSystem->openFileForRead(fullDestinationPath_); + readData(readFile.get()); + + // Wrong endpoint info specified in hdfs file path. + const std::string wrongFullDestinationPath = + "hdfs://not_exist_host:" + std::string(miniCluster->nameNodePort()) + + kDestinationPath; + VELOX_ASSERT_THROW( + filesystems::getFileSystem(wrongFullDestinationPath, config), + "Unable to connect to HDFS"); +} + +TEST_F(HdfsFileSystemTest, fallbackToUseConfig) { + auto config = std::make_shared( + std::unordered_map(configurationValues)); + auto hdfsFileSystem = + filesystems::getFileSystem(fullDestinationPath_, config); + auto readFile = hdfsFileSystem->openFileForRead(fullDestinationPath_); + readData(readFile.get()); +} + +TEST_F(HdfsFileSystemTest, oneFsInstanceForOneEndpoint) { + auto hdfsFileSystem1 = + filesystems::getFileSystem(fullDestinationPath_, nullptr); + auto hdfsFileSystem2 = + filesystems::getFileSystem(fullDestinationPath_, nullptr); + ASSERT_TRUE(hdfsFileSystem1 == hdfsFileSystem2); +} + +TEST_F(HdfsFileSystemTest, missingFileViaFileSystem) { + auto config = std::make_shared( + std::unordered_map(configurationValues)); + auto hdfsFileSystem = + filesystems::getFileSystem(fullDestinationPath_, config); + + VELOX_ASSERT_RUNTIME_THROW_CODE( + hdfsFileSystem->openFileForRead( + "hdfs://localhost:7777/path/that/does/not/exist"), + error_code::kFileNotFound, + "Unable to get file path info for file: /path/that/does/not/exist. got error: FileNotFoundException: Path /path/that/does/not/exist does not exist."); +} + +TEST_F(HdfsFileSystemTest, missingHost) { + std::unordered_map missingHostConfiguration( + {{"hive.hdfs.port", std::string(miniCluster->nameNodePort())}}); + auto config = std::make_shared( + std::move(missingHostConfiguration)); + + VELOX_ASSERT_THROW( + filesystems::HdfsFileSystem::getServiceEndpoint( + kSimpleDestinationPath, config.get()), + "hdfsHost is empty, configuration missing for hdfs host"); +} + +TEST_F(HdfsFileSystemTest, missingPort) { + std::unordered_map missingPortConfiguration( + {{"hive.hdfs.host", std::string(miniCluster->host())}}); + auto config = std::make_shared( + std::move(missingPortConfiguration)); + + VELOX_ASSERT_THROW( + filesystems::HdfsFileSystem::getServiceEndpoint( + kSimpleDestinationPath, config.get()), + "hdfsPort is empty, configuration missing for hdfs port"); +} + +TEST_F(HdfsFileSystemTest, missingFileViaReadFile) { + filesystems::arrow::io::internal::LibHdfsShim* driver; + auto hdfs = connectHdfsDriver( + &driver, + std::string(miniCluster->host()), + std::string(miniCluster->nameNodePort())); + VELOX_ASSERT_THROW( + std::make_shared( + driver, hdfs, "/path/that/does/not/exist"), + "Unable to get file path info for file: /path/that/does/not/exist. got error: FileNotFoundException: Path /path/that/does/not/exist does not exist."); +} + +TEST_F(HdfsFileSystemTest, schemeMatching) { + VELOX_ASSERT_THROW( + std::dynamic_pointer_cast( + filesystems::getFileSystem("file://", nullptr)), + "No registered file system matched with file path 'file://'") + + auto fs = std::dynamic_pointer_cast( + filesystems::getFileSystem(fullDestinationPath_, nullptr)); + ASSERT_TRUE(fs->isHdfsFile(fullDestinationPath_)); + + fs = std::dynamic_pointer_cast( + filesystems::getFileSystem(kViewfsDestinationPath, nullptr)); + ASSERT_TRUE(fs->isHdfsFile(kViewfsDestinationPath)); +} + +TEST_F(HdfsFileSystemTest, writeSupported) { + auto config = std::make_shared( + std::unordered_map(configurationValues)); + auto hdfsFileSystem = + filesystems::getFileSystem(fullDestinationPath_, config); + hdfsFileSystem->openFileForWrite("/path"); +} + +TEST_F(HdfsFileSystemTest, multipleThreadsWithReadFile) { + startThreads = false; + + filesystems::arrow::io::internal::LibHdfsShim* driver; + auto hdfs = connectHdfsDriver( + &driver, + std::string(miniCluster->host()), + std::string(miniCluster->nameNodePort())); + std::vector threads; + std::mt19937 generator(std::random_device{}()); + std::vector sleepTimesInMicroseconds = {0, 500, 50000}; + std::uniform_int_distribution distribution( + 0, sleepTimesInMicroseconds.size() - 1); + for (int i = 0; i < 25; i++) { + auto thread = std::thread( + [&driver, &hdfs, &distribution, &generator, &sleepTimesInMicroseconds] { + int index = distribution(generator); + while (!HdfsFileSystemTest::startThreads) { + std::this_thread::yield(); + } + std::this_thread::sleep_for( + std::chrono::microseconds(sleepTimesInMicroseconds[index])); + HdfsReadFile readFile(driver, hdfs, kDestinationPath); + readData(&readFile); + }); + threads.emplace_back(std::move(thread)); + } + startThreads = true; + for (auto& thread : threads) { + thread.join(); + } +} + +TEST_F(HdfsFileSystemTest, multipleThreadsWithFileSystem) { + startThreads = false; + auto config = std::make_shared( + std::unordered_map(configurationValues)); + auto hdfsFileSystem = + filesystems::getFileSystem(fullDestinationPath_, config); + + std::vector threads; + std::mt19937 generator(std::random_device{}()); + std::vector sleepTimesInMicroseconds = {0, 500, 50000}; + std::uniform_int_distribution distribution( + 0, sleepTimesInMicroseconds.size() - 1); + for (int i = 0; i < 25; i++) { + auto thread = std::thread([&hdfsFileSystem, + &distribution, + &generator, + &sleepTimesInMicroseconds] { + int index = distribution(generator); + while (!HdfsFileSystemTest::startThreads) { + std::this_thread::yield(); + } + std::this_thread::sleep_for( + std::chrono::microseconds(sleepTimesInMicroseconds[index])); + auto readFile = hdfsFileSystem->openFileForRead(fullDestinationPath_); + readData(readFile.get()); + }); + threads.emplace_back(std::move(thread)); + } + startThreads = true; + for (auto& thread : threads) { + thread.join(); + } +} + +TEST_F(HdfsFileSystemTest, write) { + const std::string_view path = "/a.txt"; + auto writeFile = openFileForWrite(path); + const std::string_view data = "abcdefghijk"; + writeFile->append(data); + writeFile->flush(); + ASSERT_EQ(writeFile->size(), 0); + writeFile->append(data); + writeFile->append(data); + writeFile->flush(); + writeFile->close(); + ASSERT_EQ(writeFile->size(), data.size() * 3); +} + +TEST_F(HdfsFileSystemTest, missingFileForWrite) { + const std::string_view filePath = + "hdfs://localhost:7777/path/that/does/not/exist"; + const std::string_view errorMsg = + "Failed to open hdfs file: hdfs://localhost:7777/path/that/does/not/exist"; + VELOX_ASSERT_THROW(openFileForWrite(filePath), errorMsg); +} + +TEST_F(HdfsFileSystemTest, writeDataFailures) { + auto writeFile = openFileForWrite("/a.txt"); + writeFile->close(); + VELOX_ASSERT_THROW( + writeFile->append("abcde"), + "Cannot append to HDFS file because file handle is null, file path: /a.txt"); +} + +TEST_F(HdfsFileSystemTest, writeFlushFailures) { + auto writeFile = openFileForWrite("/a.txt"); + writeFile->close(); + VELOX_ASSERT_THROW( + writeFile->flush(), + "Cannot flush HDFS file because file handle is null, file path: /a.txt"); +} + +TEST_F(HdfsFileSystemTest, writeWithParentDirNotExist) { + const std::string_view path = "/parent/directory/that/does/not/exist/a.txt"; + auto writeFile = openFileForWrite(path); + const std::string_view data = "abcdefghijk"; + writeFile->append(data); + writeFile->flush(); + ASSERT_EQ(writeFile->size(), 0); + writeFile->append(data); + writeFile->append(data); + writeFile->flush(); + writeFile->close(); + ASSERT_EQ(writeFile->size(), data.size() * 3); +} + +TEST_F(HdfsFileSystemTest, list) { + auto config = std::make_shared( + std::unordered_map(configurationValues)); + auto hdfsFileSystem = + filesystems::getFileSystem(fullDestinationPath_, config); + + auto result = hdfsFileSystem->list(fullDestinationPath_); + + ASSERT_EQ(result.size(), 1); + ASSERT_TRUE(result[0].find(kDestinationPath) != std::string::npos); +} + +TEST_F(HdfsFileSystemTest, readFailures) { + filesystems::arrow::io::internal::LibHdfsShim* driver; + auto hdfs = connectHdfsDriver( + &driver, + std::string(miniCluster->host()), + std::string(miniCluster->nameNodePort())); + verifyFailures(driver, hdfs); +} diff --git a/velox/connectors/lakehouse/storage_adapters/hdfs/tests/HdfsInsertTest.cpp b/velox/connectors/lakehouse/storage_adapters/hdfs/tests/HdfsInsertTest.cpp new file mode 100644 index 000000000000..9ec9a1254154 --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/hdfs/tests/HdfsInsertTest.cpp @@ -0,0 +1,70 @@ +/* + * 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 "gtest/gtest.h" + +#include "velox/common/memory/Memory.h" +#include "velox/connectors/hive/storage_adapters/hdfs/RegisterHdfsFileSystem.h" +#include "velox/connectors/hive/storage_adapters/hdfs/tests/HdfsMiniCluster.h" +#include "velox/connectors/hive/storage_adapters/test_common/InsertTest.h" + +using namespace facebook::velox; +using namespace facebook::velox::exec::test; +using namespace facebook::velox::connector; +using namespace facebook::velox::test; + +class HdfsInsertTest : public testing::Test, public InsertTest { + public: + static void SetUpTestSuite() { + memory::MemoryManager::testingSetInstance(memory::MemoryManager::Options{}); + } + + void SetUp() override { + filesystems::registerHdfsFileSystem(); + if (miniCluster == nullptr) { + miniCluster = std::make_shared(); + miniCluster->start(); + } + ioExecutor_ = std::make_unique(3); + + InsertTest::SetUp( + std::make_shared( + std::unordered_map()), + ioExecutor_.get()); + } + + void TearDown() override { + InsertTest::TearDown(); + miniCluster->stop(); + } + + static std::shared_ptr miniCluster; + std::unique_ptr ioExecutor_; +}; + +std::shared_ptr + HdfsInsertTest::miniCluster = nullptr; + +TEST_F(HdfsInsertTest, hdfsInsertTest) { + const int64_t kExpectedRows = 1'000; + runInsertTest(fmt::format("{}/", miniCluster->url()), kExpectedRows, pool()); +} + +int main(int argc, char** argv) { + testing::InitGoogleTest(&argc, argv); + folly::Init init{&argc, &argv, false}; + return RUN_ALL_TESTS(); +} diff --git a/velox/connectors/lakehouse/storage_adapters/hdfs/tests/HdfsMiniCluster.cpp b/velox/connectors/lakehouse/storage_adapters/hdfs/tests/HdfsMiniCluster.cpp new file mode 100644 index 000000000000..07b78dbea9be --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/hdfs/tests/HdfsMiniCluster.cpp @@ -0,0 +1,123 @@ +/* + * 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/storage_adapters/hdfs/tests/HdfsMiniCluster.h" + +#include "velox/exec/tests/utils/PortUtil.h" + +namespace facebook::velox::filesystems::test { +void HdfsMiniCluster::start() { + try { + serverProcess_ = std::make_unique( + env_, + exePath_, + kJarCommand, + env_["HADOOP_HOME"].to_string() + kMiniclusterJar, + kMiniclusterCommand, + kNoMapReduceOption, + kFormatNameNodeOption, + kHttpPortOption, + httpPort_, + kNameNodePortOption, + nameNodePort_, + kConfigurationOption, + kTurnOffPermissions); + serverProcess_->wait_for(std::chrono::duration(60000)); + VELOX_CHECK_EQ( + serverProcess_->exit_code(), + 383, + "Minicluster process exited, code: {}", + serverProcess_->exit_code()); + } catch (const std::exception& e) { + VELOX_FAIL("Failed to launch Minicluster server: {}", e.what()); + } +} + +void HdfsMiniCluster::stop() { + if (serverProcess_ && serverProcess_->valid()) { + serverProcess_->terminate(); + serverProcess_->wait(); + serverProcess_.reset(); + } +} + +bool HdfsMiniCluster::isRunning() { + if (serverProcess_) { + return true; + } + return false; +} + +// requires hadoop executable to be on the PATH +HdfsMiniCluster::HdfsMiniCluster() { + env_ = (boost::process::environment)boost::this_process::environment(); + env_["PATH"] = env_["PATH"].to_string() + kHadoopSearchPath; + auto path = env_["PATH"].to_vector(); + exePath_ = boost::process::search_path( + kMiniClusterExecutableName, + std::vector(path.begin(), path.end())); + if (exePath_.empty()) { + VELOX_FAIL( + "Failed to find minicluster executable {}'", + kMiniClusterExecutableName); + } + constexpr auto kHostAddressTemplate = "hdfs://{}:{}"; + auto ports = facebook::velox::exec::test::getFreePorts(2); + nameNodePort_ = fmt::format("{}", ports[0]); + httpPort_ = fmt::format("{}", ports[1]); + filesystemUrl_ = fmt::format(kHostAddressTemplate, host(), nameNodePort_); + boost::filesystem::path hadoopHomeDirectory = exePath_; + hadoopHomeDirectory.remove_leaf().remove_leaf(); + setupEnvironment(hadoopHomeDirectory.string()); +} + +void HdfsMiniCluster::addFile(std::string source, std::string destination) { + auto filePutProcess = std::make_shared( + env_, + exePath_, + kFilesystemCommand, + kFilesystemUrlOption, + filesystemUrl_, + kFilePutOption, + source, + destination); + bool isExited = + filePutProcess->wait_for(std::chrono::duration(15000)); + if (!isExited) { + VELOX_FAIL( + "Failed to add file to hdfs, exit code: {}", + filePutProcess->exit_code()); + } +} + +HdfsMiniCluster::~HdfsMiniCluster() { + stop(); +} + +void HdfsMiniCluster::setupEnvironment(const std::string& homeDirectory) { + env_["HADOOP_HOME"] = homeDirectory; + env_["HADOOP_INSTALL"] = homeDirectory; + env_["HADOOP_MAPRED_HOME"] = homeDirectory; + env_["HADOOP_COMMON_HOME"] = homeDirectory; + env_["HADOOP_HDFS_HOME"] = homeDirectory; + env_["YARN_HOME"] = homeDirectory; + env_["HADOOP_COMMON_LIB_NATIVE_DIR"] = homeDirectory + "/lib/native"; + env_["HADOOP_CONF_DIR"] = homeDirectory; + env_["HADOOP_PREFIX"] = homeDirectory; + env_["HADOOP_LIBEXEC_DIR"] = homeDirectory + "/libexec"; + env_["HADOOP_CONF_DIR"] = homeDirectory + "/etc/hadoop"; +} +} // namespace facebook::velox::filesystems::test diff --git a/velox/connectors/lakehouse/storage_adapters/hdfs/tests/HdfsMiniCluster.h b/velox/connectors/lakehouse/storage_adapters/hdfs/tests/HdfsMiniCluster.h new file mode 100644 index 000000000000..c54ae9589b3e --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/hdfs/tests/HdfsMiniCluster.h @@ -0,0 +1,77 @@ +/* + * 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/exec/tests/utils/TempDirectoryPath.h" + +#include +#include +#include +#include "boost/process.hpp" + +namespace facebook::velox::filesystems::test { +static const std::string kMiniClusterExecutableName{"hadoop"}; +static const std::string kHadoopSearchPath{":/usr/local/hadoop/bin"}; +static const std::string kJarCommand{"jar"}; +static const std::string kMiniclusterJar{ + "/share/hadoop/mapreduce/hadoop-mapreduce-client-jobclient-3.3.0-tests.jar"}; +static const std::string kMiniclusterCommand{"minicluster"}; +static const std::string kNoMapReduceOption{"-nomr"}; +static const std::string kFormatNameNodeOption{"-format"}; +static const std::string kHttpPortOption{"-nnhttpport"}; +static const std::string kNameNodePortOption{"-nnport"}; +static const std::string kConfigurationOption{"-D"}; +static const std::string kTurnOffPermissions{"dfs.permissions=false"}; +static const std::string kFilesystemCommand{"fs"}; +static const std::string kFilesystemUrlOption{"-fs"}; +static const std::string kFilePutOption{"-put"}; + +class HdfsMiniCluster { + public: + HdfsMiniCluster(); + + void start(); + + void stop(); + + bool isRunning(); + + void addFile(std::string source, std::string destination); + virtual ~HdfsMiniCluster(); + + std::string_view nameNodePort() const { + return nameNodePort_; + } + + std::string_view url() const { + return filesystemUrl_; + } + + std::string_view host() const { + static const std::string_view kLocalhost = "localhost"; + return kLocalhost; + } + + private: + void setupEnvironment(const std::string& homeDirectory); + + std::unique_ptr<::boost::process::child> serverProcess_; + boost::filesystem::path exePath_; + boost::process::environment env_; + std::string nameNodePort_; + std::string httpPort_; + std::string filesystemUrl_; +}; +} // namespace facebook::velox::filesystems::test diff --git a/velox/connectors/lakehouse/storage_adapters/hdfs/tests/HdfsUtilTest.cpp b/velox/connectors/lakehouse/storage_adapters/hdfs/tests/HdfsUtilTest.cpp new file mode 100644 index 000000000000..da4dd2b33148 --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/hdfs/tests/HdfsUtilTest.cpp @@ -0,0 +1,35 @@ +/* + * 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/storage_adapters/hdfs/HdfsUtil.h" + +#include "gtest/gtest.h" + +using namespace facebook::velox::filesystems; + +TEST(HdfsUtilTest, getHdfsPath) { + const std::string& kScheme = "hdfs://"; + std::string path1 = + getHdfsPath("hdfs://hdfsCluster/user/hive/a.txt", kScheme); + EXPECT_EQ("/user/hive/a.txt", path1); + + std::string path2 = + getHdfsPath("hdfs://localhost:9000/user/hive/a.txt", kScheme); + EXPECT_EQ("/user/hive/a.txt", path2); + + std::string path3 = getHdfsPath("hdfs:///user/hive/a.txt", kScheme); + EXPECT_EQ("/user/hive/a.txt", path3); +} diff --git a/velox/connectors/lakehouse/storage_adapters/s3fs/CMakeLists.txt b/velox/connectors/lakehouse/storage_adapters/s3fs/CMakeLists.txt new file mode 100644 index 000000000000..b6adaed08389 --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/s3fs/CMakeLists.txt @@ -0,0 +1,35 @@ +# 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. + +# for generated headers + +velox_add_library(velox_s3fs RegisterS3FileSystem.cpp) +if(VELOX_ENABLE_S3) + velox_sources( + velox_s3fs + PRIVATE + S3FileSystem.cpp + S3Util.cpp + S3Config.cpp + S3WriteFile.cpp + S3ReadFile.cpp) + + velox_include_directories(velox_s3fs PRIVATE ${AWSSDK_INCLUDE_DIRS}) + velox_link_libraries(velox_s3fs PRIVATE velox_dwio_common Folly::folly + ${AWSSDK_LIBRARIES}) + + if(${VELOX_BUILD_TESTING}) + add_subdirectory(tests) + endif() +endif() diff --git a/velox/connectors/lakehouse/storage_adapters/s3fs/RegisterS3FileSystem.cpp b/velox/connectors/lakehouse/storage_adapters/s3fs/RegisterS3FileSystem.cpp new file mode 100644 index 000000000000..62d014090a47 --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/s3fs/RegisterS3FileSystem.cpp @@ -0,0 +1,162 @@ +/* + * 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/storage_adapters/s3fs/RegisterS3FileSystem.h" // @manual + +#ifdef VELOX_ENABLE_S3 +#include "velox/common/base/StatsReporter.h" +#include "velox/connectors/hive/storage_adapters/s3fs/S3Config.h" // @manual +#include "velox/connectors/hive/storage_adapters/s3fs/S3Counters.h" // @manual +#include "velox/connectors/hive/storage_adapters/s3fs/S3FileSystem.h" // @manual +#include "velox/connectors/hive/storage_adapters/s3fs/S3Util.h" // @manual +#include "velox/dwio/common/FileSink.h" +#endif + +namespace facebook::velox::filesystems { + +#ifdef VELOX_ENABLE_S3 +using FileSystemMap = folly::Synchronized< + std::unordered_map>>; + +/// Multiple S3 filesystems are supported. +/// Key is the endpoint value specified in the config using hive.s3.endpoint. +/// If the endpoint is empty, it will default to AWS S3 Library. +/// Different S3 buckets can be accessed with different client configurations. +/// This allows for different endpoints, data read and write strategies. +/// The bucket specific option is set by replacing the hive.s3. prefix on an +/// option with hive.s3.bucket.BUCKETNAME., where BUCKETNAME is the name of the +/// bucket. When connecting to a bucket, all options explicitly set will +/// override the base hive.s3. values. + +FileSystemMap& fileSystems() { + static FileSystemMap instances; + return instances; +} + +CacheKeyFn cacheKeyFunc; + +std::shared_ptr fileSystemGenerator( + std::shared_ptr properties, + std::string_view s3Path) { + std::string cacheKey, bucketName, key; + getBucketAndKeyFromPath(getPath(s3Path), bucketName, key); + if (!cacheKeyFunc) { + cacheKey = S3Config::cacheKey(bucketName, properties); + } else { + cacheKey = cacheKeyFunc(properties, s3Path); + } + + // Check if an instance exists with a read lock (shared). + auto fs = fileSystems().withRLock( + [&](auto& instanceMap) -> std::shared_ptr { + auto iterator = instanceMap.find(cacheKey); + if (iterator != instanceMap.end()) { + return iterator->second; + } + return nullptr; + }); + if (fs != nullptr) { + return fs; + } + + return fileSystems().withWLock( + [&](auto& instanceMap) -> std::shared_ptr { + // Repeat the checks with a write lock. + auto iterator = instanceMap.find(cacheKey); + if (iterator != instanceMap.end()) { + return iterator->second; + } + + auto logLevel = + properties->get(S3Config::kS3LogLevel, std::string("FATAL")); + std::optional logLocation = + static_cast>( + properties->get(S3Config::kS3LogLocation)); + initializeS3(logLevel, logLocation); + auto fs = std::make_shared(bucketName, properties); + instanceMap.insert({cacheKey, fs}); + return fs; + }); +} + +std::unique_ptr s3WriteFileSinkGenerator( + const std::string& fileURI, + const velox::dwio::common::FileSink::Options& options) { + if (isS3File(fileURI)) { + auto fileSystem = + filesystems::getFileSystem(fileURI, options.connectorProperties); + return std::make_unique( + fileSystem->openFileForWrite(fileURI, {{}, options.pool, std::nullopt}), + fileURI, + options.metricLogger, + options.stats); + } + return nullptr; +} +#endif + +void registerS3FileSystem(CacheKeyFn identityFunction) { +#ifdef VELOX_ENABLE_S3 + fileSystems().withWLock([&](auto& instanceMap) { + if (instanceMap.empty()) { + cacheKeyFunc = identityFunction; + registerFileSystem(isS3File, std::function(fileSystemGenerator)); + dwio::common::FileSink::registerFactory( + std::function(s3WriteFileSinkGenerator)); + } + }); +#endif +} + +void finalizeS3FileSystem() { +#ifdef VELOX_ENABLE_S3 + bool singleUseCount = true; + fileSystems().withWLock([&](auto& instanceMap) { + for (const auto& [id, fs] : instanceMap) { + singleUseCount &= (fs.use_count() == 1); + } + VELOX_CHECK(singleUseCount, "Cannot finalize S3FileSystem while in use"); + instanceMap.clear(); + }); + + finalizeS3(); +#endif +} + +void registerS3Metrics() { +#ifdef VELOX_ENABLE_S3 + DEFINE_METRIC(kMetricS3ActiveConnections, velox::StatType::SUM); + DEFINE_METRIC(kMetricS3StartedUploads, velox::StatType::COUNT); + DEFINE_METRIC(kMetricS3FailedUploads, velox::StatType::COUNT); + DEFINE_METRIC(kMetricS3SuccessfulUploads, velox::StatType::COUNT); + DEFINE_METRIC(kMetricS3MetadataCalls, velox::StatType::COUNT); + DEFINE_METRIC(kMetricS3GetObjectCalls, velox::StatType::COUNT); + DEFINE_METRIC(kMetricS3GetObjectErrors, velox::StatType::COUNT); + DEFINE_METRIC(kMetricS3GetMetadataErrors, velox::StatType::COUNT); + DEFINE_METRIC(kMetricS3GetObjectRetries, velox::StatType::COUNT); + DEFINE_METRIC(kMetricS3GetMetadataRetries, velox::StatType::COUNT); +#endif +} + +void registerAWSCredentialsProvider( + const std::string& providerName, + const AWSCredentialsProviderFactory& provider) { +#ifdef VELOX_ENABLE_S3 + registerCredentialsProvider(providerName, provider); +#endif +} + +} // namespace facebook::velox::filesystems diff --git a/velox/connectors/lakehouse/storage_adapters/s3fs/RegisterS3FileSystem.h b/velox/connectors/lakehouse/storage_adapters/s3fs/RegisterS3FileSystem.h new file mode 100644 index 000000000000..346379b72b7d --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/s3fs/RegisterS3FileSystem.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 +#include +#include + +namespace Aws::Auth { +// Forward-declare the AWSCredentialsProvider class from the AWS SDK. +class AWSCredentialsProvider; +} // namespace Aws::Auth + +namespace facebook::velox::config { +class ConfigBase; +} + +namespace facebook::velox::filesystems { + +using CacheKeyFn = std::function< + std::string(std::shared_ptr, std::string_view)>; + +// Register the S3 filesystem. +void registerS3FileSystem(CacheKeyFn cacheKeyFunc = nullptr); + +void registerS3Metrics(); + +/// Teardown the AWS SDK C++. +/// Velox users need to manually invoke this before exiting an application. +/// This is because Velox uses a static object to hold the S3 FileSystem +/// instance. AWS C++ SDK library also uses static global objects in its code. +/// The order of static object destruction is not determined by the C++ +/// standard. +/// This could lead to a segmentation fault during the program exit. +/// Ref https://github.com/aws/aws-sdk-cpp/issues/1550#issuecomment-1412601061 +void finalizeS3FileSystem(); + +class S3Config; + +using AWSCredentialsProviderFactory = + std::function( + const S3Config& config)>; + +void registerAWSCredentialsProvider( + const std::string& providerName, + const AWSCredentialsProviderFactory& provider); + +} // namespace facebook::velox::filesystems diff --git a/velox/connectors/lakehouse/storage_adapters/s3fs/S3Config.cpp b/velox/connectors/lakehouse/storage_adapters/s3fs/S3Config.cpp new file mode 100644 index 000000000000..a82a3be45626 --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/s3fs/S3Config.cpp @@ -0,0 +1,90 @@ +/* + * 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/storage_adapters/s3fs/S3Config.h" + +#include "velox/common/config/Config.h" +#include "velox/connectors/hive/storage_adapters/s3fs/S3Util.h" + +namespace facebook::velox::filesystems { + +std::string S3Config::cacheKey( + std::string_view bucket, + std::shared_ptr config) { + auto bucketEndpoint = bucketConfigKey(Keys::kEndpoint, bucket); + if (config->valueExists(bucketEndpoint)) { + return fmt::format( + "{}-{}", config->get(bucketEndpoint).value(), bucket); + } + auto baseEndpoint = baseConfigKey(Keys::kEndpoint); + if (config->valueExists(baseEndpoint)) { + return fmt::format( + "{}-{}", config->get(baseEndpoint).value(), bucket); + } + return std::string(bucket); +} + +S3Config::S3Config( + std::string_view bucket, + const std::shared_ptr properties) + : bucket_(bucket) { + for (int key = static_cast(Keys::kBegin); + key < static_cast(Keys::kEnd); + key++) { + auto s3Key = static_cast(key); + auto value = S3Config::configTraits().find(s3Key)->second; + auto configSuffix = value.first; + auto configDefault = value.second; + + // Set bucket S3 config "hive.s3.bucket.*" if present. + std::stringstream bucketConfig; + bucketConfig << kS3BucketPrefix << bucket << "." << configSuffix; + auto configVal = static_cast>( + properties->get(bucketConfig.str())); + if (configVal.has_value()) { + config_[s3Key] = configVal.value(); + } else { + // Set base config "hive.s3.*" if present. + std::stringstream baseConfig; + baseConfig << kS3Prefix << configSuffix; + configVal = static_cast>( + properties->get(baseConfig.str())); + if (configVal.has_value()) { + config_[s3Key] = configVal.value(); + } else { + // Set the default value. + config_[s3Key] = configDefault; + } + } + } + payloadSigningPolicy_ = + properties->get(kS3PayloadSigningPolicy, "Never"); +} + +std::optional S3Config::endpointRegion() const { + auto region = config_.find(Keys::kEndpointRegion)->second; + if (!region.has_value()) { + // If region is not set, try inferring from the endpoint value for AWS + // endpoints. + auto endpointValue = endpoint(); + if (endpointValue.has_value()) { + region = parseAWSStandardRegionName(endpointValue.value()); + } + } + return region; +} + +} // namespace facebook::velox::filesystems diff --git a/velox/connectors/lakehouse/storage_adapters/s3fs/S3Config.h b/velox/connectors/lakehouse/storage_adapters/s3fs/S3Config.h new file mode 100644 index 000000000000..4fad43799256 --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/s3fs/S3Config.h @@ -0,0 +1,252 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#pragma once + +#include +#include +#include "velox/common/base/Exceptions.h" + +namespace facebook::velox::config { +class ConfigBase; +} + +namespace facebook::velox::filesystems { + +/// Build config required to initialize an S3FileSystem instance. +/// All hive.s3 options can be set on a per-bucket basis. +/// The bucket-specific option is set by replacing the hive.s3. prefix on an +/// option with hive.s3.bucket.BUCKETNAME., where BUCKETNAME is the name of the +/// bucket. +/// When connecting to a bucket, all options explicitly set will override the +/// base hive.s3. values. +/// These semantics are similar to the Apache Hadoop-Aws module. +/// https://hadoop.apache.org/docs/current/hadoop-aws/tools/hadoop-aws/index.html +class S3Config { + public: + S3Config() = delete; + + /// S3 config prefix. + static constexpr const char* kS3Prefix = "hive.s3."; + + /// S3 bucket config prefix + static constexpr const char* kS3BucketPrefix = "hive.s3.bucket."; + + /// Log granularity of AWS C++ SDK. + static constexpr const char* kS3LogLevel = "hive.s3.log-level"; + + /// Payload signing policy. + static constexpr const char* kS3PayloadSigningPolicy = + "hive.s3.payload-signing-policy"; + + /// S3FileSystem default identity. + static constexpr const char* kDefaultS3Identity = "s3-default-identity"; + + /// Log location of AWS C++ SDK. + static constexpr const char* kS3LogLocation = "hive.s3.log-location"; + + /// Keys to identify the config. + enum class Keys { + kBegin, + kEndpoint = kBegin, + kEndpointRegion, + kAccessKey, + kSecretKey, + kPathStyleAccess, + kSSLEnabled, + kUseInstanceCredentials, + kIamRole, + kIamRoleSessionName, + kConnectTimeout, + kSocketTimeout, + kMaxConnections, + kMaxAttempts, + kRetryMode, + kUseProxyFromEnv, + kCredentialsProvider, + kEnd + }; + + /// Map of keys -> . + /// New config must be added here along with a getter function below. + static const std::unordered_map< + Keys, + std::pair>>& + configTraits() { + static const std::unordered_map< + Keys, + std::pair>> + config = { + {Keys::kEndpoint, std::make_pair("endpoint", std::nullopt)}, + {Keys::kEndpointRegion, + std::make_pair("endpoint.region", std::nullopt)}, + {Keys::kAccessKey, std::make_pair("aws-access-key", std::nullopt)}, + {Keys::kSecretKey, std::make_pair("aws-secret-key", std::nullopt)}, + {Keys::kPathStyleAccess, + std::make_pair("path-style-access", "false")}, + {Keys::kSSLEnabled, std::make_pair("ssl.enabled", "true")}, + {Keys::kUseInstanceCredentials, + std::make_pair("use-instance-credentials", "false")}, + {Keys::kIamRole, std::make_pair("iam-role", std::nullopt)}, + {Keys::kIamRoleSessionName, + std::make_pair("iam-role-session-name", "velox-session")}, + {Keys::kConnectTimeout, + std::make_pair("connect-timeout", std::nullopt)}, + {Keys::kSocketTimeout, + std::make_pair("socket-timeout", std::nullopt)}, + {Keys::kMaxConnections, + std::make_pair("max-connections", std::nullopt)}, + {Keys::kMaxAttempts, std::make_pair("max-attempts", std::nullopt)}, + {Keys::kRetryMode, std::make_pair("retry-mode", std::nullopt)}, + {Keys::kUseProxyFromEnv, + std::make_pair("use-proxy-from-env", "false")}, + {Keys::kCredentialsProvider, + std::make_pair("aws-credentials-provider", std::nullopt)}, + }; + return config; + } + + S3Config( + std::string_view bucket, + std::shared_ptr config); + + /// cacheKey is used as a key for the S3FileSystem instance map. + /// This will be the bucket endpoint or the base endpoint if they exist plus + /// bucket name. + static std::string cacheKey( + std::string_view bucket, + std::shared_ptr config); + + /// Return the base config for the input Key. + static std::string baseConfigKey(Keys key) { + std::stringstream buffer; + buffer << kS3Prefix << configTraits().find(key)->second.first; + return buffer.str(); + } + + /// Return the bucket config for the input key. + static std::string bucketConfigKey(Keys key, std::string_view bucket) { + std::stringstream buffer; + buffer << kS3BucketPrefix << bucket << "." + << configTraits().find(key)->second.first; + return buffer.str(); + } + + /// The S3 storage endpoint server. This can be used to connect to an + /// S3-compatible storage system instead of AWS. + std::optional endpoint() const { + return config_.find(Keys::kEndpoint)->second; + } + + /// The S3 storage endpoint region. + std::optional endpointRegion() const; + + /// Access key to use. + std::optional accessKey() const { + return config_.find(Keys::kAccessKey)->second; + } + + /// Secret key to use + std::optional secretKey() const { + return config_.find(Keys::kSecretKey)->second; + } + + /// Virtual addressing is used for AWS S3 and is the default + /// (path-style-access is false). Path access style is used for some on-prem + /// systems like Minio. + bool useVirtualAddressing() const { + auto value = config_.find(Keys::kPathStyleAccess)->second.value(); + return !folly::to(value); + } + + /// Use HTTPS to communicate with the S3 API. + bool useSSL() const { + auto value = config_.find(Keys::kSSLEnabled)->second.value(); + return folly::to(value); + } + + /// Use the EC2 metadata service to retrieve API credentials. + bool useInstanceCredentials() const { + auto value = config_.find(Keys::kUseInstanceCredentials)->second.value(); + return folly::to(value); + } + + /// IAM role to assume. + std::optional iamRole() const { + return config_.find(Keys::kIamRole)->second; + } + + /// Session name associated with the IAM role. + std::string iamRoleSessionName() const { + return config_.find(Keys::kIamRoleSessionName)->second.value(); + } + + /// Socket connect timeout. + std::optional connectTimeout() const { + return config_.find(Keys::kConnectTimeout)->second; + } + + /// Socket read timeout. + std::optional socketTimeout() const { + return config_.find(Keys::kSocketTimeout)->second; + } + + /// Maximum concurrent TCP connections for a single http client. + std::optional maxConnections() const { + auto val = config_.find(Keys::kMaxConnections)->second; + if (val.has_value()) { + return folly::to(val.value()); + } + return std::optional(); + } + + /// Maximum retry attempts for a single http client. + std::optional maxAttempts() const { + auto val = config_.find(Keys::kMaxAttempts)->second; + if (val.has_value()) { + return folly::to(val.value()); + } + return std::optional(); + } + + /// Retry mode for a single http client. + std::optional retryMode() const { + return config_.find(Keys::kRetryMode)->second; + } + + bool useProxyFromEnv() const { + auto value = config_.find(Keys::kUseProxyFromEnv)->second.value(); + return folly::to(value); + } + + std::string payloadSigningPolicy() const { + return payloadSigningPolicy_; + } + + std::string bucket() const { + return bucket_; + } + + std::optional credentialsProvider() const { + return config_.find(Keys::kCredentialsProvider)->second; + } + + private: + std::unordered_map> config_; + std::string payloadSigningPolicy_; + std::string bucket_; +}; + +} // namespace facebook::velox::filesystems diff --git a/velox/connectors/lakehouse/storage_adapters/s3fs/S3Counters.h b/velox/connectors/lakehouse/storage_adapters/s3fs/S3Counters.h new file mode 100644 index 000000000000..021e37e3bf07 --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/s3fs/S3Counters.h @@ -0,0 +1,56 @@ +/* + * 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 + +namespace facebook::velox::filesystems { + +// The number of connections open for S3 read operations. +constexpr std::string_view kMetricS3ActiveConnections{ + "velox.s3_active_connections"}; + +// The number of S3 upload calls that started. +constexpr std::string_view kMetricS3StartedUploads{"velox.s3_started_uploads"}; + +// The number of S3 upload calls that were completed. +constexpr std::string_view kMetricS3SuccessfulUploads{ + "velox.s3_successful_uploads"}; + +// The number of S3 upload calls that failed. +constexpr std::string_view kMetricS3FailedUploads{"velox.s3_failed_uploads"}; + +// The number of S3 head (metadata) calls. +constexpr std::string_view kMetricS3MetadataCalls{"velox.s3_metadata_calls"}; + +// The number of S3 head (metadata) calls that failed. +constexpr std::string_view kMetricS3GetMetadataErrors{ + "velox.s3_get_metadata_errors"}; + +// The number of retries made during S3 head (metadata) calls. +constexpr std::string_view kMetricS3GetMetadataRetries{ + "velox.s3_get_metadata_retries"}; + +// The number of S3 getObject calls. +constexpr std::string_view kMetricS3GetObjectCalls{"velox.s3_get_object_calls"}; + +// The number of S3 getObject calls that failed. +constexpr std::string_view kMetricS3GetObjectErrors{ + "velox.s3_get_object_errors"}; + +// The number of retries made during S3 getObject calls. +constexpr std::string_view kMetricS3GetObjectRetries{ + "velox.s3_get_object_retries"}; + +} // namespace facebook::velox::filesystems diff --git a/velox/connectors/lakehouse/storage_adapters/s3fs/S3FileSystem.cpp b/velox/connectors/lakehouse/storage_adapters/s3fs/S3FileSystem.cpp new file mode 100644 index 000000000000..0250dd85e668 --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/s3fs/S3FileSystem.cpp @@ -0,0 +1,517 @@ +/* + * 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/storage_adapters/s3fs/S3FileSystem.h" +#include "velox/common/base/StatsReporter.h" +#include "velox/common/config/Config.h" +#include "velox/common/file/File.h" +#include "velox/connectors/hive/storage_adapters/s3fs/S3Config.h" +#include "velox/connectors/hive/storage_adapters/s3fs/S3Counters.h" +#include "velox/connectors/hive/storage_adapters/s3fs/S3ReadFile.h" +#include "velox/connectors/hive/storage_adapters/s3fs/S3Util.h" +#include "velox/connectors/hive/storage_adapters/s3fs/S3WriteFile.h" +#include "velox/dwio/common/DataBuffer.h" + +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include + +namespace facebook::velox::filesystems { +namespace { + +folly::Synchronized< + std::unordered_map>& +credentialsProviderFactories() { + static folly::Synchronized< + std::unordered_map> + factories; + return factories; +} + +std::shared_ptr getCredentialsProviderByName( + const std::string& providerName, + const S3Config& s3Config) { + return credentialsProviderFactories().withRLock([&](const auto& factories) { + const auto it = factories.find(providerName); + VELOX_CHECK( + it != factories.end(), + "CredentialsProviderFactory for '{}' not registered", + providerName); + const auto& factory = it->second; + return factory(s3Config); + }); +} + +Aws::Utils::Logging::LogLevel inferS3LogLevel(std::string_view logLevel) { + std::string level = std::string(logLevel); + // Convert to upper case. + std::transform( + level.begin(), level.end(), level.begin(), [](unsigned char c) { + return std::toupper(c); + }); + if (level == "FATAL") { + return Aws::Utils::Logging::LogLevel::Fatal; + } else if (level == "TRACE") { + return Aws::Utils::Logging::LogLevel::Trace; + } else if (level == "OFF") { + return Aws::Utils::Logging::LogLevel::Off; + } else if (level == "ERROR") { + return Aws::Utils::Logging::LogLevel::Error; + } else if (level == "WARN") { + return Aws::Utils::Logging::LogLevel::Warn; + } else if (level == "INFO") { + return Aws::Utils::Logging::LogLevel::Info; + } else if (level == "DEBUG") { + return Aws::Utils::Logging::LogLevel::Debug; + } + return Aws::Utils::Logging::LogLevel::Fatal; +} + +// Supported values are "Always", "RequestDependent", "Never"(default). +Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy inferPayloadSign( + std::string sign) { + // Convert to upper case. + std::transform(sign.begin(), sign.end(), sign.begin(), [](unsigned char c) { + return std::toupper(c); + }); + if (sign == "ALWAYS") { + return Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy::Always; + } else if (sign == "REQUESTDEPENDENT") { + return Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy::RequestDependent; + } + return Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy::Never; +} +} // namespace + +// Initialize and Finalize the AWS SDK C++ library. +// Initialization must be done before creating a S3FileSystem. +// Finalization must be done after all S3FileSystem instances have been deleted. +// After Finalize, no new S3FileSystem can be created. +struct AwsInstance { + AwsInstance() : isInitialized_(false), isFinalized_(false) {} + ~AwsInstance() { + finalize(/*from_destructor=*/true); + } + + // Returns true iff the instance was newly initialized with config. + bool initialize( + std::string_view logLevel, + std::optional logLocation) { + if (isFinalized_.load()) { + VELOX_FAIL("Attempt to initialize S3 after it has been finalized."); + } + if (!isInitialized_.exchange(true)) { + // Not already initialized. + doInitialize(logLevel, logLocation); + return true; + } + return false; + } + + bool isInitialized() const { + return !isFinalized_ && isInitialized_; + } + + void finalize(bool fromDestructor = false) { + if (isFinalized_.exchange(true)) { + // Already finalized. + return; + } + if (isInitialized_.exchange(false)) { + // Was initialized. + if (fromDestructor) { + VLOG(0) + << "finalizeS3FileSystem() was not called even though S3 was initialized." + "This could lead to a segmentation fault at exit"; + } + Aws::ShutdownAPI(awsOptions_); + } + } + + std::string getLogLevelName() const { + return Aws::Utils::Logging::GetLogLevelName( + awsOptions_.loggingOptions.logLevel); + } + + std::string getLogPrefix() const { + return logPrefix_; + } + + private: + void doInitialize( + std::string_view logLevel, + std::optional logLocation) { + awsOptions_.loggingOptions.logLevel = inferS3LogLevel(logLevel); + if (logLocation.has_value()) { + logPrefix_ = fmt::format( + "{}{}{}", + logLocation.value(), + logLocation.value().back() == '/' ? "" : "/", + Aws::DEFAULT_LOG_PREFIX); + awsOptions_.loggingOptions.defaultLogPrefix = logPrefix_.c_str(); + VLOG(0) << "Custom S3 log location prefix: " << logPrefix_; + } + // In some situations, curl triggers a SIGPIPE signal causing the entire + // process to be terminated without any notification. + // This behavior is seen via Prestissimo on AmazonLinux2 on AWS EC2. + // Relevant documentation in AWS SDK C++ + // https://github.com/aws/aws-sdk-cpp/blob/276ee83080fcc521d41d456dbbe61d49392ddf77/src/aws-cpp-sdk-core/include/aws/core/Aws.h#L96 + // This option allows the AWS SDK C++ to catch the SIGPIPE signal and + // log a message. + awsOptions_.httpOptions.installSigPipeHandler = true; + Aws::InitAPI(awsOptions_); + } + + Aws::SDKOptions awsOptions_; + std::atomic isInitialized_; + std::atomic isFinalized_; + std::string logPrefix_; +}; + +// Singleton to initialize AWS S3. +AwsInstance* getAwsInstance() { + static auto instance = std::make_unique(); + return instance.get(); +} + +bool initializeS3( + std::string_view logLevel, + std::optional logLocation) { + return getAwsInstance()->initialize(logLevel, logLocation); +} + +static std::atomic fileSystemCount = 0; + +void finalizeS3() { + VELOX_CHECK((fileSystemCount == 0), "Cannot finalize S3 while in use"); + getAwsInstance()->finalize(); +} + +void registerCredentialsProvider( + const std::string& providerName, + const AWSCredentialsProviderFactory& factory) { + VELOX_CHECK( + !providerName.empty(), "CredentialsProviderFactory name cannot be empty"); + credentialsProviderFactories().withWLock([&](auto& factories) { + VELOX_CHECK( + factories.find(providerName) == factories.end(), + "CredentialsProviderFactory '{}' already registered", + providerName); + factories.insert({providerName, factory}); + }); +} + +class S3FileSystem::Impl { + public: + Impl(const S3Config& s3Config) { + VELOX_CHECK(getAwsInstance()->isInitialized(), "S3 is not initialized"); + Aws::S3::S3ClientConfiguration clientConfig; + if (s3Config.endpoint().has_value()) { + clientConfig.endpointOverride = s3Config.endpoint().value(); + } + + if (s3Config.endpointRegion().has_value()) { + clientConfig.region = s3Config.endpointRegion().value(); + } + + if (s3Config.useProxyFromEnv()) { + auto proxyConfig = + S3ProxyConfigurationBuilder( + s3Config.endpoint().has_value() ? s3Config.endpoint().value() + : "") + .useSsl(s3Config.useSSL()) + .build(); + if (proxyConfig.has_value()) { + clientConfig.proxyScheme = Aws::Http::SchemeMapper::FromString( + proxyConfig.value().scheme().c_str()); + clientConfig.proxyHost = awsString(proxyConfig.value().host()); + clientConfig.proxyPort = proxyConfig.value().port(); + clientConfig.proxyUserName = awsString(proxyConfig.value().username()); + clientConfig.proxyPassword = awsString(proxyConfig.value().password()); + } + } + + if (s3Config.useSSL()) { + clientConfig.scheme = Aws::Http::Scheme::HTTPS; + } else { + clientConfig.scheme = Aws::Http::Scheme::HTTP; + } + + if (s3Config.connectTimeout().has_value()) { + clientConfig.connectTimeoutMs = + std::chrono::duration_cast( + facebook::velox::config::toDuration( + s3Config.connectTimeout().value())) + .count(); + } + + if (s3Config.socketTimeout().has_value()) { + clientConfig.requestTimeoutMs = + std::chrono::duration_cast( + facebook::velox::config::toDuration( + s3Config.socketTimeout().value())) + .count(); + } + + if (s3Config.maxConnections().has_value()) { + clientConfig.maxConnections = s3Config.maxConnections().value(); + } + + auto retryStrategy = getRetryStrategy(s3Config); + if (retryStrategy.has_value()) { + clientConfig.retryStrategy = retryStrategy.value(); + } + + clientConfig.useVirtualAddressing = s3Config.useVirtualAddressing(); + clientConfig.payloadSigningPolicy = + inferPayloadSign(s3Config.payloadSigningPolicy()); + + auto credentialsProvider = getCredentialsProvider(s3Config); + + client_ = std::make_shared( + credentialsProvider, nullptr /* endpointProvider */, clientConfig); + ++fileSystemCount; + } + + ~Impl() { + client_.reset(); + --fileSystemCount; + } + + // Configure and return an AWSCredentialsProvider with access key and secret + // key. + std::shared_ptr + getAccessKeySecretKeyCredentialsProvider( + const std::string& accessKey, + const std::string& secretKey) const { + return std::make_shared( + awsString(accessKey), awsString(secretKey)); + } + + // Return a default AWSCredentialsProvider. + std::shared_ptr + getDefaultCredentialsProvider() const { + return std::make_shared(); + } + + // Configure and return an AWSCredentialsProvider with S3 IAM Role. + std::shared_ptr + getIAMRoleCredentialsProvider( + const std::string& s3IAMRole, + const std::string& sessionName) const { + return std::make_shared( + awsString(s3IAMRole), awsString(sessionName)); + } + + // Return an AWSCredentialsProvider based on the config. + std::shared_ptr getCredentialsProvider( + const S3Config& s3Config) const { + auto credentialsProvider = s3Config.credentialsProvider(); + if (credentialsProvider.has_value()) { + const auto& name = credentialsProvider.value(); + // Create the credentials provider using the registered factory. + return getCredentialsProviderByName(name, s3Config); + } + + auto accessKey = s3Config.accessKey(); + auto secretKey = s3Config.secretKey(); + const auto iamRole = s3Config.iamRole(); + + int keyCount = accessKey.has_value() + secretKey.has_value(); + // keyCount=0 means both are not specified + // keyCount=2 means both are specified + // keyCount=1 means only one of them is specified and is an error + VELOX_USER_CHECK( + (keyCount != 1), + "Invalid configuration: both access key and secret key must be specified"); + + int configCount = (accessKey.has_value() && secretKey.has_value()) + + iamRole.has_value() + s3Config.useInstanceCredentials(); + VELOX_USER_CHECK( + (configCount <= 1), + "Invalid configuration: specify only one among 'access/secret keys', 'use instance credentials', 'IAM role'"); + + if (accessKey.has_value() && secretKey.has_value()) { + return getAccessKeySecretKeyCredentialsProvider( + accessKey.value(), secretKey.value()); + } + + if (s3Config.useInstanceCredentials()) { + return getDefaultCredentialsProvider(); + } + + if (iamRole.has_value()) { + return getIAMRoleCredentialsProvider( + iamRole.value(), s3Config.iamRoleSessionName()); + } + + return getDefaultCredentialsProvider(); + } + + // Return a client RetryStrategy based on the config. + std::optional> getRetryStrategy( + const S3Config& s3Config) const { + auto retryMode = s3Config.retryMode(); + auto maxAttempts = s3Config.maxAttempts(); + if (retryMode.has_value()) { + if (retryMode.value() == "standard") { + if (maxAttempts.has_value()) { + VELOX_USER_CHECK_GE( + maxAttempts.value(), + 0, + "Invalid configuration: specified 'hive.s3.max-attempts' value {} is < 0.", + maxAttempts.value()); + return std::make_shared( + maxAttempts.value()); + } else { + // Otherwise, use default value 3. + return std::make_shared(); + } + } else if (retryMode.value() == "adaptive") { + if (maxAttempts.has_value()) { + VELOX_USER_CHECK_GE( + maxAttempts.value(), + 0, + "Invalid configuration: specified 'hive.s3.max-attempts' value {} is < 0.", + maxAttempts.value()); + return std::make_shared( + maxAttempts.value()); + } else { + // Otherwise, use default value 3. + return std::make_shared(); + } + } else if (retryMode.value() == "legacy") { + if (maxAttempts.has_value()) { + VELOX_USER_CHECK_GE( + maxAttempts.value(), + 0, + "Invalid configuration: specified 'hive.s3.max-attempts' value {} is < 0.", + maxAttempts.value()); + return std::make_shared( + maxAttempts.value()); + } else { + // Otherwise, use default value maxRetries = 10, scaleFactor = 25 + return std::make_shared(); + } + } else { + VELOX_USER_FAIL("Invalid retry mode for S3: {}", retryMode.value()); + } + } + return std::nullopt; + } + + // Make it clear that the S3FileSystem instance owns the S3Client. + // Once the S3FileSystem is destroyed, the S3Client fails to work + // due to the Aws::ShutdownAPI invocation in the destructor. + Aws::S3::S3Client* s3Client() const { + return client_.get(); + } + + std::string getLogLevelName() const { + return getAwsInstance()->getLogLevelName(); + } + + std::string getLogPrefix() const { + return getAwsInstance()->getLogPrefix(); + } + + private: + std::shared_ptr client_; +}; + +S3FileSystem::S3FileSystem( + std::string_view bucketName, + const std::shared_ptr config) + : FileSystem(config) { + S3Config s3Config(bucketName, config); + impl_ = std::make_shared(s3Config); +} + +std::string S3FileSystem::getLogLevelName() const { + return impl_->getLogLevelName(); +} + +std::string S3FileSystem::getLogPrefix() const { + return impl_->getLogPrefix(); +} + +std::unique_ptr S3FileSystem::openFileForRead( + std::string_view s3Path, + const FileOptions& options) { + const auto path = getPath(s3Path); + auto s3file = std::make_unique(path, impl_->s3Client()); + s3file->initialize(options); + return s3file; +} + +std::unique_ptr S3FileSystem::openFileForWrite( + std::string_view s3Path, + const FileOptions& options) { + const auto path = getPath(s3Path); + auto s3file = + std::make_unique(path, impl_->s3Client(), options.pool); + return s3file; +} + +std::string S3FileSystem::name() const { + return "S3"; +} + +std::vector S3FileSystem::list(std::string_view path) { + std::string bucket; + std::string key; + getBucketAndKeyFromPath(getPath(path), bucket, key); + + Aws::S3::Model::ListObjectsRequest request; + request.SetBucket(awsString(bucket)); + request.SetPrefix(awsString(key)); + + auto outcome = impl_->s3Client()->ListObjects(request); + VELOX_CHECK_AWS_OUTCOME( + outcome, "Failed to list objects in S3 bucket", bucket, key); + + std::vector objectKeys; + const auto& result = outcome.GetResult(); + for (const auto& object : result.GetContents()) { + objectKeys.emplace_back(object.GetKey()); + } + + return objectKeys; +} + +bool S3FileSystem::exists(std::string_view path) { + std::string bucket; + std::string key; + getBucketAndKeyFromPath(getPath(path), bucket, key); + + Aws::S3::Model::HeadObjectRequest request; + request.SetBucket(awsString(bucket)); + request.SetKey(awsString(key)); + + return impl_->s3Client()->HeadObject(request).IsSuccess(); +} + +} // namespace facebook::velox::filesystems diff --git a/velox/connectors/lakehouse/storage_adapters/s3fs/S3FileSystem.h b/velox/connectors/lakehouse/storage_adapters/s3fs/S3FileSystem.h new file mode 100644 index 000000000000..1c47f89195a0 --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/s3fs/S3FileSystem.h @@ -0,0 +1,98 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#pragma once + +#include "velox/common/file/FileSystems.h" + +namespace Aws::Auth { +// Forward-declare the AWSCredentialsProvider class from the AWS SDK. +class AWSCredentialsProvider; +} // namespace Aws::Auth + +namespace facebook::velox::filesystems { + +bool initializeS3( + std::string_view logLevel = "FATAL", + std::optional logLocation = std::nullopt); + +void finalizeS3(); + +class S3Config; + +using AWSCredentialsProviderFactory = + std::function( + const S3Config& config)>; + +void registerCredentialsProvider( + const std::string& providerName, + const AWSCredentialsProviderFactory& factory); + +/// Implementation of S3 filesystem and file interface. +/// We provide a registration method for read and write files so the appropriate +/// type of file can be constructed based on a filename. +class S3FileSystem : public FileSystem { + public: + S3FileSystem( + std::string_view bucketName, + const std::shared_ptr config); + + std::string name() const override; + + std::unique_ptr openFileForRead( + std::string_view s3Path, + const FileOptions& options = {}) override; + + std::unique_ptr openFileForWrite( + std::string_view s3Path, + const FileOptions& options) override; + + void remove(std::string_view path) override { + VELOX_UNSUPPORTED("remove for S3 not implemented"); + } + + void rename( + std::string_view path, + std::string_view newPath, + bool overWrite = false) override { + VELOX_UNSUPPORTED("rename for S3 not implemented"); + } + + /// Checks that the path exists. + bool exists(std::string_view path) override; + + /// List the objects associated to a path. + std::vector list(std::string_view path) override; + + void mkdir(std::string_view path, const DirectoryOptions& options = {}) + override { + VELOX_UNSUPPORTED("mkdir for S3 not implemented"); + } + + void rmdir(std::string_view path) override { + VELOX_UNSUPPORTED("rmdir for S3 not implemented"); + } + + std::string getLogLevelName() const; + + std::string getLogPrefix() const; + + protected: + class Impl; + std::shared_ptr impl_; +}; + +} // namespace facebook::velox::filesystems diff --git a/velox/connectors/lakehouse/storage_adapters/s3fs/S3ReadFile.cpp b/velox/connectors/lakehouse/storage_adapters/s3fs/S3ReadFile.cpp new file mode 100644 index 000000000000..06d180b19f7a --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/s3fs/S3ReadFile.cpp @@ -0,0 +1,220 @@ +/* + * 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/storage_adapters/s3fs/S3ReadFile.h" +#include "velox/common/base/StatsReporter.h" +#include "velox/connectors/hive/storage_adapters/s3fs/S3Counters.h" +#include "velox/connectors/hive/storage_adapters/s3fs/S3Util.h" + +#include +#include +#include +#include + +namespace facebook::velox::filesystems { + +namespace { + +// By default, the AWS SDK reads object data into an auto-growing StringStream. +// To avoid copies, read directly into a pre-allocated buffer instead. +// See https://github.com/aws/aws-sdk-cpp/issues/64 for an alternative but +// functionally similar recipe. +Aws::IOStreamFactory AwsWriteableStreamFactory(void* data, int64_t nbytes) { + return [=]() { return Aws::New("", data, nbytes); }; +} + +} // namespace + +class S3ReadFile ::Impl { + public: + explicit Impl(std::string_view path, Aws::S3::S3Client* client) + : client_(client) { + getBucketAndKeyFromPath(path, bucket_, key_); + } + + // Gets the length of the file. + // Checks if there are any issues reading the file. + void initialize(const filesystems::FileOptions& options) { + if (options.fileSize.has_value()) { + VELOX_CHECK_GE( + options.fileSize.value(), 0, "File size must be non-negative"); + length_ = options.fileSize.value(); + } + + // Make it a no-op if invoked twice. + if (length_ != -1) { + return; + } + + Aws::S3::Model::HeadObjectRequest request; + request.SetBucket(awsString(bucket_)); + request.SetKey(awsString(key_)); + + RECORD_METRIC_VALUE(kMetricS3MetadataCalls); + auto outcome = client_->HeadObject(request); + if (!outcome.IsSuccess()) { + RECORD_METRIC_VALUE(kMetricS3GetMetadataErrors); + } + RECORD_METRIC_VALUE(kMetricS3GetMetadataRetries, outcome.GetRetryCount()); + VELOX_CHECK_AWS_OUTCOME( + outcome, "Failed to get metadata for S3 object", bucket_, key_); + length_ = outcome.GetResult().GetContentLength(); + VELOX_CHECK_GE(length_, 0); + } + + std::string_view pread( + uint64_t offset, + uint64_t length, + void* buffer, + File::IoStats* stats) const { + preadInternal(offset, length, static_cast(buffer)); + return {static_cast(buffer), length}; + } + + std::string pread(uint64_t offset, uint64_t length, File::IoStats* stats) + const { + std::string result(length, 0); + char* position = result.data(); + preadInternal(offset, length, position); + return result; + } + + uint64_t preadv( + uint64_t offset, + const std::vector>& buffers, + File::IoStats* stats) const { + // 'buffers' contains Ranges(data, size) with some gaps (data = nullptr) in + // between. This call must populate the ranges (except gap ranges) + // sequentially starting from 'offset'. AWS S3 GetObject does not support + // multi-range. AWS S3 also charges by number of read requests and not size. + // The idea here is to use a single read spanning all the ranges and then + // populate individual ranges. We pre-allocate a buffer to support this. + size_t length = 0; + for (const auto range : buffers) { + length += range.size(); + } + // TODO: allocate from a memory pool + std::string result(length, 0); + preadInternal(offset, length, static_cast(result.data())); + size_t resultOffset = 0; + for (auto range : buffers) { + if (range.data()) { + memcpy(range.data(), &(result.data()[resultOffset]), range.size()); + } + resultOffset += range.size(); + } + return length; + } + + uint64_t size() const { + return length_; + } + + uint64_t memoryUsage() const { + // TODO: Check if any buffers are being used by the S3 library + return sizeof(Aws::S3::S3Client) + kS3MaxKeySize + 2 * sizeof(std::string) + + sizeof(int64_t); + } + + bool shouldCoalesce() const { + return false; + } + + std::string getName() const { + return fmt::format("s3://{}/{}", bucket_, key_); + } + + private: + // The assumption here is that "position" has space for at least "length" + // bytes. + void preadInternal(uint64_t offset, uint64_t length, char* position) const { + // Read the desired range of bytes. + Aws::S3::Model::GetObjectRequest request; + Aws::S3::Model::GetObjectResult result; + + request.SetBucket(awsString(bucket_)); + request.SetKey(awsString(key_)); + std::stringstream ss; + ss << "bytes=" << offset << "-" << offset + length - 1; + request.SetRange(awsString(ss.str())); + request.SetResponseStreamFactory( + AwsWriteableStreamFactory(position, length)); + RECORD_METRIC_VALUE(kMetricS3ActiveConnections); + RECORD_METRIC_VALUE(kMetricS3GetObjectCalls); + auto outcome = client_->GetObject(request); + if (!outcome.IsSuccess()) { + RECORD_METRIC_VALUE(kMetricS3GetObjectErrors); + } + RECORD_METRIC_VALUE(kMetricS3GetObjectRetries, outcome.GetRetryCount()); + RECORD_METRIC_VALUE(kMetricS3ActiveConnections, -1); + VELOX_CHECK_AWS_OUTCOME(outcome, "Failed to get S3 object", bucket_, key_); + } + + Aws::S3::S3Client* client_; + std::string bucket_; + std::string key_; + int64_t length_ = -1; +}; + +S3ReadFile::S3ReadFile(std::string_view path, Aws::S3::S3Client* client) { + impl_ = std::make_shared(path, client); +} + +S3ReadFile::~S3ReadFile() = default; + +void S3ReadFile::initialize(const filesystems::FileOptions& options) { + return impl_->initialize(options); +} + +std::string_view S3ReadFile::pread( + uint64_t offset, + uint64_t length, + void* buf, + filesystems::File::IoStats* stats) const { + return impl_->pread(offset, length, buf, stats); +} + +std::string S3ReadFile::pread( + uint64_t offset, + uint64_t length, + filesystems::File::IoStats* stats) const { + return impl_->pread(offset, length, stats); +} + +uint64_t S3ReadFile::preadv( + uint64_t offset, + const std::vector>& buffers, + filesystems::File::IoStats* stats) const { + return impl_->preadv(offset, buffers, stats); +} + +uint64_t S3ReadFile::size() const { + return impl_->size(); +} + +uint64_t S3ReadFile::memoryUsage() const { + return impl_->memoryUsage(); +} + +bool S3ReadFile::shouldCoalesce() const { + return impl_->shouldCoalesce(); +} + +std::string S3ReadFile::getName() const { + return impl_->getName(); +} + +} // namespace facebook::velox::filesystems diff --git a/velox/connectors/lakehouse/storage_adapters/s3fs/S3ReadFile.h b/velox/connectors/lakehouse/storage_adapters/s3fs/S3ReadFile.h new file mode 100644 index 000000000000..0b08ed0ec188 --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/s3fs/S3ReadFile.h @@ -0,0 +1,71 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#pragma once + +#include "velox/common/file/File.h" + +namespace Aws::S3 { +class S3Client; +} + +namespace facebook::velox::filesystems { + +/// Implementation of s3 read file. +class S3ReadFile : public ReadFile { + public: + S3ReadFile(std::string_view path, Aws::S3::S3Client* client); + + ~S3ReadFile() override; + + std::string_view pread( + uint64_t offset, + uint64_t length, + void* buf, + filesystems::File::IoStats* stats = nullptr) const final; + + std::string pread( + uint64_t offset, + uint64_t length, + filesystems::File::IoStats* stats = nullptr) const final; + + uint64_t preadv( + uint64_t offset, + const std::vector>& buffers, + filesystems::File::IoStats* stats = nullptr) const final; + + uint64_t size() const final; + + uint64_t memoryUsage() const final; + + bool shouldCoalesce() const final; + + std::string getName() const final; + + uint64_t getNaturalReadSize() const final { + return 72 << 20; + } + + void initialize(const filesystems::FileOptions& options); + + private: + void preadInternal(uint64_t offset, uint64_t length, char* position) const; + + class Impl; + std::shared_ptr impl_; +}; + +} // namespace facebook::velox::filesystems diff --git a/velox/connectors/lakehouse/storage_adapters/s3fs/S3Util.cpp b/velox/connectors/lakehouse/storage_adapters/s3fs/S3Util.cpp new file mode 100644 index 000000000000..9470f858884a --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/s3fs/S3Util.cpp @@ -0,0 +1,185 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// Implementation of S3 filesystem and file interface. +// We provide a registration method for read and write files so the appropriate +// type of file can be constructed based on a filename. See the +// (register|generate)ReadFile and (register|generate)WriteFile functions. + +#include "folly/IPAddress.h" +#include "re2/re2.h" + +#include "velox/connectors/hive/storage_adapters/s3fs/S3Util.h" + +namespace facebook::velox::filesystems { + +std::string getErrorStringFromS3Error( + const Aws::Client::AWSError& error) { + switch (error.GetErrorType()) { + case Aws::S3::S3Errors::NO_SUCH_BUCKET: + return "No such bucket"; + case Aws::S3::S3Errors::NO_SUCH_KEY: + return "No such key"; + case Aws::S3::S3Errors::RESOURCE_NOT_FOUND: + return "Resource not found"; + case Aws::S3::S3Errors::ACCESS_DENIED: + return "Access denied"; + case Aws::S3::S3Errors::SERVICE_UNAVAILABLE: + return "Service unavailable"; + case Aws::S3::S3Errors::NETWORK_CONNECTION: + return "Network connection"; + default: + return "Unknown error"; + } +} + +/// The noProxyList is a comma separated list of subdomains, domains or IP +/// ranges (using CIDR). For a given hostname check if it has a matching +/// subdomain, domain or IP range in the noProxyList. +bool isHostExcludedFromProxy( + const std::string& hostname, + const std::string& noProxyList) { + std::vector noProxyListElements{}; + + if (noProxyList.empty()) { + return false; + } + + auto hostAsIp = folly::IPAddress::tryFromString(hostname); + folly::split(',', noProxyList, noProxyListElements); + for (auto elem : noProxyListElements) { + // Elem contains "/" which separates IP and subnet mask e.g. 192.168.1.0/24. + if (elem.find("/") != std::string::npos && hostAsIp.hasValue()) { + return hostAsIp.value().inSubnet(elem); + } + // Match subdomain, domain names and IP address strings. + else if ( + elem.length() < hostname.length() && elem[0] == '.' && + !hostname.compare( + hostname.length() - elem.length(), elem.length(), elem)) { + return true; + } else if ( + elem.length() < hostname.length() && elem[0] == '*' && elem[1] == '.' && + !hostname.compare( + hostname.length() - elem.length() + 1, + elem.length() - 1, + elem.substr(1))) { + return true; + } else if (elem.length() == hostname.length() && !hostname.compare(elem)) { + return true; + } + } + return false; +} + +/// Reading the various proxy related environment variables. +/// There is a lacking standard. The environment variables can be +/// defined lower case or upper case. The lower case values are checked +/// first and, if set, returned, therefore taking precendence. +/// Note, the envVar input is expected to be lower case. +namespace { +std::string readProxyEnvVar(std::string envVar) { + auto httpProxy = getenv(envVar.c_str()); + if (httpProxy) { + return std::string(httpProxy); + } + + std::transform(envVar.begin(), envVar.end(), envVar.begin(), ::toupper); + httpProxy = getenv(envVar.c_str()); + if (httpProxy) { + return std::string(httpProxy); + } + return ""; +}; +} // namespace + +std::string getHttpProxyEnvVar() { + return readProxyEnvVar("http_proxy"); +} + +std::string getHttpsProxyEnvVar() { + return readProxyEnvVar("https_proxy"); +}; + +std::string getNoProxyEnvVar() { + return readProxyEnvVar("no_proxy"); +}; + +std::optional S3ProxyConfigurationBuilder::build() { + std::string proxyUrl; + if (useSsl_) { + proxyUrl = getHttpsProxyEnvVar(); + } else { + proxyUrl = getHttpProxyEnvVar(); + } + + if (proxyUrl.empty()) { + return std::nullopt; + } + folly::Uri proxyUri(proxyUrl); + + /// The endpoint is usually a domain with port or an + /// IP address with port. It is assumed that there are + /// 2 parts separated by a colon. + std::vector endpointElements{}; + folly::split(':', s3Endpoint_, endpointElements); + if (FOLLY_UNLIKELY(endpointElements.size() > 2)) { + LOG(ERROR) << fmt::format( + "Too many parts in S3 endpoint URI {} ", s3Endpoint_); + return std::nullopt; + } + + auto noProxy = getNoProxyEnvVar(); + if (isHostExcludedFromProxy(endpointElements[0], noProxy)) { + return std::nullopt; + } + return proxyUri; +} + +std::optional parseAWSStandardRegionName( + std::string_view endpoint) { + // The assumption is that the endpoint ends with + // ".amazonaws.com" or ".amazonaws.com/". That means for AWS we don't + // expect a port in the endpoint. + const std::string_view kAmazonHostSuffix = ".amazonaws.com"; + auto index = endpoint.size() - kAmazonHostSuffix.size(); + // Handle the case where the endpoint ends in a trailing slash. + if (endpoint.back() == '/') { + index--; + } + if (endpoint.rfind(kAmazonHostSuffix) != index) { + return std::nullopt; + } + // Remove the kAmazonHostSuffix. + std::string_view endpointPrefix = endpoint.substr(0, index); + const re2::RE2 pattern("^(?:.+\\.)?s3[-.]([a-z0-9-]+)$"); + std::string region; + if (re2::RE2::FullMatch(endpointPrefix, pattern, ®ion)) { + // endpointPrefix is 'bucket.s3-[region]' or 'bucket.s3.[region]' + return region; + } + + index = endpointPrefix.rfind('.'); + if (index != std::string::npos) { + // endpointPrefix was 'service.[region]'. + return std::string(endpointPrefix.substr(index + 1)); + } + + // Use default region set by the SDK. + return std::nullopt; +} + +} // namespace facebook::velox::filesystems diff --git a/velox/connectors/lakehouse/storage_adapters/s3fs/S3Util.h b/velox/connectors/lakehouse/storage_adapters/s3fs/S3Util.h new file mode 100644 index 000000000000..ab2e25790d04 --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/s3fs/S3Util.h @@ -0,0 +1,243 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// Implementation of S3 filesystem and file interface. +// We provide a registration method for read and write files so the appropriate +// type of file can be constructed based on a filename. See the +// (register|generate)ReadFile and (register|generate)WriteFile functions. + +#pragma once + +#include +#include +#include +#include + +#include "velox/common/base/Exceptions.h" + +#include + +namespace facebook::velox::filesystems { + +namespace { +static std::string_view kSep{"/"}; +// AWS S3 EMRFS, Hadoop block storage filesystem on-top of Amazon S3 buckets. +static std::string_view kS3Scheme{"s3://"}; +// This should not be mixed with s3 nor the s3a. +// S3A Hadoop 3.x (previous connectors "s3" and "s3n" are deprecated). +static std::string_view kS3aScheme{"s3a://"}; +// DEPRECATED: s3n are deprecated in Hadoop 3.x but we are supporting s3n for +// data that hasn't been migrated yet. +static std::string_view kS3nScheme{"s3n://"}; +// OSS Alibaba support S3 format, usage only with SSL. +static std::string_view kOssScheme{"oss://"}; +// Tencent COS support S3 format. +static std::string_view kCosScheme{"cos://"}; +static std::string_view kCosNScheme{"cosn://"}; + +// From AWS documentation +constexpr int kS3MaxKeySize{1024}; +} // namespace + +inline bool isS3AwsFile(const std::string_view filename) { + return filename.substr(0, kS3Scheme.size()) == kS3Scheme; +} + +inline bool isS3aFile(const std::string_view filename) { + return filename.substr(0, kS3aScheme.size()) == kS3aScheme; +} + +inline bool isS3nFile(const std::string_view filename) { + return filename.substr(0, kS3nScheme.size()) == kS3nScheme; +} + +inline bool isOssFile(const std::string_view filename) { + return filename.substr(0, kOssScheme.size()) == kOssScheme; +} + +inline bool isCosFile(const std::string_view filename) { + return filename.substr(0, kCosScheme.size()) == kCosScheme; +} + +inline bool isCosNFile(const std::string_view filename) { + return filename.substr(0, kCosNScheme.size()) == kCosNScheme; +} + +inline bool isS3File(const std::string_view filename) { + // TODO: Each prefix should be implemented as its own filesystem. + return isS3AwsFile(filename) || isS3aFile(filename) || isS3nFile(filename) || + isOssFile(filename) || isCosFile(filename) || isCosNFile(filename); +} + +// The input `path` must not have the S3 prefix. +inline void getBucketAndKeyFromPath( + std::string_view path, + std::string& bucket, + std::string& key) { + auto firstSep = path.find_first_of(kSep); + bucket = path.substr(0, firstSep); + key = path.substr(firstSep + 1); +} + +// TODO: Correctness check for bucket name. +// 1. Length between 3 and 63: +// 3 < length(bucket) < 63 +// 2. Mandatory label notation - regexp: +// regexp="(^[a-z0-9])([.-]?[a-z0-9]+){2,62}([/]?$)" +// 3. Disallowed IPv4 notation - regexp: +// regexp="^((25[0-5]|(2[0-4]|1\d|[1-9]|)\d)\.?\b){4}[/]?$" +inline std::string s3URI(std::string_view bucket, std::string_view key) { + std::stringstream ss; + ss << kS3Scheme << bucket << kSep << key; + return ss.str(); +} + +inline std::string_view getPath(std::string_view path) { + // Remove one of the prefixes 's3://', 'oss://', 's3a://' if any from the + // given path. + // TODO: Each prefix should be implemented as its own filesystem. + if (isS3AwsFile(path)) { + return path.substr(kS3Scheme.length()); + } else if (isS3aFile(path)) { + return path.substr(kS3aScheme.length()); + } else if (isS3nFile(path)) { + return path.substr(kS3nScheme.length()); + } else if (isOssFile(path)) { + return path.substr(kOssScheme.length()); + } else if (isCosFile(path)) { + return path.substr(kCosScheme.length()); + } else if (isCosNFile(path)) { + return path.substr(kCosNScheme.length()); + } + return path; +} + +inline Aws::String awsString(const std::string& s) { + return Aws::String(s.begin(), s.end()); +} + +std::string getErrorStringFromS3Error( + const Aws::Client::AWSError& error); + +namespace { +inline std::string getS3BackendService( + const Aws::Http::HeaderValueCollection& headers) { + const auto it = headers.find("server"); + if (it != headers.end()) { + return it->second; + } + return "Unknown"; +} + +inline std::string getRequestID( + const Aws::Http::HeaderValueCollection& headers) { + static const std::vector kRequestIds{ + "x-amz-request-id", "x-oss-request-id"}; + + for (const auto& kRequestId : kRequestIds) { + const auto it = headers.find(kRequestId); + if (it != headers.end()) { + return it->second; + } + } + return ""; +} +} // namespace + +/// Only Amazon (amz) and Alibaba (oss) request IDs are supported. +#define VELOX_CHECK_AWS_OUTCOME(outcome, errorMsgPrefix, bucket, key) \ + { \ + if (!outcome.IsSuccess()) { \ + auto error = outcome.GetError(); \ + auto errMsg = fmt::format( \ + "{} due to: '{}'. Path:'{}', SDK Error Type:{}, HTTP Status Code:{}, S3 Service:'{}', Message:'{}', RequestID:'{}'.", \ + errorMsgPrefix, \ + getErrorStringFromS3Error(error), \ + s3URI(bucket, key), \ + static_cast(error.GetErrorType()), \ + error.GetResponseCode(), \ + getS3BackendService(error.GetResponseHeaders()), \ + error.GetMessage(), \ + getRequestID(error.GetResponseHeaders())); \ + if (IsRetryableHttpResponseCode(error.GetResponseCode())) { \ + auto retryHint = fmt::format( \ + " Request failed after retrying {} times. Try increasing the value of 'hive.s3.max-attempts'.", \ + outcome.GetRetryCount()); \ + errMsg.append(retryHint); \ + } \ + if (error.GetResponseCode() == Aws::Http::HttpResponseCode::NOT_FOUND) { \ + VELOX_FILE_NOT_FOUND_ERROR(errMsg); \ + } \ + VELOX_FAIL(errMsg); \ + } \ + } + +bool isHostExcludedFromProxy( + const std::string& hostname, + const std::string& noProxyList); + +std::string getHttpProxyEnvVar(); +std::string getHttpsProxyEnvVar(); +std::string getNoProxyEnvVar(); + +// Adopted from the AWS Java SDK +// Endpoint can be 'service.[region].amazonaws.com' or +// 'bucket.s3-[region].amazonaws.com' or bucket.s3.[region].amazonaws.com' +// Return value is a region string value if present. +// The endpoint may contain a trailing '/' that is handled. +std::optional parseAWSStandardRegionName( + std::string_view endpoint); + +class S3ProxyConfigurationBuilder { + public: + S3ProxyConfigurationBuilder(const std::string& s3Endpoint) + : s3Endpoint_(s3Endpoint){}; + + S3ProxyConfigurationBuilder& useSsl(const bool& useSsl) { + useSsl_ = useSsl; + return *this; + } + + std::optional build(); + + private: + const std::string s3Endpoint_; + bool useSsl_; +}; + +// Reference: https://issues.apache.org/jira/browse/ARROW-8692 +// https://github.com/apache/arrow/blob/master/cpp/src/arrow/filesystem/s3fs.cc#L843 +// A non-copying iostream. See +// https://stackoverflow.com/questions/35322033/aws-c-sdk-uploadpart-times-out +// https://stackoverflow.com/questions/13059091/creating-an-input-stream-from-constant-memory +class StringViewStream : Aws::Utils::Stream::PreallocatedStreamBuf, + public std::iostream { + public: + StringViewStream(const void* data, int64_t nbytes) + : Aws::Utils::Stream::PreallocatedStreamBuf( + reinterpret_cast(const_cast(data)), + static_cast(nbytes)), + std::iostream(this) {} +}; + +} // namespace facebook::velox::filesystems + +template <> +struct fmt::formatter : formatter { + auto format(Aws::Http::HttpResponseCode s, format_context& ctx) { + return formatter::format(static_cast(s), ctx); + } +}; diff --git a/velox/connectors/lakehouse/storage_adapters/s3fs/S3WriteFile.cpp b/velox/connectors/lakehouse/storage_adapters/s3fs/S3WriteFile.cpp new file mode 100644 index 000000000000..2bddbe687f40 --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/s3fs/S3WriteFile.cpp @@ -0,0 +1,269 @@ +/* + * 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/storage_adapters/s3fs/S3WriteFile.h" +#include "velox/common/base/StatsReporter.h" +#include "velox/connectors/hive/storage_adapters/s3fs/S3Counters.h" +#include "velox/connectors/hive/storage_adapters/s3fs/S3Util.h" +#include "velox/dwio/common/DataBuffer.h" + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace facebook::velox::filesystems { + +class S3WriteFile::Impl { + public: + explicit Impl( + std::string_view path, + Aws::S3::S3Client* client, + memory::MemoryPool* pool) + : client_(client), pool_(pool) { + VELOX_CHECK_NOT_NULL(client); + VELOX_CHECK_NOT_NULL(pool); + getBucketAndKeyFromPath(path, bucket_, key_); + currentPart_ = std::make_unique>(*pool_); + currentPart_->reserve(kPartUploadSize); + // Check that the object doesn't exist, if it does throw an error. + { + Aws::S3::Model::HeadObjectRequest request; + request.SetBucket(awsString(bucket_)); + request.SetKey(awsString(key_)); + RECORD_METRIC_VALUE(kMetricS3MetadataCalls); + auto objectMetadata = client_->HeadObject(request); + if (!objectMetadata.IsSuccess()) { + RECORD_METRIC_VALUE(kMetricS3GetMetadataErrors); + } + RECORD_METRIC_VALUE( + kMetricS3GetObjectRetries, objectMetadata.GetRetryCount()); + VELOX_CHECK(!objectMetadata.IsSuccess(), "S3 object already exists"); + } + + // Create bucket if not present. + { + Aws::S3::Model::HeadBucketRequest request; + request.SetBucket(awsString(bucket_)); + auto bucketMetadata = client_->HeadBucket(request); + if (!bucketMetadata.IsSuccess()) { + Aws::S3::Model::CreateBucketRequest request; + request.SetBucket(bucket_); + auto outcome = client_->CreateBucket(request); + VELOX_CHECK_AWS_OUTCOME( + outcome, "Failed to create S3 bucket", bucket_, ""); + } + } + + // Initiate the multi-part upload. + { + Aws::S3::Model::CreateMultipartUploadRequest request; + request.SetBucket(awsString(bucket_)); + request.SetKey(awsString(key_)); + + /// If we do not set anything then the SDK will default to application/xml + /// which confuses some tools + /// (https://github.com/apache/arrow/issues/11934). So we instead default + /// to application/octet-stream which is less misleading. + request.SetContentType(kApplicationOctetStream); + // The default algorithm used is MD5. However, MD5 is not supported with + // fips and can cause a SIGSEGV. Set CRC32 instead which is a standard for + // checksum computation and is not restricted by fips. + request.SetChecksumAlgorithm(Aws::S3::Model::ChecksumAlgorithm::CRC32); + + auto outcome = client_->CreateMultipartUpload(request); + VELOX_CHECK_AWS_OUTCOME( + outcome, "Failed initiating multiple part upload", bucket_, key_); + uploadState_.id = outcome.GetResult().GetUploadId(); + } + + fileSize_ = 0; + } + + // Appends data to the end of the file. + void append(std::string_view data) { + VELOX_CHECK(!closed(), "File is closed"); + if (data.size() + currentPart_->size() >= kPartUploadSize) { + upload(data); + } else { + // Append to current part. + currentPart_->unsafeAppend(data.data(), data.size()); + } + fileSize_ += data.size(); + } + + // No-op. + void flush() { + VELOX_CHECK(!closed(), "File is closed"); + /// currentPartSize must be less than kPartUploadSize since + /// append() would have already flushed after reaching kUploadPartSize. + VELOX_CHECK_LT(currentPart_->size(), kPartUploadSize); + } + + // Complete the multipart upload and close the file. + void close() { + if (closed()) { + return; + } + RECORD_METRIC_VALUE(kMetricS3StartedUploads); + uploadPart({currentPart_->data(), currentPart_->size()}, true); + VELOX_CHECK_EQ(uploadState_.partNumber, uploadState_.completedParts.size()); + // Complete the multipart upload. + { + Aws::S3::Model::CompletedMultipartUpload completedUpload; + completedUpload.SetParts(uploadState_.completedParts); + Aws::S3::Model::CompleteMultipartUploadRequest request; + request.SetBucket(awsString(bucket_)); + request.SetKey(awsString(key_)); + request.SetUploadId(uploadState_.id); + request.SetMultipartUpload(std::move(completedUpload)); + + auto outcome = client_->CompleteMultipartUpload(request); + if (outcome.IsSuccess()) { + RECORD_METRIC_VALUE(kMetricS3SuccessfulUploads); + } else { + RECORD_METRIC_VALUE(kMetricS3FailedUploads); + } + VELOX_CHECK_AWS_OUTCOME( + outcome, "Failed to complete multiple part upload", bucket_, key_); + } + currentPart_->clear(); + } + + // Current file size, i.e. the sum of all previous appends. + uint64_t size() const { + return fileSize_; + } + + int numPartsUploaded() const { + return uploadState_.partNumber; + } + + private: + static constexpr int64_t kPartUploadSize = 10 * 1024 * 1024; + static constexpr const char* kApplicationOctetStream = + "application/octet-stream"; + + bool closed() const { + return (currentPart_->capacity() == 0); + } + + // Holds state for the multipart upload. + struct UploadState { + Aws::Vector completedParts; + int64_t partNumber = 0; + Aws::String id; + }; + UploadState uploadState_; + + // Data can be smaller or larger than the kPartUploadSize. + // Complete the currentPart_ and upload kPartUploadSize chunks of data. + // Save the remaining into currentPart_. + void upload(const std::string_view data) { + auto dataPtr = data.data(); + auto dataSize = data.size(); + // Fill-up the remaining currentPart_. + auto remainingBufferSize = currentPart_->capacity() - currentPart_->size(); + currentPart_->unsafeAppend(dataPtr, remainingBufferSize); + uploadPart({currentPart_->data(), currentPart_->size()}); + dataPtr += remainingBufferSize; + dataSize -= remainingBufferSize; + while (dataSize > kPartUploadSize) { + uploadPart({dataPtr, kPartUploadSize}); + dataPtr += kPartUploadSize; + dataSize -= kPartUploadSize; + } + // Stash the remaining at the beginning of currentPart. + currentPart_->unsafeAppend(0, dataPtr, dataSize); + } + + void uploadPart(const std::string_view part, bool isLast = false) { + // Only the last part can be less than kPartUploadSize. + VELOX_CHECK(isLast || (!isLast && (part.size() == kPartUploadSize))); + // Upload the part. + { + Aws::S3::Model::UploadPartRequest request; + request.SetBucket(bucket_); + request.SetKey(key_); + request.SetUploadId(uploadState_.id); + request.SetPartNumber(++uploadState_.partNumber); + request.SetContentLength(part.size()); + request.SetBody( + std::make_shared(part.data(), part.size())); + // The default algorithm used is MD5. However, MD5 is not supported with + // fips and can cause a SIGSEGV. Set CRC32 instead which is a standard for + // checksum computation and is not restricted by fips. + request.SetChecksumAlgorithm(Aws::S3::Model::ChecksumAlgorithm::CRC32); + auto outcome = client_->UploadPart(request); + VELOX_CHECK_AWS_OUTCOME(outcome, "Failed to upload", bucket_, key_); + // Append ETag and part number for this uploaded part. + // This will be needed for upload completion in Close(). + auto result = outcome.GetResult(); + Aws::S3::Model::CompletedPart part; + + part.SetPartNumber(uploadState_.partNumber); + part.SetETag(result.GetETag()); + // Don't add the checksum to the part if the checksum is empty. + // Some filesystems such as IBM COS require this to be not set. + if (!result.GetChecksumCRC32().empty()) { + part.SetChecksumCRC32(result.GetChecksumCRC32()); + } + uploadState_.completedParts.push_back(std::move(part)); + } + } + + Aws::S3::S3Client* client_; + memory::MemoryPool* pool_; + std::unique_ptr> currentPart_; + std::string bucket_; + std::string key_; + size_t fileSize_ = -1; +}; + +S3WriteFile::S3WriteFile( + std::string_view path, + Aws::S3::S3Client* client, + memory::MemoryPool* pool) { + impl_ = std::make_shared(path, client, pool); +} + +void S3WriteFile::append(std::string_view data) { + return impl_->append(data); +} + +void S3WriteFile::flush() { + impl_->flush(); +} + +void S3WriteFile::close() { + impl_->close(); +} + +uint64_t S3WriteFile::size() const { + return impl_->size(); +} + +int S3WriteFile::numPartsUploaded() const { + return impl_->numPartsUploaded(); +} + +} // namespace facebook::velox::filesystems diff --git a/velox/connectors/lakehouse/storage_adapters/s3fs/S3WriteFile.h b/velox/connectors/lakehouse/storage_adapters/s3fs/S3WriteFile.h new file mode 100644 index 000000000000..929eed20c370 --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/s3fs/S3WriteFile.h @@ -0,0 +1,76 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#pragma once + +#include "velox/common/file/File.h" +#include "velox/common/memory/MemoryPool.h" + +namespace Aws::S3 { +class S3Client; +} + +namespace facebook::velox::filesystems { + +/// S3WriteFile uses the Apache Arrow implementation as a reference. +/// AWS C++ SDK allows streaming writes via the MultiPart upload API. +/// Multipart upload allows you to upload a single object as a set of parts. +/// Each part is a contiguous portion of the object's data. +/// While AWS and Minio support different sizes for each +/// part (only requiring a minimum of 5MB), Cloudflare R2 requires that every +/// part be exactly equal (except for the last part). We set this to 10 MiB, so +/// that in combination with the maximum number of parts of 10,000, this gives a +/// file limit of 100k MiB (or about 98 GiB). +/// (see https://docs.aws.amazon.com/AmazonS3/latest/userguide/qfacts.html) +/// (for rational, see: https://github.com/apache/arrow/issues/34363) +/// You can upload these object parts independently and in any order. +/// After all parts of your object are uploaded, Amazon S3 assembles these parts +/// and creates the object. +/// https://docs.aws.amazon.com/AmazonS3/latest/userguide/mpuoverview.html +/// https://github.com/apache/arrow/blob/main/cpp/src/arrow/filesystem/s3fs.cc +/// S3WriteFile is not thread-safe. +/// UploadPart is currently synchronous during append and flush. +/// TODO: Evaluate and add option for asynchronous part uploads. +/// TODO: Implement retry on failure. +class S3WriteFile : public WriteFile { + public: + S3WriteFile( + std::string_view path, + Aws::S3::S3Client* client, + memory::MemoryPool* pool); + + /// Appends data to the end of the file. + /// Uploads a part on reaching part size limit. + void append(std::string_view data) override; + + /// No-op. Append handles the flush. + void flush() override; + + /// Close the file. Any cleanup (disk flush, etc.) will be done here. + void close() override; + + /// Current file size, i.e. the sum of all previous Appends. + uint64_t size() const override; + + /// Return the number of parts uploaded so far. + int numPartsUploaded() const; + + protected: + class Impl; + std::shared_ptr impl_; +}; + +} // namespace facebook::velox::filesystems diff --git a/velox/connectors/lakehouse/storage_adapters/s3fs/tests/CMakeLists.txt b/velox/connectors/lakehouse/storage_adapters/s3fs/tests/CMakeLists.txt new file mode 100644 index 000000000000..f2bc59aa20f9 --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/s3fs/tests/CMakeLists.txt @@ -0,0 +1,115 @@ +# Copyright (c) Facebook, Inc. and its affiliates. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +add_executable(velox_s3config_test S3ConfigTest.cpp) +add_test(velox_s3config_test velox_s3config_test) +target_link_libraries( + velox_s3config_test + velox_common_config + velox_s3fs + GTest::gtest + GTest::gtest_main) + +add_executable(velox_s3file_test S3FileSystemTest.cpp S3UtilTest.cpp) +add_test(velox_s3file_test velox_s3file_test) +target_link_libraries( + velox_s3file_test + velox_file + velox_s3fs + velox_core + velox_exec_test_lib + velox_dwio_common_exception + velox_exec + GTest::gtest + GTest::gtest_main) + +add_executable(velox_s3registration_test S3FileSystemRegistrationTest.cpp) +add_test(velox_s3registration_test velox_s3registration_test) +target_link_libraries( + velox_s3registration_test + velox_file + velox_s3fs + velox_core + velox_exec_test_lib + velox_dwio_parquet_reader + velox_dwio_common_exception + velox_exec + GTest::gtest + GTest::gtest_main) + +add_executable(velox_s3finalize_test S3FileSystemFinalizeTest.cpp) +add_test(velox_s3finalize_test velox_s3finalize_test) +target_link_libraries( + velox_s3finalize_test + velox_s3fs + velox_file + velox_core + GTest::gtest + GTest::gtest_main) + +add_executable(velox_s3insert_test S3InsertTest.cpp) +add_test(velox_s3insert_test velox_s3insert_test) +target_link_libraries( + velox_s3insert_test + velox_file + velox_s3fs + velox_core + velox_exec_test_lib + velox_dwio_parquet_writer + velox_dwio_parquet_reader + velox_dwio_common_exception + velox_exec + GTest::gtest + GTest::gtest_main) + +add_executable(velox_s3read_test S3ReadTest.cpp) +add_test( + NAME velox_s3read_test + COMMAND velox_s3read_test + WORKING_DIRECTORY ${CMAKE_CURRENT_SOURCE_DIR}) +target_link_libraries( + velox_s3read_test + velox_file + velox_s3fs + velox_core + velox_exec_test_lib + velox_dwio_parquet_reader + velox_dwio_common_exception + velox_exec + GTest::gtest + GTest::gtest_main) + +add_executable(velox_s3metrics_test S3FileSystemMetricsTest.cpp) +add_test(velox_s3metrics_test velox_s3metrics_test) +target_link_libraries( + velox_s3metrics_test + velox_s3fs + velox_exec_test_lib + GTest::gtest + GTest::gtest_main) + +add_executable(velox_s3multiendpoints_test S3MultipleEndpointsTest.cpp) +add_test(velox_s3multiendpoints_test velox_s3multiendpoints_test) +target_link_libraries( + velox_s3multiendpoints_test + velox_file + velox_s3fs + velox_core + velox_exec_test_lib + velox_dwio_parquet_reader + velox_dwio_parquet_writer + velox_dwio_common_exception + velox_exec + GTest::gtest + GTest::gtest_main) diff --git a/velox/connectors/lakehouse/storage_adapters/s3fs/tests/MinioServer.h b/velox/connectors/lakehouse/storage_adapters/s3fs/tests/MinioServer.h new file mode 100644 index 000000000000..591ed403f350 --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/s3fs/tests/MinioServer.h @@ -0,0 +1,119 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#pragma once + +#include "velox/common/config/Config.h" +#include "velox/exec/tests/utils/PortUtil.h" +#include "velox/exec/tests/utils/TempDirectoryPath.h" + +#include "boost/process.hpp" + +using namespace facebook::velox; + +namespace { +constexpr char const* kMinioExecutableName{"minio-2022-05-26"}; +constexpr char const* kMinioAccessKey{"minio"}; +constexpr char const* kMinioSecretKey{"miniopass"}; +} // namespace + +// A minio server, managed as a child process. +// Adapted from the Apache Arrow library. +class MinioServer { + public: + MinioServer() : tempPath_(::exec::test::TempDirectoryPath::create()) { + constexpr auto kHostAddressTemplate = "127.0.0.1:{}"; + auto ports = facebook::velox::exec::test::getFreePorts(2); + connectionString_ = fmt::format(kHostAddressTemplate, ports[0]); + consoleAddress_ = fmt::format(kHostAddressTemplate, ports[1]); + } + + void start(); + + void stop(); + + void addBucket(const char* bucket) { + const std::string path = tempPath_->getPath() + "/" + bucket; + mkdir(path.c_str(), S_IRWXU | S_IRWXG); + } + + std::string path() const { + return tempPath_->getPath(); + } + + std::shared_ptr hiveConfig( + const std::unordered_map configOverride = {}) + const { + std::unordered_map config({ + {"hive.s3.aws-access-key", accessKey_}, + {"hive.s3.aws-secret-key", secretKey_}, + {"hive.s3.endpoint", connectionString_}, + {"hive.s3.ssl.enabled", "false"}, + {"hive.s3.path-style-access", "true"}, + }); + + // Update the default config map with the supplied configOverride map + for (const auto& [configName, configValue] : configOverride) { + config[configName] = configValue; + } + + return std::make_shared(std::move(config)); + } + + private: + const std::shared_ptr tempPath_; + std::string connectionString_; + std::string consoleAddress_; + const std::string accessKey_ = kMinioAccessKey; + const std::string secretKey_ = kMinioSecretKey; + std::shared_ptr<::boost::process::child> serverProcess_; +}; + +void MinioServer::start() { + boost::process::environment env = boost::this_process::environment(); + env["MINIO_ACCESS_KEY"] = accessKey_; + env["MINIO_SECRET_KEY"] = secretKey_; + + auto exePath = boost::process::search_path(kMinioExecutableName); + if (exePath.empty()) { + VELOX_FAIL("Failed to find minio executable {}'", kMinioExecutableName); + } + + const auto path = tempPath_->getPath(); + try { + serverProcess_ = std::make_shared( + env, + exePath, + "server", + "--quiet", + "--compat", + "--address", + connectionString_, + "--console-address", + consoleAddress_, + path.c_str()); + } catch (const std::exception& e) { + VELOX_FAIL("Failed to launch Minio server: {}", e.what()); + } +} + +void MinioServer::stop() { + if (serverProcess_ && serverProcess_->valid()) { + // Brutal shutdown + serverProcess_->terminate(); + serverProcess_->wait(); + } +} diff --git a/velox/connectors/lakehouse/storage_adapters/s3fs/tests/S3ConfigTest.cpp b/velox/connectors/lakehouse/storage_adapters/s3fs/tests/S3ConfigTest.cpp new file mode 100644 index 000000000000..e8d48fe6edd4 --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/s3fs/tests/S3ConfigTest.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/storage_adapters/s3fs/S3Config.h" +#include "velox/common/config/Config.h" + +#include + +namespace facebook::velox::filesystems { +namespace { +TEST(S3ConfigTest, defaultConfig) { + auto config = std::make_shared( + std::unordered_map()); + auto s3Config = S3Config("", config); + ASSERT_EQ(s3Config.useVirtualAddressing(), true); + ASSERT_EQ(s3Config.useSSL(), true); + ASSERT_EQ(s3Config.useInstanceCredentials(), false); + ASSERT_EQ(s3Config.endpoint(), std::nullopt); + ASSERT_EQ(s3Config.endpointRegion(), std::nullopt); + ASSERT_EQ(s3Config.accessKey(), std::nullopt); + ASSERT_EQ(s3Config.secretKey(), std::nullopt); + ASSERT_EQ(s3Config.iamRole(), std::nullopt); + ASSERT_EQ(s3Config.iamRoleSessionName(), "velox-session"); + ASSERT_EQ(s3Config.payloadSigningPolicy(), "Never"); + ASSERT_EQ(s3Config.cacheKey("foo", config), "foo"); + ASSERT_EQ(s3Config.bucket(), ""); +} + +TEST(S3ConfigTest, overrideConfig) { + std::unordered_map configFromFile = { + {S3Config::baseConfigKey(S3Config::Keys::kPathStyleAccess), "true"}, + {S3Config::baseConfigKey(S3Config::Keys::kSSLEnabled), "false"}, + {S3Config::baseConfigKey(S3Config::Keys::kUseInstanceCredentials), + "true"}, + {"hive.s3.payload-signing-policy", "RequestDependent"}, + {S3Config::baseConfigKey(S3Config::Keys::kEndpoint), "endpoint"}, + {S3Config::baseConfigKey(S3Config::Keys::kEndpointRegion), "region"}, + {S3Config::baseConfigKey(S3Config::Keys::kAccessKey), "access"}, + {S3Config::baseConfigKey(S3Config::Keys::kSecretKey), "secret"}, + {S3Config::baseConfigKey(S3Config::Keys::kIamRole), "iam"}, + {S3Config::baseConfigKey(S3Config::Keys::kIamRoleSessionName), "velox"}, + {S3Config::baseConfigKey(S3Config::Keys::kCredentialsProvider), + "my-credentials-provider"}}; + auto configBase = + std::make_shared(std::move(configFromFile)); + auto s3Config = S3Config("bucket", configBase); + ASSERT_EQ(s3Config.useVirtualAddressing(), false); + ASSERT_EQ(s3Config.useSSL(), false); + ASSERT_EQ(s3Config.useInstanceCredentials(), true); + ASSERT_EQ(s3Config.endpoint(), "endpoint"); + ASSERT_EQ(s3Config.endpointRegion(), "region"); + ASSERT_EQ(s3Config.accessKey(), std::optional("access")); + ASSERT_EQ(s3Config.secretKey(), std::optional("secret")); + ASSERT_EQ(s3Config.iamRole(), std::optional("iam")); + ASSERT_EQ(s3Config.iamRoleSessionName(), "velox"); + ASSERT_EQ(s3Config.payloadSigningPolicy(), "RequestDependent"); + ASSERT_EQ(s3Config.cacheKey("foo", configBase), "endpoint-foo"); + ASSERT_EQ(s3Config.cacheKey("bar", configBase), "endpoint-bar"); + ASSERT_EQ(s3Config.bucket(), "bucket"); + ASSERT_EQ(s3Config.credentialsProvider(), "my-credentials-provider"); +} + +TEST(S3ConfigTest, overrideBucketConfig) { + std::string_view bucket = "bucket"; + std::unordered_map bucketConfigFromFile = { + {S3Config::baseConfigKey(S3Config::Keys::kPathStyleAccess), "true"}, + {S3Config::baseConfigKey(S3Config::Keys::kSSLEnabled), "false"}, + {S3Config::baseConfigKey(S3Config::Keys::kUseInstanceCredentials), + "true"}, + {S3Config::baseConfigKey(S3Config::Keys::kEndpoint), "endpoint"}, + {S3Config::bucketConfigKey(S3Config::Keys::kEndpoint, bucket), + "bucket.s3-region.amazonaws.com"}, + {S3Config::baseConfigKey(S3Config::Keys::kAccessKey), "access"}, + {S3Config::bucketConfigKey(S3Config::Keys::kAccessKey, bucket), + "bucket-access"}, + {"hive.s3.payload-signing-policy", "Always"}, + {S3Config::baseConfigKey(S3Config::Keys::kSecretKey), "secret"}, + {S3Config::bucketConfigKey(S3Config::Keys::kSecretKey, bucket), + "bucket-secret"}, + {S3Config::baseConfigKey(S3Config::Keys::kIamRole), "iam"}, + {S3Config::baseConfigKey(S3Config::Keys::kIamRoleSessionName), "velox"}, + {S3Config::baseConfigKey(S3Config::Keys::kCredentialsProvider), + "my-credentials-provider"}, + {S3Config::bucketConfigKey(S3Config::Keys::kCredentialsProvider, bucket), + "override-credentials-provider"}}; + auto configBase = + std::make_shared(std::move(bucketConfigFromFile)); + auto s3Config = S3Config(bucket, configBase); + ASSERT_EQ(s3Config.useVirtualAddressing(), false); + ASSERT_EQ(s3Config.useSSL(), false); + ASSERT_EQ(s3Config.useInstanceCredentials(), true); + ASSERT_EQ(s3Config.endpoint(), "bucket.s3-region.amazonaws.com"); + // Inferred from the endpoint. + ASSERT_EQ(s3Config.endpointRegion(), "region"); + ASSERT_EQ(s3Config.accessKey(), std::optional("bucket-access")); + ASSERT_EQ(s3Config.secretKey(), std::optional("bucket-secret")); + ASSERT_EQ(s3Config.iamRole(), std::optional("iam")); + ASSERT_EQ(s3Config.iamRoleSessionName(), "velox"); + ASSERT_EQ(s3Config.payloadSigningPolicy(), "Always"); + ASSERT_EQ( + s3Config.cacheKey(bucket, configBase), + "bucket.s3-region.amazonaws.com-bucket"); + ASSERT_EQ(s3Config.cacheKey("foo", configBase), "endpoint-foo"); + ASSERT_EQ(s3Config.credentialsProvider(), "override-credentials-provider"); +} + +} // namespace +} // namespace facebook::velox::filesystems diff --git a/velox/connectors/lakehouse/storage_adapters/s3fs/tests/S3FileSystemFinalizeTest.cpp b/velox/connectors/lakehouse/storage_adapters/s3fs/tests/S3FileSystemFinalizeTest.cpp new file mode 100644 index 000000000000..9bebc4162490 --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/s3fs/tests/S3FileSystemFinalizeTest.cpp @@ -0,0 +1,43 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "velox/common/base/tests/GTestUtils.h" +#include "velox/common/config/Config.h" +#include "velox/connectors/hive/storage_adapters/s3fs/S3FileSystem.h" + +#include "gtest/gtest.h" + +namespace facebook::velox { +namespace { + +TEST(S3FileSystemFinalizeTest, finalize) { + auto s3Config = std::make_shared( + std::unordered_map()); + ASSERT_TRUE(filesystems::initializeS3()); + ASSERT_FALSE(filesystems::initializeS3()); + { + filesystems::S3FileSystem s3fs("", s3Config); + VELOX_ASSERT_THROW( + filesystems::finalizeS3(), "Cannot finalize S3 while in use"); + } + filesystems::finalizeS3(); + VELOX_ASSERT_THROW( + filesystems::initializeS3(), + "Attempt to initialize S3 after it has been finalized."); +} + +} // namespace +} // namespace facebook::velox diff --git a/velox/connectors/lakehouse/storage_adapters/s3fs/tests/S3FileSystemMetricsTest.cpp b/velox/connectors/lakehouse/storage_adapters/s3fs/tests/S3FileSystemMetricsTest.cpp new file mode 100644 index 000000000000..ca941a5feb3b --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/s3fs/tests/S3FileSystemMetricsTest.cpp @@ -0,0 +1,178 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#include + +#include "velox/common/memory/Memory.h" +#include "velox/connectors/hive/storage_adapters/s3fs/RegisterS3FileSystem.h" +#include "velox/connectors/hive/storage_adapters/s3fs/S3Counters.h" +#include "velox/connectors/hive/storage_adapters/s3fs/S3WriteFile.h" +#include "velox/connectors/hive/storage_adapters/s3fs/tests/S3Test.h" + +#include + +namespace facebook::velox::filesystems { +namespace { +class S3TestReporter : public BaseStatsReporter { + public: + mutable std::mutex m; + mutable std::map counterMap; + mutable std::unordered_map statTypeMap; + mutable std::unordered_map> + histogramPercentilesMap; + + void clear() { + std::lock_guard l(m); + counterMap.clear(); + statTypeMap.clear(); + histogramPercentilesMap.clear(); + } + void registerMetricExportType(const char* key, StatType statType) + const override { + statTypeMap[key] = statType; + } + + void registerMetricExportType(folly::StringPiece key, StatType statType) + const override { + statTypeMap[key.str()] = statType; + } + + void registerHistogramMetricExportType( + const char* key, + int64_t /* bucketWidth */, + int64_t /* min */, + int64_t /* max */, + const std::vector& pcts) const override { + histogramPercentilesMap[key] = pcts; + } + + void registerHistogramMetricExportType( + folly::StringPiece key, + int64_t /* bucketWidth */, + int64_t /* min */, + int64_t /* max */, + const std::vector& pcts) const override { + histogramPercentilesMap[key.str()] = pcts; + } + + void addMetricValue(const std::string& key, const size_t value) + const override { + std::lock_guard l(m); + counterMap[key] += value; + } + + void addMetricValue(const char* key, const size_t value) const override { + std::lock_guard l(m); + counterMap[key] += value; + } + + void addMetricValue(folly::StringPiece key, size_t value) const override { + std::lock_guard l(m); + counterMap[key.str()] += value; + } + + void addHistogramMetricValue(const std::string& key, size_t value) + const override { + counterMap[key] = std::max(counterMap[key], value); + } + + void addHistogramMetricValue(const char* key, size_t value) const override { + counterMap[key] = std::max(counterMap[key], value); + } + + void addHistogramMetricValue(folly::StringPiece key, size_t value) + const override { + counterMap[key.str()] = std::max(counterMap[key.str()], value); + } + + std::string fetchMetrics() override { + std::stringstream ss; + ss << "["; + auto sep = ""; + for (const auto& [key, value] : counterMap) { + ss << sep << key << ":" << value; + sep = ","; + } + ss << "]"; + return ss.str(); + } +}; + +folly::Singleton reporter([]() { + return new S3TestReporter(); +}); + +class S3FileSystemMetricsTest : public S3Test { + protected: + static void SetUpTestSuite() { + memory::MemoryManager::testingSetInstance(memory::MemoryManager::Options{}); + } + + void SetUp() override { + S3Test::SetUp(); + filesystems::initializeS3("Info"); + s3Reporter = std::dynamic_pointer_cast( + folly::Singleton::try_get()); + s3Reporter->clear(); + } + + static void TearDownTestSuite() { + filesystems::finalizeS3(); + } + std::shared_ptr s3Reporter; +}; + +} // namespace + +TEST_F(S3FileSystemMetricsTest, metrics) { + registerS3Metrics(); + + const auto bucketName = "metrics"; + const auto file = "test.txt"; + const auto filename = localPath(bucketName) + "/" + file; + const auto s3File = s3URI(bucketName, file); + auto hiveConfig = minioServer_->hiveConfig(); + S3FileSystem s3fs(bucketName, hiveConfig); + auto pool = memory::memoryManager()->addLeafPool("S3FileSystemMetricsTest"); + + auto writeFile = + s3fs.openFileForWrite(s3File, {{}, pool.get(), std::nullopt}); + EXPECT_EQ(1, s3Reporter->counterMap[std::string{kMetricS3MetadataCalls}]); + EXPECT_EQ(1, s3Reporter->counterMap[std::string{kMetricS3GetMetadataErrors}]); + + constexpr std::string_view kDataContent = + "Dance me to your beauty with a burning violin" + "Dance me through the panic till I'm gathered safely in" + "Lift me like an olive branch and be my homeward dove" + "Dance me to the end of love"; + writeFile->append(kDataContent); + writeFile->close(); + EXPECT_EQ(1, s3Reporter->counterMap[std::string{kMetricS3StartedUploads}]); + EXPECT_EQ(1, s3Reporter->counterMap[std::string{kMetricS3SuccessfulUploads}]); + + auto readFile = s3fs.openFileForRead(s3File); + EXPECT_EQ(2, s3Reporter->counterMap[std::string{kMetricS3MetadataCalls}]); + readFile->pread(0, kDataContent.length()); + EXPECT_EQ(1, s3Reporter->counterMap[std::string{kMetricS3GetObjectCalls}]); +} + +} // namespace facebook::velox::filesystems + +int main(int argc, char** argv) { + testing::InitGoogleTest(&argc, argv); + folly::Init init{&argc, &argv, false}; + BaseStatsReporter::registered = true; + return RUN_ALL_TESTS(); +} diff --git a/velox/connectors/lakehouse/storage_adapters/s3fs/tests/S3FileSystemRegistrationTest.cpp b/velox/connectors/lakehouse/storage_adapters/s3fs/tests/S3FileSystemRegistrationTest.cpp new file mode 100644 index 000000000000..256fe6e481e5 --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/s3fs/tests/S3FileSystemRegistrationTest.cpp @@ -0,0 +1,95 @@ +/* + * 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/storage_adapters/s3fs/RegisterS3FileSystem.h" +#include "velox/connectors/hive/storage_adapters/s3fs/tests/S3Test.h" + +namespace facebook::velox::filesystems { +namespace { + +std::string cacheKeyFunc( + std::shared_ptr config, + std::string_view path) { + return config->get("hive.s3.endpoint").value(); +} + +class S3FileSystemRegistrationTest : public S3Test { + protected: + static void SetUpTestCase() { + memory::MemoryManager::testingSetInstance(memory::MemoryManager::Options{}); + filesystems::registerS3FileSystem(cacheKeyFunc); + } + + static void TearDownTestCase() { + filesystems::finalizeS3FileSystem(); + } +}; +} // namespace + +TEST_F(S3FileSystemRegistrationTest, readViaRegistry) { + const char* bucketName = "data2"; + const char* file = "test.txt"; + const std::string filename = localPath(bucketName) + "/" + file; + const std::string s3File = s3URI(bucketName, file); + addBucket(bucketName); + { + LocalWriteFile writeFile(filename); + writeData(&writeFile); + } + auto hiveConfig = minioServer_->hiveConfig(); + { + auto s3fs = filesystems::getFileSystem(s3File, hiveConfig); + auto readFile = s3fs->openFileForRead(s3File); + readData(readFile.get()); + } +} + +TEST_F(S3FileSystemRegistrationTest, fileHandle) { + const char* bucketName = "data3"; + const char* file = "test.txt"; + const std::string filename = localPath(bucketName) + "/" + file; + const std::string s3File = s3URI(bucketName, file); + addBucket(bucketName); + { + LocalWriteFile writeFile(filename); + writeData(&writeFile); + } + auto hiveConfig = minioServer_->hiveConfig(); + FileHandleFactory factory( + std::make_unique>(1000), + std::make_unique(hiveConfig)); + FileHandleKey key{s3File}; + auto fileHandleCachePtr = factory.generate(key); + readData(fileHandleCachePtr->file.get()); +} + +TEST_F(S3FileSystemRegistrationTest, cacheKey) { + auto hiveConfig = minioServer_->hiveConfig(); + auto s3fs = filesystems::getFileSystem(kDummyPath, hiveConfig); + std::string_view kDummyPath2 = "s3://dummy2/foo.txt"; + auto s3fs_new = filesystems::getFileSystem(kDummyPath2, hiveConfig); + // The cacheKeyFunc function allows fs caching based on the endpoint value. + ASSERT_EQ(s3fs, s3fs_new); +} + +TEST_F(S3FileSystemRegistrationTest, finalize) { + auto hiveConfig = minioServer_->hiveConfig(); + auto s3fs = filesystems::getFileSystem(kDummyPath, hiveConfig); + VELOX_ASSERT_THROW( + filesystems::finalizeS3FileSystem(), + "Cannot finalize S3FileSystem while in use"); +} +} // namespace facebook::velox::filesystems diff --git a/velox/connectors/lakehouse/storage_adapters/s3fs/tests/S3FileSystemTest.cpp b/velox/connectors/lakehouse/storage_adapters/s3fs/tests/S3FileSystemTest.cpp new file mode 100644 index 000000000000..6e24372ac61a --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/s3fs/tests/S3FileSystemTest.cpp @@ -0,0 +1,352 @@ +/* + * 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 + +#include "velox/common/memory/Memory.h" +#include "velox/connectors/hive/storage_adapters/s3fs/RegisterS3FileSystem.h" +#include "velox/connectors/hive/storage_adapters/s3fs/S3WriteFile.h" +#include "velox/connectors/hive/storage_adapters/s3fs/tests/S3Test.h" + +#include + +namespace facebook::velox::filesystems { +namespace { + +class S3FileSystemTest : public S3Test { + protected: + static void SetUpTestCase() { + memory::MemoryManager::testingSetInstance(memory::MemoryManager::Options{}); + } + + void SetUp() override { + S3Test::SetUp(); + auto hiveConfig = minioServer_->hiveConfig({}); + filesystems::initializeS3("Info", kLogLocation_); + } + + static void TearDownTestSuite() { + filesystems::finalizeS3(); + } + + std::string_view kLogLocation_ = "/tmp/foobar/"; +}; + +class MyCredentialsProvider : public Aws::Auth::AWSCredentialsProvider { + public: + MyCredentialsProvider() = default; + + Aws::Auth::AWSCredentials GetAWSCredentials() override { + return Aws::Auth::AWSCredentials(); + } +}; + +} // namespace + +TEST_F(S3FileSystemTest, writeAndRead) { + /// The hive config used for Minio defaults to turning + /// off using proxy settings if the environment provides them. + setenv("HTTP_PROXY", "http://test:test@127.0.0.1:8888", 1); + const char* bucketName = "data"; + const char* file = "test.txt"; + const auto filename = localPath(bucketName) + "/" + file; + const auto s3File = s3URI(bucketName, file); + addBucket(bucketName); + { + LocalWriteFile writeFile(filename); + writeData(&writeFile); + } + auto hiveConfig = minioServer_->hiveConfig(); + filesystems::S3FileSystem s3fs(bucketName, hiveConfig); + auto readFile = s3fs.openFileForRead(s3File); + readData(readFile.get()); +} + +TEST_F(S3FileSystemTest, invalidCredentialsConfig) { + { + std::unordered_map config( + {{"hive.s3.use-instance-credentials", "true"}, + {"hive.s3.iam-role", "dummy-iam-role"}}); + auto hiveConfig = + std::make_shared(std::move(config)); + + // Both instance credentials and iam-role cannot be specified + VELOX_ASSERT_THROW( + filesystems::S3FileSystem("", hiveConfig), + "Invalid configuration: specify only one among 'access/secret keys', 'use instance credentials', 'IAM role'"); + } + { + std::unordered_map config( + {{"hive.s3.aws-secret-key", "dummy-key"}, + {"hive.s3.aws-access-key", "dummy-key"}, + {"hive.s3.iam-role", "dummy-iam-role"}}); + auto hiveConfig = + std::make_shared(std::move(config)); + // Both access/secret keys and iam-role cannot be specified + VELOX_ASSERT_THROW( + filesystems::S3FileSystem("", hiveConfig), + "Invalid configuration: specify only one among 'access/secret keys', 'use instance credentials', 'IAM role'"); + } + { + std::unordered_map config( + {{"hive.s3.aws-secret-key", "dummy"}, + {"hive.s3.aws-access-key", "dummy"}, + {"hive.s3.use-instance-credentials", "true"}}); + auto hiveConfig = + std::make_shared(std::move(config)); + // Both access/secret keys and instance credentials cannot be specified + VELOX_ASSERT_THROW( + filesystems::S3FileSystem("", hiveConfig), + "Invalid configuration: specify only one among 'access/secret keys', 'use instance credentials', 'IAM role'"); + } + { + std::unordered_map config( + {{"hive.s3.aws-secret-key", "dummy"}}); + auto hiveConfig = + std::make_shared(std::move(config)); + // Both access key and secret key must be specified + VELOX_ASSERT_THROW( + filesystems::S3FileSystem("", hiveConfig), + "Invalid configuration: both access key and secret key must be specified"); + } +} + +TEST_F(S3FileSystemTest, missingFile) { + const char* bucketName = "data1"; + const char* file = "i-do-not-exist.txt"; + const std::string s3File = s3URI(bucketName, file); + addBucket(bucketName); + auto hiveConfig = minioServer_->hiveConfig(); + filesystems::S3FileSystem s3fs(bucketName, hiveConfig); + VELOX_ASSERT_RUNTIME_THROW_CODE( + s3fs.openFileForRead(s3File), + error_code::kFileNotFound, + "Failed to get metadata for S3 object due to: 'Resource not found'. Path:'s3://data1/i-do-not-exist.txt', SDK Error Type:16, HTTP Status Code:404, S3 Service:'MinIO', Message:'No response body.'"); +} + +TEST_F(S3FileSystemTest, missingBucket) { + auto hiveConfig = minioServer_->hiveConfig(); + filesystems::S3FileSystem s3fs("", hiveConfig); + VELOX_ASSERT_RUNTIME_THROW_CODE( + s3fs.openFileForRead(kDummyPath), + error_code::kFileNotFound, + "Failed to get metadata for S3 object due to: 'Resource not found'. Path:'s3://dummy/foo.txt', SDK Error Type:16, HTTP Status Code:404, S3 Service:'MinIO', Message:'No response body.'"); +} + +TEST_F(S3FileSystemTest, invalidAccessKey) { + auto hiveConfig = + minioServer_->hiveConfig({{"hive.s3.aws-access-key", "dummy-key"}}); + filesystems::S3FileSystem s3fs("", hiveConfig); + // Minio credentials are wrong and this should throw + VELOX_ASSERT_THROW( + s3fs.openFileForRead(kDummyPath), + "Failed to get metadata for S3 object due to: 'Access denied'. Path:'s3://dummy/foo.txt', SDK Error Type:15, HTTP Status Code:403, S3 Service:'MinIO', Message:'No response body.'"); +} + +TEST_F(S3FileSystemTest, invalidSecretKey) { + auto hiveConfig = + minioServer_->hiveConfig({{"hive.s3.aws-secret-key", "dummy-key"}}); + filesystems::S3FileSystem s3fs("", hiveConfig); + // Minio credentials are wrong and this should throw. + VELOX_ASSERT_THROW( + s3fs.openFileForRead("s3://dummy/foo.txt"), + "Failed to get metadata for S3 object due to: 'Access denied'. Path:'s3://dummy/foo.txt', SDK Error Type:15, HTTP Status Code:403, S3 Service:'MinIO', Message:'No response body.'"); +} + +TEST_F(S3FileSystemTest, noBackendServer) { + auto hiveConfig = + minioServer_->hiveConfig({{"hive.s3.aws-secret-key", "dummy-key"}}); + filesystems::S3FileSystem s3fs("", hiveConfig); + // Stop Minio and check error. + minioServer_->stop(); + VELOX_ASSERT_THROW( + s3fs.openFileForRead(kDummyPath), + "Failed to get metadata for S3 object due to: 'Network connection'. Path:'s3://dummy/foo.txt', SDK Error Type:99, HTTP Status Code:-1, S3 Service:'Unknown', Message:'curlCode: 7, Couldn't connect to server'"); + // Start Minio again. + minioServer_->start(); +} + +TEST_F(S3FileSystemTest, logLevel) { + std::unordered_map config; + auto checkLogLevelName = [&config](std::string_view expected) { + auto s3Config = + std::make_shared(std::move(config)); + filesystems::S3FileSystem s3fs("", s3Config); + EXPECT_EQ(s3fs.getLogLevelName(), expected); + }; + + // Test is configured with INFO. + checkLogLevelName("INFO"); + + // S3 log level is set once during initialization. + // It does not change with a new config. + config["hive.s3.log-level"] = "Trace"; + checkLogLevelName("INFO"); +} + +TEST_F(S3FileSystemTest, logLocation) { + // From aws-cpp-sdk-core/include/aws/core/Aws.h . + std::string_view kDefaultPrefix = "aws_sdk_"; + std::unordered_map config; + auto checkLogPrefix = [&config](std::string_view expected) { + auto s3Config = + std::make_shared(std::move(config)); + filesystems::S3FileSystem s3fs("", s3Config); + EXPECT_EQ(s3fs.getLogPrefix(), expected); + }; + + const auto expected = fmt::format("{}{}", kLogLocation_, kDefaultPrefix); + // Test is configured with the default. + checkLogPrefix(expected); + + // S3 log location is set once during initialization. + // It does not change with a new config. + config["hive.s3.log-location"] = "/home/foobar"; + checkLogPrefix(expected); +} + +TEST_F(S3FileSystemTest, writeFileAndRead) { + const auto bucketName = "writedata"; + const auto file = "test.txt"; + const auto filename = localPath(bucketName) + "/" + file; + const auto s3File = s3URI(bucketName, file); + + auto hiveConfig = minioServer_->hiveConfig(); + filesystems::S3FileSystem s3fs(bucketName, hiveConfig); + auto pool = memory::memoryManager()->addLeafPool("S3FileSystemTest"); + auto writeFile = + s3fs.openFileForWrite(s3File, {{}, pool.get(), std::nullopt}); + auto s3WriteFile = dynamic_cast(writeFile.get()); + std::string dataContent = + "Dance me to your beauty with a burning violin" + "Dance me through the panic till I'm gathered safely in" + "Lift me like an olive branch and be my homeward dove" + "Dance me to the end of love"; + + EXPECT_EQ(writeFile->size(), 0); + std::int64_t contentSize = dataContent.length(); + // dataContent length is 178. + EXPECT_EQ(contentSize, 178); + + // Append and flush a small batch of data. + writeFile->append(dataContent.substr(0, 10)); + EXPECT_EQ(writeFile->size(), 10); + writeFile->append(dataContent.substr(10, contentSize - 10)); + EXPECT_EQ(writeFile->size(), contentSize); + writeFile->flush(); + // No parts must have been uploaded. + EXPECT_EQ(s3WriteFile->numPartsUploaded(), 0); + + // Append data 178 * 100'000 ~ 16MiB. + // Should have 1 part in total with kUploadPartSize = 10MiB. + for (int i = 0; i < 100'000; ++i) { + writeFile->append(dataContent); + } + EXPECT_EQ(s3WriteFile->numPartsUploaded(), 1); + EXPECT_EQ(writeFile->size(), 100'001 * contentSize); + + // Append a large data buffer 178 * 150'000 ~ 25MiB (2 parts). + std::vector largeBuffer(contentSize * 150'000); + for (int i = 0; i < 150'000; ++i) { + memcpy( + largeBuffer.data() + (i * contentSize), + dataContent.data(), + contentSize); + } + + writeFile->append({largeBuffer.data(), largeBuffer.size()}); + EXPECT_EQ(writeFile->size(), 250'001 * contentSize); + // Total data = ~41 MB = 5 parts. + // But parts uploaded will be 4. + EXPECT_EQ(s3WriteFile->numPartsUploaded(), 4); + + // Upload the last part. + writeFile->close(); + EXPECT_EQ(s3WriteFile->numPartsUploaded(), 5); + + VELOX_ASSERT_THROW( + writeFile->append(dataContent.substr(0, 10)), "File is closed"); + + auto readFile = s3fs.openFileForRead(s3File); + ASSERT_EQ(readFile->size(), contentSize * 250'001); + // Sample and verify every 1'000 dataContent chunks. + for (int i = 0; i < 250; ++i) { + ASSERT_EQ( + readFile->pread(i * (1'000 * contentSize), contentSize), dataContent); + } + // Verify the last chunk. + ASSERT_EQ(readFile->pread(contentSize * 250'000, contentSize), dataContent); + + // Verify the S3 list function. + auto result = s3fs.list(s3File); + + ASSERT_EQ(result.size(), 1); + ASSERT_TRUE(result[0] == file); + + ASSERT_TRUE(s3fs.exists(s3File)); +} + +TEST_F(S3FileSystemTest, invalidConnectionSettings) { + auto hiveConfig = + minioServer_->hiveConfig({{"hive.s3.connect-timeout", "400"}}); + VELOX_ASSERT_THROW( + filesystems::S3FileSystem("", hiveConfig), "Invalid duration"); + + hiveConfig = minioServer_->hiveConfig({{"hive.s3.socket-timeout", "abc"}}); + VELOX_ASSERT_THROW( + filesystems::S3FileSystem("", hiveConfig), "Invalid duration"); +} + +TEST_F(S3FileSystemTest, registerCredentialProviderFactories) { + const std::string credentialsProvider = "my-credentials-provider"; + const std::string invalidCredentialsProvider = "invalid-credentials-provider"; + registerAWSCredentialsProvider( + credentialsProvider, [](const S3Config& config) { + return std::make_shared(); + }); + + auto hiveConfig = minioServer_->hiveConfig( + {{"hive.s3.aws-credentials-provider", credentialsProvider}}); + ASSERT_NO_THROW(filesystems::S3FileSystem("", hiveConfig)); + + // Configure with unregistered credential provider. + hiveConfig = minioServer_->hiveConfig( + {{"hive.s3.aws-credentials-provider", invalidCredentialsProvider}}); + VELOX_ASSERT_THROW( + filesystems::S3FileSystem({"", hiveConfig}), + "CredentialsProviderFactory for 'invalid-credentials-provider' not registered"); + + // Register invalid credentials provider name. + VELOX_ASSERT_THROW( + registerAWSCredentialsProvider( + "", + [](const S3Config& config) { + return std::make_shared(); + }), + "CredentialsProviderFactory name cannot be empty"); + + // Register the same credential provider name again. + VELOX_ASSERT_THROW( + registerAWSCredentialsProvider( + credentialsProvider, + [](const S3Config& config) { + return std::make_shared(); + }), + "CredentialsProviderFactory 'my-credentials-provider' already registered"); +} + +} // namespace facebook::velox::filesystems diff --git a/velox/connectors/lakehouse/storage_adapters/s3fs/tests/S3InsertTest.cpp b/velox/connectors/lakehouse/storage_adapters/s3fs/tests/S3InsertTest.cpp new file mode 100644 index 000000000000..2f1e753e7baf --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/s3fs/tests/S3InsertTest.cpp @@ -0,0 +1,60 @@ +/* + * 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 + +#include "velox/connectors/hive/storage_adapters/s3fs/RegisterS3FileSystem.h" +#include "velox/connectors/hive/storage_adapters/s3fs/tests/S3Test.h" +#include "velox/connectors/hive/storage_adapters/test_common/InsertTest.h" + +namespace facebook::velox::filesystems { +namespace { + +class S3InsertTest : public S3Test, public test::InsertTest { + protected: + static void SetUpTestCase() { + memory::MemoryManager::testingSetInstance(memory::MemoryManager::Options{}); + } + + void SetUp() override { + S3Test::SetUp(); + filesystems::registerS3FileSystem(); + InsertTest::SetUp(minioServer_->hiveConfig(), ioExecutor_.get()); + } + + void TearDown() override { + S3Test::TearDown(); + InsertTest::TearDown(); + filesystems::finalizeS3FileSystem(); + } +}; +} // namespace + +TEST_F(S3InsertTest, s3InsertTest) { + const int64_t kExpectedRows = 1'000; + const std::string_view kOutputDirectory{"s3://writedata/"}; + minioServer_->addBucket("writedata"); + + runInsertTest(kOutputDirectory, kExpectedRows, pool()); +} +} // namespace facebook::velox::filesystems + +int main(int argc, char** argv) { + testing::InitGoogleTest(&argc, argv); + folly::Init init{&argc, &argv, false}; + return RUN_ALL_TESTS(); +} diff --git a/velox/connectors/lakehouse/storage_adapters/s3fs/tests/S3MultipleEndpointsTest.cpp b/velox/connectors/lakehouse/storage_adapters/s3fs/tests/S3MultipleEndpointsTest.cpp new file mode 100644 index 000000000000..4b22a912ce92 --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/s3fs/tests/S3MultipleEndpointsTest.cpp @@ -0,0 +1,239 @@ +/* + * 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 "gtest/gtest.h" +#include "velox/connectors/hive/HiveConnector.h" +#include "velox/connectors/hive/storage_adapters/s3fs/RegisterS3FileSystem.h" +#include "velox/connectors/hive/storage_adapters/s3fs/S3Util.h" +#include "velox/connectors/hive/storage_adapters/s3fs/tests/S3Test.h" +#include "velox/dwio/parquet/RegisterParquetReader.h" +#include "velox/dwio/parquet/RegisterParquetWriter.h" +#include "velox/exec/TableWriter.h" +#include "velox/exec/tests/utils/AssertQueryBuilder.h" +#include "velox/exec/tests/utils/PlanBuilder.h" + +static const std::string_view kConnectorId1 = "test-hive1"; +static const std::string_view kConnectorId2 = "test-hive2"; +static const std::string_view kBucketName = "writedata"; + +using namespace facebook::velox::exec::test; + +namespace facebook::velox { +namespace { + +class S3MultipleEndpoints : public S3Test, public ::test::VectorTestBase { + public: + static void SetUpTestCase() { + memory::MemoryManager::testingSetInstance(memory::MemoryManager::Options{}); + } + static void TearDownTestCase() { + filesystems::finalizeS3FileSystem(); + } + + void SetUp() override { + S3Test::SetUp(); + minioSecondServer_ = std::make_unique(); + minioSecondServer_->start(); + minioServer_->addBucket(kBucketName.data()); + minioSecondServer_->addBucket(kBucketName.data()); + + filesystems::registerS3FileSystem(); + connector::registerConnectorFactory( + std::make_shared()); + parquet::registerParquetReaderFactory(); + parquet::registerParquetWriterFactory(); + } + + void registerConnectors( + std::string_view connectorId1, + std::string_view connectorId2, + const std::unordered_map config1Override = {}, + const std::unordered_map config2Override = {}) { + auto hiveConnector1 = + connector::getConnectorFactory( + connector::hive::HiveConnectorFactory::kHiveConnectorName) + ->newConnector( + std::string(connectorId1), + minioServer_->hiveConfig(config1Override), + ioExecutor_.get()); + auto hiveConnector2 = + connector::getConnectorFactory( + connector::hive::HiveConnectorFactory::kHiveConnectorName) + ->newConnector( + std::string(connectorId2), + minioSecondServer_->hiveConfig(config2Override), + ioExecutor_.get()); + connector::registerConnector(hiveConnector1); + connector::registerConnector(hiveConnector2); + } + + void TearDown() override { + parquet::unregisterParquetReaderFactory(); + parquet::unregisterParquetWriterFactory(); + connector::unregisterConnectorFactory( + connector::hive::HiveConnectorFactory::kHiveConnectorName); + S3Test::TearDown(); + } + + folly::dynamic writeData( + const RowVectorPtr input, + const std::string& outputDirectory, + const std::string& connectorId) { + auto plan = PlanBuilder() + .values({input}) + .tableWrite( + outputDirectory.data(), + {}, + 0, + {}, + {}, + dwio::common::FileFormat::PARQUET, + {}, + connectorId) + .planNode(); + // Execute the write plan. + auto results = AssertQueryBuilder(plan).copyResults(pool()); + // Second column contains details about written files. + auto details = results->childAt(exec::TableWriteTraits::kFragmentChannel) + ->as>(); + folly::dynamic obj = folly::parseJson(details->valueAt(1)); + return obj["fileWriteInfos"]; + } + + std::shared_ptr createSplit( + folly::dynamic tableWriteInfo, + std::string outputDirectory, + std::string connectorId) { + auto writeFileName = tableWriteInfo[0]["writeFileName"].asString(); + auto filePath = fmt::format("{}{}", outputDirectory, writeFileName); + const int64_t fileSize = tableWriteInfo[0]["fileSize"].asInt(); + + return HiveConnectorSplitBuilder(filePath) + .connectorId(connectorId) + .fileFormat(dwio::common::FileFormat::PARQUET) + .length(fileSize) + .build(); + } + + void testJoin( + int numRows, + std::string_view outputDirectory, + std::string_view connectorId1, + std::string_view connectorId2) { + auto rowType1 = ROW( + {"a0", "a1", "a2", "a3"}, {BIGINT(), INTEGER(), SMALLINT(), DOUBLE()}); + auto rowType2 = ROW( + {"b0", "b1", "b2", "b3"}, {BIGINT(), INTEGER(), SMALLINT(), DOUBLE()}); + + auto input1 = makeRowVector( + rowType1->names(), + {makeFlatVector(numRows, [](auto row) { return row; }), + makeFlatVector(numRows, [](auto row) { return row; }), + makeFlatVector(numRows, [](auto row) { return row; }), + makeFlatVector(numRows, [](auto row) { return row; })}); + auto input2 = makeRowVector(rowType2->names(), input1->children()); + + // Insert input data into both tables. + auto table1WriteInfo = + writeData(input1, outputDirectory.data(), std::string(connectorId1)); + auto table2WriteInfo = + writeData(input2, outputDirectory.data(), std::string(connectorId2)); + + // Inner Join both the tables. + core::PlanNodeId scan1, scan2; + auto planNodeIdGenerator = std::make_shared(); + auto table1Scan = PlanBuilder(planNodeIdGenerator, pool()) + .startTableScan() + .tableName("hive_table1") + .outputType(rowType1) + .connectorId(std::string(connectorId1)) + .endTableScan() + .capturePlanNodeId(scan1) + .planNode(); + auto join = + PlanBuilder(planNodeIdGenerator, pool()) + .startTableScan() + .tableName("hive_table1") + .outputType(rowType2) + .connectorId(std::string(connectorId2)) + .endTableScan() + .capturePlanNodeId(scan2) + .hashJoin({"b0"}, {"a0"}, table1Scan, "", {"a0", "a1", "a2", "a3"}) + .planNode(); + + auto split1 = createSplit( + table1WriteInfo, outputDirectory.data(), std::string(connectorId1)); + auto split2 = createSplit( + table2WriteInfo, outputDirectory.data(), std::string(connectorId2)); + auto results = AssertQueryBuilder(join) + .split(scan1, split1) + .split(scan2, split2) + .copyResults(pool()); + assertEqualResults({input1}, {results}); + } + + std::unique_ptr minioSecondServer_; +}; +} // namespace + +TEST_F(S3MultipleEndpoints, baseEndpoints) { + const int64_t kExpectedRows = 1'000; + const auto outputDirectory{filesystems::s3URI(kBucketName, "")}; + + registerConnectors(kConnectorId1, kConnectorId2); + + testJoin(kExpectedRows, outputDirectory, kConnectorId1, kConnectorId2); + + connector::unregisterConnector(std::string(kConnectorId1)); + connector::unregisterConnector(std::string(kConnectorId2)); +} + +TEST_F(S3MultipleEndpoints, bucketEndpoints) { + const int64_t kExpectedRows = 1'000; + const auto outputDirectory{filesystems::s3URI(kBucketName, "")}; + + auto configOverride = [](std::shared_ptr config) { + return std::unordered_map{ + {"hive.s3.bucket.writedata.endpoint", + config->get("hive.s3.endpoint").value()}, + {"hive.s3.bucket.writedata.aws-access-key", + config->get("hive.s3.aws-access-key").value()}, + {"hive.s3.bucket.writedata.aws-secret-key", + config->get("hive.s3.aws-secret-key").value()}, + {"hive.s3.endpoint", "fail"}, + {"hive.s3.aws-access-key", "fail"}, + {"hive.s3.aws-secret-key", "fail"}, + }; + }; + auto config1 = configOverride(minioServer_->hiveConfig()); + auto config2 = configOverride(minioSecondServer_->hiveConfig()); + registerConnectors(kConnectorId1, kConnectorId2, config1, config2); + + testJoin(kExpectedRows, outputDirectory, kConnectorId1, kConnectorId2); + + connector::unregisterConnector(std::string(kConnectorId1)); + connector::unregisterConnector(std::string(kConnectorId2)); +} + +} // namespace facebook::velox + +int main(int argc, char** argv) { + testing::InitGoogleTest(&argc, argv); + folly::Init init{&argc, &argv, false}; + return RUN_ALL_TESTS(); +} diff --git a/velox/connectors/lakehouse/storage_adapters/s3fs/tests/S3ReadTest.cpp b/velox/connectors/lakehouse/storage_adapters/s3fs/tests/S3ReadTest.cpp new file mode 100644 index 000000000000..d292b5d35d8c --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/s3fs/tests/S3ReadTest.cpp @@ -0,0 +1,101 @@ +/* + * 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 + +#include "velox/common/memory/Memory.h" +#include "velox/connectors/hive/HiveConnector.h" +#include "velox/connectors/hive/storage_adapters/s3fs/RegisterS3FileSystem.h" +#include "velox/connectors/hive/storage_adapters/s3fs/tests/S3Test.h" +#include "velox/dwio/common/tests/utils/DataFiles.h" +#include "velox/dwio/parquet/RegisterParquetReader.h" +#include "velox/exec/tests/utils/AssertQueryBuilder.h" +#include "velox/exec/tests/utils/PlanBuilder.h" + +using namespace facebook::velox::exec::test; + +namespace facebook::velox::filesystems { +namespace { + +class S3ReadTest : public S3Test, public ::test::VectorTestBase { + protected: + static void SetUpTestCase() { + memory::MemoryManager::testingSetInstance(memory::MemoryManager::Options{}); + } + + void SetUp() override { + S3Test::SetUp(); + filesystems::registerS3FileSystem(); + connector::registerConnectorFactory( + std::make_shared()); + auto hiveConnector = + connector::getConnectorFactory( + connector::hive::HiveConnectorFactory::kHiveConnectorName) + ->newConnector(kHiveConnectorId, minioServer_->hiveConfig()); + connector::registerConnector(hiveConnector); + parquet::registerParquetReaderFactory(); + } + + void TearDown() override { + parquet::unregisterParquetReaderFactory(); + filesystems::finalizeS3FileSystem(); + connector::unregisterConnectorFactory( + connector::hive::HiveConnectorFactory::kHiveConnectorName); + connector::unregisterConnector(kHiveConnectorId); + S3Test::TearDown(); + } +}; +} // namespace + +TEST_F(S3ReadTest, s3ReadTest) { + const auto sourceFile = test::getDataFilePath( + "velox/connectors/hive/storage_adapters/s3fs/tests", + "../../../../../dwio/parquet/tests/examples/int.parquet"); + const char* bucketName = "data"; + const auto destinationFile = S3Test::localPath(bucketName) + "/int.parquet"; + minioServer_->addBucket(bucketName); + std::ifstream src(sourceFile, std::ios::binary); + std::ofstream dest(destinationFile, std::ios::binary); + // Copy source file to destination bucket. + dest << src.rdbuf(); + ASSERT_GT(dest.tellp(), 0) << "Unable to copy from source " << sourceFile; + dest.close(); + + // Read the parquet file via the S3 bucket. + auto rowType = ROW({"int", "bigint"}, {INTEGER(), BIGINT()}); + auto plan = PlanBuilder().tableScan(rowType).planNode(); + auto split = HiveConnectorSplitBuilder(s3URI(bucketName, "int.parquet")) + .fileFormat(dwio::common::FileFormat::PARQUET) + .build(); + auto copy = AssertQueryBuilder(plan).split(split).copyResults(pool()); + + // expectedResults is the data in int.parquet file. + const int64_t kExpectedRows = 10; + auto expectedResults = makeRowVector( + {makeFlatVector( + kExpectedRows, [](auto row) { return row + 100; }), + makeFlatVector( + kExpectedRows, [](auto row) { return row + 1000; })}); + assertEqualResults({expectedResults}, {copy}); +} +} // namespace facebook::velox::filesystems + +int main(int argc, char** argv) { + testing::InitGoogleTest(&argc, argv); + folly::Init init{&argc, &argv, false}; + return RUN_ALL_TESTS(); +} diff --git a/velox/connectors/lakehouse/storage_adapters/s3fs/tests/S3Test.h b/velox/connectors/lakehouse/storage_adapters/s3fs/tests/S3Test.h new file mode 100644 index 000000000000..6a190684e2b9 --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/s3fs/tests/S3Test.h @@ -0,0 +1,101 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "velox/common/base/tests/GTestUtils.h" +#include "velox/common/file/File.h" +#include "velox/connectors/hive/FileHandle.h" +#include "velox/connectors/hive/storage_adapters/s3fs/S3FileSystem.h" +#include "velox/connectors/hive/storage_adapters/s3fs/S3Util.h" +#include "velox/connectors/hive/storage_adapters/s3fs/tests/MinioServer.h" +#include "velox/exec/tests/utils/HiveConnectorTestBase.h" +#include "velox/exec/tests/utils/TempFilePath.h" + +#include "gtest/gtest.h" + +using namespace facebook::velox; + +constexpr int kOneMB = 1 << 20; + +static constexpr std::string_view kDummyPath = "s3://dummy/foo.txt"; + +class S3Test : public testing::Test { + protected: + void SetUp() override { + minioServer_ = std::make_unique(); + minioServer_->start(); + ioExecutor_ = std::make_unique(3); + } + + void TearDown() override { + minioServer_->stop(); + } + + void addBucket(const char* bucket) { + minioServer_->addBucket(bucket); + } + + std::string localPath(const char* directory) { + return minioServer_->path() + "/" + directory; + } + + void writeData(WriteFile* writeFile) { + writeFile->append("aaaaa"); + writeFile->append("bbbbb"); + writeFile->append(std::string(kOneMB, 'c')); + writeFile->append("ddddd"); + ASSERT_EQ(writeFile->size(), 15 + kOneMB); + } + + void readData(ReadFile* readFile) { + ASSERT_EQ(readFile->size(), 15 + kOneMB); + char buffer1[5]; + ASSERT_EQ(readFile->pread(10 + kOneMB, 5, &buffer1), "ddddd"); + char buffer2[10]; + ASSERT_EQ(readFile->pread(0, 10, &buffer2), "aaaaabbbbb"); + char buffer3[kOneMB]; + ASSERT_EQ(readFile->pread(10, kOneMB, &buffer3), std::string(kOneMB, 'c')); + ASSERT_EQ(readFile->size(), 15 + kOneMB); + char buffer4[10]; + const std::string_view arf = readFile->pread(5, 10, &buffer4); + const std::string zarf = readFile->pread(kOneMB, 15); + auto buf = std::make_unique(8); + const std::string_view warf = readFile->pread(4, 8, buf.get()); + const std::string_view warfFromBuf(buf.get(), 8); + ASSERT_EQ(arf, "bbbbbccccc"); + ASSERT_EQ(zarf, "ccccccccccddddd"); + ASSERT_EQ(warf, "abbbbbcc"); + ASSERT_EQ(warfFromBuf, "abbbbbcc"); + char head[12]; + char middle[4]; + char tail[7]; + std::vector> buffers = { + folly::Range(head, sizeof(head)), + folly::Range(nullptr, (char*)(uint64_t)500000), + folly::Range(middle, sizeof(middle)), + folly::Range( + nullptr, + (char*)(uint64_t)(15 + kOneMB - 500000 - sizeof(head) - + sizeof(middle) - sizeof(tail))), + folly::Range(tail, sizeof(tail))}; + ASSERT_EQ(15 + kOneMB, readFile->preadv(0, buffers)); + ASSERT_EQ(std::string_view(head, sizeof(head)), "aaaaabbbbbcc"); + ASSERT_EQ(std::string_view(middle, sizeof(middle)), "cccc"); + ASSERT_EQ(std::string_view(tail, sizeof(tail)), "ccddddd"); + } + + std::unique_ptr minioServer_; + std::unique_ptr ioExecutor_; +}; diff --git a/velox/connectors/lakehouse/storage_adapters/s3fs/tests/S3UtilTest.cpp b/velox/connectors/lakehouse/storage_adapters/s3fs/tests/S3UtilTest.cpp new file mode 100644 index 000000000000..c71065ba2cda --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/s3fs/tests/S3UtilTest.cpp @@ -0,0 +1,269 @@ +/* + * 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/storage_adapters/s3fs/S3Util.h" + +#include "gtest/gtest.h" + +namespace facebook::velox::filesystems { + +// TODO: Each prefix should be implemented as its own filesystem. +TEST(S3UtilTest, isS3File) { + EXPECT_FALSE(isS3File("ss3://")); + EXPECT_FALSE(isS3File("s3:/")); + EXPECT_FALSE(isS3File("oss:")); + EXPECT_FALSE(isS3File("cos:")); + EXPECT_FALSE(isS3File("cosn:")); + EXPECT_FALSE(isS3File("S3A://bucket/some/file.txt")); + EXPECT_FALSE(isS3File("OSS://other-bucket/some/file.txt")); + EXPECT_FALSE(isS3File("COS://other-bucket/some/file.txt")); + EXPECT_FALSE(isS3File("COSN://other-bucket/some/file.txt")); + EXPECT_FALSE(isS3File("s3::/bucket")); + EXPECT_FALSE(isS3File("s3:/bucket")); + EXPECT_FALSE(isS3File("file://bucket")); + EXPECT_TRUE(isS3File("s3://bucket/file.txt")); + EXPECT_TRUE(isS3File("s3n://bucket/file.txt")); +} + +TEST(S3UtilTest, isS3AwsFile) { + EXPECT_FALSE(isS3AwsFile("s3:")); + EXPECT_FALSE(isS3AwsFile("s3::/bucket")); + EXPECT_FALSE(isS3AwsFile("s3:/bucket")); + EXPECT_TRUE(isS3AwsFile("s3://bucket/file.txt")); +} + +TEST(S3UtilTest, isS3aFile) { + EXPECT_FALSE(isS3aFile("s3a:")); + EXPECT_FALSE(isS3aFile("s3a::/bucket")); + EXPECT_FALSE(isS3aFile("s3a:/bucket")); + EXPECT_FALSE(isS3aFile("S3A://bucket-name/file.txt")); + EXPECT_TRUE(isS3aFile("s3a://bucket/file.txt")); +} + +TEST(S3UtilTest, isS3nFile) { + EXPECT_FALSE(isS3nFile("s3n:")); + EXPECT_FALSE(isS3nFile("s3n::/bucket")); + EXPECT_FALSE(isS3nFile("s3n:/bucket")); + EXPECT_FALSE(isS3nFile("S3N://bucket-name/file.txt")); + EXPECT_TRUE(isS3nFile("s3n://bucket/file.txt")); +} + +TEST(S3UtilTest, isOssFile) { + EXPECT_FALSE(isOssFile("oss:")); + EXPECT_FALSE(isOssFile("oss::/bucket")); + EXPECT_FALSE(isOssFile("oss:/bucket")); + EXPECT_FALSE(isOssFile("OSS://BUCKET/sub-key/file.txt")); + EXPECT_TRUE(isOssFile("oss://bucket/file.txt")); +} + +TEST(S3UtilTest, isCosFile) { + EXPECT_FALSE(isCosFile("cos:")); + EXPECT_FALSE(isCosFile("cos::/bucket")); + EXPECT_FALSE(isCosFile("cos:/bucket")); + EXPECT_FALSE(isCosFile("COS://BUCKET/sub-key/file.txt")); + EXPECT_TRUE(isCosFile("cos://bucket/file.txt")); +} + +TEST(S3UtilTest, isCosNFile) { + EXPECT_FALSE(isCosNFile("cosn:")); + EXPECT_FALSE(isCosNFile("cosn::/bucket")); + EXPECT_FALSE(isCosNFile("cosn:/bucket")); + EXPECT_FALSE(isCosNFile("COSN://BUCKET/sub-key/file.txt")); + EXPECT_TRUE(isCosNFile("cosn://bucket/file.txt")); +} + +TEST(S3UtilTest, s3Path) { + auto path_0 = getPath("s3://bucket/file.txt"); + auto path_1 = getPath("oss://bucket-name/file.txt"); + auto path_2 = getPath("S3A://bucket-NAME/sub-PATH/my-file.txt"); + auto path_3 = getPath("s3N://bucket-NAME/sub-PATH/my-file.txt"); + auto path_4 = getPath("cos://bucket-name/file.txt"); + auto path_5 = getPath("cosn://bucket-name/file.txt"); + EXPECT_EQ(path_0, "bucket/file.txt"); + EXPECT_EQ(path_1, "bucket-name/file.txt"); + EXPECT_NE(path_2, "bucket-NAME/sub-PATH/my-file.txt"); + EXPECT_NE(path_3, "bucket-NAME/sub-PATH/my-file.txt"); + EXPECT_EQ(path_4, "bucket-name/file.txt"); + EXPECT_EQ(path_5, "bucket-name/file.txt"); +} + +TEST(S3UtilTest, bucketAndKeyFromgetPath) { + std::string bucket, key; + auto path = "bucket/file.txt"; + getBucketAndKeyFromPath(path, bucket, key); + EXPECT_EQ(bucket, "bucket"); + EXPECT_EQ(key, "file.txt"); +} + +TEST(S3UtilTest, isDomainExcludedFromProxy) { + auto hostname = "test.foobar.com"; + + std::vector> tests = { + {"localhost,.foobar.com", true}, + {"localhost,.,foobar.com,.com", true}, + {"localhost,test.foobar.com", true}, + {"localhost,foobar.com,*.com", true}, + {"localhost,*.foobar.com", true}, + {"localhost", false}, + {"localhost,foobar.com", false}, + {"", false}, + }; + + for (auto pair : tests) { + EXPECT_EQ(isHostExcludedFromProxy(hostname, pair.first), pair.second); + } +} + +TEST(S3UtilTest, parseAWSRegion) { + // bucket.s3.[region] + EXPECT_EQ( + parseAWSStandardRegionName("foo.s3.region.amazonaws.com"), "region"); + EXPECT_EQ( + parseAWSStandardRegionName("foo.s3.region.amazonaws.com/"), "region"); + // bucket.s3-[region] + EXPECT_EQ( + parseAWSStandardRegionName("foo.s3-region.amazonaws.com"), "region"); + EXPECT_EQ( + parseAWSStandardRegionName("foo.s3-region.amazonaws.com/"), "region"); + // service.[region] + EXPECT_EQ(parseAWSStandardRegionName("foo.a3-reg.amazonaws.com"), "a3-reg"); + EXPECT_EQ(parseAWSStandardRegionName("foo.a3-reg.amazonaws.com/"), "a3-reg"); + // Not the right suffix + EXPECT_EQ( + parseAWSStandardRegionName("foo.a3-region.amazon.com"), std::nullopt); + EXPECT_EQ(parseAWSStandardRegionName(""), std::nullopt); + EXPECT_EQ(parseAWSStandardRegionName("velox"), std::nullopt); +} + +TEST(S3UtilTest, isIpExcludedFromProxy) { + auto hostname = "127.0.0.1"; + + std::vector> tests = { + {"localhost,127.0.0.1,.foobar.com", true}, + {"localhost,127.0.0.0/24,.foobar.com", true}, + {"localhost,foobar.com,127.0.0.0/16,.1,.com", true}, + {"localhost,foobar.com,.1,.com", true}, + {"localhost,test.foobar.com", false}, + {"localhost,foobar.com,*.1,*.com", true}, + {"localhost", false}, + {"localhost,127.1.0.1", false}, + {"", false}, + }; + + for (auto pair : tests) { + EXPECT_EQ(isHostExcludedFromProxy(hostname, pair.first), pair.second) + << pair.first; + } +} + +class S3UtilProxyTest : public ::testing::TestWithParam {}; + +TEST_P(S3UtilProxyTest, proxyBuilderBadEndpoint) { + auto s3Endpoint = "http://127.0.0.1:8888"; + auto useSsl = GetParam(); + + setenv("HTTP_PROXY", "http://127.0.0.1:12345", 1); + setenv("HTTPS_PROXY", "http://127.0.0.1:12345", 1); + EXPECT_FALSE(S3ProxyConfigurationBuilder(s3Endpoint) + .useSsl(useSsl) + .build() + .has_value()); +} + +TEST_P(S3UtilProxyTest, proxyBuilderNoProxy) { + auto s3Endpoint = "127.0.0.1:8888"; + auto useSsl = GetParam(); + + setenv("HTTP_PROXY", "", 1); + setenv("HTTPS_PROXY", "", 1); + EXPECT_FALSE(S3ProxyConfigurationBuilder(s3Endpoint) + .useSsl(useSsl) + .build() + .has_value()); +} + +TEST_P(S3UtilProxyTest, proxyBuilderSameHttpProxy) { + auto s3Endpoint = "192.168.0.1:12345"; + auto useSsl = GetParam(); + + setenv("HTTP_PROXY", "http://127.0.0.1:8888", 1); + setenv("HTTPS_PROXY", "http://127.0.0.1:8888", 1); + auto proxyConfig = + S3ProxyConfigurationBuilder(s3Endpoint).useSsl(useSsl).build(); + ASSERT_TRUE(proxyConfig.has_value()); + EXPECT_EQ(proxyConfig.value().scheme(), "http"); + EXPECT_EQ(proxyConfig.value().host(), "127.0.0.1"); + EXPECT_EQ(proxyConfig.value().port(), 8888); + EXPECT_EQ(proxyConfig.value().username(), ""); + EXPECT_EQ(proxyConfig.value().password(), ""); +} + +TEST_P(S3UtilProxyTest, proxyBuilderMixProxy) { + auto s3Endpoint = "192.168.0.1:12345"; + auto useSsl = GetParam(); + + const std::string httpProxy = "https://test1:testpw1@80.67.3.1:35631"; + setenv("HTTP_PROXY", httpProxy.c_str(), 1); + EXPECT_EQ(getHttpProxyEnvVar(), httpProxy) + << "HTTP_PROXY environment variable not set."; + const std::string httpsProxy = "http://test2:testpw2@80.80.5.1:45631"; + setenv("HTTPS_PROXY", httpsProxy.c_str(), 1); + EXPECT_EQ(getHttpsProxyEnvVar(), httpsProxy) + << "HTTPS_PROXY environment variable not set."; + auto proxyConfig = + S3ProxyConfigurationBuilder(s3Endpoint).useSsl(useSsl).build(); + ASSERT_TRUE(proxyConfig.has_value()); + EXPECT_EQ(proxyConfig.value().scheme(), (useSsl ? "http" : "https")); + EXPECT_EQ(proxyConfig.value().host(), (useSsl ? "80.80.5.1" : "80.67.3.1")); + EXPECT_EQ(proxyConfig.value().port(), (useSsl ? 45631 : 35631)); + EXPECT_EQ(proxyConfig.value().username(), (useSsl ? "test2" : "test1")); + EXPECT_EQ(proxyConfig.value().password(), (useSsl ? "testpw2" : "testpw1")); +} + +TEST_P(S3UtilProxyTest, proxyBuilderMixProxyLowerCase) { + auto s3Endpoint = "192.168.0.1:12345"; + auto useSsl = GetParam(); + + const std::string lcHttpProxy = "https://lctest1:lctestpw1@80.67.3.1:35631"; + const std::string ucHttpProxy = "https://uctest1:uctestpw1@80.67.3.2:35632"; + setenv("http_proxy", lcHttpProxy.c_str(), 1); + setenv("HTTP_PROXY", ucHttpProxy.c_str(), 1); + // Lower case value takes precedence. + EXPECT_EQ(getHttpProxyEnvVar(), lcHttpProxy) + << "http_proxy environment variable not set."; + const std::string lcHttpsProxy = "http://lctest2:lctestpw2@80.80.5.1:45631"; + const std::string ucHttpsProxy = "http://uctest2:uctestpw2@80.80.5.2:45632"; + setenv("https_proxy", lcHttpsProxy.c_str(), 1); + setenv("HTTPS_PROXY", ucHttpsProxy.c_str(), 1); + EXPECT_EQ(getHttpsProxyEnvVar(), lcHttpsProxy) + << "https_proxy environment variable not set."; + auto proxyConfig = + S3ProxyConfigurationBuilder(s3Endpoint).useSsl(useSsl).build(); + ASSERT_TRUE(proxyConfig.has_value()); + EXPECT_EQ(proxyConfig.value().scheme(), (useSsl ? "http" : "https")); + EXPECT_EQ(proxyConfig.value().host(), (useSsl ? "80.80.5.1" : "80.67.3.1")); + EXPECT_EQ(proxyConfig.value().port(), (useSsl ? 45631 : 35631)); + EXPECT_EQ(proxyConfig.value().username(), (useSsl ? "lctest2" : "lctest1")); + EXPECT_EQ( + proxyConfig.value().password(), (useSsl ? "lctestpw2" : "lctestpw1")); +} + +INSTANTIATE_TEST_SUITE_P( + S3UtilTest, + S3UtilProxyTest, + ::testing::Values(true, false)); + +} // namespace facebook::velox::filesystems diff --git a/velox/connectors/lakehouse/storage_adapters/test_common/InsertTest.h b/velox/connectors/lakehouse/storage_adapters/test_common/InsertTest.h new file mode 100644 index 000000000000..3fa6ca056d67 --- /dev/null +++ b/velox/connectors/lakehouse/storage_adapters/test_common/InsertTest.h @@ -0,0 +1,117 @@ +/* + * 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 + +#include "velox/common/memory/Memory.h" +#include "velox/connectors/hive/HiveConnector.h" +#include "velox/dwio/parquet/RegisterParquetReader.h" +#include "velox/dwio/parquet/RegisterParquetWriter.h" +#include "velox/exec/TableWriter.h" +#include "velox/exec/tests/utils/AssertQueryBuilder.h" +#include "velox/exec/tests/utils/HiveConnectorTestBase.h" +#include "velox/exec/tests/utils/PlanBuilder.h" +#include "velox/vector/tests/utils/VectorTestBase.h" + +namespace facebook::velox::test { + +class InsertTest : public velox::test::VectorTestBase { + protected: + void SetUp( + std::shared_ptr hiveConfig, + folly::Executor* ioExecutor) { + connector::registerConnectorFactory( + std::make_shared()); + auto hiveConnector = + connector::getConnectorFactory( + connector::hive::HiveConnectorFactory::kHiveConnectorName) + ->newConnector( + exec::test::kHiveConnectorId, hiveConfig, ioExecutor); + connector::registerConnector(hiveConnector); + + parquet::registerParquetReaderFactory(); + parquet::registerParquetWriterFactory(); + } + + void TearDown() { + parquet::unregisterParquetReaderFactory(); + parquet::unregisterParquetWriterFactory(); + connector::unregisterConnectorFactory( + connector::hive::HiveConnectorFactory::kHiveConnectorName); + connector::unregisterConnector(exec::test::kHiveConnectorId); + } + + void runInsertTest( + std::string_view outputDirectory, + int numRows, + memory::MemoryPool* pool) { + auto rowType = ROW( + {"c0", "c1", "c2", "c3"}, {BIGINT(), INTEGER(), SMALLINT(), DOUBLE()}); + + auto input = makeRowVector( + {makeFlatVector(numRows, [](auto row) { return row; }), + makeFlatVector(numRows, [](auto row) { return row; }), + makeFlatVector(numRows, [](auto row) { return row; }), + makeFlatVector(numRows, [](auto row) { return row; })}); + + // Insert with one writer. + auto plan = + exec::test::PlanBuilder() + .values({input}) + .tableWrite( + outputDirectory.data(), dwio::common::FileFormat::PARQUET) + .planNode(); + + // Execute the write plan. + auto results = exec::test::AssertQueryBuilder(plan).copyResults(pool); + + // First column has number of rows written in the first row and nulls in + // other rows. + auto rowCount = results->childAt(exec::TableWriteTraits::kRowCountChannel) + ->as>(); + ASSERT_FALSE(rowCount->isNullAt(0)); + ASSERT_EQ(numRows, rowCount->valueAt(0)); + ASSERT_TRUE(rowCount->isNullAt(1)); + + // Second column contains details about written files. + auto details = results->childAt(exec::TableWriteTraits::kFragmentChannel) + ->as>(); + ASSERT_TRUE(details->isNullAt(0)); + ASSERT_FALSE(details->isNullAt(1)); + folly::dynamic obj = folly::parseJson(details->valueAt(1)); + + ASSERT_EQ(numRows, obj["rowCount"].asInt()); + auto fileWriteInfos = obj["fileWriteInfos"]; + ASSERT_EQ(1, fileWriteInfos.size()); + + auto writeFileName = fileWriteInfos[0]["writeFileName"].asString(); + + // Read from 'writeFileName' and verify the data matches the original. + plan = exec::test::PlanBuilder().tableScan(rowType).planNode(); + + auto filePath = fmt::format("{}{}", outputDirectory, writeFileName); + const int64_t fileSize = fileWriteInfos[0]["fileSize"].asInt(); + auto split = exec::test::HiveConnectorSplitBuilder(filePath) + .fileFormat(dwio::common::FileFormat::PARQUET) + .length(fileSize) + .build(); + auto copy = + exec::test::AssertQueryBuilder(plan).split(split).copyResults(pool); + exec::test::assertEqualResults({input}, {copy}); + } +}; +} // namespace facebook::velox::test