diff --git a/.clang-tidy b/.clang-tidy index 4824fc2f2ed1..1a810ecbd0b2 100644 --- a/.clang-tidy +++ b/.clang-tidy @@ -28,6 +28,7 @@ Checks: [ '-bugprone-unchecked-optional-access', '-bugprone-crtp-constructor-accessibility', '-bugprone-not-null-terminated-result', + '-bugprone-forward-declaration-namespace', '-cert-dcl16-c', '-cert-err58-cpp', diff --git a/ci/jobs/scripts/check_style/aspell-ignore/en/aspell-dict.txt b/ci/jobs/scripts/check_style/aspell-ignore/en/aspell-dict.txt index a94681f791ca..501587372d6a 100644 --- a/ci/jobs/scripts/check_style/aspell-ignore/en/aspell-dict.txt +++ b/ci/jobs/scripts/check_style/aspell-ignore/en/aspell-dict.txt @@ -800,8 +800,6 @@ PagerDuty ParallelFormattingOutputFormatThreads ParallelFormattingOutputFormatThreadsActive ParallelParsingInputFormat -ParallelParsingInputFormatThreads -ParallelParsingInputFormatThreadsActive ParallelReplicasMode ParquetCompression ParquetMetadata diff --git a/docs/en/operations/system-tables/metrics.md b/docs/en/operations/system-tables/metrics.md index 89466dca65e0..8463ce7d9822 100644 --- a/docs/en/operations/system-tables/metrics.md +++ b/docs/en/operations/system-tables/metrics.md @@ -524,14 +524,6 @@ Number of threads in the ParallelFormattingOutputFormatThreads thread pool. Number of threads in the ParallelFormattingOutputFormatThreads thread pool running a task. -### ParallelParsingInputFormatThreads {#parallelparsinginputformatthreads} - -Number of threads in the ParallelParsingInputFormat thread pool. - -### ParallelParsingInputFormatThreadsActive {#parallelparsinginputformatthreadsactive} - -Number of threads in the ParallelParsingInputFormat thread pool running a task. - ### PartMutation {#partmutation} Number of mutations (ALTER DELETE/UPDATE) diff --git a/programs/keeper-bench/Runner.cpp b/programs/keeper-bench/Runner.cpp index 32764bfdc611..69f43637c600 100644 --- a/programs/keeper-bench/Runner.cpp +++ b/programs/keeper-bench/Runner.cpp @@ -576,8 +576,7 @@ struct ZooKeeperRequestFromLogReader context, context->getSettingsRef()[DB::Setting::max_block_size], format_settings, - 1, - std::nullopt, + DB::FormatParserGroup::singleThreaded(context->getSettingsRef()), /*is_remote_fs*/ false, DB::CompressionMethod::None, false); diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index c49501190a78..3ae856dabedb 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -129,6 +129,9 @@ namespace ServerSetting extern const ServerSettingsUInt64 max_prefixes_deserialization_thread_pool_size; extern const ServerSettingsUInt64 max_prefixes_deserialization_thread_pool_free_size; extern const ServerSettingsUInt64 prefixes_deserialization_thread_pool_thread_pool_queue_size; + extern const ServerSettingsUInt64 max_format_parsing_thread_pool_size; + extern const ServerSettingsUInt64 max_format_parsing_thread_pool_free_size; + extern const ServerSettingsUInt64 format_parsing_thread_pool_queue_size; } namespace ErrorCodes @@ -266,6 +269,11 @@ void LocalServer::initialize(Poco::Util::Application & self) server_settings[ServerSetting::max_prefixes_deserialization_thread_pool_size], server_settings[ServerSetting::max_prefixes_deserialization_thread_pool_free_size], server_settings[ServerSetting::prefixes_deserialization_thread_pool_thread_pool_queue_size]); + + getFormatParsingThreadPool().initialize( + server_settings[ServerSetting::max_format_parsing_thread_pool_size], + server_settings[ServerSetting::max_format_parsing_thread_pool_free_size], + server_settings[ServerSetting::format_parsing_thread_pool_queue_size]); } diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index cb2286cc28e8..3247dfe2f514 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -316,6 +316,9 @@ namespace ServerSetting extern const ServerSettingsUInt64 max_prefixes_deserialization_thread_pool_size; extern const ServerSettingsUInt64 max_prefixes_deserialization_thread_pool_free_size; extern const ServerSettingsUInt64 prefixes_deserialization_thread_pool_thread_pool_queue_size; + extern const ServerSettingsUInt64 max_format_parsing_thread_pool_size; + extern const ServerSettingsUInt64 max_format_parsing_thread_pool_free_size; + extern const ServerSettingsUInt64 format_parsing_thread_pool_queue_size; extern const ServerSettingsUInt64 page_cache_history_window_ms; extern const ServerSettingsString page_cache_policy; extern const ServerSettingsDouble page_cache_size_ratio; @@ -1365,6 +1368,11 @@ try server_settings[ServerSetting::max_prefixes_deserialization_thread_pool_free_size], server_settings[ServerSetting::prefixes_deserialization_thread_pool_thread_pool_queue_size]); + getFormatParsingThreadPool().initialize( + server_settings[ServerSetting::max_format_parsing_thread_pool_size], + server_settings[ServerSetting::max_format_parsing_thread_pool_free_size], + server_settings[ServerSetting::format_parsing_thread_pool_queue_size]); + std::string path_str = getCanonicalPath(config().getString("path", DBMS_DEFAULT_PATH)); fs::path path = path_str; @@ -2088,6 +2096,11 @@ try new_server_settings[ServerSetting::max_prefixes_deserialization_thread_pool_free_size], new_server_settings[ServerSetting::prefixes_deserialization_thread_pool_thread_pool_queue_size]); + getFormatParsingThreadPool().reloadConfiguration( + new_server_settings[ServerSetting::max_format_parsing_thread_pool_size], + new_server_settings[ServerSetting::max_format_parsing_thread_pool_free_size], + new_server_settings[ServerSetting::format_parsing_thread_pool_queue_size]); + global_context->setMergeWorkload(new_server_settings[ServerSetting::merge_workload]); global_context->setMutationWorkload(new_server_settings[ServerSetting::mutation_workload]); global_context->setThrowOnUnknownWorkload(new_server_settings[ServerSetting::throw_on_unknown_workload]); diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index e482072ad1a8..81b72b77b6fa 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -70,9 +70,11 @@ #include #include #include +#include #include #include +#include #include #include diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index ea3bd714930f..eb843a8b97d2 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -153,9 +153,6 @@ M(ParallelFormattingOutputFormatThreads, "Number of threads in the ParallelFormattingOutputFormatThreads thread pool.") \ M(ParallelFormattingOutputFormatThreadsActive, "Number of threads in the ParallelFormattingOutputFormatThreads thread pool running a task.") \ M(ParallelFormattingOutputFormatThreadsScheduled, "Number of queued or active jobs in the ParallelFormattingOutputFormatThreads thread pool.") \ - M(ParallelParsingInputFormatThreads, "Number of threads in the ParallelParsingInputFormat thread pool.") \ - M(ParallelParsingInputFormatThreadsActive, "Number of threads in the ParallelParsingInputFormat thread pool running a task.") \ - M(ParallelParsingInputFormatThreadsScheduled, "Number of queued or active jobs in the ParallelParsingInputFormat thread pool.") \ M(MergeTreeBackgroundExecutorThreads, "Number of threads in the MergeTreeBackgroundExecutor thread pool.") \ M(MergeTreeBackgroundExecutorThreadsActive, "Number of threads in the MergeTreeBackgroundExecutor thread pool running a task.") \ M(MergeTreeBackgroundExecutorThreadsScheduled, "Number of queued or active jobs in the MergeTreeBackgroundExecutor thread pool.") \ @@ -239,21 +236,15 @@ M(QueryPipelineExecutorThreads, "Number of threads in the PipelineExecutor thread pool.") \ M(QueryPipelineExecutorThreadsActive, "Number of threads in the PipelineExecutor thread pool running a task.") \ M(QueryPipelineExecutorThreadsScheduled, "Number of queued or active jobs in the PipelineExecutor thread pool.") \ - M(ParquetDecoderThreads, "Number of threads in the ParquetBlockInputFormat thread pool.") \ - M(ParquetDecoderThreadsActive, "Number of threads in the ParquetBlockInputFormat thread pool running a task.") \ - M(ParquetDecoderThreadsScheduled, "Number of queued or active jobs in the ParquetBlockInputFormat thread pool.") \ - M(ParquetDecoderIOThreads, "Number of threads in the ParquetBlockInputFormat io thread pool.") \ - M(ParquetDecoderIOThreadsActive, "Number of threads in the ParquetBlockInputFormat io thread pool running a task.") \ - M(ParquetDecoderIOThreadsScheduled, "Number of queued or active jobs in the ParquetBlockInputFormat io thread pool.") \ M(ParquetEncoderThreads, "Number of threads in ParquetBlockOutputFormat thread pool.") \ M(ParquetEncoderThreadsActive, "Number of threads in ParquetBlockOutputFormat thread pool running a task.") \ M(ParquetEncoderThreadsScheduled, "Number of queued or active jobs in ParquetBlockOutputFormat thread pool.") \ M(MergeTreeSubcolumnsReaderThreads, "Number of threads in the thread pool used for subcolumns reading in MergeTree.") \ M(MergeTreeSubcolumnsReaderThreadsActive, "Number of threads in the thread pool used for subcolumns reading in MergeTree running a task.") \ M(MergeTreeSubcolumnsReaderThreadsScheduled, "Number of queued or active jobs in the thread pool used for subcolumns reading in MergeTree.") \ - M(DWARFReaderThreads, "Number of threads in the DWARFBlockInputFormat thread pool.") \ - M(DWARFReaderThreadsActive, "Number of threads in the DWARFBlockInputFormat thread pool running a task.") \ - M(DWARFReaderThreadsScheduled, "Number of queued or active jobs in the DWARFBlockInputFormat thread pool.") \ + M(FormatParsingThreads, "Number of threads in the thread pool used for parsing input.") \ + M(FormatParsingThreadsActive, "Number of threads in the thread pool used for parsing input running a task.") \ + M(FormatParsingThreadsScheduled, "Number of queued or active jobs in the thread pool used for parsing input.") \ M(OutdatedPartsLoadingThreads, "Number of threads in the threadpool for loading Outdated data parts.") \ M(OutdatedPartsLoadingThreadsActive, "Number of active threads in the threadpool for loading Outdated data parts.") \ M(OutdatedPartsLoadingThreadsScheduled, "Number of queued or active jobs in the threadpool for loading Outdated data parts.") \ diff --git a/src/Common/threadPoolCallbackRunner.cpp b/src/Common/threadPoolCallbackRunner.cpp new file mode 100644 index 000000000000..a49b612150ae --- /dev/null +++ b/src/Common/threadPoolCallbackRunner.cpp @@ -0,0 +1,260 @@ +#include + +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +ThreadPoolCallbackRunnerFast::ThreadPoolCallbackRunnerFast() = default; + +void ThreadPoolCallbackRunnerFast::initThreadPool(ThreadPool & pool_, size_t max_threads_, std::string thread_name_, ThreadGroupPtr thread_group_) +{ + chassert(!pool); + mode = Mode::ThreadPool; + pool = &pool_; + max_threads = max_threads_; + thread_name = thread_name_; + thread_group = thread_group_; + + /// We could dynamically add and remove threads based on load, but it's not clear whether it's + /// worth the added complexity. + for (size_t i = 0; i < max_threads; ++i) + { + pool->scheduleOrThrowOnError([this] { threadFunction(); }); + ++threads; // only if scheduleOrThrowOnError didn't throw + } +} + +ThreadPoolCallbackRunnerFast::ThreadPoolCallbackRunnerFast(Mode mode_) : mode(mode_) +{ + chassert(mode != Mode::ThreadPool); +} + +ThreadPoolCallbackRunnerFast::~ThreadPoolCallbackRunnerFast() +{ + shutdown(); +} + +void ThreadPoolCallbackRunnerFast::shutdown() +{ + /// May be called multiple times. + std::unique_lock lock(mutex); + shutdown_requested = true; +#ifdef OS_LINUX + const UInt32 a_lot = UINT32_MAX / 4; + queue_size += a_lot; + futexWake(&queue_size, a_lot); +#else + queue_cv.notify_all(); +#endif + shutdown_cv.wait(lock, [&] { return threads == 0; }); +} + +void ThreadPoolCallbackRunnerFast::operator()(std::function f) +{ + if (mode == Mode::Disabled) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Thread pool runner is not initialized"); + + { + std::unique_lock lock(mutex); + queue.push_back(std::move(f)); + } + + if (mode == Mode::ThreadPool) + { +#ifdef OS_LINUX + UInt32 prev_size = queue_size.fetch_add(1, std::memory_order_release); + if (prev_size < max_threads) + futexWake(&queue_size, 1); +#else + queue_cv.notify_one(); +#endif + } +} + +void ThreadPoolCallbackRunnerFast::bulkSchedule(std::vector> fs) +{ + if (fs.empty()) + return; + + if (mode == Mode::Disabled) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Thread pool runner is not initialized"); + + { + std::unique_lock lock(mutex); + queue.insert(queue.end(), std::move_iterator(fs.begin()), std::move_iterator(fs.end())); + } + + if (mode == Mode::ThreadPool) + { +#ifdef OS_LINUX + UInt32 prev_size = queue_size.fetch_add(fs.size(), std::memory_order_release); + if (prev_size < max_threads) + futexWake(&queue_size, fs.size()); +#else + if (fs.size() < 4) + for (size_t i = 0; i < fs.size(); ++i) + queue_cv.notify_one(); + else + queue_cv.notify_all(); +#endif + } +} + +bool ThreadPoolCallbackRunnerFast::runTaskInline() +{ + std::function f; + { + std::unique_lock lock(mutex); + if (queue.empty()) + return false; + f = std::move(queue.front()); + queue.pop_front(); + } + f(); + return true; +} + +void ThreadPoolCallbackRunnerFast::threadFunction() +{ + ThreadGroupSwitcher switcher(thread_group, thread_name.c_str()); + + while (true) + { +#ifdef OS_LINUX + UInt32 x = queue_size.load(std::memory_order_relaxed); + while (true) + { + if (x == 0) + { + futexWait(&queue_size, 0); + x = queue_size.load(std::memory_order_relaxed); + } + else if (queue_size.compare_exchange_weak( + x, x - 1, std::memory_order_acquire, std::memory_order_relaxed)) + break; + } +#endif + + std::function f; + { + std::unique_lock lock(mutex); + +#ifndef OS_LINUX + queue_cv.wait(lock, [&] { return shutdown_requested || !queue.empty(); }); +#endif + + if (shutdown_requested) + { + threads -= 1; + if (threads == 0) + shutdown_cv.notify_all(); + return; + } + + chassert(!queue.empty()); + + f = std::move(queue.front()); + queue.pop_front(); + } + + try + { + f(); + + CurrentThread::updatePerformanceCountersIfNeeded(); + } + catch (...) + { + tryLogCurrentException("FastThreadPool"); + chassert(false); + } + } +} + +bool ShutdownHelper::try_lock_shared() +{ + Int64 n = val.fetch_add(1, std::memory_order_acquire) + 1; + chassert(n != SHUTDOWN_START); + if (n >= SHUTDOWN_START) + { + unlock_shared(); + return false; + } + return true; +} + +void ShutdownHelper::unlock_shared() +{ + Int64 n = val.fetch_sub(1, std::memory_order_release) - 1; + chassert(n >= 0); + if (n == SHUTDOWN_START) + { + /// We're the last completed task. Add SHUTDOWN_END to indicate that no further waiting + /// or cv notifying is needed, even though `val` can get briefly bumped up and down by + /// unsuccessful try_lock_shared() calls. + val.fetch_add(SHUTDOWN_END); + { + /// Lock and unlock the mutex. This may look weird, but this is usually (always?) + /// required to avoid race conditions when combining condition_variable with atomics. + /// + /// In this case, the prevented race condition is: + /// 1. unlock_shared() sees n == SHUTDOWN_START, + /// 2. shutdown thread enters cv.wait(lock, [&] { return val.load() >= SHUTDOWN_END; }); + /// the callback does val.load(), gets SHUTDOWN_START, and is about + /// to return false; at this point, the cv.wait call is not monitoring + /// condition_variable notifications (remember that cv.wait with callback is + /// equivalent to a wait without callback in a loop), + /// 3. the unlock_shared() assigns `val` and calls cv.notify_all(), which does + /// nothing because no thread is blocked on the condition variable, + /// 4. the cv.wait callback returns false; the wait goes back to sleep and never + /// wakes up. + std::unique_lock lock(mutex); + } + cv.notify_all(); + } +} + +bool ShutdownHelper::shutdown_requested() +{ + return val.load(std::memory_order_relaxed) >= SHUTDOWN_START; +} + +bool ShutdownHelper::begin_shutdown() +{ + Int64 n = val.fetch_add(SHUTDOWN_START) + SHUTDOWN_START; + bool already_called = n >= SHUTDOWN_START * 2; + if (already_called) + n = val.fetch_sub(SHUTDOWN_START) - SHUTDOWN_START; + if (n == SHUTDOWN_START) + { + val.fetch_add(SHUTDOWN_END); + { + std::unique_lock lock(mutex); + } + cv.notify_all(); + } + return !already_called; +} + +void ShutdownHelper::wait_shutdown() +{ + std::unique_lock lock(mutex); + cv.wait(lock, [&] { return val.load() >= SHUTDOWN_END; }); +} + +void ShutdownHelper::shutdown() +{ + begin_shutdown(); + wait_shutdown(); +} + +template ThreadPoolCallbackRunnerUnsafe threadPoolCallbackRunnerUnsafe(ThreadPool &, const std::string &); +template class ThreadPoolCallbackRunnerLocal; + +} diff --git a/src/Common/threadPoolCallbackRunner.h b/src/Common/threadPoolCallbackRunner.h index 594eec2f7e00..8d861c20bbd0 100644 --- a/src/Common/threadPoolCallbackRunner.h +++ b/src/Common/threadPoolCallbackRunner.h @@ -226,7 +226,144 @@ class ThreadPoolCallbackRunnerLocal final tasks.clear(); } +}; + +/// Has a task queue and a set of threads from ThreadPool. +/// Per-task overhead is lower than in ThreadPool because ThreadGroup is not switched, stack trace is +/// not propagated, etc. +/// ThreadPool is ok for maybe thousands of tasks per second. +/// ThreadPoolCallbackRunnerFast is ok for maybe tens of thousands. +/// (For hundreds of thousands you'd want a faster queue and some tricks to avoid doing FUTEX_WAKE a lot. +/// But more importantly you'd want to reconsider the design to avoid having such small tasks.) +class ThreadPoolCallbackRunnerFast +{ +public: + enum class Mode + { + /// Normal mode, tasks are queued and ran by a thread pool. + ThreadPool, + /// Tasks can be enqueued, but there's no thread pool to pick them up. + /// You have to call runTaskInline() to run them. + Manual, + /// operator() will throw. + Disabled, + }; + + /// TODO [parquet]: Add metrics for queue size and active threads, and maybe event for tasks executed. + + ThreadPoolCallbackRunnerFast(); + + void initManual() + { + mode = Mode::Manual; + } + + void initThreadPool(ThreadPool & pool_, size_t max_threads_, std::string thread_name_, ThreadGroupPtr thread_group_); + + /// Manual or Disabled. + explicit ThreadPoolCallbackRunnerFast(Mode mode_); + + ~ThreadPoolCallbackRunnerFast(); + + void shutdown(); + void operator()(std::function f); + + void bulkSchedule(std::vector> fs); + + /// Returns true if a task was run, false if queue is empty. + bool runTaskInline(); + + Mode getMode() const { return mode; } + size_t getMaxThreads() const { return mode == Mode::ThreadPool ? max_threads : 0; } + bool isDisabled() const { return mode == Mode::Disabled; } + bool isManual() const { return mode == Mode::Manual; } + +private: + Mode mode = Mode::Disabled; + ThreadPool * pool = nullptr; + size_t max_threads = 0; + std::string thread_name; + ThreadGroupPtr thread_group; + + std::mutex mutex; + size_t threads = 0; + bool shutdown_requested = false; + std::condition_variable shutdown_cv; + + std::deque> queue; + +#ifdef OS_LINUX + /// Use futex when available. It's faster than condition_variable, especially on the enqueue side. + std::atomic queue_size {0}; +#else + std::condition_variable queue_cv; +#endif + + void threadFunction(); }; +/// Usage: +/// +/// struct Foo +/// { +/// std::shared_ptr shutdown = std::make_shared(); +/// +/// ~Foo() +/// { +/// shutdown->shutdown(); +/// +/// // No running background tasks remain. +/// // Some tasks may be left in thread pool queues; these tasks will detect +/// // shutdown and return early without accessing `this`. +/// } +/// +/// void someBackgroundTask(std::shared_ptr shutdown_) +/// { +/// std::shared_lock shutdown_lock(*shutdown, std::try_to_lock); +/// if (!shutdown_lock.owns_lock()) +/// return; // shutdown was requested, `this` may be destroyed +/// +/// // `this` is safe to access as long as `shutdown_lock` is held. +/// } +/// } +/// +/// Fun fact: ShutdownHelper can almost be replaced with std::shared_mutex. +/// Background tasks would do try_lock_shared(). Shutdown would do lock() and never unlock. +/// Alas, std::shared_mutex::try_lock_shared() is allowed to spuriously fail, so this doesn't work. +/// (In Common/SharedMutex.h, the futex-based implementation has reliable try_lock_shared(), but the +/// fallback absl implementation can fail spuriously. In Common/CancelableSharedMutex.h there's +/// another suitable futex-based linux-only implementation.) +class ShutdownHelper +{ +public: + bool try_lock_shared(); + void unlock_shared(); + + /// For re-checking in the middle of long-running operation while already holding a lock. + bool shutdown_requested(); + + /// Returns false if shutdown was already requested before. + bool begin_shutdown(); + void wait_shutdown(); + + /// Equivalent to `begin_shutdown(); end_shutdown();`. Ok to call multiple times. + void shutdown(); + +private: + static constexpr Int64 SHUTDOWN_START = 1l << 42; // shutdown requested + static constexpr Int64 SHUTDOWN_END = 1l << 52; // no shared locks remain + + /// If >= SHUTDOWN_START, no new try_lock_shared() calls will succeed. + /// Whoever changes the value to exactly SHUTDOWN_START (i.e. shutdown requested, no shared locks) + /// must then add SHUTDOWN_END to it. + /// Note that SHUTDOWN_END might be added multiple times because of benign race conditions. + std::atomic val {0}; + std::mutex mutex; + std::condition_variable cv; +}; + +extern template ThreadPoolCallbackRunnerUnsafe threadPoolCallbackRunnerUnsafe(ThreadPool &, const std::string &); +extern template class ThreadPoolCallbackRunnerLocal; + } diff --git a/src/Core/ServerSettings.cpp b/src/Core/ServerSettings.cpp index fc964e21892f..45403316d8a3 100644 --- a/src/Core/ServerSettings.cpp +++ b/src/Core/ServerSettings.cpp @@ -89,6 +89,19 @@ namespace DB DECLARE(UInt64, prefixes_deserialization_thread_pool_thread_pool_queue_size, 10000, R"( The maximum number of jobs that can be scheduled on the prefixes deserialization Thread pool. + :::note + A value of `0` means unlimited. + ::: + )", 0) \ + DECLARE(UInt64, max_format_parsing_thread_pool_size, 100, R"( + Maximum total number of threads to use for parsing input. + )", 0) \ + DECLARE(UInt64, max_format_parsing_thread_pool_free_size, 0, R"( + Maximum number of idle standby threads to keep in the thread pool for parsing input. + )", 0) \ + DECLARE(UInt64, format_parsing_thread_pool_queue_size, 10000, R"( + The maximum number of jobs that can be scheduled on thread pool for parsing input. + :::note A value of `0` means unlimited. ::: @@ -1248,6 +1261,12 @@ void ServerSettings::dumpToSystemServerSettingsColumns(ServerSettingColumnsParam {getMergeTreePrefixesDeserializationThreadPool().isInitialized() ? std::to_string(getMergeTreePrefixesDeserializationThreadPool().get().getMaxFreeThreads()) : "0", ChangeableWithoutRestart::Yes}}, {"prefixes_deserialization_thread_pool_thread_pool_queue_size", {getMergeTreePrefixesDeserializationThreadPool().isInitialized() ? std::to_string(getMergeTreePrefixesDeserializationThreadPool().get().getQueueSize()) : "0", ChangeableWithoutRestart::Yes}}, + {"max_format_parsing_thread_pool_size", + {getFormatParsingThreadPool().isInitialized() ? std::to_string(getFormatParsingThreadPool().get().getMaxThreads()) : "0", ChangeableWithoutRestart::Yes}}, + {"max_format_parsing_thread_pool_free_size", + {getFormatParsingThreadPool().isInitialized() ? std::to_string(getFormatParsingThreadPool().get().getMaxFreeThreads()) : "0", ChangeableWithoutRestart::Yes}}, + {"format_parsing_thread_pool_queue_size", + {getFormatParsingThreadPool().isInitialized() ? std::to_string(getFormatParsingThreadPool().get().getQueueSize()) : "0", ChangeableWithoutRestart::Yes}}, }; if (context->areBackgroundExecutorsInitialized()) diff --git a/src/Databases/DataLake/DatabaseDataLake.cpp b/src/Databases/DataLake/DatabaseDataLake.cpp index 96bebea69ea1..94fe50c2d7e1 100644 --- a/src/Databases/DataLake/DatabaseDataLake.cpp +++ b/src/Databases/DataLake/DatabaseDataLake.cpp @@ -176,7 +176,7 @@ std::shared_ptr DatabaseDataLake::getCatalog() const return catalog_impl; } -std::shared_ptr DatabaseDataLake::getConfiguration( +std::shared_ptr DatabaseDataLake::getConfiguration( DatabaseDataLakeStorageType type, DataLakeStorageSettingsPtr storage_settings) const { @@ -428,7 +428,7 @@ StoragePtr DatabaseDataLake::tryGetTableImpl(const String & name, ContextPtr con /// with_table_structure = false: because there will be /// no table structure in table definition AST. - StorageObjectStorage::Configuration::initialize(*configuration, args, context_copy, /* with_table_structure */false); + StorageObjectStorageConfiguration::initialize(*configuration, args, context_copy, /* with_table_structure */false); return std::make_shared( configuration, diff --git a/src/Databases/DataLake/DatabaseDataLake.h b/src/Databases/DataLake/DatabaseDataLake.h index fbdd0e3d9b32..b787288341f3 100644 --- a/src/Databases/DataLake/DatabaseDataLake.h +++ b/src/Databases/DataLake/DatabaseDataLake.h @@ -72,7 +72,7 @@ class DatabaseDataLake final : public IDatabase, WithContext void validateSettings(); std::shared_ptr getCatalog() const; - std::shared_ptr getConfiguration( + std::shared_ptr getConfiguration( DatabaseDataLakeStorageType type, DataLakeStorageSettingsPtr storage_settings) const; diff --git a/src/Databases/DataLake/GlueCatalog.cpp b/src/Databases/DataLake/GlueCatalog.cpp index 4a397a28e79d..40e976f4a62c 100644 --- a/src/Databases/DataLake/GlueCatalog.cpp +++ b/src/Databases/DataLake/GlueCatalog.cpp @@ -441,7 +441,7 @@ bool GlueCatalog::classifyTimestampTZ(const String & column_name, const TableMet auto storage_settings = std::make_shared(); storage_settings->loadFromSettingsChanges(settings.allChanged()); auto configuration = std::make_shared(storage_settings); - DB::StorageObjectStorage::Configuration::initialize(*configuration, args, getContext(), false); + DB::StorageObjectStorageConfiguration::initialize(*configuration, args, getContext(), false); auto object_storage = configuration->createObjectStorage(getContext(), true); const auto & read_settings = getContext()->getReadSettings(); diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index db22070bd7f6..b0a6e5c04bbe 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -2,6 +2,7 @@ #include #include +#include #include #include #include @@ -38,10 +39,8 @@ FORMAT_FACTORY_SETTINGS(DECLARE_FORMAT_EXTERN, INITIALIZE_SETTING_EXTERN) extern const SettingsBool input_format_parallel_parsing; extern const SettingsBool log_queries; extern const SettingsUInt64 max_download_buffer_size; - extern const SettingsMaxThreads max_download_threads; extern const SettingsSeconds max_execution_time; extern const SettingsUInt64 max_parser_depth; - extern const SettingsMaxThreads max_parsing_threads; extern const SettingsUInt64 max_memory_usage; extern const SettingsUInt64 max_memory_usage_for_user; extern const SettingsMaxThreads max_threads; @@ -371,8 +370,7 @@ InputFormatPtr FormatFactory::getInput( const ContextPtr & context, UInt64 max_block_size, const std::optional & _format_settings, - std::optional _max_parsing_threads, - std::optional _max_download_threads, + std::shared_ptr parser_group, bool is_remote_fs, CompressionMethod compression, bool need_only_count) const @@ -381,10 +379,19 @@ InputFormatPtr FormatFactory::getInput( if (!creators.input_creator && !creators.random_access_input_creator) throw Exception(ErrorCodes::FORMAT_IS_NOT_SUITABLE_FOR_INPUT, "Format {} is not suitable for input", name); + /// Some formats use this thread pool. Lazily initialize it. + /// This doesn't affect server and clickhouse-local, they initialize threads pools on startup. + getFormatParsingThreadPool().initializeWithDefaultSettingsIfNotInitialized(); + auto format_settings = _format_settings ? *_format_settings : getFormatSettings(context); const Settings & settings = context->getSettingsRef(); - size_t max_parsing_threads = _max_parsing_threads.value_or(settings[Setting::max_parsing_threads]); - size_t max_download_threads = _max_download_threads.value_or(settings[Setting::max_download_threads]); + + if (!parser_group) + parser_group = std::make_shared( + settings, + /*num_streams_=*/ 1, + /*filter_actions_dag_=*/ nullptr, + /*context_=*/ nullptr); RowInputFormatParams row_input_format_params; row_input_format_params.max_block_size = max_block_size; @@ -399,11 +406,12 @@ InputFormatPtr FormatFactory::getInput( // Add ParallelReadBuffer and decompression if needed. - auto owned_buf = wrapReadBufferIfNeeded(_buf, compression, creators, format_settings, settings, is_remote_fs, max_download_threads); + auto owned_buf = wrapReadBufferIfNeeded(_buf, compression, creators, format_settings, settings, is_remote_fs, parser_group); auto & buf = owned_buf ? *owned_buf : _buf; // Decide whether to use ParallelParsingInputFormat. + size_t max_parsing_threads = parser_group->getParsingThreadsPerReader(); bool parallel_parsing = max_parsing_threads > 1 && settings[Setting::input_format_parallel_parsing] && creators.file_segmentation_engine_creator && !creators.random_access_input_creator && !need_only_count; @@ -435,6 +443,8 @@ InputFormatPtr FormatFactory::getInput( (ReadBuffer & input) -> InputFormatPtr { return input_getter(input, sample, row_input_format_params, format_settings); }; + /// TODO: Try using parser_group->parsing_runner instead of creating a ThreadPool in + /// ParallelParsingInputFormat. ParallelParsingInputFormat::Params params{ buf, sample, @@ -452,7 +462,7 @@ InputFormatPtr FormatFactory::getInput( else if (creators.random_access_input_creator) { format = creators.random_access_input_creator( - buf, sample, format_settings, context->getReadSettings(), is_remote_fs, max_download_threads, max_parsing_threads); + buf, sample, format_settings, context->getReadSettings(), is_remote_fs, parser_group); } else { @@ -485,10 +495,11 @@ std::unique_ptr FormatFactory::wrapReadBufferIfNeeded( const FormatSettings & format_settings, const Settings & settings, bool is_remote_fs, - size_t max_download_threads) const + const FormatParserGroupPtr & parser_group) const { std::unique_ptr res; + size_t max_download_threads = parser_group->getIOThreadsPerReader(); bool parallel_read = is_remote_fs && max_download_threads > 1 && format_settings.seekable_read && isBufferWithFileSize(buf); if (creators.random_access_input_creator) parallel_read &= compression != CompressionMethod::None; @@ -520,6 +531,7 @@ std::unique_ptr FormatFactory::wrapReadBufferIfNeeded( max_download_threads, settings[Setting::max_download_buffer_size].value); + /// TODO: Consider using parser_group->io_runner instead of threadPoolCallbackRunnerUnsafe. res = wrapInParallelReadBufferIfSupported( buf, threadPoolCallbackRunnerUnsafe(getIOThreadPool().get(), "ParallelRead"), diff --git a/src/Formats/FormatFactory.h b/src/Formats/FormatFactory.h index b308eab7738b..45d85457d6ea 100644 --- a/src/Formats/FormatFactory.h +++ b/src/Formats/FormatFactory.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include #include @@ -48,6 +49,8 @@ using RowOutputFormatPtr = std::shared_ptr; template struct Memory; +struct FormatParserGroup; + FormatSettings getFormatSettings(const ContextPtr & context); FormatSettings getFormatSettings(const ContextPtr & context, const Settings & settings); @@ -97,8 +100,7 @@ class FormatFactory final : private boost::noncopyable const FormatSettings & settings, const ReadSettings & read_settings, bool is_remote_fs, - size_t max_download_threads, - size_t max_parsing_threads)>; + FormatParserGroupPtr parser_group)>; using OutputCreator = std::function & format_settings = std::nullopt, - std::optional max_parsing_threads = std::nullopt, - std::optional max_download_threads = std::nullopt, + std::shared_ptr parser_group = nullptr, // affects things like buffer sizes and parallel reading bool is_remote_fs = false, // allows to do: buf -> parallel read -> decompression, @@ -287,7 +289,7 @@ class FormatFactory final : private boost::noncopyable const FormatSettings & format_settings, const Settings & settings, bool is_remote_fs, - size_t max_download_threads) const; + const FormatParserGroupPtr & parser_group) const; }; } diff --git a/src/Formats/FormatParserGroup.cpp b/src/Formats/FormatParserGroup.cpp new file mode 100644 index 000000000000..9c1f0223ffc7 --- /dev/null +++ b/src/Formats/FormatParserGroup.cpp @@ -0,0 +1,117 @@ +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +namespace Setting +{ + extern const SettingsMaxThreads max_download_threads; + extern const SettingsMaxThreads max_parsing_threads; +} + +void ColumnMapper::setStorageColumnEncoding(std::unordered_map && storage_encoding_) +{ + storage_encoding = std::move(storage_encoding_); +} + +std::pair, std::unordered_map> ColumnMapper::makeMapping( + const Block & header, + const std::unordered_map & format_encoding) +{ + std::unordered_map clickhouse_to_parquet_names; + std::unordered_map parquet_names_to_clickhouse; + + for (size_t i = 0; i < header.columns(); ++i) + { + auto column_name = header.getNames()[i]; + int64_t field_id; + if (auto it = storage_encoding.find(column_name); it != storage_encoding.end()) + field_id = it->second; + else + continue; + clickhouse_to_parquet_names[column_name] = format_encoding.at(field_id); + parquet_names_to_clickhouse.emplace(format_encoding.at(field_id), column_name); + } + return {clickhouse_to_parquet_names, parquet_names_to_clickhouse}; +} + +FormatParserGroup::FormatParserGroup(const Settings & settings, size_t num_streams_, std::shared_ptr filter_actions_dag_, const ContextPtr & context_) + : max_parsing_threads(settings[Setting::max_parsing_threads]) + , max_io_threads(settings[Setting::max_download_threads]) + , num_streams(num_streams_) + , filter_actions_dag(std::move(filter_actions_dag_)) + , context(ContextWeakPtr(context_)) {} + +FormatParserGroupPtr FormatParserGroup::singleThreaded(const Settings & settings) +{ + auto parser_group = std::make_shared(settings, 1, nullptr, nullptr); + parser_group->max_parsing_threads = 1; + return parser_group; +} + +bool FormatParserGroup::hasFilter() const +{ + return filter_actions_dag != nullptr; +} + +void FormatParserGroup::finishStream() +{ + num_streams.fetch_sub(1, std::memory_order_relaxed); +} + +size_t FormatParserGroup::getParsingThreadsPerReader() const +{ + size_t n = num_streams.load(std::memory_order_relaxed); + n = std::max(n, 1ul); + return (max_parsing_threads + n - 1) / n; +} + +size_t FormatParserGroup::getIOThreadsPerReader() const +{ + size_t n = num_streams.load(std::memory_order_relaxed); + n = std::max(n, 1ul); + return (max_io_threads + n - 1) / n; +} + +void FormatParserGroup::initKeyCondition(const Block & keys) +{ + if (!filter_actions_dag) + return; + + auto ctx = context.lock(); + if (!ctx) throw Exception(ErrorCodes::LOGICAL_ERROR, "Context has expired"); + + ActionsDAGWithInversionPushDown inverted_dag(filter_actions_dag->getOutputs().front(), ctx); + key_condition = std::make_shared( + inverted_dag, ctx, keys.getNames(), + std::make_shared(ActionsDAG(keys.getColumnsWithTypeAndName()))); +} + +void FormatParserGroup::initOnce(std::function f) +{ + std::call_once(init_flag, [&] + { + if (init_exception) + std::rethrow_exception(init_exception); + + try + { + f(); + } + catch (...) + { + init_exception = std::current_exception(); + throw; + } + }); +} + +} diff --git a/src/Formats/FormatParserGroup.h b/src/Formats/FormatParserGroup.h new file mode 100644 index 000000000000..be4a03d5ccc9 --- /dev/null +++ b/src/Formats/FormatParserGroup.h @@ -0,0 +1,89 @@ +#pragma once + +#include +#include + +namespace DB +{ + +class ActionsDAG; +struct Settings; +class KeyCondition; +struct FormatParserGroup; + +using FormatParserGroupPtr = std::shared_ptr; + +/// Some formats needs to custom mapping between columns in file and clickhouse columns. +class ColumnMapper +{ +public: + /// clickhouse_column_name -> field_id + void setStorageColumnEncoding(std::unordered_map && storage_encoding_); + + /// clickhouse_column_name -> format_column_name (just join the maps above by field_id). + std::pair, std::unordered_map> makeMapping( + const Block & header, + const std::unordered_map & format_encoding); + +private: + std::unordered_map storage_encoding; +}; + +using ColumnMapperPtr = std::shared_ptr; + +/// When reading many files in one query, e.g. `SELECT ... FROM file('part{00..99}.parquet')`, +/// we want the file readers to share some resource limits, e.g. number of threads. +/// They may also want to share some data structures to avoid initializing multiple copies, +/// e.g. KeyCondition. +/// This struct is shared among such group of readers (IInputFormat instances). +/// All nontrivial parts of this struct are lazily initialized by the IInputFormat implementation, +/// because most implementations don't use most of this struct. +struct FormatParserGroup +{ + /// Total limits across all readers in the group. + size_t max_parsing_threads = 0; + size_t max_io_threads = 0; + + std::atomic num_streams {0}; + + std::shared_ptr filter_actions_dag; + ContextWeakPtr context; // required only if `filter_actions_dag` is set + /// TODO: std::optional prewhere_actions; + + ThreadPoolCallbackRunnerFast parsing_runner; + ThreadPoolCallbackRunnerFast io_runner; + + /// Optionally created from filter_actions_dag, if the format needs it. + std::shared_ptr key_condition; + + /// IInputFormat implementation may put arbitrary state here. + std::shared_ptr opaque; + + ColumnMapperPtr column_mapper; + +private: + /// For lazily initializing the fields above. + std::once_flag init_flag; + std::exception_ptr init_exception; + +public: + FormatParserGroup(const Settings & settings, size_t num_streams_, std::shared_ptr filter_actions_dag_, const ContextPtr & context_); + + static FormatParserGroupPtr singleThreaded(const Settings & settings); + + bool hasFilter() const; + + void finishStream(); + + size_t getParsingThreadsPerReader() const; + size_t getIOThreadsPerReader() const; + + /// Creates `key_condition`. Call inside call_once(init_flag, ...). + void initKeyCondition(const Block & keys); + + /// Does std::call_once(init_flag, ...). + /// If a previous init attempt threw exception, rethrows it instead retrying. + void initOnce(std::function f); +}; + +} diff --git a/src/IO/ReadBufferFromS3.cpp b/src/IO/ReadBufferFromS3.cpp index 1a27e98c4ae1..59fc9097d874 100644 --- a/src/IO/ReadBufferFromS3.cpp +++ b/src/IO/ReadBufferFromS3.cpp @@ -13,6 +13,7 @@ #include #include #include +#include #include #include diff --git a/src/IO/ReadBufferFromS3.h b/src/IO/ReadBufferFromS3.h index 47ef87f15116..1cbdfa2520f6 100644 --- a/src/IO/ReadBufferFromS3.h +++ b/src/IO/ReadBufferFromS3.h @@ -8,7 +8,6 @@ #include #include -#include #include #include #include diff --git a/src/IO/SharedThreadPools.cpp b/src/IO/SharedThreadPools.cpp index 6dfba2d555ab..24355202cae3 100644 --- a/src/IO/SharedThreadPools.cpp +++ b/src/IO/SharedThreadPools.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include namespace CurrentMetrics @@ -35,6 +36,9 @@ namespace CurrentMetrics extern const Metric MergeTreeSubcolumnsReaderThreads; extern const Metric MergeTreeSubcolumnsReaderThreadsActive; extern const Metric MergeTreeSubcolumnsReaderThreadsScheduled; + extern const Metric FormatParsingThreads; + extern const Metric FormatParsingThreadsActive; + extern const Metric FormatParsingThreadsScheduled; } namespace DB @@ -63,6 +67,23 @@ void StaticThreadPool::initialize(size_t max_threads, size_t max_free_threads, s if (instance) throw Exception(ErrorCodes::LOGICAL_ERROR, "The {} is initialized twice", name); + std::call_once(init_flag, [&] + { + initializeImpl(max_threads, max_free_threads, queue_size); + }); +} + +void StaticThreadPool::initializeWithDefaultSettingsIfNotInitialized() +{ + std::call_once(init_flag, [&] + { + size_t max_threads = getNumberOfCPUCoresToUse(); + initializeImpl(max_threads, /*max_free_threads*/ 0, /*queue_size*/ 10000); + }); +} + +void StaticThreadPool::initializeImpl(size_t max_threads, size_t max_free_threads, size_t queue_size) +{ /// By default enabling "turbo mode" won't affect the number of threads anyhow max_threads_turbo = max_threads; max_threads_normal = max_threads; @@ -199,4 +220,10 @@ StaticThreadPool & getMergeTreePrefixesDeserializationThreadPool() return instance; } +StaticThreadPool & getFormatParsingThreadPool() +{ + static StaticThreadPool instance("FormatParsingThreadPool", CurrentMetrics::FormatParsingThreads, CurrentMetrics::FormatParsingThreadsActive, CurrentMetrics::FormatParsingThreadsScheduled); + return instance; +} + } diff --git a/src/IO/SharedThreadPools.h b/src/IO/SharedThreadPools.h index 6544ffc7d146..28161aafea46 100644 --- a/src/IO/SharedThreadPools.h +++ b/src/IO/SharedThreadPools.h @@ -27,6 +27,10 @@ class StaticThreadPool bool isInitialized() const; void reloadConfiguration(size_t max_threads, size_t max_free_threads, size_t queue_size); + /// Server and clickhouse-local initialize all thread pools on startup, with settings from config. + /// Client and misc tools may initialize the pools they use lazily using this method instead. + void initializeWithDefaultSettingsIfNotInitialized(); + /// At runtime we can increase the number of threads up the specified limit /// This is needed to utilize as much a possible resources to accomplish some task. void setMaxTurboThreads(size_t max_threads_turbo_); @@ -40,11 +44,14 @@ class StaticThreadPool const CurrentMetrics::Metric threads_scheduled_metric; std::unique_ptr instance; + std::once_flag init_flag; std::mutex mutex; size_t max_threads_turbo = 0; size_t max_threads_normal = 0; /// If this counter is > 0 - this specific mode is enabled size_t turbo_mode_enabled = 0; + + void initializeImpl(size_t max_threads, size_t max_free_threads, size_t queue_size); }; /// ThreadPool used for the IO. @@ -79,4 +86,6 @@ StaticThreadPool & getDatabaseCatalogDropTablesThreadPool(); /// ThreadPool used for parallel prefixes deserialization of subcolumns in Wide MergeTree parts. StaticThreadPool & getMergeTreePrefixesDeserializationThreadPool(); +StaticThreadPool & getFormatParsingThreadPool(); + } diff --git a/src/IO/WithFileSize.cpp b/src/IO/WithFileSize.cpp index f111754470e6..8fb9d9e27b1c 100644 --- a/src/IO/WithFileSize.cpp +++ b/src/IO/WithFileSize.cpp @@ -1,7 +1,6 @@ #include "WithFileSize.h" #include #include -#include #include #include diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index db58e2352267..604ef9ec5795 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -5469,9 +5469,9 @@ size_t Context::getConfigReloaderInterval() const return shared->config_reload_interval_ms.load(std::memory_order_relaxed); } -InputFormatPtr Context::getInputFormat(const String & name, ReadBuffer & buf, const Block & sample, UInt64 max_block_size, const std::optional & format_settings, std::optional max_parsing_threads) const +InputFormatPtr Context::getInputFormat(const String & name, ReadBuffer & buf, const Block & sample, UInt64 max_block_size, const std::optional & format_settings) const { - return FormatFactory::instance().getInput(name, buf, sample, shared_from_this(), max_block_size, format_settings, max_parsing_threads); + return FormatFactory::instance().getInput(name, buf, sample, shared_from_this(), max_block_size, format_settings); } OutputFormatPtr Context::getOutputFormat(const String & name, WriteBuffer & buf, const Block & sample, const std::optional & format_settings) const diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index f26899ea3c72..592472d8534d 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -1022,7 +1022,7 @@ class Context: public ContextData, public std::enable_shared_from_this /// I/O formats. InputFormatPtr getInputFormat(const String & name, ReadBuffer & buf, const Block & sample, UInt64 max_block_size, - const std::optional & format_settings = std::nullopt, std::optional max_parsing_threads = std::nullopt) const; + const std::optional & format_settings = std::nullopt) const; OutputFormatPtr getOutputFormat(const String & name, WriteBuffer & buf, const Block & sample, const std::optional & format_settings = std::nullopt) const; OutputFormatPtr getOutputFormatParallelIfPossible(const String & name, WriteBuffer & buf, const Block & sample, const std::optional & format_settings = std::nullopt) const; diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 964fc202cb4a..0e5a979df492 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -52,6 +52,7 @@ #include #include #include +#include #include @@ -231,6 +232,11 @@ FiltersForTableExpressionMap collectFiltersForAnalysis(const QueryTreeNodePtr & collect_filters = true; break; } + if (typeid_cast(storage.get())) + { + collect_filters = true; + break; + } } if (!collect_filters) @@ -276,7 +282,7 @@ FiltersForTableExpressionMap collectFiltersForAnalysis(const QueryTreeNodePtr & if (auto filter_actions = read_from_dummy->detachFilterActionsDAG()) { const auto & table_node = dummy_storage_to_table.at(&read_from_dummy->getStorage()); - res[table_node] = FiltersForTableExpression{std::move(filter_actions), read_from_dummy->getPrewhereInfo()}; + res[table_node] = FiltersForTableExpression{filter_actions, read_from_dummy->getPrewhereInfo()}; } } diff --git a/src/Planner/PlannerContext.h b/src/Planner/PlannerContext.h index 92af2687c59b..7eb9638b2912 100644 --- a/src/Planner/PlannerContext.h +++ b/src/Planner/PlannerContext.h @@ -22,7 +22,7 @@ class TableNode; struct FiltersForTableExpression { - std::optional filter_actions; + std::shared_ptr filter_actions; PrewhereInfoPtr prewhere_info; }; diff --git a/src/Processors/Formats/IInputFormat.cpp b/src/Processors/Formats/IInputFormat.cpp index 3009e91c45ad..37e60e0638be 100644 --- a/src/Processors/Formats/IInputFormat.cpp +++ b/src/Processors/Formats/IInputFormat.cpp @@ -6,8 +6,7 @@ namespace DB { -IInputFormat::IInputFormat(Block header, ReadBuffer * in_) - : SourceWithKeyCondition(std::move(header)), in(in_) +IInputFormat::IInputFormat(Block header, ReadBuffer * in_) : ISource(std::move(header)), in(in_) { column_mapping = std::make_shared(); } diff --git a/src/Processors/Formats/IInputFormat.h b/src/Processors/Formats/IInputFormat.h index c59f1e714038..cf2397ff1cb6 100644 --- a/src/Processors/Formats/IInputFormat.h +++ b/src/Processors/Formats/IInputFormat.h @@ -3,9 +3,8 @@ #include #include #include -#include -#include #include +#include namespace DB @@ -17,7 +16,7 @@ using ColumnMappingPtr = std::shared_ptr; /** Input format is a source, that reads data from ReadBuffer. */ -class IInputFormat : public SourceWithKeyCondition +class IInputFormat : public ISource { protected: diff --git a/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp b/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp index 8ac78180c0b0..5d744c0610c8 100644 --- a/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp @@ -174,6 +174,7 @@ void ArrowBlockInputFormat::prepareReader() getPort().getHeader(), "Arrow", format_settings, + std::nullopt, format_settings.arrow.allow_missing_columns, format_settings.null_as_default, format_settings.date_time_overflow_behavior, diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index c0b836371254..4d3df207b838 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -1468,6 +1468,7 @@ ArrowColumnToCHColumn::ArrowColumnToCHColumn( const Block & header_, const std::string & format_name_, const FormatSettings & format_settings_, + const std::optional> & parquet_columns_to_clickhouse_, bool allow_missing_columns_, bool null_as_default_, FormatSettings::DateTimeOverflowBehavior date_time_overflow_behavior_, @@ -1485,6 +1486,7 @@ ArrowColumnToCHColumn::ArrowColumnToCHColumn( , case_insensitive_matching(case_insensitive_matching_) , is_stream(is_stream_) , enable_json_parsing(enable_json_parsing_) + , parquet_columns_to_clickhouse(parquet_columns_to_clickhouse_) { } @@ -1504,12 +1506,14 @@ Chunk ArrowColumnToCHColumn::arrowTableToCHChunk( auto arrow_field = table->schema()->GetFieldByName(column_name); + if (parquet_columns_to_clickhouse) + column_name = parquet_columns_to_clickhouse->at(column_name); + if (case_insensitive_matching) boost::to_lower(column_name); name_to_arrow_column[std::move(column_name)] = {std::move(arrow_column), std::move(arrow_field)}; } - return arrowColumnsToCHChunk(name_to_arrow_column, num_rows, metadata, block_missing_values); } diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h index ad0d9a4b3b18..26ec4f0e377d 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include "config.h" @@ -25,6 +26,7 @@ class ArrowColumnToCHColumn const Block & header_, const std::string & format_name_, const FormatSettings & format_settings_, + const std::optional> & parquet_columns_to_clickhouse_, bool allow_missing_columns_, bool null_as_default_, FormatSettings::DateTimeOverflowBehavior date_time_overflow_behavior_, @@ -90,6 +92,8 @@ class ArrowColumnToCHColumn /// To avoid converting dictionary from Arrow Dictionary /// to LowCardinality every chunk we save it and reuse. std::unordered_map dictionary_infos; + + std::optional> parquet_columns_to_clickhouse; }; } diff --git a/src/Processors/Formats/Impl/ArrowFieldIndexUtil.h b/src/Processors/Formats/Impl/ArrowFieldIndexUtil.h index 6e027dfbd7d0..8118f5ebe7e4 100644 --- a/src/Processors/Formats/Impl/ArrowFieldIndexUtil.h +++ b/src/Processors/Formats/Impl/ArrowFieldIndexUtil.h @@ -80,7 +80,8 @@ class ArrowFieldIndexUtil std::vector findRequiredIndices( const Block & header, const arrow::Schema & schema, - const parquet::FileMetaData & file) + const parquet::FileMetaData & file, + const std::optional> & clickhouse_to_parquet_names) { std::vector required_indices; std::unordered_set added_indices; @@ -90,6 +91,13 @@ class ArrowFieldIndexUtil { const auto & named_col = header.getByPosition(i); std::string col_name = named_col.name; + if (clickhouse_to_parquet_names) + { + if (auto it = clickhouse_to_parquet_names->find(col_name); it != clickhouse_to_parquet_names->end()) + col_name = it->second; + else + continue; + } if (ignore_case) boost::to_lower(col_name); findRequiredIndices(col_name, i, named_col.type, fields_indices, added_indices, required_indices, file); diff --git a/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp b/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp index 142777131e29..d0169829a4e5 100644 --- a/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp @@ -6,6 +6,7 @@ #include #include #include +#include namespace DB { diff --git a/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp b/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp index 7cd12723575f..1d82e5f0c725 100644 --- a/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp +++ b/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Processors/Formats/Impl/ConstantExpressionTemplate.h b/src/Processors/Formats/Impl/ConstantExpressionTemplate.h index 41a5a49bb211..5e0842697f1a 100644 --- a/src/Processors/Formats/Impl/ConstantExpressionTemplate.h +++ b/src/Processors/Formats/Impl/ConstantExpressionTemplate.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include #include #include @@ -16,6 +15,9 @@ struct SpecialParserType; class ExpressionActions; using ExpressionActionsPtr = std::shared_ptr; +struct FormatSettings; +struct Settings; + /// Deduces template of an expression by replacing literals with dummy columns. /// It allows to parse and evaluate similar expressions without using heavy IParsers and ExpressionAnalyzer. /// Using ConstantExpressionTemplate for one expression is slower then evaluateConstantExpression(...), diff --git a/src/Processors/Formats/Impl/DWARFBlockInputFormat.cpp b/src/Processors/Formats/Impl/DWARFBlockInputFormat.cpp index 44c7cb4d6d83..7d24ffd5bd13 100644 --- a/src/Processors/Formats/Impl/DWARFBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/DWARFBlockInputFormat.cpp @@ -20,17 +20,11 @@ #include #include #include +#include #include #include #include -namespace CurrentMetrics -{ - extern const Metric DWARFReaderThreads; - extern const Metric DWARFReaderThreadsActive; - extern const Metric DWARFReaderThreadsScheduled; -} - namespace DB { @@ -238,15 +232,13 @@ void DWARFBlockInputFormat::initializeIfNeeded() LOG_DEBUG(getLogger("DWARF"), "{} units, reading in {} threads", units_queue.size(), num_threads); - pool.emplace(CurrentMetrics::DWARFReaderThreads, CurrentMetrics::DWARFReaderThreadsActive, CurrentMetrics::DWARFReaderThreadsScheduled, num_threads); + runner.emplace(getFormatParsingThreadPool().get(), "DWARFDecoder"); for (size_t i = 0; i < num_threads; ++i) - pool->scheduleOrThrowOnError( + runner.value()( [this, thread_group = CurrentThread::getGroup()]() { try { - ThreadGroupSwitcher switcher(thread_group, "DWARFDecoder"); - std::unique_lock lock(mutex); while (!units_queue.empty() && !is_stopped) { @@ -293,8 +285,8 @@ void DWARFBlockInputFormat::stopThreads() is_stopped = true; } wake_up_threads.notify_all(); - if (pool) - pool->wait(); + if (runner) + runner->waitForAllToFinishAndRethrowFirstError(); } static inline void throwIfError(llvm::Error & e, const char * what) @@ -949,7 +941,7 @@ void DWARFBlockInputFormat::resetParser() { stopThreads(); - pool.reset(); + runner.reset(); background_exception = nullptr; is_stopped = false; units_queue.clear(); @@ -991,14 +983,13 @@ void registerInputFormatDWARF(FormatFactory & factory) const FormatSettings & settings, const ReadSettings &, bool /* is_remote_fs */, - size_t /* max_download_threads */, - size_t max_parsing_threads) + FormatParserGroupPtr parser_group) { return std::make_shared( buf, sample, settings, - max_parsing_threads); + parser_group->getParsingThreadsPerReader()); }); factory.markFormatSupportsSubsetOfColumns("DWARF"); } diff --git a/src/Processors/Formats/Impl/DWARFBlockInputFormat.h b/src/Processors/Formats/Impl/DWARFBlockInputFormat.h index 2d94d1667087..6026fb5ebf1b 100644 --- a/src/Processors/Formats/Impl/DWARFBlockInputFormat.h +++ b/src/Processors/Formats/Impl/DWARFBlockInputFormat.h @@ -11,7 +11,7 @@ #include #include #include -#include +#include #include namespace DB @@ -79,7 +79,7 @@ class DWARFBlockInputFormat : public IInputFormat std::atomic is_stopped{0}; size_t approx_bytes_read_for_chunk = 0; - std::optional pool; + std::optional> runner; std::mutex mutex; std::condition_variable deliver_chunk; std::condition_variable wake_up_threads; diff --git a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp index f770238ee08e..60b36c44a093 100644 --- a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp @@ -1,52 +1,53 @@ -#include "NativeORCBlockInputFormat.h" +#include #if USE_ORC -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include - -# include "ArrowBufferedStreams.h" - -# include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include + +#include namespace @@ -944,7 +945,7 @@ updateIncludeTypeIds(DataTypePtr type, const orc::Type * orc_type, bool ignore_c } NativeORCBlockInputFormat::NativeORCBlockInputFormat( - ReadBuffer & in_, Block header_, const FormatSettings & format_settings_, bool use_prefetch_, size_t min_bytes_for_seek_) + ReadBuffer & in_, Block header_, const FormatSettings & format_settings_, bool use_prefetch_, size_t min_bytes_for_seek_, FormatParserGroupPtr parser_group_) : IInputFormat(std::move(header_), &in_) , memory_pool(std::make_unique()) , block_missing_values(getPort().getHeader().columns()) @@ -952,6 +953,7 @@ NativeORCBlockInputFormat::NativeORCBlockInputFormat( , skip_stripes(format_settings.orc.skip_stripes) , use_prefetch(use_prefetch_) , min_bytes_for_seek(min_bytes_for_seek_) + , parser_group(std::move(parser_group_)) { } @@ -961,6 +963,11 @@ void NativeORCBlockInputFormat::prepareFileReader() if (is_stopped) return; + parser_group->initOnce([&] + { + parser_group->initKeyCondition(getPort().getHeader()); + }); + std::unique_ptr stripe_info; if (file_reader->getNumberOfStripes()) stripe_info = file_reader->getStripe(0); @@ -985,8 +992,8 @@ void NativeORCBlockInputFormat::prepareFileReader() } include_indices.assign(include_typeids.begin(), include_typeids.end()); - if (format_settings.orc.filter_push_down && key_condition && !sargs) - sargs = buildORCSearchArgument(*key_condition, getPort().getHeader(), file_reader->getType(), format_settings); + if (format_settings.orc.filter_push_down && parser_group->key_condition && !sargs) + sargs = buildORCSearchArgument(*parser_group->key_condition, getPort().getHeader(), file_reader->getType(), format_settings); selected_stripes = calculateSelectedStripes(static_cast(file_reader->getNumberOfStripes()), skip_stripes); read_iterator = 0; diff --git a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.h b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.h index e3eb51b8a9cb..636cc9ffa422 100644 --- a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.h +++ b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.h @@ -3,19 +3,22 @@ #include "config.h" #if USE_ORC -# include -# include -# include -# include -# include -# include -# include -# include -# include + +#include +#include +#include +#include +#include +#include +#include +#include +#include namespace DB { +class KeyCondition; + class ORCInputStream : public orc::InputStream { public: @@ -60,7 +63,7 @@ class NativeORCBlockInputFormat : public IInputFormat { public: NativeORCBlockInputFormat( - ReadBuffer & in_, Block header_, const FormatSettings & format_settings_, bool use_prefetch_, size_t min_bytes_for_seek_); + ReadBuffer & in_, Block header_, const FormatSettings & format_settings_, bool use_prefetch_, size_t min_bytes_for_seek_, FormatParserGroupPtr parser_group_); String getName() const override { return "ORCBlockInputFormat"; } @@ -101,6 +104,7 @@ class NativeORCBlockInputFormat : public IInputFormat const std::unordered_set & skip_stripes; const bool use_prefetch; const size_t min_bytes_for_seek; + FormatParserGroupPtr parser_group; std::vector selected_stripes; size_t read_iterator; diff --git a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp index 14cc2defe7e1..0041dc1fb323 100644 --- a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp @@ -139,6 +139,7 @@ void ORCBlockInputFormat::prepareReader() getPort().getHeader(), "ORC", format_settings, + std::nullopt, format_settings.orc.allow_missing_columns, format_settings.null_as_default, format_settings.date_time_overflow_behavior, @@ -207,8 +208,7 @@ void registerInputFormatORC(FormatFactory & factory) const FormatSettings & settings, const ReadSettings & read_settings, bool is_remote_fs, - size_t /* max_download_threads */, - size_t /* max_parsing_threads */) + FormatParserGroupPtr parser_group) { InputFormatPtr res; if (settings.orc.use_fast_decoder) @@ -218,7 +218,7 @@ void registerInputFormatORC(FormatFactory & factory) const bool use_prefetch = is_remote_fs && read_settings.remote_fs_prefetch && has_file_size && seekable_in && seekable_in->checkIfActuallySeekable() && seekable_in->supportsReadAt() && settings.seekable_read; const size_t min_bytes_for_seek = use_prefetch ? read_settings.remote_read_min_bytes_for_seek : 0; - res = std::make_shared(buf, sample, settings, use_prefetch, min_bytes_for_seek); + res = std::make_shared(buf, sample, settings, use_prefetch, min_bytes_for_seek, parser_group); } else res = std::make_shared(buf, sample, settings); diff --git a/src/Processors/Formats/Impl/ParallelParsingInputFormat.cpp b/src/Processors/Formats/Impl/ParallelParsingInputFormat.cpp index de640b5237ed..c49cafaf493b 100644 --- a/src/Processors/Formats/Impl/ParallelParsingInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParallelParsingInputFormat.cpp @@ -58,10 +58,8 @@ void ParallelParsingInputFormat::segmentatorThreadFunction(ThreadGroupPtr thread } } -void ParallelParsingInputFormat::parserThreadFunction(ThreadGroupPtr thread_group, size_t current_ticket_number) +void ParallelParsingInputFormat::parserThreadFunction(size_t current_ticket_number) { - ThreadGroupSwitcher switcher(thread_group, "ChunkParser"); - const auto parser_unit_number = current_ticket_number % processing_units.size(); auto & unit = processing_units[parser_unit_number]; diff --git a/src/Processors/Formats/Impl/ParallelParsingInputFormat.h b/src/Processors/Formats/Impl/ParallelParsingInputFormat.h index 6ef44d79b356..a3df992b2220 100644 --- a/src/Processors/Formats/Impl/ParallelParsingInputFormat.h +++ b/src/Processors/Formats/Impl/ParallelParsingInputFormat.h @@ -7,19 +7,14 @@ #include #include #include +#include #include #include +#include #include #include -namespace CurrentMetrics -{ - extern const Metric ParallelParsingInputFormatThreads; - extern const Metric ParallelParsingInputFormatThreadsActive; - extern const Metric ParallelParsingInputFormatThreadsScheduled; -} - namespace DB { @@ -104,7 +99,7 @@ class ParallelParsingInputFormat : public IInputFormat , max_block_size(params.max_block_size) , last_block_missing_values(getPort().getHeader().columns()) , is_server(params.is_server) - , pool(CurrentMetrics::ParallelParsingInputFormatThreads, CurrentMetrics::ParallelParsingInputFormatThreadsActive, CurrentMetrics::ParallelParsingInputFormatThreadsScheduled, params.max_threads) + , runner(getFormatParsingThreadPool().get(), "ChunkParser") { // One unit for each thread, including segmentator and reader, plus a // couple more units so that the segmentation thread doesn't spuriously @@ -242,9 +237,9 @@ class ParallelParsingInputFormat : public IInputFormat const bool is_server; - /// There are multiple "parsers", that's why we use thread pool. - ThreadPool pool; - /// Reading and segmentating the file + /// Parsing threads. + ThreadPoolCallbackRunnerLocal runner; + /// Reading and segmentating the file. ThreadFromGlobalPool segmentator_thread; enum ProcessingUnitStatus @@ -289,9 +284,9 @@ class ParallelParsingInputFormat : public IInputFormat void scheduleParserThreadForUnitWithNumber(size_t ticket_number) { - pool.scheduleOrThrowOnError([this, ticket_number, group = CurrentThread::getGroup()]() + runner([this, ticket_number]() { - parserThreadFunction(group, ticket_number); + parserThreadFunction(ticket_number); }); /// We have to wait here to possibly extract ColumnMappingPtr from the first parser. if (ticket_number == 0) @@ -324,7 +319,7 @@ class ParallelParsingInputFormat : public IInputFormat try { - pool.wait(); + runner.waitForAllToFinishAndRethrowFirstError(); } catch (...) { @@ -333,7 +328,7 @@ class ParallelParsingInputFormat : public IInputFormat } void segmentatorThreadFunction(ThreadGroupPtr thread_group); - void parserThreadFunction(ThreadGroupPtr thread_group, size_t current_ticket_number); + void parserThreadFunction(size_t current_ticket_number); /// Save/log a background exception, set termination flag, wake up all /// threads. This function is used by segmentator and parsed threads. diff --git a/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.cpp b/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.cpp index 415c084b88c5..e94cdb8a27e8 100644 --- a/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.cpp +++ b/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.cpp @@ -318,6 +318,7 @@ ParquetRecordReader::ParquetRecordReader( std::shared_ptr<::arrow::io::RandomAccessFile> arrow_file, const FormatSettings & format_settings, std::vector row_groups_indices_, + const std::optional> & column_indices_, std::shared_ptr metadata) : file_reader(createFileReader(std::move(arrow_file), reader_properties_, std::move(metadata))) , arrow_properties(arrow_properties_) @@ -338,20 +339,26 @@ ParquetRecordReader::ParquetRecordReader( parquet_col_indice.reserve(header.columns()); column_readers.reserve(header.columns()); - for (const auto & col_with_name : header) + if (!column_indices_) { - auto it = parquet_columns.find(col_with_name.name); - if (it == parquet_columns.end()) - throw Exception(ErrorCodes::PARQUET_EXCEPTION, "no column with '{}' in parquet file", col_with_name.name); + for (const auto & col_with_name : header) + { + auto it = parquet_columns.find(col_with_name.name); + if (it == parquet_columns.end()) + throw Exception(ErrorCodes::PARQUET_EXCEPTION, "no column with '{}' in parquet file", col_with_name.name); - const auto & node = it->second; - if (!node->is_primitive()) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "arrays and maps are not implemented in native parquet reader"); + const auto & node = it->second; + if (!node->is_primitive()) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "arrays and maps are not implemented in native parquet reader"); - auto idx = file_reader->metadata()->schema()->ColumnIndex(*node); - chassert(idx >= 0); - parquet_col_indice.push_back(idx); + auto idx = file_reader->metadata()->schema()->ColumnIndex(*node); + chassert(idx >= 0); + parquet_col_indice.push_back(idx); + } } + else + parquet_col_indice = *column_indices_; + if (arrow_properties.pre_buffer()) { THROW_PARQUET_EXCEPTION(file_reader->PreBuffer( diff --git a/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.h b/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.h index a846f4d66b06..9f502c2a3762 100644 --- a/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.h +++ b/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.h @@ -25,6 +25,7 @@ class ParquetRecordReader std::shared_ptr<::arrow::io::RandomAccessFile> arrow_file, const FormatSettings & format_settings, std::vector row_groups_indices_, + const std::optional> & column_indices_, std::shared_ptr metadata = nullptr); Chunk readChunk(); diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index 67994192a6b3..21c8024c8945 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -1,4 +1,5 @@ -#include "ParquetBlockInputFormat.h" +#include +#include #if USE_PARQUET @@ -8,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -30,7 +32,9 @@ #include #include #include +#include +#include #include namespace ProfileEvents @@ -42,13 +46,9 @@ namespace ProfileEvents namespace CurrentMetrics { - extern const Metric ParquetDecoderThreads; - extern const Metric ParquetDecoderThreadsActive; - extern const Metric ParquetDecoderThreadsScheduled; - - extern const Metric ParquetDecoderIOThreads; - extern const Metric ParquetDecoderIOThreadsActive; - extern const Metric ParquetDecoderIOThreadsScheduled; + extern const Metric IOThreads; + extern const Metric IOThreadsActive; + extern const Metric IOThreadsScheduled; } namespace DB @@ -572,29 +572,33 @@ ParquetBlockInputFormat::ParquetBlockInputFormat( ReadBuffer & buf, const Block & header_, const FormatSettings & format_settings_, - size_t max_decoding_threads_, - size_t max_io_threads_, + FormatParserGroupPtr parser_group_, size_t min_bytes_for_seek_) : IInputFormat(header_, &buf) , format_settings(format_settings_) , skip_row_groups(format_settings.parquet.skip_row_groups) - , max_decoding_threads(max_decoding_threads_) - , max_io_threads(max_io_threads_) + , parser_group(std::move(parser_group_)) , min_bytes_for_seek(min_bytes_for_seek_) , pending_chunks(PendingChunk::Compare { .row_group_first = format_settings_.parquet.preserve_order }) , previous_block_missing_values(getPort().getHeader().columns()) { - if (max_decoding_threads > 1) - pool = std::make_unique(CurrentMetrics::ParquetDecoderThreads, CurrentMetrics::ParquetDecoderThreadsActive, CurrentMetrics::ParquetDecoderThreadsScheduled, max_decoding_threads); - if (supportPrefetch()) - io_pool = std::make_shared(CurrentMetrics::ParquetDecoderIOThreads, CurrentMetrics::ParquetDecoderIOThreadsActive, CurrentMetrics::ParquetDecoderIOThreadsScheduled, max_io_threads); + use_thread_pool = parser_group->max_parsing_threads > 1; + + bool row_group_prefetch = + !use_thread_pool && parser_group->max_io_threads > 0 && + format_settings.parquet.enable_row_group_prefetch && + !format_settings.parquet.use_native_reader; + if (row_group_prefetch) + io_pool = std::make_shared( + CurrentMetrics::IOThreads, CurrentMetrics::IOThreadsActive, CurrentMetrics::IOThreadsScheduled, + parser_group->getIOThreadsPerReader()); } ParquetBlockInputFormat::~ParquetBlockInputFormat() { is_stopped = true; - if (pool) - pool->wait(); + if (use_thread_pool) + shutdown->shutdown(); if (io_pool) io_pool->wait(); } @@ -604,16 +608,23 @@ void ParquetBlockInputFormat::initializeIfNeeded() if (std::exchange(is_initialized, true)) return; + parser_group->initOnce([&] + { + parser_group->initKeyCondition(getPort().getHeader()); + + if (use_thread_pool) + parser_group->parsing_runner.initThreadPool( + getFormatParsingThreadPool().get(), parser_group->max_parsing_threads, "ParquetDecoder", CurrentThread::getGroup()); + }); + // Create arrow file adapter. - // TODO: Make the adapter do prefetching on IO threads, based on the full set of ranges that - // we'll need to read (which we know in advance). Use max_download_threads for that. arrow_file = asArrowFile(*in, format_settings, is_stopped, "Parquet", PARQUET_MAGIC_BYTES, /* avoid_buffering */ true, io_pool); if (is_stopped) return; metadata = parquet::ReadMetaData(arrow_file); - const bool prefetch_group = supportPrefetch(); + const bool prefetch_group = io_pool != nullptr; std::shared_ptr schema; THROW_ARROW_NOT_OK(parquet::arrow::FromParquetSchema(metadata->schema(), &schema)); @@ -622,7 +633,22 @@ void ParquetBlockInputFormat::initializeIfNeeded() format_settings.parquet.case_insensitive_column_matching, format_settings.parquet.allow_missing_columns); - auto index_mapping = field_util.findRequiredIndices(getPort().getHeader(), *schema, *metadata); + std::optional> clickhouse_to_parquet_names; + if (parser_group && parser_group->column_mapper) + { + auto header = getPort().getHeader(); + const auto & group_node = metadata->schema()->group_node(); + + std::unordered_map parquet_field_ids; + parquet_names_to_clickhouse = std::unordered_map{}; + for (int i = 0; i < group_node->field_count(); ++i) + parquet_field_ids[group_node->field(i)->field_id()] = group_node->field(i)->name(); + + auto result = parser_group->column_mapper->makeMapping(header, parquet_field_ids); + clickhouse_to_parquet_names = std::move(result.first); + parquet_names_to_clickhouse = std::move(result.second); + } + auto index_mapping = field_util.findRequiredIndices(getPort().getHeader(), *schema, *metadata, clickhouse_to_parquet_names); for (const auto & [clickhouse_header_index, parquet_indexes] : index_mapping) { @@ -665,9 +691,9 @@ void ParquetBlockInputFormat::initializeIfNeeded() std::unique_ptr key_condition_with_bloom_filter_data; - if (key_condition) + if (parser_group->key_condition) { - key_condition_with_bloom_filter_data = std::make_unique(*key_condition); + key_condition_with_bloom_filter_data = std::make_unique(*parser_group->key_condition); if (format_settings.parquet.bloom_filter_push_down) { @@ -768,7 +794,7 @@ void ParquetBlockInputFormat::initializeIfNeeded() void ParquetBlockInputFormat::initializeRowGroupBatchReader(size_t row_group_batch_idx) { - const bool row_group_prefetch = supportPrefetch(); + const bool row_group_prefetch = io_pool != nullptr; auto & row_group_batch = row_group_batches[row_group_batch_idx]; parquet::ArrowReaderProperties arrow_properties; @@ -838,7 +864,8 @@ void ParquetBlockInputFormat::initializeRowGroupBatchReader(size_t row_group_bat reader_properties, arrow_file, format_settings, - row_group_batch.row_groups_idxs); + row_group_batch.row_groups_idxs, + column_indices); } else { @@ -865,6 +892,7 @@ void ParquetBlockInputFormat::initializeRowGroupBatchReader(size_t row_group_bat getPort().getHeader(), "Parquet", format_settings, + parquet_names_to_clickhouse, format_settings.parquet.allow_missing_columns, format_settings.null_as_default, format_settings.date_time_overflow_behavior, @@ -884,13 +912,15 @@ void ParquetBlockInputFormat::scheduleRowGroup(size_t row_group_batch_idx) status = RowGroupBatchState::Status::Running; - pool->scheduleOrThrowOnError( - [this, row_group_batch_idx, thread_group = CurrentThread::getGroup()]() + parser_group->parsing_runner( + [this, row_group_batch_idx, shutdown_ = shutdown]() { + std::shared_lock shutdown_lock(*shutdown_, std::try_to_lock); + if (!shutdown_lock.owns_lock()) + return; + try { - ThreadGroupSwitcher switcher(thread_group, "ParquetDecoder"); - threadFunction(row_group_batch_idx); } catch (...) @@ -923,11 +953,6 @@ void ParquetBlockInputFormat::threadFunction(size_t row_group_batch_idx) return; } } -bool ParquetBlockInputFormat::supportPrefetch() const -{ - return max_decoding_threads == 1 && max_io_threads > 0 && format_settings.parquet.enable_row_group_prefetch && !format_settings.parquet.use_native_reader; -} - std::shared_ptr ParquetBlockInputFormat::RowGroupPrefetchIterator::nextRowGroupReader() { if (prefetched_row_groups.empty()) return nullptr; @@ -1064,8 +1089,9 @@ void ParquetBlockInputFormat::scheduleMoreWorkIfNeeded(std::optional row ++row_group_batches_completed; } - if (pool) + if (use_thread_pool) { + size_t max_decoding_threads = parser_group->getParsingThreadsPerReader(); while (row_group_batches_started - row_group_batches_completed < max_decoding_threads && row_group_batches_started < row_group_batches.size()) scheduleRowGroup(row_group_batches_started++); @@ -1126,7 +1152,7 @@ Chunk ParquetBlockInputFormat::read() if (row_group_batches_completed == row_group_batches.size()) return {}; - if (pool) + if (use_thread_pool) condvar.wait(lock); else decodeOneChunk(row_group_batches_completed, lock); @@ -1136,8 +1162,11 @@ Chunk ParquetBlockInputFormat::read() void ParquetBlockInputFormat::resetParser() { is_stopped = true; - if (pool) - pool->wait(); + if (use_thread_pool) + { + shutdown->shutdown(); + shutdown = std::make_shared(); + } arrow_file.reset(); metadata.reset(); @@ -1237,17 +1266,16 @@ void registerInputFormatParquet(FormatFactory & factory) const FormatSettings & settings, const ReadSettings & read_settings, bool is_remote_fs, - size_t max_download_threads, - size_t max_parsing_threads) + FormatParserGroupPtr parser_group) -> InputFormatPtr { size_t min_bytes_for_seek = is_remote_fs ? read_settings.remote_read_min_bytes_for_seek : settings.parquet.local_read_min_bytes_for_seek; - return std::make_shared( + auto ptr = std::make_shared( buf, sample, settings, - max_parsing_threads, - max_download_threads, + std::move(parser_group), min_bytes_for_seek); + return ptr; }); factory.markFormatSupportsSubsetOfColumns("Parquet"); } diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.h b/src/Processors/Formats/Impl/ParquetBlockInputFormat.h index f941bb70ebce..a646b537c12f 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.h @@ -5,7 +5,7 @@ #include #include #include -#include +#include #include @@ -57,10 +57,9 @@ class ParquetBlockInputFormat : public IInputFormat ParquetBlockInputFormat( ReadBuffer & buf, const Block & header, - const FormatSettings & format_settings, - size_t max_decoding_threads, - size_t max_io_threads, - size_t min_bytes_for_seek); + const FormatSettings & format_settings_, + FormatParserGroupPtr parser_group_, + size_t min_bytes_for_seek_); ~ParquetBlockInputFormat() override; @@ -90,8 +89,6 @@ class ParquetBlockInputFormat : public IInputFormat void threadFunction(size_t row_group_batch_idx); - inline bool supportPrefetch() const; - // Data layout in the file: // // row group 0 @@ -297,8 +294,7 @@ class ParquetBlockInputFormat : public IInputFormat const FormatSettings format_settings; const std::unordered_set & skip_row_groups; - size_t max_decoding_threads; - size_t max_io_threads; + FormatParserGroupPtr parser_group; size_t min_bytes_for_seek; const size_t max_pending_chunks_per_row_group_batch = 2; @@ -329,7 +325,8 @@ class ParquetBlockInputFormat : public IInputFormat // These are only used when max_decoding_threads > 1. size_t row_group_batches_started = 0; - std::unique_ptr pool; + bool use_thread_pool = false; + std::shared_ptr shutdown = std::make_shared(); std::shared_ptr io_pool; BlockMissingValues previous_block_missing_values; @@ -338,6 +335,7 @@ class ParquetBlockInputFormat : public IInputFormat std::exception_ptr background_exception = nullptr; std::atomic is_stopped{0}; bool is_initialized = false; + std::optional> parquet_names_to_clickhouse; }; class ParquetSchemaReader : public ISchemaReader diff --git a/src/Processors/Formats/Impl/ParquetMetadataInputFormat.cpp b/src/Processors/Formats/Impl/ParquetMetadataInputFormat.cpp index d9b55b2fc79c..80a0c62bc43a 100644 --- a/src/Processors/Formats/Impl/ParquetMetadataInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetMetadataInputFormat.cpp @@ -508,8 +508,7 @@ void registerInputFormatParquetMetadata(FormatFactory & factory) const FormatSettings & settings, const ReadSettings &, bool /* is_remote_fs */, - size_t /* max_download_threads */, - size_t /* max_parsing_threads */) + FormatParserGroupPtr) { return std::make_shared(buf, sample, settings); }); diff --git a/src/Processors/Formats/RowInputFormatWithDiagnosticInfo.cpp b/src/Processors/Formats/RowInputFormatWithDiagnosticInfo.cpp index 12e9b650acd2..9cf6fcd88dbf 100644 --- a/src/Processors/Formats/RowInputFormatWithDiagnosticInfo.cpp +++ b/src/Processors/Formats/RowInputFormatWithDiagnosticInfo.cpp @@ -2,6 +2,7 @@ #include #include #include +#include namespace DB diff --git a/src/Processors/ISource.cpp b/src/Processors/ISource.cpp index 68749c474535..db658704476d 100644 --- a/src/Processors/ISource.cpp +++ b/src/Processors/ISource.cpp @@ -48,7 +48,6 @@ ISource::Status ISource::prepare() if (got_exception) { - finished = true; output.finish(); return Status::Finished; } @@ -103,6 +102,9 @@ void ISource::work() { try { + if (finished) + return; + read_progress_was_set = false; if (auto chunk = tryGenerate()) @@ -120,11 +122,17 @@ void ISource::work() if (isCancelled()) finished = true; + + if (finished) + onFinish(); } catch (...) { - finished = true; got_exception = true; + + if (!std::exchange(finished, true)) + onFinish(); + throw; } } diff --git a/src/Processors/ISource.h b/src/Processors/ISource.h index 83f5c615923d..253afb9f683b 100644 --- a/src/Processors/ISource.h +++ b/src/Processors/ISource.h @@ -31,6 +31,7 @@ class ISource : public IProcessor virtual Chunk generate(); virtual std::optional tryGenerate(); + virtual void onFinish() {} void progress(size_t read_rows, size_t read_bytes); diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 46b8927c7d07..addc5fbb051a 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1873,14 +1873,15 @@ void ReadFromMergeTree::applyFilters(ActionDAGNodes added_filter_nodes) { if (!indexes) { - filter_actions_dag = ActionsDAG::buildFilterActionsDAG(added_filter_nodes.nodes, query_info.buildNodeNameToInputNodeColumn()); + auto dag = ActionsDAG::buildFilterActionsDAG(added_filter_nodes.nodes, query_info.buildNodeNameToInputNodeColumn()); + filter_actions_dag = dag ? std::make_shared(std::move(*dag)) : nullptr; /// NOTE: Currently we store two DAGs for analysis: - /// (1) SourceStepWithFilter::filter_nodes, (2) query_info.filter_actions_dag. Make sure there are consistent. + /// (1) SourceStepWithFilter::filter_nodes, (2) query_info.filter_actions_dag. Make sure they are consistent. /// TODO: Get rid of filter_actions_dag in query_info after we move analysis of /// parallel replicas and unused shards into optimization, similar to projection analysis. if (filter_actions_dag) - query_info.filter_actions_dag = std::make_shared(filter_actions_dag->clone()); + query_info.filter_actions_dag = filter_actions_dag; buildIndexes( indexes, diff --git a/src/Processors/QueryPlan/ReadFromObjectStorageStep.cpp b/src/Processors/QueryPlan/ReadFromObjectStorageStep.cpp index c152d991d62d..764edad3337b 100644 --- a/src/Processors/QueryPlan/ReadFromObjectStorageStep.cpp +++ b/src/Processors/QueryPlan/ReadFromObjectStorageStep.cpp @@ -28,7 +28,7 @@ namespace Setting ReadFromObjectStorageStep::ReadFromObjectStorageStep( ObjectStoragePtr object_storage_, - ConfigurationPtr configuration_, + StorageObjectStorageConfigurationPtr configuration_, const String & name_, const Names & columns_to_read, const NamesAndTypesList & virtual_columns_, @@ -67,7 +67,6 @@ void ReadFromObjectStorageStep::initializePipeline(QueryPipelineBuilder & pipeli Pipes pipes; auto context = getContext(); - const size_t max_threads = context->getSettingsRef()[Setting::max_threads]; size_t estimated_keys_count = iterator_wrapper->estimatedKeysCount(); if (estimated_keys_count > 1) @@ -79,18 +78,17 @@ void ReadFromObjectStorageStep::initializePipeline(QueryPipelineBuilder & pipeli num_streams = 1; } - const size_t max_parsing_threads = num_streams >= max_threads ? 1 : (max_threads / std::max(num_streams, 1ul)); + auto parser_group = std::make_shared(context->getSettingsRef(), num_streams, filter_actions_dag, context); + parser_group->column_mapper = configuration->getColumnMapper(); for (size_t i = 0; i < num_streams; ++i) { auto source = std::make_shared( getName(), object_storage, configuration, info, format_settings, - context, max_block_size, iterator_wrapper, max_parsing_threads, need_only_count); + context, max_block_size, iterator_wrapper, parser_group, need_only_count); - source->setKeyCondition(filter_actions_dag, context); pipes.emplace_back(std::move(source)); } - auto pipe = Pipe::unitePipes(std::move(pipes)); if (pipe.empty()) pipe = Pipe(std::make_shared(info.source_header)); @@ -107,14 +105,14 @@ void ReadFromObjectStorageStep::createIterator() return; const ActionsDAG::Node * predicate = nullptr; - if (filter_actions_dag.has_value()) + if (filter_actions_dag) predicate = filter_actions_dag->getOutputs().at(0); auto context = getContext(); iterator_wrapper = StorageObjectStorageSource::createFileIterator( configuration, configuration->getQuerySettings(context), object_storage, distributed_processing, - context, predicate, filter_actions_dag, virtual_columns, info.hive_partition_columns_to_read_from_file_path, nullptr, context->getFileProgressCallback()); + context, predicate, filter_actions_dag.get(), + virtual_columns, info.hive_partition_columns_to_read_from_file_path, nullptr, context->getFileProgressCallback()); } - } diff --git a/src/Processors/QueryPlan/ReadFromObjectStorageStep.h b/src/Processors/QueryPlan/ReadFromObjectStorageStep.h index 1fe3f6b21b01..22958008f929 100644 --- a/src/Processors/QueryPlan/ReadFromObjectStorageStep.h +++ b/src/Processors/QueryPlan/ReadFromObjectStorageStep.h @@ -10,11 +10,9 @@ namespace DB class ReadFromObjectStorageStep : public SourceStepWithFilter { public: - using ConfigurationPtr = StorageObjectStorage::ConfigurationPtr; - ReadFromObjectStorageStep( ObjectStoragePtr object_storage_, - ConfigurationPtr configuration_, + StorageObjectStorageConfigurationPtr configuration_, const String & name_, const Names & columns_to_read, const NamesAndTypesList & virtual_columns_, @@ -36,7 +34,7 @@ class ReadFromObjectStorageStep : public SourceStepWithFilter private: ObjectStoragePtr object_storage; - ConfigurationPtr configuration; + StorageObjectStorageConfigurationPtr configuration; std::shared_ptr iterator_wrapper; const ReadFromFormatInfo info; diff --git a/src/Processors/QueryPlan/ReadFromPreparedSource.h b/src/Processors/QueryPlan/ReadFromPreparedSource.h index 1b12c3afefe9..ea559f32b88c 100644 --- a/src/Processors/QueryPlan/ReadFromPreparedSource.h +++ b/src/Processors/QueryPlan/ReadFromPreparedSource.h @@ -8,6 +8,9 @@ namespace DB { +class IStorage; +using StoragePtr = std::shared_ptr; + /// Create source from prepared pipe. class ReadFromPreparedSource : public ISourceStep { diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index 0aedbec5cbd1..e500ded5a85e 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -491,9 +491,7 @@ void ReadFromRemote::addLazyPipe( add_extremes = context->getSettingsRef()[Setting::extremes]; } - std::shared_ptr pushed_down_filters; - if (filter_actions_dag) - pushed_down_filters = std::make_shared(filter_actions_dag->clone()); + std::shared_ptr pushed_down_filters = filter_actions_dag; const StorageID resolved_id = context->resolveStorageID(shard.main_table ? shard.main_table : main_table); const StoragePtr storage = DatabaseCatalog::instance().tryGetTable(resolved_id, context); diff --git a/src/Processors/QueryPlan/SourceStepWithFilter.cpp b/src/Processors/QueryPlan/SourceStepWithFilter.cpp index 46df96ba6ac4..96e14102abee 100644 --- a/src/Processors/QueryPlan/SourceStepWithFilter.cpp +++ b/src/Processors/QueryPlan/SourceStepWithFilter.cpp @@ -80,12 +80,14 @@ Block SourceStepWithFilter::applyPrewhereActions(Block block, const PrewhereInfo void SourceStepWithFilterBase::applyFilters(ActionDAGNodes added_filter_nodes) { - filter_actions_dag = ActionsDAG::buildFilterActionsDAG(added_filter_nodes.nodes, {}); + auto dag = ActionsDAG::buildFilterActionsDAG(added_filter_nodes.nodes, {}); + filter_actions_dag = dag ? std::make_shared(std::move(*dag)) : nullptr; } void SourceStepWithFilter::applyFilters(ActionDAGNodes added_filter_nodes) { - filter_actions_dag = ActionsDAG::buildFilterActionsDAG(added_filter_nodes.nodes, query_info.buildNodeNameToInputNodeColumn()); + auto dag = ActionsDAG::buildFilterActionsDAG(added_filter_nodes.nodes, query_info.buildNodeNameToInputNodeColumn()); + filter_actions_dag = dag ? std::make_shared(std::move(*dag)) : nullptr; } void SourceStepWithFilter::updatePrewhereInfo(const PrewhereInfoPtr & prewhere_info_value) diff --git a/src/Processors/QueryPlan/SourceStepWithFilter.h b/src/Processors/QueryPlan/SourceStepWithFilter.h index f681c20791ce..589c682181c2 100644 --- a/src/Processors/QueryPlan/SourceStepWithFilter.h +++ b/src/Processors/QueryPlan/SourceStepWithFilter.h @@ -36,7 +36,7 @@ class SourceStepWithFilterBase : public ISourceStep } if (other.filter_actions_dag) - filter_actions_dag = other.filter_actions_dag->clone(); + filter_actions_dag = std::make_shared(other.filter_actions_dag->clone()); } SourceStepWithFilterBase(SourceStepWithFilterBase &&) = default; @@ -64,8 +64,8 @@ class SourceStepWithFilterBase : public ISourceStep virtual void applyFilters(ActionDAGNodes added_filter_nodes); virtual PrewhereInfoPtr getPrewhereInfo() const { return nullptr; } - const std::optional & getFilterActionsDAG() const { return filter_actions_dag; } - std::optional detachFilterActionsDAG() { return std::move(filter_actions_dag); } + const std::shared_ptr & getFilterActionsDAG() const { return filter_actions_dag; } + std::shared_ptr detachFilterActionsDAG() { return std::move(filter_actions_dag); } bool hasCorrelatedExpressions() const override { @@ -81,7 +81,7 @@ class SourceStepWithFilterBase : public ISourceStep protected: std::optional limit; - std::optional filter_actions_dag; + std::shared_ptr filter_actions_dag; }; /** Source step that can use filters and limit for more efficient pipeline initialization. diff --git a/src/Processors/SourceWithKeyCondition.cpp b/src/Processors/SourceWithKeyCondition.cpp deleted file mode 100644 index 12f8b39e1ae8..000000000000 --- a/src/Processors/SourceWithKeyCondition.cpp +++ /dev/null @@ -1,19 +0,0 @@ -#include - -#include -#include - -namespace DB -{ - -void SourceWithKeyCondition::setKeyConditionImpl(const std::optional & filter_actions_dag, ContextPtr context, const Block & keys) -{ - ActionsDAGWithInversionPushDown inverted_dag(filter_actions_dag ? filter_actions_dag->getOutputs().front() : nullptr, context); - key_condition = std::make_shared( - inverted_dag, - context, - keys.getNames(), - std::make_shared(ActionsDAG(keys.getColumnsWithTypeAndName()))); -} - -} diff --git a/src/Processors/SourceWithKeyCondition.h b/src/Processors/SourceWithKeyCondition.h deleted file mode 100644 index af40cbccaab4..000000000000 --- a/src/Processors/SourceWithKeyCondition.h +++ /dev/null @@ -1,31 +0,0 @@ -#pragma once - -#include -#include - -namespace DB -{ - -class KeyCondition; -class ActionsDAG; - -/// Source with KeyCondition to push down filters. -class SourceWithKeyCondition : public ISource -{ -protected: - /// Represents pushed down filters in source - std::shared_ptr key_condition; - - void setKeyConditionImpl(const std::optional & filter_actions_dag, ContextPtr context, const Block & keys); - -public: - using Base = ISource; - using Base::Base; - - /// Set key_condition directly. It is used for filter push down in source. - virtual void setKeyCondition(const std::shared_ptr & key_condition_) { key_condition = key_condition_; } - - /// Set key_condition created by filter_actions_dag and context. - virtual void setKeyCondition(const std::optional & /*filter_actions_dag*/, ContextPtr /*context*/) { } -}; -} diff --git a/src/Processors/Sources/PostgreSQLSource.cpp b/src/Processors/Sources/PostgreSQLSource.cpp index b9bda46bd10c..884229cce83e 100644 --- a/src/Processors/Sources/PostgreSQLSource.cpp +++ b/src/Processors/Sources/PostgreSQLSource.cpp @@ -105,7 +105,15 @@ IProcessor::Status PostgreSQLSource::prepare() auto status = ISource::prepare(); if (status == Status::Finished) - onFinish(); + { + if (stream) + stream->close(); + + if (tx && auto_commit) + tx->commit(); + + is_completed = true; + } return status; } @@ -172,18 +180,6 @@ Chunk PostgreSQLSource::generate() } -template -void PostgreSQLSource::onFinish() -{ - if (stream) - stream->close(); - - if (tx && auto_commit) - tx->commit(); - - is_completed = true; -} - template PostgreSQLSource::~PostgreSQLSource() { diff --git a/src/Processors/Sources/PostgreSQLSource.h b/src/Processors/Sources/PostgreSQLSource.h index 64d84bf08574..59915a2e1afd 100644 --- a/src/Processors/Sources/PostgreSQLSource.h +++ b/src/Processors/Sources/PostgreSQLSource.h @@ -43,8 +43,6 @@ class PostgreSQLSource : public ISource void onStart(); - void onFinish(); - private: void init(const Block & sample_block); diff --git a/src/Processors/Sources/ShellCommandSource.cpp b/src/Processors/Sources/ShellCommandSource.cpp index f18204122286..0f1e2c1cd63c 100644 --- a/src/Processors/Sources/ShellCommandSource.cpp +++ b/src/Processors/Sources/ShellCommandSource.cpp @@ -13,6 +13,7 @@ #include #include #include +#include #include #include diff --git a/src/Storages/FileLog/FileLogSource.cpp b/src/Storages/FileLog/FileLogSource.cpp index c02053048640..ae48524f6931 100644 --- a/src/Storages/FileLog/FileLogSource.cpp +++ b/src/Storages/FileLog/FileLogSource.cpp @@ -51,7 +51,7 @@ FileLogSource::~FileLogSource() try { if (!finished) - onFinish(); + close(); } catch (...) { @@ -59,7 +59,7 @@ FileLogSource::~FileLogSource() } } -void FileLogSource::onFinish() +void FileLogSource::close() { storage.closeFilesAndStoreMeta(start, end); storage.reduceStreams(); @@ -73,9 +73,9 @@ Chunk FileLogSource::generate() if (!consumer || consumer->noRecords()) { - /// There is no onFinish for ISource, we call it + /// There is no close for ISource, we call it /// when no records return to close files - onFinish(); + close(); return {}; } @@ -83,7 +83,7 @@ Chunk FileLogSource::generate() EmptyReadBuffer empty_buf; auto input_format = FormatFactory::instance().getInput( - storage.getFormatName(), empty_buf, non_virtual_header, context, max_block_size, std::nullopt, 1); + storage.getFormatName(), empty_buf, non_virtual_header, context, max_block_size, std::nullopt, FormatParserGroup::singleThreaded(context->getSettingsRef())); std::optional exception_message; size_t total_rows = 0; @@ -160,7 +160,7 @@ Chunk FileLogSource::generate() if (total_rows == 0) { - onFinish(); + close(); return {}; } diff --git a/src/Storages/FileLog/FileLogSource.h b/src/Storages/FileLog/FileLogSource.h index c29b4539152b..8a7c7eb225d1 100644 --- a/src/Storages/FileLog/FileLogSource.h +++ b/src/Storages/FileLog/FileLogSource.h @@ -29,7 +29,7 @@ class FileLogSource : public ISource bool noRecords() { return !consumer || consumer->noRecords(); } - void onFinish(); + void close(); ~FileLogSource() override; diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index 1289aaf58d04..d0a474eedb02 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -279,21 +279,23 @@ class StorageHiveSource : public ISource, WithContext else read_buf = std::move(raw_read_buf); + ContextPtr context = getContext(); + auto input_format = FormatFactory::instance().getInput( format, *read_buf, to_read_block, - getContext(), + context, max_block_size, updateFormatSettings(current_file), - /* max_parsing_threads */ 1); + FormatParserGroup::singleThreaded(context->getSettingsRef())); Pipe pipe(input_format); if (columns_description.hasDefaults()) { pipe.addSimpleTransform([&](const Block & header) { - return std::make_shared(header, columns_description, *input_format, getContext()); + return std::make_shared(header, columns_description, *input_format, context); }); } pipeline = std::make_unique(std::move(pipe)); @@ -616,14 +618,15 @@ HiveFiles StorageHive::collectHiveFilesFromPartition( writeString("\n", wb); ReadBufferFromString buffer(wb.str()); + ContextPtr context = getContext(); auto format = FormatFactory::instance().getInput( "CSV", buffer, partition_key_expr->getSampleBlock(), - getContext(), - getContext()->getSettingsRef()[Setting::max_block_size], + context, + context->getSettingsRef()[Setting::max_block_size], std::nullopt, - /* max_parsing_threads */ 1); + FormatParserGroup::singleThreaded(context->getSettingsRef())); auto pipeline = QueryPipeline(std::move(format)); auto reader = std::make_unique(pipeline); Block block; @@ -643,7 +646,7 @@ HiveFiles StorageHive::collectHiveFilesFromPartition( ranges.emplace_back(fields[i]); ActionsDAGWithInversionPushDown inverted_dag(filter_actions_dag->getOutputs().front(), context_); - const KeyCondition partition_key_condition(inverted_dag, getContext(), partition_names, partition_minmax_idx_expr); + const KeyCondition partition_key_condition(inverted_dag, context, partition_names, partition_minmax_idx_expr); if (!partition_key_condition.checkInHyperrectangle(ranges, partition_types).can_be_true) return {}; } diff --git a/src/Storages/HivePartitioningUtils.cpp b/src/Storages/HivePartitioningUtils.cpp index 2ed5b002ba98..2dbb792ef84d 100644 --- a/src/Storages/HivePartitioningUtils.cpp +++ b/src/Storages/HivePartitioningUtils.cpp @@ -180,7 +180,7 @@ void extractPartitionColumnsFromPathAndEnrichStorageColumns( HivePartitionColumnsWithFileColumnsPair setupHivePartitioningForObjectStorage( ColumnsDescription & columns, - const StorageObjectStorage::ConfigurationPtr & configuration, + const StorageObjectStorageConfigurationPtr & configuration, const std::string & sample_path, bool inferred_schema, std::optional format_settings, diff --git a/src/Storages/HivePartitioningUtils.h b/src/Storages/HivePartitioningUtils.h index 0550b5694750..fa4980219912 100644 --- a/src/Storages/HivePartitioningUtils.h +++ b/src/Storages/HivePartitioningUtils.h @@ -25,7 +25,7 @@ using HivePartitionColumnsWithFileColumnsPair = std::pair format_settings, diff --git a/src/Storages/IStorageCluster.cpp b/src/Storages/IStorageCluster.cpp index 7c8d410cee53..415803ba6722 100644 --- a/src/Storages/IStorageCluster.cpp +++ b/src/Storages/IStorageCluster.cpp @@ -117,7 +117,11 @@ void ReadFromCluster::createExtension(const ActionsDAG::Node * predicate, size_t if (extension) return; - extension = storage->getTaskIteratorExtension(predicate, context, number_of_replicas); + extension = storage->getTaskIteratorExtension( + predicate, + filter_actions_dag ? filter_actions_dag.get() : query_info.filter_actions_dag.get(), + context, + number_of_replicas); } /// The code executes on initiator diff --git a/src/Storages/IStorageCluster.h b/src/Storages/IStorageCluster.h index 6017613c7bea..5f31f7d9f03c 100644 --- a/src/Storages/IStorageCluster.h +++ b/src/Storages/IStorageCluster.h @@ -34,8 +34,13 @@ class IStorageCluster : public IStorage size_t /*num_streams*/) override; ClusterPtr getCluster(ContextPtr context) const; + /// Query is needed for pruning by virtual columns (_file, _path) - virtual RemoteQueryExecutor::Extension getTaskIteratorExtension(const ActionsDAG::Node * predicate, const ContextPtr & context, size_t number_of_replicas) const = 0; + virtual RemoteQueryExecutor::Extension getTaskIteratorExtension( + const ActionsDAG::Node * predicate, + const ActionsDAG * filter_actions_dag, + const ContextPtr & context, + size_t number_of_replicas) const = 0; QueryProcessingStage::Enum getQueryProcessingStage(ContextPtr, QueryProcessingStage::Enum, const StorageSnapshotPtr &, SelectQueryInfo &) const override; diff --git a/src/Storages/KVStorageUtils.cpp b/src/Storages/KVStorageUtils.cpp index 151717371e2f..baa8a450836e 100644 --- a/src/Storages/KVStorageUtils.cpp +++ b/src/Storages/KVStorageUtils.cpp @@ -3,9 +3,7 @@ #include #include -#include #include -#include #include #include @@ -15,7 +13,6 @@ #include - namespace DB { @@ -26,106 +23,8 @@ namespace ErrorCodes namespace { -// returns keys may be filter by condition -bool traverseASTFilter( - const std::string & primary_key, const DataTypePtr & primary_key_type, const ASTPtr & elem, const PreparedSetsPtr & prepared_sets, const ContextPtr & context, FieldVectorPtr & res) -{ - const auto * function = elem->as(); - if (!function) - return false; - - if (function->name == "and") - { - // one child has the key filter condition is ok - for (const auto & child : function->arguments->children) - if (traverseASTFilter(primary_key, primary_key_type, child, prepared_sets, context, res)) - return true; - return false; - } - if (function->name == "or") - { - // make sure every child has the key filter condition - for (const auto & child : function->arguments->children) - if (!traverseASTFilter(primary_key, primary_key_type, child, prepared_sets, context, res)) - return false; - return true; - } - if (function->name == "equals" || function->name == "in") - { - const auto & args = function->arguments->as(); - const ASTIdentifier * ident; - std::shared_ptr value; - - if (args.children.size() != 2) - return false; - - if (function->name == "in") - { - if (!prepared_sets) - return false; - - ident = args.children.at(0)->as(); - if (!ident) - return false; - - if (ident->name() != primary_key) - return false; - value = args.children.at(1); - - PreparedSets::Hash set_key = value->getTreeHash(/*ignore_aliases=*/true); - FutureSetPtr future_set; - - if ((value->as() || value->as())) - future_set = prepared_sets->findSubquery(set_key); - else - future_set = prepared_sets->findTuple(set_key, {primary_key_type}); - - if (!future_set) - return false; - - future_set->buildOrderedSetInplace(context); - - auto set = future_set->get(); - if (!set) - return false; - - if (!set->hasExplicitSetElements()) - return false; - - set->checkColumnsNumber(1); - const auto & set_column = *set->getSetElements()[0]; - - if (set_column.getDataType() != primary_key_type->getTypeId()) - return false; - - for (size_t row = 0; row < set_column.size(); ++row) - res->push_back(set_column[row]); - return true; - } - - if ((ident = args.children.at(0)->as())) - value = args.children.at(1); - else if ((ident = args.children.at(1)->as())) - value = args.children.at(0); - else - return false; - - if (ident->name() != primary_key) - return false; - - const auto node = evaluateConstantExpressionAsLiteral(value, context); - /// function->name == "equals" - if (const auto * literal = node->as()) - { - auto converted_field = convertFieldToType(literal->value, *primary_key_type); - if (!converted_field.isNull()) - res->push_back(converted_field); - return true; - } - } - return false; -} +// returns keys may be filtered by condition bool traverseDAGFilter( const std::string & primary_key, const DataTypePtr & primary_key_type, const ActionsDAG::Node * elem, const ContextPtr & context, FieldVectorPtr & res) { @@ -227,7 +126,7 @@ bool traverseDAGFilter( } std::pair getFilterKeys( - const String & primary_key, const DataTypePtr & primary_key_type, const std::optional & filter_actions_dag, const ContextPtr & context) + const String & primary_key, const DataTypePtr & primary_key_type, const ActionsDAG * filter_actions_dag, const ContextPtr & context) { if (!filter_actions_dag) return {{}, true}; @@ -239,18 +138,6 @@ std::pair getFilterKeys( return std::make_pair(res, !matched_keys); } -std::pair getFilterKeys( - const String & primary_key, const DataTypePtr & primary_key_type, const SelectQueryInfo & query_info, const ContextPtr & context) -{ - const auto & select = query_info.query->as(); - if (!select.where()) - return {{}, true}; - - FieldVectorPtr res = std::make_shared(); - auto matched_keys = traverseASTFilter(primary_key, primary_key_type, select.where(), query_info.prepared_sets, context, res); - return std::make_pair(res, !matched_keys); -} - std::vector serializeKeysToRawString( FieldVector::const_iterator & it, FieldVector::const_iterator end, diff --git a/src/Storages/KVStorageUtils.h b/src/Storages/KVStorageUtils.h index 692fa26cbd7f..e73bdd0d253d 100644 --- a/src/Storages/KVStorageUtils.h +++ b/src/Storages/KVStorageUtils.h @@ -22,7 +22,7 @@ std::pair getFilterKeys( const std::string & primary_key, const DataTypePtr & primary_key_type, const SelectQueryInfo & query_info, const ContextPtr & context); std::pair getFilterKeys( - const String & primary_key, const DataTypePtr & primary_key_type, const std::optional & filter_actions_dag, const ContextPtr & context); + const String & primary_key, const DataTypePtr & primary_key_type, const ActionsDAG * filter_actions_dag, const ContextPtr & context); template void fillColumns(const K & key, const V & value, size_t key_pos, const Block & header, MutableColumns & columns) diff --git a/src/Storages/Kafka/KafkaSource.cpp b/src/Storages/Kafka/KafkaSource.cpp index 5de3a2280b67..f51a732a5a30 100644 --- a/src/Storages/Kafka/KafkaSource.cpp +++ b/src/Storages/Kafka/KafkaSource.cpp @@ -110,7 +110,7 @@ Chunk KafkaSource::generateImpl() EmptyReadBuffer empty_buf; auto input_format = FormatFactory::instance().getInput( - storage.getFormatName(), empty_buf, non_virtual_header, context, max_block_size, std::nullopt, 1); + storage.getFormatName(), empty_buf, non_virtual_header, context, max_block_size, std::nullopt, FormatParserGroup::singleThreaded(context->getSettingsRef())); std::optional exception_message; size_t total_rows = 0; diff --git a/src/Storages/Kafka/StorageKafka2.cpp b/src/Storages/Kafka/StorageKafka2.cpp index a154dd85dd1b..73d840b71d2b 100644 --- a/src/Storages/Kafka/StorageKafka2.cpp +++ b/src/Storages/Kafka/StorageKafka2.cpp @@ -1018,7 +1018,7 @@ StorageKafka2::PolledBatchInfo StorageKafka2::pollConsumer( EmptyReadBuffer empty_buf; auto input_format = FormatFactory::instance().getInput( - getFormatName(), empty_buf, non_virtual_header, modified_context, getMaxBlockSize(), std::nullopt, 1); + getFormatName(), empty_buf, non_virtual_header, modified_context, getMaxBlockSize(), std::nullopt, FormatParserGroup::singleThreaded(modified_context->getSettingsRef())); std::optional exception_message; size_t total_rows = 0; diff --git a/src/Storages/NATS/NATSSource.cpp b/src/Storages/NATS/NATSSource.cpp index b7b6a6129aac..bc960e0acc8f 100644 --- a/src/Storages/NATS/NATSSource.cpp +++ b/src/Storages/NATS/NATSSource.cpp @@ -110,7 +110,7 @@ Chunk NATSSource::generate() MutableColumns virtual_columns = virtual_header.cloneEmptyColumns(); EmptyReadBuffer empty_buf; auto input_format = FormatFactory::instance().getInput( - storage.getFormatName(), empty_buf, non_virtual_header, context, max_block_size, std::nullopt, 1); + storage.getFormatName(), empty_buf, non_virtual_header, context, max_block_size, std::nullopt, FormatParserGroup::singleThreaded(context->getSettingsRef())); std::optional exception_message; size_t total_rows = 0; auto on_error = [&](const MutableColumns & result_columns, const ColumnCheckpoints & checkpoints, Exception & e) diff --git a/src/Storages/ObjectStorage/Azure/Configuration.cpp b/src/Storages/ObjectStorage/Azure/Configuration.cpp index 13cec9b867d4..aee2c215ff36 100644 --- a/src/Storages/ObjectStorage/Azure/Configuration.cpp +++ b/src/Storages/ObjectStorage/Azure/Configuration.cpp @@ -63,13 +63,13 @@ void StorageAzureConfiguration::check(ContextPtr context) const { auto url = Poco::URI(connection_params.getConnectionURL()); context->getGlobalContext()->getRemoteHostFilter().checkURL(url); - Configuration::check(context); + StorageObjectStorageConfiguration::check(context); } -StorageObjectStorage::QuerySettings StorageAzureConfiguration::getQuerySettings(const ContextPtr & context) const +StorageObjectStorageQuerySettings StorageAzureConfiguration::getQuerySettings(const ContextPtr & context) const { const auto & settings = context->getSettingsRef(); - return StorageObjectStorage::QuerySettings{ + return StorageObjectStorageQuerySettings{ .truncate_on_insert = settings[Setting::azure_truncate_on_insert], .create_new_file_on_insert = settings[Setting::azure_create_new_file_on_insert], .schema_inference_use_cache = settings[Setting::schema_inference_use_cache_for_azure], diff --git a/src/Storages/ObjectStorage/Azure/Configuration.h b/src/Storages/ObjectStorage/Azure/Configuration.h index 303232b72184..addcac522d94 100644 --- a/src/Storages/ObjectStorage/Azure/Configuration.h +++ b/src/Storages/ObjectStorage/Azure/Configuration.h @@ -12,15 +12,13 @@ namespace DB { class BackupFactory; -class StorageAzureConfiguration : public StorageObjectStorage::Configuration +class StorageAzureConfiguration : public StorageObjectStorageConfiguration { friend class BackupReaderAzureBlobStorage; friend class BackupWriterAzureBlobStorage; friend void registerBackupEngineAzureBlobStorage(BackupFactory & factory); public: - using ConfigurationPtr = StorageObjectStorage::ConfigurationPtr; - static constexpr auto type = ObjectStorageType::Azure; static constexpr auto type_name = "azure"; static constexpr auto engine_name = "Azure"; @@ -74,7 +72,7 @@ class StorageAzureConfiguration : public StorageObjectStorage::Configuration String getNamespace() const override { return connection_params.getContainer(); } String getDataSourceDescription() const override { return std::filesystem::path(connection_params.getConnectionURL()) / connection_params.getContainer(); } - StorageObjectStorage::QuerySettings getQuerySettings(const ContextPtr &) const override; + StorageObjectStorageQuerySettings getQuerySettings(const ContextPtr &) const override; void check(ContextPtr context) const override; diff --git a/src/Storages/ObjectStorage/DataLakes/Common.cpp b/src/Storages/ObjectStorage/DataLakes/Common.cpp index d2bc6ce6be59..f3e0617743fc 100644 --- a/src/Storages/ObjectStorage/DataLakes/Common.cpp +++ b/src/Storages/ObjectStorage/DataLakes/Common.cpp @@ -10,7 +10,7 @@ namespace DB std::vector listFiles( const IObjectStorage & object_storage, - const StorageObjectStorage::Configuration & configuration, + const StorageObjectStorageConfiguration & configuration, const String & prefix, const String & suffix) { auto key = std::filesystem::path(configuration.getPathForRead().path) / prefix; diff --git a/src/Storages/ObjectStorage/DataLakes/Common.h b/src/Storages/ObjectStorage/DataLakes/Common.h index db3afa9e4a6d..4fbffd179348 100644 --- a/src/Storages/ObjectStorage/DataLakes/Common.h +++ b/src/Storages/ObjectStorage/DataLakes/Common.h @@ -9,7 +9,7 @@ class IObjectStorage; std::vector listFiles( const IObjectStorage & object_storage, - const StorageObjectStorage::Configuration & configuration, + const StorageObjectStorageConfiguration & configuration, const String & prefix, const String & suffix); } diff --git a/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h b/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h index 844f631f5d7d..c5684fe68a91 100644 --- a/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h +++ b/src/Storages/ObjectStorage/DataLakes/DataLakeConfiguration.h @@ -13,7 +13,8 @@ #include #include #include -#include "Storages/ColumnsDescription.h" +#include +#include #include #include @@ -39,14 +40,12 @@ namespace DataLakeStorageSetting template -concept StorageConfiguration = std::derived_from; +concept StorageConfiguration = std::derived_from; template -class DataLakeConfiguration : public BaseStorageConfiguration, public std::enable_shared_from_this +class DataLakeConfiguration : public BaseStorageConfiguration, public std::enable_shared_from_this { public: - using Configuration = StorageObjectStorage::Configuration; - explicit DataLakeConfiguration(DataLakeStorageSettingsPtr settings_) : settings(settings_) {} bool isDataLakeConfiguration() const override { return true; } @@ -168,6 +167,11 @@ class DataLakeConfiguration : public BaseStorageConfiguration, public std::enabl current_metadata->modifyFormatSettings(settings_); } + ColumnMapperPtr getColumnMapper() const override + { + return current_metadata->getColumnMapper(); + } + private: DataLakeMetadataPtr current_metadata; LoggerPtr log = getLogger("DataLakeConfiguration"); diff --git a/src/Storages/ObjectStorage/DataLakes/DeltaLake/KernelHelper.cpp b/src/Storages/ObjectStorage/DataLakes/DeltaLake/KernelHelper.cpp index 48be506daa55..a5bccf513139 100644 --- a/src/Storages/ObjectStorage/DataLakes/DeltaLake/KernelHelper.cpp +++ b/src/Storages/ObjectStorage/DataLakes/DeltaLake/KernelHelper.cpp @@ -160,7 +160,7 @@ namespace S3AuthSetting } DeltaLake::KernelHelperPtr getKernelHelper( - const StorageObjectStorage::ConfigurationPtr & configuration, + const StorageObjectStorageConfigurationPtr & configuration, const ObjectStoragePtr & object_storage) { switch (configuration->getType()) diff --git a/src/Storages/ObjectStorage/DataLakes/DeltaLake/KernelHelper.h b/src/Storages/ObjectStorage/DataLakes/DeltaLake/KernelHelper.h index f8dae7e0ec9d..58865c8ea0f5 100644 --- a/src/Storages/ObjectStorage/DataLakes/DeltaLake/KernelHelper.h +++ b/src/Storages/ObjectStorage/DataLakes/DeltaLake/KernelHelper.h @@ -47,7 +47,7 @@ namespace DB /// Depending on the type of the passed StorageObjectStorage::IConfiguration object, /// it would create S3KernelHelper, AzureKernelHelper, etc. DeltaLake::KernelHelperPtr getKernelHelper( - const StorageObjectStorage::ConfigurationPtr & configuration, + const StorageObjectStorageConfigurationPtr & configuration, const ObjectStoragePtr & object_storage); } diff --git a/src/Storages/ObjectStorage/DataLakes/DeltaLake/PartitionPruner.cpp b/src/Storages/ObjectStorage/DataLakes/DeltaLake/PartitionPruner.cpp index c2b4754e116b..fc5a5cbf4f6a 100644 --- a/src/Storages/ObjectStorage/DataLakes/DeltaLake/PartitionPruner.cpp +++ b/src/Storages/ObjectStorage/DataLakes/DeltaLake/PartitionPruner.cpp @@ -9,7 +9,6 @@ #include #include -#include #include diff --git a/src/Storages/ObjectStorage/DataLakes/DeltaLake/PartitionPruner.h b/src/Storages/ObjectStorage/DataLakes/DeltaLake/PartitionPruner.h index bb64aa64aebe..63be4a9afd64 100644 --- a/src/Storages/ObjectStorage/DataLakes/DeltaLake/PartitionPruner.h +++ b/src/Storages/ObjectStorage/DataLakes/DeltaLake/PartitionPruner.h @@ -2,6 +2,8 @@ #include "config.h" #include #include +#include +#include #if USE_DELTA_KERNEL_RS diff --git a/src/Storages/ObjectStorage/DataLakes/DeltaLake/TableSnapshot.h b/src/Storages/ObjectStorage/DataLakes/DeltaLake/TableSnapshot.h index b545cf96afb5..320bc4aab91b 100644 --- a/src/Storages/ObjectStorage/DataLakes/DeltaLake/TableSnapshot.h +++ b/src/Storages/ObjectStorage/DataLakes/DeltaLake/TableSnapshot.h @@ -25,8 +25,6 @@ namespace DeltaLake class TableSnapshot { public: - using ConfigurationWeakPtr = DB::StorageObjectStorage::ConfigurationObserverPtr; - explicit TableSnapshot( KernelHelperPtr helper_, DB::ObjectStoragePtr object_storage_, diff --git a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp index e640e467c6d9..aed1435a79b3 100644 --- a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp +++ b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp @@ -112,17 +112,15 @@ DataTypePtr getComplexTypeFromObject(const Poco::JSON::Object::Ptr & type) struct DeltaLakeMetadataImpl { - using ConfigurationObserverPtr = DeltaLakeMetadata::ConfigurationObserverPtr; - ObjectStoragePtr object_storage; - ConfigurationObserverPtr configuration; + StorageObjectStorageConfigurationWeakPtr configuration; ContextPtr context; /** * Useful links: * - https://github.com/delta-io/delta/blob/master/PROTOCOL.md#data-files */ - DeltaLakeMetadataImpl(ObjectStoragePtr object_storage_, ConfigurationObserverPtr configuration_, ContextPtr context_) + DeltaLakeMetadataImpl(ObjectStoragePtr object_storage_, StorageObjectStorageConfigurationWeakPtr configuration_, ContextPtr context_) : object_storage(object_storage_), configuration(configuration_), context(context_) { } @@ -508,6 +506,7 @@ struct DeltaLakeMetadataImpl ArrowColumnToCHColumn column_reader( header, "Parquet", format_settings, + std::nullopt, format_settings.parquet.allow_missing_columns, /* null_as_default */true, format_settings.date_time_overflow_behavior, @@ -605,7 +604,7 @@ struct DeltaLakeMetadataImpl LoggerPtr log = getLogger("DeltaLakeMetadataParser"); }; -DeltaLakeMetadata::DeltaLakeMetadata(ObjectStoragePtr object_storage_, ConfigurationObserverPtr configuration_, ContextPtr context_) +DeltaLakeMetadata::DeltaLakeMetadata(ObjectStoragePtr object_storage_, StorageObjectStorageConfigurationWeakPtr configuration_, ContextPtr context_) { auto impl = DeltaLakeMetadataImpl(object_storage_, configuration_, context_); auto result = impl.processMetadataFiles(); @@ -620,7 +619,7 @@ DeltaLakeMetadata::DeltaLakeMetadata(ObjectStoragePtr object_storage_, Configura DataLakeMetadataPtr DeltaLakeMetadata::create( ObjectStoragePtr object_storage, - ConfigurationObserverPtr configuration, + StorageObjectStorageConfigurationWeakPtr configuration, ContextPtr local_context) { #if USE_DELTA_KERNEL_RS diff --git a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.h b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.h index e68a673c740f..1d04741b3e3d 100644 --- a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.h +++ b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.h @@ -30,10 +30,9 @@ using DeltaLakePartitionColumns = std::unordered_map( diff --git a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadataDeltaKernel.h b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadataDeltaKernel.h index 679214d5c489..d8b69b60f583 100644 --- a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadataDeltaKernel.h +++ b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadataDeltaKernel.h @@ -22,12 +22,11 @@ namespace DB class DeltaLakeMetadataDeltaKernel final : public IDataLakeMetadata { public: - using ConfigurationObserverPtr = StorageObjectStorage::ConfigurationObserverPtr; static constexpr auto name = "DeltaLake"; DeltaLakeMetadataDeltaKernel( ObjectStoragePtr object_storage_, - ConfigurationObserverPtr configuration_); + StorageObjectStorageConfigurationWeakPtr configuration_); bool supportsUpdate() const override { return true; } @@ -47,7 +46,7 @@ class DeltaLakeMetadataDeltaKernel final : public IDataLakeMetadata static DataLakeMetadataPtr create( ObjectStoragePtr object_storage, - ConfigurationObserverPtr configuration, + StorageObjectStorageConfigurationWeakPtr configuration, ContextPtr /* context */) { auto configuration_ptr = configuration.lock(); diff --git a/src/Storages/ObjectStorage/DataLakes/HudiMetadata.cpp b/src/Storages/ObjectStorage/DataLakes/HudiMetadata.cpp index a1f47e34be6e..8417b232a845 100644 --- a/src/Storages/ObjectStorage/DataLakes/HudiMetadata.cpp +++ b/src/Storages/ObjectStorage/DataLakes/HudiMetadata.cpp @@ -86,7 +86,7 @@ Strings HudiMetadata::getDataFilesImpl() const return result; } -HudiMetadata::HudiMetadata(ObjectStoragePtr object_storage_, ConfigurationObserverPtr configuration_, ContextPtr context_) +HudiMetadata::HudiMetadata(ObjectStoragePtr object_storage_, StorageObjectStorageConfigurationWeakPtr configuration_, ContextPtr context_) : WithContext(context_), object_storage(object_storage_), configuration(configuration_) { } diff --git a/src/Storages/ObjectStorage/DataLakes/HudiMetadata.h b/src/Storages/ObjectStorage/DataLakes/HudiMetadata.h index 7fd94e0d14c4..f074f672ad59 100644 --- a/src/Storages/ObjectStorage/DataLakes/HudiMetadata.h +++ b/src/Storages/ObjectStorage/DataLakes/HudiMetadata.h @@ -13,11 +13,9 @@ namespace DB class HudiMetadata final : public IDataLakeMetadata, private WithContext { public: - using ConfigurationObserverPtr = StorageObjectStorage::ConfigurationObserverPtr; - static constexpr auto name = "Hudi"; - HudiMetadata(ObjectStoragePtr object_storage_, ConfigurationObserverPtr configuration_, ContextPtr context_); + HudiMetadata(ObjectStoragePtr object_storage_, StorageObjectStorageConfigurationWeakPtr configuration_, ContextPtr context_); NamesAndTypesList getTableSchema() const override { return {}; } @@ -31,7 +29,7 @@ class HudiMetadata final : public IDataLakeMetadata, private WithContext static DataLakeMetadataPtr create( ObjectStoragePtr object_storage, - ConfigurationObserverPtr configuration, + StorageObjectStorageConfigurationWeakPtr configuration, ContextPtr local_context) { return std::make_unique(object_storage, configuration, local_context); @@ -46,7 +44,7 @@ class HudiMetadata final : public IDataLakeMetadata, private WithContext private: const ObjectStoragePtr object_storage; - const ConfigurationObserverPtr configuration; + const StorageObjectStorageConfigurationWeakPtr configuration; mutable Strings data_files; Strings getDataFilesImpl() const; diff --git a/src/Storages/ObjectStorage/DataLakes/IDataLakeMetadata.h b/src/Storages/ObjectStorage/DataLakes/IDataLakeMetadata.h index 39f66faf106f..b6c0b618ee5c 100644 --- a/src/Storages/ObjectStorage/DataLakes/IDataLakeMetadata.h +++ b/src/Storages/ObjectStorage/DataLakes/IDataLakeMetadata.h @@ -5,6 +5,7 @@ #include "Interpreters/ActionsDAG.h" #include #include +#include namespace DB { @@ -57,6 +58,10 @@ class IDataLakeMetadata : boost::noncopyable virtual std::optional totalRows(ContextPtr) const { return {}; } virtual std::optional totalBytes(ContextPtr) const { return {}; } + /// Some data lakes specify information for reading files from disks. + /// For example, Iceberg has Parquet schema field ids in its metadata for reading files. + virtual ColumnMapperPtr getColumnMapper() const { return nullptr; } + protected: ObjectIterator createKeysIterator( Strings && data_files_, diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.cpp b/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.cpp index c2b8caa64c55..5b33bf0bca48 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.cpp +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.cpp @@ -1,4 +1,8 @@ -#include "config.h" +#include +#include +#include +#include +#include #if USE_AVRO @@ -101,7 +105,7 @@ std::string normalizeUuid(const std::string & uuid) Poco::JSON::Object::Ptr getMetadataJSONObject( const String & metadata_file_path, ObjectStoragePtr object_storage, - StorageObjectStorage::ConfigurationPtr configuration_ptr, + StorageObjectStorageConfigurationPtr configuration_ptr, IcebergMetadataFilesCachePtr cache_ptr, const ContextPtr & local_context, LoggerPtr log) @@ -138,7 +142,7 @@ Poco::JSON::Object::Ptr getMetadataJSONObject( IcebergMetadata::IcebergMetadata( ObjectStoragePtr object_storage_, - ConfigurationObserverPtr configuration_, + StorageObjectStorageConfigurationWeakPtr configuration_, const ContextPtr & context_, Int32 metadata_version_, Int32 format_version_, @@ -314,7 +318,7 @@ struct ShortMetadataFileInfo */ static std::pair getLatestMetadataFileAndVersion( const ObjectStoragePtr & object_storage, - StorageObjectStorage::ConfigurationPtr configuration_ptr, + StorageObjectStorageConfigurationPtr configuration_ptr, IcebergMetadataFilesCachePtr cache_ptr, const ContextPtr & local_context, const std::optional & table_uuid) @@ -394,7 +398,7 @@ static std::pair getLatestMetadataFileAndVersion( static std::pair getLatestOrExplicitMetadataFileAndVersion( const ObjectStoragePtr & object_storage, - StorageObjectStorage::ConfigurationPtr configuration_ptr, + StorageObjectStorageConfigurationPtr configuration_ptr, IcebergMetadataFilesCachePtr cache_ptr, const ContextPtr & local_context, Poco::Logger * log) @@ -526,6 +530,32 @@ void IcebergMetadata::updateSnapshot(ContextPtr local_context, Poco::JSON::Objec total_bytes = summary_object->getValue(f_total_files_size); } +#if USE_PARQUET + if (configuration_ptr->format == "Parquet") + column_mapper = std::make_shared(); + + if (column_mapper) + { + Int32 schema_id = snapshot->getValue(f_schema_id); + auto schemas = metadata_object->getArray(f_schemas); + std::unordered_map column_name_to_parquet_field_id; + for (UInt32 j = 0; j < schemas->size(); ++j) + { + auto schema = schemas->getObject(j); + if (schema->getValue(f_schema_id) != schema_id) + continue; + + auto fields = schema->getArray(f_fields); + for (UInt32 field_ind = 0; field_ind < fields->size(); ++field_ind) + { + auto field = fields->getObject(field_ind); + column_name_to_parquet_field_id[field->getValue(f_name)] = field->getValue(f_id); + } + } + column_mapper->setStorageColumnEncoding(std::move(column_name_to_parquet_field_id)); + } +#endif + relevant_snapshot = IcebergSnapshot{ getManifestList(local_context, getProperFilePathFromMetadataInfo( snapshot->getValue(f_manifest_list), configuration_ptr->getPathForRead().path, table_location)), @@ -649,7 +679,7 @@ std::optional IcebergMetadata::getSchemaVersionByFileIfOutdated(String da DataLakeMetadataPtr IcebergMetadata::create( const ObjectStoragePtr & object_storage, - const ConfigurationObserverPtr & configuration, + const StorageObjectStorageConfigurationWeakPtr & configuration, const ContextPtr & local_context) { auto configuration_ptr = configuration.lock(); diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.h b/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.h index 411ffa296d17..0d7a36cd8842 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.h +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.h @@ -1,4 +1,5 @@ #pragma once + #include "config.h" #if USE_AVRO @@ -17,6 +18,7 @@ #include #include #include +#include #include #include @@ -30,15 +32,13 @@ namespace DB class IcebergMetadata : public IDataLakeMetadata { public: - using ConfigurationObserverPtr = StorageObjectStorage::ConfigurationObserverPtr; - using ConfigurationPtr = StorageObjectStorage::ConfigurationPtr; using IcebergHistory = std::vector; static constexpr auto name = "Iceberg"; IcebergMetadata( ObjectStoragePtr object_storage_, - ConfigurationObserverPtr configuration_, + StorageObjectStorageConfigurationWeakPtr configuration_, const ContextPtr & context_, Int32 metadata_version_, Int32 format_version_, @@ -56,7 +56,7 @@ class IcebergMetadata : public IDataLakeMetadata static DataLakeMetadataPtr create( const ObjectStoragePtr & object_storage, - const ConfigurationObserverPtr & configuration, + const StorageObjectStorageConfigurationWeakPtr & configuration, const ContextPtr & local_context); std::shared_ptr getInitialSchemaByPath(ContextPtr local_context, const String & data_path) const override; @@ -75,6 +75,8 @@ class IcebergMetadata : public IDataLakeMetadata std::optional totalRows(ContextPtr Local_context) const override; std::optional totalBytes(ContextPtr Local_context) const override; + ColumnMapperPtr getColumnMapper() const override { return column_mapper; } + protected: ObjectIterator iterate( const ActionsDAG * filter_dag, @@ -84,7 +86,7 @@ class IcebergMetadata : public IDataLakeMetadata private: const ObjectStoragePtr object_storage; - const ConfigurationObserverPtr configuration; + const StorageObjectStorageConfigurationWeakPtr configuration; mutable IcebergSchemaProcessor schema_processor; LoggerPtr log; @@ -110,6 +112,8 @@ class IcebergMetadata : public IDataLakeMetadata mutable std::optional cached_unprunned_files_for_last_processed_snapshot TSA_GUARDED_BY(cached_unprunned_files_for_last_processed_snapshot_mutex); mutable std::mutex cached_unprunned_files_for_last_processed_snapshot_mutex; + ColumnMapperPtr column_mapper; + void updateState(const ContextPtr & local_context, Poco::JSON::Object::Ptr metadata_object, bool metadata_file_changed) TSA_REQUIRES(mutex); Strings getDataFiles(const ActionsDAG * filter_dag, ContextPtr local_context) const; diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadataFilesCache.h b/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadataFilesCache.h index 2ad276d6e38d..e0ed07ada180 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadataFilesCache.h +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadataFilesCache.h @@ -95,7 +95,7 @@ class IcebergMetadataFilesCache : public CacheBasegetDataSourceDescription()) / data_path; } diff --git a/src/Storages/ObjectStorage/HDFS/Configuration.cpp b/src/Storages/ObjectStorage/HDFS/Configuration.cpp index b2d9a1b7d99a..94548f845b85 100644 --- a/src/Storages/ObjectStorage/HDFS/Configuration.cpp +++ b/src/Storages/ObjectStorage/HDFS/Configuration.cpp @@ -45,7 +45,7 @@ void StorageHDFSConfiguration::check(ContextPtr context) const { context->getRemoteHostFilter().checkURL(Poco::URI(url)); checkHDFSURL(fs::path(url) / path.path.substr(1)); - Configuration::check(context); + StorageObjectStorageConfiguration::check(context); } ObjectStoragePtr StorageHDFSConfiguration::createObjectStorage( /// NOLINT @@ -59,10 +59,10 @@ ObjectStoragePtr StorageHDFSConfiguration::createObjectStorage( /// NOLINT url, std::move(hdfs_settings), context->getConfigRef(), /* lazy_initialize */true); } -StorageObjectStorage::QuerySettings StorageHDFSConfiguration::getQuerySettings(const ContextPtr & context) const +StorageObjectStorageQuerySettings StorageHDFSConfiguration::getQuerySettings(const ContextPtr & context) const { const auto & settings = context->getSettingsRef(); - return StorageObjectStorage::QuerySettings{ + return StorageObjectStorageQuerySettings{ .truncate_on_insert = settings[Setting::hdfs_truncate_on_insert], .create_new_file_on_insert = settings[Setting::hdfs_create_new_file_on_insert], .schema_inference_use_cache = settings[Setting::schema_inference_use_cache_for_hdfs], diff --git a/src/Storages/ObjectStorage/HDFS/Configuration.h b/src/Storages/ObjectStorage/HDFS/Configuration.h index 81022c54aadd..df382860f56e 100644 --- a/src/Storages/ObjectStorage/HDFS/Configuration.h +++ b/src/Storages/ObjectStorage/HDFS/Configuration.h @@ -9,11 +9,9 @@ namespace DB { -class StorageHDFSConfiguration : public StorageObjectStorage::Configuration +class StorageHDFSConfiguration : public StorageObjectStorageConfiguration { public: - using ConfigurationPtr = StorageObjectStorage::ConfigurationPtr; - static constexpr auto type = ObjectStorageType::HDFS; static constexpr auto type_name = "hdfs"; static constexpr auto engine_name = "HDFS"; @@ -57,7 +55,7 @@ class StorageHDFSConfiguration : public StorageObjectStorage::Configuration String getNamespace() const override { return ""; } String getDataSourceDescription() const override { return url; } - StorageObjectStorage::QuerySettings getQuerySettings(const ContextPtr &) const override; + StorageObjectStorageQuerySettings getQuerySettings(const ContextPtr &) const override; void check(ContextPtr context) const override; diff --git a/src/Storages/ObjectStorage/Local/Configuration.cpp b/src/Storages/ObjectStorage/Local/Configuration.cpp index bae84714273c..d843f728a8d2 100644 --- a/src/Storages/ObjectStorage/Local/Configuration.cpp +++ b/src/Storages/ObjectStorage/Local/Configuration.cpp @@ -67,10 +67,10 @@ void StorageLocalConfiguration::fromAST(ASTs & args, ContextPtr context, bool wi paths = {path}; } -StorageObjectStorage::QuerySettings StorageLocalConfiguration::getQuerySettings(const ContextPtr & context) const +StorageObjectStorageQuerySettings StorageLocalConfiguration::getQuerySettings(const ContextPtr & context) const { const auto & settings = context->getSettingsRef(); - return StorageObjectStorage::QuerySettings{ + return StorageObjectStorageQuerySettings{ .truncate_on_insert = settings[Setting::engine_file_truncate_on_insert], .create_new_file_on_insert = false, .schema_inference_use_cache = settings[Setting::schema_inference_use_cache_for_file], diff --git a/src/Storages/ObjectStorage/Local/Configuration.h b/src/Storages/ObjectStorage/Local/Configuration.h index 1654bb36d79f..27d6a2ee9ae1 100644 --- a/src/Storages/ObjectStorage/Local/Configuration.h +++ b/src/Storages/ObjectStorage/Local/Configuration.h @@ -13,11 +13,9 @@ namespace fs = std::filesystem; namespace DB { -class StorageLocalConfiguration : public StorageObjectStorage::Configuration +class StorageLocalConfiguration : public StorageObjectStorageConfiguration { public: - using ConfigurationPtr = StorageObjectStorage::ConfigurationPtr; - static constexpr auto type = ObjectStorageType::Local; static constexpr auto type_name = "local"; /// All possible signatures for Local engine with structure argument (for example for local table function). @@ -52,7 +50,7 @@ class StorageLocalConfiguration : public StorageObjectStorage::Configuration String getNamespace() const override { return ""; } String getDataSourceDescription() const override { return ""; } - StorageObjectStorage::QuerySettings getQuerySettings(const ContextPtr &) const override; + StorageObjectStorageQuerySettings getQuerySettings(const ContextPtr &) const override; ObjectStoragePtr createObjectStorage(ContextPtr, bool readonly) override { diff --git a/src/Storages/ObjectStorage/ReadBufferIterator.cpp b/src/Storages/ObjectStorage/ReadBufferIterator.cpp index bc2a43af9fd1..323c0105630f 100644 --- a/src/Storages/ObjectStorage/ReadBufferIterator.cpp +++ b/src/Storages/ObjectStorage/ReadBufferIterator.cpp @@ -23,7 +23,7 @@ namespace ErrorCodes ReadBufferIterator::ReadBufferIterator( ObjectStoragePtr object_storage_, - ConfigurationPtr configuration_, + StorageObjectStorageConfigurationPtr configuration_, const ObjectIterator & file_iterator_, const std::optional & format_settings_, SchemaCache & schema_cache_, diff --git a/src/Storages/ObjectStorage/ReadBufferIterator.h b/src/Storages/ObjectStorage/ReadBufferIterator.h index 1f064368fb06..e14aef3863af 100644 --- a/src/Storages/ObjectStorage/ReadBufferIterator.h +++ b/src/Storages/ObjectStorage/ReadBufferIterator.h @@ -10,12 +10,11 @@ namespace DB class ReadBufferIterator : public IReadBufferIterator, WithContext { public: - using ConfigurationPtr = StorageObjectStorage::ConfigurationPtr; using ObjectInfos = StorageObjectStorage::ObjectInfos; ReadBufferIterator( ObjectStoragePtr object_storage_, - ConfigurationPtr configuration_, + StorageObjectStorageConfigurationPtr configuration_, const ObjectIterator & file_iterator_, const std::optional & format_settings_, SchemaCache & schema_cache_, @@ -45,10 +44,10 @@ class ReadBufferIterator : public IReadBufferIterator, WithContext const ObjectInfos::iterator & begin, const ObjectInfos::iterator & end); ObjectStoragePtr object_storage; - const ConfigurationPtr configuration; + const StorageObjectStorageConfigurationPtr configuration; const ObjectIterator file_iterator; const std::optional & format_settings; - const StorageObjectStorage::QuerySettings query_settings; + const StorageObjectStorageQuerySettings query_settings; SchemaCache & schema_cache; ObjectInfos & read_keys; std::optional format; diff --git a/src/Storages/ObjectStorage/S3/Configuration.cpp b/src/Storages/ObjectStorage/S3/Configuration.cpp index 4fd306707581..7a7d163ae8fb 100644 --- a/src/Storages/ObjectStorage/S3/Configuration.cpp +++ b/src/Storages/ObjectStorage/S3/Configuration.cpp @@ -119,7 +119,7 @@ void StorageS3Configuration::check(ContextPtr context) const validateNamespace(url.bucket); context->getGlobalContext()->getRemoteHostFilter().checkURL(url.uri); context->getGlobalContext()->getHTTPHeaderFilter().checkHeaders(headers_from_ast); - Configuration::check(context); + StorageObjectStorageConfiguration::check(context); } void StorageS3Configuration::validateNamespace(const String & name) const @@ -127,10 +127,10 @@ void StorageS3Configuration::validateNamespace(const String & name) const S3::URI::validateBucket(name, {}); } -StorageObjectStorage::QuerySettings StorageS3Configuration::getQuerySettings(const ContextPtr & context) const +StorageObjectStorageQuerySettings StorageS3Configuration::getQuerySettings(const ContextPtr & context) const { const auto & settings = context->getSettingsRef(); - return StorageObjectStorage::QuerySettings{ + return StorageObjectStorageQuerySettings{ .truncate_on_insert = settings[Setting::s3_truncate_on_insert], .create_new_file_on_insert = settings[Setting::s3_create_new_file_on_insert], .schema_inference_use_cache = settings[Setting::schema_inference_use_cache_for_s3], diff --git a/src/Storages/ObjectStorage/S3/Configuration.h b/src/Storages/ObjectStorage/S3/Configuration.h index 830d36494d5a..f4f784239704 100644 --- a/src/Storages/ObjectStorage/S3/Configuration.h +++ b/src/Storages/ObjectStorage/S3/Configuration.h @@ -11,11 +11,9 @@ namespace DB { -class StorageS3Configuration : public StorageObjectStorage::Configuration +class StorageS3Configuration : public StorageObjectStorageConfiguration { public: - using ConfigurationPtr = StorageObjectStorage::ConfigurationPtr; - static constexpr auto type = ObjectStorageType::S3; static constexpr auto type_name = "s3"; static constexpr auto namespace_name = "bucket"; @@ -83,7 +81,7 @@ class StorageS3Configuration : public StorageObjectStorage::Configuration String getNamespace() const override { return url.bucket; } String getDataSourceDescription() const override; - StorageObjectStorage::QuerySettings getQuerySettings(const ContextPtr &) const override; + StorageObjectStorageQuerySettings getQuerySettings(const ContextPtr &) const override; bool isArchive() const override { return url.archive_pattern.has_value(); } std::string getPathInArchive() const override; diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index 77d88e350ff2..d65b71badf08 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -26,6 +26,7 @@ #include #include #include +#include #include #include #include @@ -37,7 +38,6 @@ namespace DB { namespace Setting { - extern const SettingsMaxThreads max_threads; extern const SettingsBool optimize_count_from_files; extern const SettingsBool use_hive_partitioning; } @@ -86,7 +86,7 @@ String StorageObjectStorage::getPathSample(ContextPtr context) } StorageObjectStorage::StorageObjectStorage( - ConfigurationPtr configuration_, + StorageObjectStorageConfigurationPtr configuration_, ObjectStoragePtr object_storage_, ContextPtr context, const StorageID & table_id_, @@ -224,16 +224,6 @@ bool StorageObjectStorage::supportsSubsetOfColumns(const ContextPtr & context) c return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(configuration->format, context, format_settings); } -bool StorageObjectStorage::Configuration::update( ///NOLINT - ObjectStoragePtr object_storage_ptr, - ContextPtr context, - bool /* if_not_updated_before */, - bool /* check_consistent_with_previous_metadata */) -{ - IObjectStorage::ApplyNewSettingsOptions options{.allow_client_change = !isStaticConfiguration()}; - object_storage_ptr->applyNewSettings(context->getConfigRef(), getTypeName() + ".", context, options); - return true; -} IDataLakeMetadata * StorageObjectStorage::getExternalMetadata(ContextPtr query_context) { @@ -299,22 +289,6 @@ std::optional StorageObjectStorage::totalBytes(ContextPtr query_context) return configuration->totalBytes(query_context); } -ReadFromFormatInfo StorageObjectStorage::Configuration::prepareReadingFromFormat( - ObjectStoragePtr, - const Strings & requested_columns, - const StorageSnapshotPtr & storage_snapshot, - bool supports_subset_of_columns, - ContextPtr local_context, - const PrepareReadingFromFormatHiveParams & hive_parameters) -{ - return DB::prepareReadingFromFormat(requested_columns, storage_snapshot, local_context, supports_subset_of_columns, hive_parameters); -} - -std::optional StorageObjectStorage::Configuration::tryGetTableStructureFromMetadata() const -{ - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method tryGetTableStructureFromMetadata is not implemented for basic configuration"); -} - void StorageObjectStorage::read( QueryPlan & query_plan, const Names & column_names, @@ -469,7 +443,7 @@ void StorageObjectStorage::truncate( std::unique_ptr StorageObjectStorage::createReadBufferIterator( const ObjectStoragePtr & object_storage, - const ConfigurationPtr & configuration, + const StorageObjectStorageConfigurationPtr & configuration, const std::optional & format_settings, ObjectInfos & read_keys, const ContextPtr & context) @@ -493,7 +467,7 @@ std::unique_ptr StorageObjectStorage::createReadBufferIterat ColumnsDescription StorageObjectStorage::resolveSchemaFromData( const ObjectStoragePtr & object_storage, - const ConfigurationPtr & configuration, + const StorageObjectStorageConfigurationPtr & configuration, const std::optional & format_settings, std::string & sample_path, const ContextPtr & context) @@ -507,7 +481,7 @@ ColumnsDescription StorageObjectStorage::resolveSchemaFromData( std::string StorageObjectStorage::resolveFormatFromData( const ObjectStoragePtr & object_storage, - const ConfigurationPtr & configuration, + const StorageObjectStorageConfigurationPtr & configuration, const std::optional & format_settings, std::string & sample_path, const ContextPtr & context) @@ -521,7 +495,7 @@ std::string StorageObjectStorage::resolveFormatFromData( std::pair StorageObjectStorage::resolveSchemaAndFormatFromData( const ObjectStoragePtr & object_storage, - const ConfigurationPtr & configuration, + const StorageObjectStorageConfigurationPtr & configuration, const std::optional & format_settings, std::string & sample_path, const ContextPtr & context) @@ -570,152 +544,4 @@ SchemaCache & StorageObjectStorage::getSchemaCache(const ContextPtr & context, c throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unsupported storage type: {}", storage_type_name); } -void StorageObjectStorage::Configuration::initialize( - Configuration & configuration_to_initialize, - ASTs & engine_args, - ContextPtr local_context, - bool with_table_structure) -{ - if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args, local_context)) - configuration_to_initialize.fromNamedCollection(*named_collection, local_context); - else - configuration_to_initialize.fromAST(engine_args, local_context, with_table_structure); - - if (configuration_to_initialize.isNamespaceWithGlobs()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Expression can not have wildcards inside {} name", configuration_to_initialize.getNamespaceType()); - - if (configuration_to_initialize.isDataLakeConfiguration()) - { - if (configuration_to_initialize.partition_strategy_type != PartitionStrategyFactory::StrategyType::NONE) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "The `partition_strategy` argument is incompatible with data lakes"); - } - } - else if (configuration_to_initialize.partition_strategy_type == PartitionStrategyFactory::StrategyType::NONE) - { - // Promote to wildcard in case it is not data lake to make it backwards compatible - configuration_to_initialize.partition_strategy_type = PartitionStrategyFactory::StrategyType::WILDCARD; - } - - if (configuration_to_initialize.format == "auto") - { - if (configuration_to_initialize.isDataLakeConfiguration()) - { - configuration_to_initialize.format = "Parquet"; - } - else - { - configuration_to_initialize.format - = FormatFactory::instance() - .tryGetFormatFromFileName(configuration_to_initialize.isArchive() ? configuration_to_initialize.getPathInArchive() : configuration_to_initialize.getRawPath().path) - .value_or("auto"); - } - } - else - FormatFactory::instance().checkFormatName(configuration_to_initialize.format); - - /// It might be changed on `StorageObjectStorage::Configuration::initPartitionStrategy` - configuration_to_initialize.read_path = configuration_to_initialize.getRawPath(); - configuration_to_initialize.initialized = true; - -} - -void StorageObjectStorage::Configuration::initPartitionStrategy(ASTPtr partition_by, const ColumnsDescription & columns, ContextPtr context) -{ - partition_strategy = PartitionStrategyFactory::get( - partition_strategy_type, - partition_by, - columns.getOrdinary(), - context, - format, - getRawPath().hasGlobs(), - getRawPath().hasPartitionWildcard(), - partition_columns_in_data_file); - - if (partition_strategy) - { - read_path = partition_strategy->getPathForRead(getRawPath().path); - LOG_DEBUG(getLogger("StorageObjectStorageConfiguration"), "Initialized partition strategy {}", magic_enum::enum_name(partition_strategy_type)); - } -} - -const StorageObjectStorage::Configuration::Path & StorageObjectStorage::Configuration::getPathForRead() const -{ - return read_path; -} - -StorageObjectStorage::Configuration::Path StorageObjectStorage::Configuration::getPathForWrite(const std::string & partition_id) const -{ - auto raw_path = getRawPath(); - - if (!partition_strategy) - { - return raw_path; - } - - return Path {partition_strategy->getPathForWrite(raw_path.path, partition_id)}; -} - - -bool StorageObjectStorage::Configuration::Path::hasPartitionWildcard() const -{ - static const String PARTITION_ID_WILDCARD = "{_partition_id}"; - return path.find(PARTITION_ID_WILDCARD) != String::npos; -} - -bool StorageObjectStorage::Configuration::Path::hasGlobsIgnorePartitionWildcard() const -{ - if (!hasPartitionWildcard()) - return hasGlobs(); - return PartitionedSink::replaceWildcards(path, "").find_first_of("*?{") != std::string::npos; -} - -bool StorageObjectStorage::Configuration::Path::hasGlobs() const -{ - return path.find_first_of("*?{") != std::string::npos; -} - -std::string StorageObjectStorage::Configuration::Path::cutGlobs(bool supports_partial_prefix) const -{ - if (supports_partial_prefix) - { - return path.substr(0, path.find_first_of("*?{")); - } - - auto first_glob_pos = path.find_first_of("*?{"); - auto end_of_path_without_globs = path.substr(0, first_glob_pos).rfind('/'); - if (end_of_path_without_globs == std::string::npos || end_of_path_without_globs == 0) - return "/"; - return path.substr(0, end_of_path_without_globs); -} - -void StorageObjectStorage::Configuration::check(ContextPtr) const -{ - FormatFactory::instance().checkFormatName(format); -} - -bool StorageObjectStorage::Configuration::isNamespaceWithGlobs() const -{ - return getNamespace().find_first_of("*?{") != std::string::npos; -} - -bool StorageObjectStorage::Configuration::isPathInArchiveWithGlobs() const -{ - return getPathInArchive().find_first_of("*?{") != std::string::npos; -} - -std::string StorageObjectStorage::Configuration::getPathInArchive() const -{ - throw Exception(ErrorCodes::LOGICAL_ERROR, "Path {} is not archive", getRawPath().path); -} - -void StorageObjectStorage::Configuration::assertInitialized() const -{ - if (!initialized) - { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Configuration was not initialized before usage"); - } -} - } diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.h b/src/Storages/ObjectStorage/StorageObjectStorage.h index a4407a426647..009c2f3e2834 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.h +++ b/src/Storages/ObjectStorage/StorageObjectStorage.h @@ -11,6 +11,7 @@ #include #include #include +#include #include #include @@ -22,17 +23,10 @@ namespace DB { class ReadBufferIterator; class SchemaCache; -class NamedCollection; struct StorageObjectStorageSettings; using StorageObjectStorageSettingsPtr = std::shared_ptr; struct IPartitionStrategy; -namespace ErrorCodes -{ - extern const int NOT_IMPLEMENTED; -} - - /** * A general class containing implementation for external table engines * such as StorageS3, StorageAzure, StorageHDFS. @@ -41,32 +35,12 @@ namespace ErrorCodes class StorageObjectStorage : public IStorage { public: - class Configuration; - using ConfigurationPtr = std::shared_ptr; - using ConfigurationObserverPtr = std::weak_ptr; using ObjectInfo = RelativePathWithMetadata; using ObjectInfoPtr = std::shared_ptr; using ObjectInfos = std::vector; - struct QuerySettings - { - /// Insert settings: - bool truncate_on_insert; - bool create_new_file_on_insert; - - /// Schema inference settings: - bool schema_inference_use_cache; - SchemaInferenceMode schema_inference_mode; - - /// List settings: - bool skip_empty_files; - size_t list_object_keys_size; - bool throw_on_zero_files_match; - bool ignore_non_existent_file; - }; - StorageObjectStorage( - ConfigurationPtr configuration_, + StorageObjectStorageConfigurationPtr configuration_, ObjectStoragePtr object_storage_, ContextPtr context_, const StorageID & table_id_, @@ -122,21 +96,21 @@ class StorageObjectStorage : public IStorage static ColumnsDescription resolveSchemaFromData( const ObjectStoragePtr & object_storage, - const ConfigurationPtr & configuration, + const StorageObjectStorageConfigurationPtr & configuration, const std::optional & format_settings, std::string & sample_path, const ContextPtr & context); static std::string resolveFormatFromData( const ObjectStoragePtr & object_storage, - const ConfigurationPtr & configuration, + const StorageObjectStorageConfigurationPtr & configuration, const std::optional & format_settings, std::string & sample_path, const ContextPtr & context); static std::pair resolveSchemaAndFormatFromData( const ObjectStoragePtr & object_storage, - const ConfigurationPtr & configuration, + const StorageObjectStorageConfigurationPtr & configuration, const std::optional & format_settings, std::string & sample_path, const ContextPtr & context); @@ -157,7 +131,7 @@ class StorageObjectStorage : public IStorage /// Creates ReadBufferIterator for schema inference implementation. static std::unique_ptr createReadBufferIterator( const ObjectStoragePtr & object_storage, - const ConfigurationPtr & configuration, + const StorageObjectStorageConfigurationPtr & configuration, const std::optional & format_settings, ObjectInfos & read_keys, const ContextPtr & context); @@ -165,7 +139,7 @@ class StorageObjectStorage : public IStorage /// Storage configuration (S3, Azure, HDFS, Local, DataLake). /// Contains information about table engine configuration /// and underlying storage access. - ConfigurationPtr configuration; + StorageObjectStorageConfigurationPtr configuration; /// `object_storage` to allow direct access to data storage. const ObjectStoragePtr object_storage; const std::optional format_settings; @@ -182,162 +156,4 @@ class StorageObjectStorage : public IStorage LoggerPtr log; }; -class StorageObjectStorage::Configuration -{ -public: - Configuration() = default; - virtual ~Configuration() = default; - - struct Path - { - Path() = default; - /// A partial prefix is a prefix that does not represent an actual object (directory or file), usually strings that do not end with a slash character. - /// Example: `table_root/year=20`. AWS S3 supports partial prefixes, but HDFS does not. - Path(const std::string & path_) : path(path_) {} /// NOLINT(google-explicit-constructor) - - std::string path; - - bool hasPartitionWildcard() const; - bool hasGlobsIgnorePartitionWildcard() const; - bool hasGlobs() const; - std::string cutGlobs(bool supports_partial_prefix) const; - }; - - using Paths = std::vector; - - /// Initialize configuration from either AST or NamedCollection. - static void initialize( - Configuration & configuration_to_initialize, - ASTs & engine_args, - ContextPtr local_context, - bool with_table_structure); - - /// Storage type: s3, hdfs, azure, local. - virtual ObjectStorageType getType() const = 0; - virtual std::string getTypeName() const = 0; - /// Engine name: S3, HDFS, Azure. - virtual std::string getEngineName() const = 0; - /// Sometimes object storages have something similar to chroot or namespace, for example - /// buckets in S3. If object storage doesn't have any namepaces return empty string. - virtual std::string getNamespaceType() const { return "namespace"; } - - // Path provided by the user in the query - virtual Path getRawPath() const = 0; - - const Path & getPathForRead() const; - // Path used for writing, it should not be globbed and might contain a partition key - Path getPathForWrite(const std::string & partition_id = "") const; - - void setPathForRead(const Path & path) - { - read_path = path; - } - - /* - * When using `s3_create_new_file_on_insert`, each new file path generated will be appended to the path list. - * This list is used to determine the next file name and the set of files that shall be read from remote storage. - * This is not ideal, there are much better ways to implement reads and writes. It should be eventually removed - */ - virtual const Paths & getPaths() const = 0; - - virtual void setPaths(const Paths & paths) = 0; - - virtual String getDataSourceDescription() const = 0; - virtual String getNamespace() const = 0; - - virtual StorageObjectStorage::QuerySettings getQuerySettings(const ContextPtr &) const = 0; - - /// Add/replace structure and format arguments in the AST arguments if they have 'auto' values. - virtual void addStructureAndFormatToArgsIfNeeded( - ASTs & args, const String & structure_, const String & format_, ContextPtr context, bool with_structure) = 0; - - bool isNamespaceWithGlobs() const; - - virtual bool isArchive() const { return false; } - bool isPathInArchiveWithGlobs() const; - virtual std::string getPathInArchive() const; - - virtual void check(ContextPtr context) const; - virtual void validateNamespace(const String & /* name */) const {} - - virtual ObjectStoragePtr createObjectStorage(ContextPtr context, bool is_readonly) = 0; - virtual bool isStaticConfiguration() const { return true; } - - virtual bool isDataLakeConfiguration() const { return false; } - - virtual std::optional totalRows(ContextPtr) { return {}; } - virtual std::optional totalBytes(ContextPtr) { return {}; } - - virtual bool hasExternalDynamicMetadata() { return false; } - - virtual IDataLakeMetadata * getExternalMetadata() { return nullptr; } - - virtual std::shared_ptr getInitialSchemaByPath(ContextPtr, const String &) const { return {}; } - - virtual std::shared_ptr getSchemaTransformer(ContextPtr, const String &) const { return {}; } - - virtual void modifyFormatSettings(FormatSettings &) const {} - - virtual ReadFromFormatInfo prepareReadingFromFormat( - ObjectStoragePtr object_storage, - const Strings & requested_columns, - const StorageSnapshotPtr & storage_snapshot, - bool supports_subset_of_columns, - ContextPtr local_context, - const PrepareReadingFromFormatHiveParams & hive_parameters); - - void initPartitionStrategy(ASTPtr partition_by, const ColumnsDescription & columns, ContextPtr context); - - virtual std::optional tryGetTableStructureFromMetadata() const; - - virtual bool supportsFileIterator() const { return false; } - virtual bool supportsWrites() const { return true; } - - virtual bool supportsPartialPathPrefix() const { return true; } - - virtual ObjectIterator iterate( - const ActionsDAG * /* filter_dag */, - std::function /* callback */, - size_t /* list_batch_size */, - ContextPtr /*context*/) - { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method iterate() is not implemented for configuration type {}", getTypeName()); - } - - /// Returns true, if metadata is of the latest version, false if unknown. - virtual bool update( - ObjectStoragePtr object_storage, - ContextPtr local_context, - bool if_not_updated_before, - bool check_consistent_with_previous_metadata); - - virtual const DataLakeStorageSettings & getDataLakeSettings() const - { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getDataLakeSettings() is not implemented for configuration type {}", getTypeName()); - } - - String format = "auto"; - String compression_method = "auto"; - String structure = "auto"; - PartitionStrategyFactory::StrategyType partition_strategy_type = PartitionStrategyFactory::StrategyType::NONE; - /// Whether partition column values are contained in the actual data. - /// And alternative is with hive partitioning, when they are contained in file path. - bool partition_columns_in_data_file = true; - std::shared_ptr partition_strategy; - -protected: - virtual void fromNamedCollection(const NamedCollection & collection, ContextPtr context) = 0; - virtual void fromAST(ASTs & args, ContextPtr context, bool with_structure) = 0; - - void assertInitialized() const; - - bool initialized = false; - -private: - // Path used for reading, by default it is the same as `getRawPath` - // When using `partition_strategy=hive`, a recursive reading pattern will be appended `'table_root/**.parquet' - Path read_path; - -}; - } diff --git a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp index 6e3af8de409b..da547f0f957c 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp @@ -57,7 +57,7 @@ String StorageObjectStorageCluster::getPathSample(ContextPtr context) StorageObjectStorageCluster::StorageObjectStorageCluster( const String & cluster_name_, - ConfigurationPtr configuration_, + StorageObjectStorageConfigurationPtr configuration_, ObjectStoragePtr object_storage_, const StorageID & table_id_, const ColumnsDescription & columns_in_table_or_function_definition, @@ -189,12 +189,27 @@ void StorageObjectStorageCluster::updateQueryToSendIfNeeded( } } + RemoteQueryExecutor::Extension StorageObjectStorageCluster::getTaskIteratorExtension( - const ActionsDAG::Node * predicate, const ContextPtr & local_context, const size_t number_of_replicas) const + const ActionsDAG::Node * predicate, + const ActionsDAG * filter, + const ContextPtr & local_context, + const size_t number_of_replicas) const { auto iterator = StorageObjectStorageSource::createFileIterator( - configuration, configuration->getQuerySettings(local_context), object_storage, /* distributed_processing */false, - local_context, predicate, {}, virtual_columns, hive_partition_columns_to_read_from_file_path, nullptr, local_context->getFileProgressCallback(), /*ignore_archive_globs=*/true, /*skip_object_metadata=*/true); + configuration, + configuration->getQuerySettings(local_context), + object_storage, + /* distributed_processing */false, + local_context, + predicate, + filter, + virtual_columns, + hive_partition_columns_to_read_from_file_path, + nullptr, + local_context->getFileProgressCallback(), + /*ignore_archive_globs=*/true, + /*skip_object_metadata=*/true); auto task_distributor = std::make_shared(iterator, number_of_replicas); diff --git a/src/Storages/ObjectStorage/StorageObjectStorageCluster.h b/src/Storages/ObjectStorage/StorageObjectStorageCluster.h index 1a557143076a..313b09b41fec 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageCluster.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageCluster.h @@ -10,11 +10,9 @@ namespace DB class StorageObjectStorageCluster : public IStorageCluster { public: - using ConfigurationPtr = StorageObjectStorage::ConfigurationPtr; - StorageObjectStorageCluster( const String & cluster_name_, - ConfigurationPtr configuration_, + StorageObjectStorageConfigurationPtr configuration_, ObjectStoragePtr object_storage_, const StorageID & table_id_, const ColumnsDescription & columns_in_table_or_function_definition, @@ -25,7 +23,10 @@ class StorageObjectStorageCluster : public IStorageCluster std::string getName() const override; RemoteQueryExecutor::Extension getTaskIteratorExtension( - const ActionsDAG::Node * predicate, const ContextPtr & context, size_t number_of_replicas) const override; + const ActionsDAG::Node * predicate, + const ActionsDAG * filter, + const ContextPtr & context, + size_t number_of_replicas) const override; String getPathSample(ContextPtr context); @@ -39,7 +40,7 @@ class StorageObjectStorageCluster : public IStorageCluster const ContextPtr & context) override; const String engine_name; - const StorageObjectStorage::ConfigurationPtr configuration; + const StorageObjectStorageConfigurationPtr configuration; const ObjectStoragePtr object_storage; NamesAndTypesList virtual_columns; NamesAndTypesList hive_partition_columns_to_read_from_file_path; diff --git a/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.cpp b/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.cpp new file mode 100644 index 000000000000..6f649ce57f47 --- /dev/null +++ b/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.cpp @@ -0,0 +1,194 @@ +#include + +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; + extern const int LOGICAL_ERROR; + extern const int BAD_ARGUMENTS; +} + +bool StorageObjectStorageConfiguration::update( ///NOLINT + ObjectStoragePtr object_storage_ptr, + ContextPtr context, + bool /* if_not_updated_before */, + bool /* check_consistent_with_previous_metadata */) +{ + IObjectStorage::ApplyNewSettingsOptions options{.allow_client_change = !isStaticConfiguration()}; + object_storage_ptr->applyNewSettings(context->getConfigRef(), getTypeName() + ".", context, options); + return true; +} + +ReadFromFormatInfo StorageObjectStorageConfiguration::prepareReadingFromFormat( + ObjectStoragePtr, + const Strings & requested_columns, + const StorageSnapshotPtr & storage_snapshot, + bool supports_subset_of_columns, + ContextPtr local_context, + const PrepareReadingFromFormatHiveParams & hive_parameters) +{ + return DB::prepareReadingFromFormat(requested_columns, storage_snapshot, local_context, supports_subset_of_columns, hive_parameters); +} + +std::optional StorageObjectStorageConfiguration::tryGetTableStructureFromMetadata() const +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method tryGetTableStructureFromMetadata is not implemented for basic configuration"); +} + +void StorageObjectStorageConfiguration::initialize( + StorageObjectStorageConfiguration & configuration_to_initialize, + ASTs & engine_args, + ContextPtr local_context, + bool with_table_structure) +{ + if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args, local_context)) + configuration_to_initialize.fromNamedCollection(*named_collection, local_context); + else + configuration_to_initialize.fromAST(engine_args, local_context, with_table_structure); + + if (configuration_to_initialize.isNamespaceWithGlobs()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Expression can not have wildcards inside {} name", configuration_to_initialize.getNamespaceType()); + + if (configuration_to_initialize.isDataLakeConfiguration()) + { + if (configuration_to_initialize.partition_strategy_type != PartitionStrategyFactory::StrategyType::NONE) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "The `partition_strategy` argument is incompatible with data lakes"); + } + } + else if (configuration_to_initialize.partition_strategy_type == PartitionStrategyFactory::StrategyType::NONE) + { + // Promote to wildcard in case it is not data lake to make it backwards compatible + configuration_to_initialize.partition_strategy_type = PartitionStrategyFactory::StrategyType::WILDCARD; + } + + if (configuration_to_initialize.format == "auto") + { + if (configuration_to_initialize.isDataLakeConfiguration()) + { + configuration_to_initialize.format = "Parquet"; + } + else + { + configuration_to_initialize.format + = FormatFactory::instance() + .tryGetFormatFromFileName(configuration_to_initialize.isArchive() ? configuration_to_initialize.getPathInArchive() : configuration_to_initialize.getRawPath().path) + .value_or("auto"); + } + } + else + FormatFactory::instance().checkFormatName(configuration_to_initialize.format); + + /// It might be changed on `StorageObjectStorageConfiguration::initPartitionStrategy` + configuration_to_initialize.read_path = configuration_to_initialize.getRawPath(); + configuration_to_initialize.initialized = true; +} + +void StorageObjectStorageConfiguration::initPartitionStrategy(ASTPtr partition_by, const ColumnsDescription & columns, ContextPtr context) +{ + partition_strategy = PartitionStrategyFactory::get( + partition_strategy_type, + partition_by, + columns.getOrdinary(), + context, + format, + getRawPath().hasGlobs(), + getRawPath().hasPartitionWildcard(), + partition_columns_in_data_file); + + if (partition_strategy) + { + read_path = partition_strategy->getPathForRead(getRawPath().path); + LOG_DEBUG(getLogger("StorageObjectStorageConfiguration"), "Initialized partition strategy {}", magic_enum::enum_name(partition_strategy_type)); + } +} + +const StorageObjectStorageConfiguration::Path & StorageObjectStorageConfiguration::getPathForRead() const +{ + return read_path; +} + +StorageObjectStorageConfiguration::Path StorageObjectStorageConfiguration::getPathForWrite(const std::string & partition_id) const +{ + auto raw_path = getRawPath(); + + if (!partition_strategy) + { + return raw_path; + } + + return Path {partition_strategy->getPathForWrite(raw_path.path, partition_id)}; +} + +bool StorageObjectStorageConfiguration::Path::hasPartitionWildcard() const +{ + static const String PARTITION_ID_WILDCARD = "{_partition_id}"; + return path.find(PARTITION_ID_WILDCARD) != String::npos; +} + +bool StorageObjectStorageConfiguration::Path::hasGlobsIgnorePartitionWildcard() const +{ + if (!hasPartitionWildcard()) + return hasGlobs(); + return PartitionedSink::replaceWildcards(path, "").find_first_of("*?{") != std::string::npos; +} + +bool StorageObjectStorageConfiguration::Path::hasGlobs() const +{ + return path.find_first_of("*?{") != std::string::npos; +} + +std::string StorageObjectStorageConfiguration::Path::cutGlobs(bool supports_partial_prefix) const +{ + if (supports_partial_prefix) + { + return path.substr(0, path.find_first_of("*?{")); + } + + auto first_glob_pos = path.find_first_of("*?{"); + auto end_of_path_without_globs = path.substr(0, first_glob_pos).rfind('/'); + if (end_of_path_without_globs == std::string::npos || end_of_path_without_globs == 0) + return "/"; + return path.substr(0, end_of_path_without_globs); +} + +void StorageObjectStorageConfiguration::check(ContextPtr) const +{ + FormatFactory::instance().checkFormatName(format); +} + +bool StorageObjectStorageConfiguration::isNamespaceWithGlobs() const +{ + return getNamespace().find_first_of("*?{") != std::string::npos; +} + +bool StorageObjectStorageConfiguration::isPathInArchiveWithGlobs() const +{ + return getPathInArchive().find_first_of("*?{") != std::string::npos; +} + +std::string StorageObjectStorageConfiguration::getPathInArchive() const +{ + throw Exception(ErrorCodes::LOGICAL_ERROR, "Path {} is not archive", getRawPath().path); +} + +void StorageObjectStorageConfiguration::assertInitialized() const +{ + if (!initialized) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Configuration was not initialized before usage"); + } +} + + +} diff --git a/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.h b/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.h new file mode 100644 index 000000000000..da36933cac79 --- /dev/null +++ b/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.h @@ -0,0 +1,203 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +class NamedCollection; + +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + + +struct StorageObjectStorageQuerySettings +{ + /// Insert settings: + bool truncate_on_insert; + bool create_new_file_on_insert; + + /// Schema inference settings: + bool schema_inference_use_cache; + SchemaInferenceMode schema_inference_mode; + + /// List settings: + bool skip_empty_files; + size_t list_object_keys_size; + bool throw_on_zero_files_match; + bool ignore_non_existent_file; +}; + + +class StorageObjectStorageConfiguration +{ +public: + StorageObjectStorageConfiguration() = default; + virtual ~StorageObjectStorageConfiguration() = default; + + struct Path + { + Path() = default; + /// A partial prefix is a prefix that does not represent an actual object (directory or file), usually strings that do not end with a slash character. + /// Example: `table_root/year=20`. AWS S3 supports partial prefixes, but HDFS does not. + Path(const std::string & path_) : path(path_) {} /// NOLINT(google-explicit-constructor) + + std::string path; + + bool hasPartitionWildcard() const; + bool hasGlobsIgnorePartitionWildcard() const; + bool hasGlobs() const; + std::string cutGlobs(bool supports_partial_prefix) const; + }; + using Paths = std::vector; + + /// Initialize configuration from either AST or NamedCollection. + static void initialize( + StorageObjectStorageConfiguration & configuration_to_initialize, + ASTs & engine_args, + ContextPtr local_context, + bool with_table_structure); + + /// Storage type: s3, hdfs, azure, local. + virtual ObjectStorageType getType() const = 0; + virtual std::string getTypeName() const = 0; + /// Engine name: S3, HDFS, Azure. + virtual std::string getEngineName() const = 0; + /// Sometimes object storages have something similar to chroot or namespace, for example + /// buckets in S3. If object storage doesn't have any namepaces return empty string. + virtual std::string getNamespaceType() const { return "namespace"; } + + // Path provided by the user in the query + virtual Path getRawPath() const = 0; + + const Path & getPathForRead() const; + // Path used for writing, it should not be globbed and might contain a partition key + Path getPathForWrite(const std::string & partition_id = "") const; + + void setPathForRead(const Path & path) + { + read_path = path; + } + + /* + * When using `s3_create_new_file_on_insert`, each new file path generated will be appended to the path list. + * This list is used to determine the next file name and the set of files that shall be read from remote storage. + * This is not ideal, there are much better ways to implement reads and writes. It should be eventually removed + */ + virtual const Paths & getPaths() const = 0; + virtual void setPaths(const Paths & paths) = 0; + + virtual String getDataSourceDescription() const = 0; + virtual String getNamespace() const = 0; + + virtual StorageObjectStorageQuerySettings getQuerySettings(const ContextPtr &) const = 0; + + /// Add/replace structure and format arguments in the AST arguments if they have 'auto' values. + virtual void addStructureAndFormatToArgsIfNeeded( + ASTs & args, const String & structure_, const String & format_, ContextPtr context, bool with_structure) = 0; + + bool isNamespaceWithGlobs() const; + + virtual bool isArchive() const { return false; } + bool isPathInArchiveWithGlobs() const; + virtual std::string getPathInArchive() const; + + virtual void check(ContextPtr context) const; + virtual void validateNamespace(const String & /* name */) const {} + + virtual ObjectStoragePtr createObjectStorage(ContextPtr context, bool is_readonly) = 0; + virtual bool isStaticConfiguration() const { return true; } + + virtual bool isDataLakeConfiguration() const { return false; } + + virtual std::optional totalRows(ContextPtr) { return {}; } + virtual std::optional totalBytes(ContextPtr) { return {}; } + + virtual bool hasExternalDynamicMetadata() { return false; } + + virtual IDataLakeMetadata * getExternalMetadata() { return nullptr; } + + virtual std::shared_ptr getInitialSchemaByPath(ContextPtr, const String &) const { return {}; } + + virtual std::shared_ptr getSchemaTransformer(ContextPtr, const String &) const { return {}; } + + virtual void modifyFormatSettings(FormatSettings &) const {} + + virtual ReadFromFormatInfo prepareReadingFromFormat( + ObjectStoragePtr object_storage, + const Strings & requested_columns, + const StorageSnapshotPtr & storage_snapshot, + bool supports_subset_of_columns, + ContextPtr local_context, + const PrepareReadingFromFormatHiveParams & hive_parameters); + + void initPartitionStrategy(ASTPtr partition_by, const ColumnsDescription & columns, ContextPtr context); + + virtual std::optional tryGetTableStructureFromMetadata() const; + + virtual bool supportsFileIterator() const { return false; } + virtual bool supportsWrites() const { return true; } + + virtual bool supportsPartialPathPrefix() const { return true; } + + virtual ObjectIterator iterate( + const ActionsDAG * /* filter_dag */, + std::function /* callback */, + size_t /* list_batch_size */, + ContextPtr /*context*/) + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method iterate() is not implemented for configuration type {}", getTypeName()); + } + + /// Returns true, if metadata is of the latest version, false if unknown. + virtual bool update( + ObjectStoragePtr object_storage, + ContextPtr local_context, + bool if_not_updated_before, + bool check_consistent_with_previous_metadata); + + virtual const DataLakeStorageSettings & getDataLakeSettings() const + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getDataLakeSettings() is not implemented for configuration type {}", getTypeName()); + } + + virtual ColumnMapperPtr getColumnMapper() const { return nullptr; } + + String format = "auto"; + String compression_method = "auto"; + String structure = "auto"; + PartitionStrategyFactory::StrategyType partition_strategy_type = PartitionStrategyFactory::StrategyType::NONE; + /// Whether partition column values are contained in the actual data. + /// And alternative is with hive partitioning, when they are contained in file path. + bool partition_columns_in_data_file = true; + std::shared_ptr partition_strategy; + +protected: + virtual void fromNamedCollection(const NamedCollection & collection, ContextPtr context) = 0; + virtual void fromAST(ASTs & args, ContextPtr context, bool with_structure) = 0; + + void assertInitialized() const; + + bool initialized = false; + +private: + // Path used for reading, by default it is the same as `getRawPath` + // When using `partition_strategy=hive`, a recursive reading pattern will be appended `'table_root/**.parquet' + Path read_path; +}; + +using StorageObjectStorageConfigurationPtr = std::shared_ptr; +using StorageObjectStorageConfigurationWeakPtr = std::weak_ptr; + + +} diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp index e23787b5aa6b..1cf68e033ee4 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp @@ -38,7 +38,7 @@ namespace PartitionedSink::validatePartitionKey(str, true); } - void validateNamespace(const String & str, StorageObjectStorage::ConfigurationPtr configuration) + void validateNamespace(const String & str, StorageObjectStorageConfigurationPtr configuration) { configuration->validateNamespace(str); @@ -52,7 +52,7 @@ namespace StorageObjectStorageSink::StorageObjectStorageSink( const std::string & path_, ObjectStoragePtr object_storage, - ConfigurationPtr configuration, + StorageObjectStorageConfigurationPtr configuration, const std::optional & format_settings_, const Block & sample_block_, ContextPtr context) @@ -129,7 +129,7 @@ void StorageObjectStorageSink::cancelBuffers() PartitionedStorageObjectStorageSink::PartitionedStorageObjectStorageSink( ObjectStoragePtr object_storage_, - ConfigurationPtr configuration_, + StorageObjectStorageConfigurationPtr configuration_, std::optional format_settings_, const Block & sample_block_, ContextPtr context_) diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSink.h b/src/Storages/ObjectStorage/StorageObjectStorageSink.h index ebfee5ab96e6..f566617834ce 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSink.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageSink.h @@ -9,12 +9,10 @@ namespace DB class StorageObjectStorageSink : public SinkToStorage { public: - using ConfigurationPtr = StorageObjectStorage::ConfigurationPtr; - StorageObjectStorageSink( const std::string & path_, ObjectStoragePtr object_storage, - ConfigurationPtr configuration, + StorageObjectStorageConfigurationPtr configuration, const std::optional & format_settings_, const Block & sample_block_, ContextPtr context); @@ -41,11 +39,9 @@ class StorageObjectStorageSink : public SinkToStorage class PartitionedStorageObjectStorageSink : public PartitionedSink { public: - using ConfigurationPtr = StorageObjectStorage::ConfigurationPtr; - PartitionedStorageObjectStorageSink( ObjectStoragePtr object_storage_, - ConfigurationPtr configuration_, + StorageObjectStorageConfigurationPtr configuration_, std::optional format_settings_, const Block & sample_block_, ContextPtr context_); @@ -54,9 +50,9 @@ class PartitionedStorageObjectStorageSink : public PartitionedSink private: ObjectStoragePtr object_storage; - ConfigurationPtr configuration; + StorageObjectStorageConfigurationPtr configuration; - const StorageObjectStorage::QuerySettings query_settings; + const StorageObjectStorageQuerySettings query_settings; const std::optional format_settings; const Block sample_block; const ContextPtr context; diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index 58345e9489a0..d3e144f04a3f 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -77,15 +77,15 @@ namespace ErrorCodes StorageObjectStorageSource::StorageObjectStorageSource( String name_, ObjectStoragePtr object_storage_, - ConfigurationPtr configuration_, + StorageObjectStorageConfigurationPtr configuration_, const ReadFromFormatInfo & info, const std::optional & format_settings_, ContextPtr context_, UInt64 max_block_size_, std::shared_ptr file_iterator_, - size_t max_parsing_threads_, + FormatParserGroupPtr parser_group_, bool need_only_count_) - : SourceWithKeyCondition(info.source_header, false) + : ISource(info.source_header, false) , name(std::move(name_)) , object_storage(object_storage_) , configuration(configuration_) @@ -93,7 +93,7 @@ StorageObjectStorageSource::StorageObjectStorageSource( , format_settings(format_settings_) , max_block_size(max_block_size_) , need_only_count(need_only_count_) - , max_parsing_threads(max_parsing_threads_) + , parser_group(std::move(parser_group_)) , read_from_format_info(info) , create_reader_pool(std::make_shared( CurrentMetrics::StorageObjectStorageThreads, @@ -111,13 +111,8 @@ StorageObjectStorageSource::~StorageObjectStorageSource() create_reader_pool->wait(); } -void StorageObjectStorageSource::setKeyCondition(const std::optional & filter_actions_dag, ContextPtr context_) -{ - setKeyConditionImpl(filter_actions_dag, context_, read_from_format_info.format_header); -} - std::string StorageObjectStorageSource::getUniqueStoragePathIdentifier( - const Configuration & configuration, + const StorageObjectStorageConfiguration & configuration, const ObjectInfo & object_info, bool include_connection_info) { @@ -131,13 +126,13 @@ std::string StorageObjectStorageSource::getUniqueStoragePathIdentifier( } std::shared_ptr StorageObjectStorageSource::createFileIterator( - ConfigurationPtr configuration, - const StorageObjectStorage::QuerySettings & query_settings, + StorageObjectStorageConfigurationPtr configuration, + const StorageObjectStorageQuerySettings & query_settings, ObjectStoragePtr object_storage, bool distributed_processing, const ContextPtr & local_context, const ActionsDAG::Node * predicate, - const std::optional & filter_actions_dag, + const ActionsDAG * filter_actions_dag, const NamesAndTypesList & virtual_columns, const NamesAndTypesList & hive_columns, ObjectInfos * read_keys, @@ -178,7 +173,7 @@ std::shared_ptr StorageObjectStorageSource::createFileIterator( else if (configuration->supportsFileIterator()) { return configuration->iterate( - filter_actions_dag.has_value() ? &filter_actions_dag.value() : nullptr, + filter_actions_dag, file_progress_callback, query_settings.list_object_keys_size, local_context); @@ -377,7 +372,7 @@ Chunk StorageObjectStorageSource::generate() } if (reader.getInputFormat() && read_context->getSettingsRef()[Setting::use_cache_for_count_from_files] - && (!key_condition || key_condition->alwaysUnknownOrTrue())) + && !parser_group->filter_actions_dag) addNumRowsToCache(*reader.getObjectInfo(), total_rows_in_file); total_rows_in_file = 0; @@ -413,28 +408,26 @@ StorageObjectStorageSource::ReaderHolder StorageObjectStorageSource::createReade object_storage, read_from_format_info, format_settings, - key_condition, read_context, &schema_cache, log, max_block_size, - max_parsing_threads, + parser_group, need_only_count); } StorageObjectStorageSource::ReaderHolder StorageObjectStorageSource::createReader( size_t processor, const std::shared_ptr & file_iterator, - const ConfigurationPtr & configuration, + const StorageObjectStorageConfigurationPtr & configuration, const ObjectStoragePtr & object_storage, ReadFromFormatInfo & read_from_format_info, const std::optional & format_settings, - const std::shared_ptr & key_condition_, const ContextPtr & context_, SchemaCache * schema_cache, const LoggerPtr & log, size_t max_block_size, - size_t max_parsing_threads, + FormatParserGroupPtr parser_group, bool need_only_count) { ObjectInfoPtr object_info; @@ -529,7 +522,6 @@ StorageObjectStorageSource::ReaderHolder StorageObjectStorageSource::createReade initial_header = sample_header; } - auto input_format = FormatFactory::instance().getInput( configuration->format, *read_buf, @@ -537,17 +529,13 @@ StorageObjectStorageSource::ReaderHolder StorageObjectStorageSource::createReade context_, max_block_size, format_settings, - need_only_count ? 1 : max_parsing_threads, - std::nullopt, + parser_group, true /* is_remote_fs */, compression_method, need_only_count); input_format->setSerializationHints(read_from_format_info.serialization_hints); - if (key_condition_) - input_format->setKeyCondition(key_condition_); - if (need_only_count) input_format->needOnlyCount(); @@ -773,7 +761,7 @@ std::unique_ptr StorageObjectStorageSource::createReadBu StorageObjectStorageSource::GlobIterator::GlobIterator( ObjectStoragePtr object_storage_, - ConfigurationPtr configuration_, + StorageObjectStorageConfigurationPtr configuration_, const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns_, const NamesAndTypesList & hive_columns_, @@ -1068,7 +1056,7 @@ StorageObjectStorageSource::ArchiveIterator::ObjectInfoInArchive::ObjectInfoInAr StorageObjectStorageSource::ArchiveIterator::ArchiveIterator( ObjectStoragePtr object_storage_, - ConfigurationPtr configuration_, + StorageObjectStorageConfigurationPtr configuration_, std::unique_ptr archives_iterator_, ContextPtr context_, ObjectInfos * read_keys_, diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.h b/src/Storages/ObjectStorage/StorageObjectStorageSource.h index 6221c1508100..8ae7c280c5da 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.h @@ -3,11 +3,11 @@ #include #include #include -#include #include #include #include #include +#include namespace DB @@ -17,12 +17,10 @@ class SchemaCache; using ReadTaskCallback = std::function; -class StorageObjectStorageSource : public SourceWithKeyCondition +class StorageObjectStorageSource : public ISource { friend class ObjectStorageQueueSource; public: - using Configuration = StorageObjectStorage::Configuration; - using ConfigurationPtr = StorageObjectStorage::ConfigurationPtr; using ObjectInfos = StorageObjectStorage::ObjectInfos; class ReadTaskIterator; @@ -33,31 +31,31 @@ class StorageObjectStorageSource : public SourceWithKeyCondition StorageObjectStorageSource( String name_, ObjectStoragePtr object_storage_, - ConfigurationPtr configuration, + StorageObjectStorageConfigurationPtr configuration, const ReadFromFormatInfo & info, const std::optional & format_settings_, ContextPtr context_, UInt64 max_block_size_, std::shared_ptr file_iterator_, - size_t max_parsing_threads_, + FormatParserGroupPtr parser_group_, bool need_only_count_); ~StorageObjectStorageSource() override; String getName() const override { return name; } - void setKeyCondition(const std::optional & filter_actions_dag, ContextPtr context_) override; - Chunk generate() override; + void onFinish() override { parser_group->finishStream(); } + static std::shared_ptr createFileIterator( - ConfigurationPtr configuration, - const StorageObjectStorage::QuerySettings & query_settings, + StorageObjectStorageConfigurationPtr configuration, + const StorageObjectStorageQuerySettings & query_settings, ObjectStoragePtr object_storage, bool distributed_processing, const ContextPtr & local_context, const ActionsDAG::Node * predicate, - const std::optional & filter_actions_dag, + const ActionsDAG * filter_actions_dag, const NamesAndTypesList & virtual_columns, const NamesAndTypesList & hive_columns, ObjectInfos * read_keys, @@ -66,7 +64,7 @@ class StorageObjectStorageSource : public SourceWithKeyCondition bool skip_object_metadata = false); static std::string getUniqueStoragePathIdentifier( - const Configuration & configuration, + const StorageObjectStorageConfiguration & configuration, const ObjectInfo & object_info, bool include_connection_info = true); @@ -80,12 +78,12 @@ class StorageObjectStorageSource : public SourceWithKeyCondition protected: const String name; ObjectStoragePtr object_storage; - const ConfigurationPtr configuration; + const StorageObjectStorageConfigurationPtr configuration; const ContextPtr read_context; const std::optional format_settings; const UInt64 max_block_size; const bool need_only_count; - const size_t max_parsing_threads; + FormatParserGroupPtr parser_group; ReadFromFormatInfo read_from_format_info; const std::shared_ptr create_reader_pool; @@ -132,16 +130,15 @@ class StorageObjectStorageSource : public SourceWithKeyCondition static ReaderHolder createReader( size_t processor, const std::shared_ptr & file_iterator, - const ConfigurationPtr & configuration, + const StorageObjectStorageConfigurationPtr & configuration, const ObjectStoragePtr & object_storage, ReadFromFormatInfo & read_from_format_info, const std::optional & format_settings, - const std::shared_ptr & key_condition_, const ContextPtr & context_, SchemaCache * schema_cache, const LoggerPtr & log, size_t max_block_size, - size_t max_parsing_threads, + FormatParserGroupPtr parser_group, bool need_only_count); ReaderHolder createReader(); @@ -173,7 +170,7 @@ class StorageObjectStorageSource::GlobIterator : public IObjectIterator, WithCon public: GlobIterator( ObjectStoragePtr object_storage_, - ConfigurationPtr configuration_, + StorageObjectStorageConfigurationPtr configuration_, const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns_, const NamesAndTypesList & hive_columns_, @@ -195,7 +192,7 @@ class StorageObjectStorageSource::GlobIterator : public IObjectIterator, WithCon void fillBufferForKey(const std::string & uri_key); const ObjectStoragePtr object_storage; - const ConfigurationPtr configuration; + const StorageObjectStorageConfigurationPtr configuration; const NamesAndTypesList virtual_columns; const NamesAndTypesList hive_columns; const bool throw_on_zero_files_match; @@ -265,7 +262,7 @@ class StorageObjectStorageSource::ArchiveIterator : public IObjectIterator, priv public: explicit ArchiveIterator( ObjectStoragePtr object_storage_, - ConfigurationPtr configuration_, + StorageObjectStorageConfigurationPtr configuration_, std::unique_ptr archives_iterator_, ContextPtr context_, ObjectInfos * read_keys_, diff --git a/src/Storages/ObjectStorage/Utils.cpp b/src/Storages/ObjectStorage/Utils.cpp index 5fefd53b37ee..5461b6ffeb43 100644 --- a/src/Storages/ObjectStorage/Utils.cpp +++ b/src/Storages/ObjectStorage/Utils.cpp @@ -13,8 +13,8 @@ namespace ErrorCodes std::optional checkAndGetNewFileOnInsertIfNeeded( const IObjectStorage & object_storage, - const StorageObjectStorage::Configuration & configuration, - const StorageObjectStorage::QuerySettings & settings, + const StorageObjectStorageConfiguration & configuration, + const StorageObjectStorageQuerySettings & settings, const String & key, size_t sequence_number) { @@ -48,7 +48,7 @@ void resolveSchemaAndFormat( ColumnsDescription & columns, std::string & format, ObjectStoragePtr object_storage, - const StorageObjectStorage::ConfigurationPtr & configuration, + const StorageObjectStorageConfigurationPtr & configuration, std::optional format_settings, std::string & sample_path, const ContextPtr & context) @@ -97,7 +97,7 @@ void resolveSchemaAndFormat( void validateSupportedColumns( ColumnsDescription & columns, - const StorageObjectStorage::Configuration & configuration) + const StorageObjectStorageConfiguration & configuration) { if (!columns.hasOnlyOrdinary()) { diff --git a/src/Storages/ObjectStorage/Utils.h b/src/Storages/ObjectStorage/Utils.h index 8a3dfaad0287..f0e830ca3feb 100644 --- a/src/Storages/ObjectStorage/Utils.h +++ b/src/Storages/ObjectStorage/Utils.h @@ -8,8 +8,8 @@ class IObjectStorage; std::optional checkAndGetNewFileOnInsertIfNeeded( const IObjectStorage & object_storage, - const StorageObjectStorage::Configuration & configuration, - const StorageObjectStorage::QuerySettings & settings, + const StorageObjectStorageConfiguration & configuration, + const StorageObjectStorageQuerySettings & settings, const std::string & key, size_t sequence_number); @@ -17,13 +17,13 @@ void resolveSchemaAndFormat( ColumnsDescription & columns, std::string & format, ObjectStoragePtr object_storage, - const StorageObjectStorage::ConfigurationPtr & configuration, + const StorageObjectStorageConfigurationPtr & configuration, std::optional format_settings, std::string & sample_path, const ContextPtr & context); void validateSupportedColumns( ColumnsDescription & columns, - const StorageObjectStorage::Configuration & configuration); + const StorageObjectStorageConfiguration & configuration); } diff --git a/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp b/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp index ce595a1eb1cd..8c51a6565aaf 100644 --- a/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp @@ -28,14 +28,14 @@ namespace #if USE_AWS_S3 || USE_AZURE_BLOB_STORAGE || USE_HDFS || USE_AVRO std::shared_ptr -createStorageObjectStorage(const StorageFactory::Arguments & args, StorageObjectStorage::ConfigurationPtr configuration) +createStorageObjectStorage(const StorageFactory::Arguments & args, StorageObjectStorageConfigurationPtr configuration) { auto & engine_args = args.engine_args; if (engine_args.empty()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "External data source must have arguments"); const auto context = args.getLocalContext(); - StorageObjectStorage::Configuration::initialize(*configuration, args.engine_args, context, false); + StorageObjectStorageConfiguration::initialize(*configuration, args.engine_args, context, false); // Use format settings from global server context + settings from // the SETTINGS clause of the create query. Settings from current diff --git a/src/Storages/ObjectStorageQueue/ObjectStorageQueueMetadata.h b/src/Storages/ObjectStorageQueue/ObjectStorageQueueMetadata.h index 986d14d5dea6..db706e4d26ba 100644 --- a/src/Storages/ObjectStorageQueue/ObjectStorageQueueMetadata.h +++ b/src/Storages/ObjectStorageQueue/ObjectStorageQueueMetadata.h @@ -20,7 +20,6 @@ class StorageObjectStorageQueue; struct ObjectStorageQueueSettings; struct ObjectStorageQueueTableMetadata; struct StorageInMemoryMetadata; -using ConfigurationPtr = StorageObjectStorage::ConfigurationPtr; /** * A class for managing ObjectStorageQueue metadata in zookeeper, e.g. diff --git a/src/Storages/ObjectStorageQueue/ObjectStorageQueueSource.cpp b/src/Storages/ObjectStorageQueue/ObjectStorageQueueSource.cpp index 99f213cc51db..fad038a89a2e 100644 --- a/src/Storages/ObjectStorageQueue/ObjectStorageQueueSource.cpp +++ b/src/Storages/ObjectStorageQueue/ObjectStorageQueueSource.cpp @@ -78,7 +78,7 @@ ObjectStorageQueueSource::ObjectStorageQueueObjectInfo::ObjectStorageQueueObject ObjectStorageQueueSource::FileIterator::FileIterator( std::shared_ptr metadata_, ObjectStoragePtr object_storage_, - ConfigurationPtr configuration_, + StorageObjectStorageConfigurationPtr configuration_, const StorageID & storage_id_, size_t list_objects_batch_size_, const ActionsDAG::Node * predicate_, @@ -721,11 +721,12 @@ ObjectStorageQueueSource::ObjectStorageQueueSource( String name_, size_t processor_id_, std::shared_ptr file_iterator_, - ConfigurationPtr configuration_, + StorageObjectStorageConfigurationPtr configuration_, ObjectStoragePtr object_storage_, ProcessingProgressPtr progress_, const ReadFromFormatInfo & read_from_format_info_, const std::optional & format_settings_, + FormatParserGroupPtr parser_group_, const CommitSettings & commit_settings_, std::shared_ptr files_metadata_, ContextPtr context_, @@ -746,6 +747,7 @@ ObjectStorageQueueSource::ObjectStorageQueueSource( , progress(progress_) , read_from_format_info(read_from_format_info_) , format_settings(format_settings_) + , parser_group(std::move(parser_group_)) , commit_settings(commit_settings_) , files_metadata(files_metadata_) , max_block_size(max_block_size_) @@ -871,12 +873,11 @@ Chunk ObjectStorageQueueSource::generateImpl() object_storage, read_from_format_info, format_settings, - nullptr, context, nullptr, log, max_block_size, - context->getSettingsRef()[Setting::max_parsing_threads].value, + parser_group, /* need_only_count */ false); if (!reader) diff --git a/src/Storages/ObjectStorageQueue/ObjectStorageQueueSource.h b/src/Storages/ObjectStorageQueue/ObjectStorageQueueSource.h index cc87667fc225..e221d1b6f7f9 100644 --- a/src/Storages/ObjectStorageQueue/ObjectStorageQueueSource.h +++ b/src/Storages/ObjectStorageQueue/ObjectStorageQueueSource.h @@ -42,7 +42,7 @@ class ObjectStorageQueueSource : public ISource, WithContext FileIterator( std::shared_ptr metadata_, ObjectStoragePtr object_storage_, - ConfigurationPtr configuration_, + StorageObjectStorageConfigurationPtr configuration_, const StorageID & storage_id_, size_t list_objects_batch_size_, const ActionsDAG::Node * predicate_, @@ -75,7 +75,7 @@ class ObjectStorageQueueSource : public ISource, WithContext const std::shared_ptr metadata; const ObjectStoragePtr object_storage; - const ConfigurationPtr configuration; + const StorageObjectStorageConfigurationPtr configuration; const NamesAndTypesList virtual_columns; const bool file_deletion_on_processed_enabled; const ObjectStorageQueueMode mode; @@ -149,11 +149,12 @@ class ObjectStorageQueueSource : public ISource, WithContext String name_, size_t processor_id_, std::shared_ptr file_iterator_, - ConfigurationPtr configuration_, + StorageObjectStorageConfigurationPtr configuration_, ObjectStoragePtr object_storage_, ProcessingProgressPtr progress_, const ReadFromFormatInfo & read_from_format_info_, const std::optional & format_settings_, + FormatParserGroupPtr parser_group_, const CommitSettings & commit_settings_, std::shared_ptr files_metadata_, ContextPtr context_, @@ -171,6 +172,8 @@ class ObjectStorageQueueSource : public ISource, WithContext Chunk generate() override; + void onFinish() override { parser_group->finishStream(); } + /// Commit files after insertion into storage finished. /// `success` defines whether insertion was successful or not. void prepareCommitRequests( @@ -205,11 +208,12 @@ class ObjectStorageQueueSource : public ISource, WithContext const String name; const size_t processor_id; const std::shared_ptr file_iterator; - const ConfigurationPtr configuration; + const StorageObjectStorageConfigurationPtr configuration; const ObjectStoragePtr object_storage; const ProcessingProgressPtr progress; ReadFromFormatInfo read_from_format_info; const std::optional format_settings; + FormatParserGroupPtr parser_group; const CommitSettings commit_settings; const std::shared_ptr files_metadata; const size_t max_block_size; diff --git a/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp b/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp index 2c78f7257aeb..be88998e68ae 100644 --- a/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp +++ b/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp @@ -179,7 +179,7 @@ namespace StorageObjectStorageQueue::StorageObjectStorageQueue( std::unique_ptr queue_settings_, - const ConfigurationPtr configuration_, + const StorageObjectStorageConfigurationPtr configuration_, const StorageID & table_id_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, @@ -425,11 +425,14 @@ void ReadFromObjectStorageQueue::initializePipeline(QueryPipelineBuilder & pipel size_t processing_threads_num = storage->getTableMetadata().processing_threads_num; createIterator(nullptr); + + auto parser_group = std::make_shared(context->getSettingsRef(), /*num_streams_=*/ processing_threads_num, nullptr, nullptr); auto progress = std::make_shared(); for (size_t i = 0; i < processing_threads_num; ++i) pipes.emplace_back(storage->createSource( i/* processor_id */, info, + parser_group, progress, iterator, max_block_size, @@ -449,6 +452,7 @@ void ReadFromObjectStorageQueue::initializePipeline(QueryPipelineBuilder & pipel std::shared_ptr StorageObjectStorageQueue::createSource( size_t processor_id, const ReadFromFormatInfo & info, + FormatParserGroupPtr parser_group, ProcessingProgressPtr progress_, std::shared_ptr file_iterator, size_t max_block_size, @@ -463,7 +467,7 @@ std::shared_ptr StorageObjectStorageQueue::createSourc return std::make_shared( getName(), processor_id, file_iterator, configuration, object_storage, progress_, - info, format_settings, + info, format_settings, parser_group, commit_settings_copy, files_metadata, local_context, max_block_size, shutdown_called, table_is_being_dropped, @@ -630,6 +634,8 @@ bool StorageObjectStorageQueue::streamToViews(size_t streaming_tasks_index) pipes.reserve(threads); sources.reserve(threads); + auto parser_group = std::make_shared(queue_context->getSettingsRef(), /*num_streams_=*/ threads, nullptr, nullptr); + auto processing_progress = std::make_shared(); for (size_t i = 0; i < threads; ++i) { @@ -637,6 +643,7 @@ bool StorageObjectStorageQueue::streamToViews(size_t streaming_tasks_index) auto source = createSource( processor_id, read_from_format_info, + parser_group, processing_progress, file_iterator, DBMS_DEFAULT_BUFFER_SIZE, diff --git a/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.h b/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.h index c77f9e185f1c..94408fc09788 100644 --- a/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.h +++ b/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.h @@ -21,11 +21,9 @@ struct ObjectStorageQueueSettings; class StorageObjectStorageQueue : public IStorage, WithContext { public: - using ConfigurationPtr = StorageObjectStorage::ConfigurationPtr; - StorageObjectStorageQueue( std::unique_ptr queue_settings_, - ConfigurationPtr configuration_, + StorageObjectStorageConfigurationPtr configuration_, const StorageID & table_id_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, @@ -93,7 +91,7 @@ class StorageObjectStorageQueue : public IStorage, WithContext std::unique_ptr temp_metadata; std::shared_ptr files_metadata; - ConfigurationPtr configuration; + StorageObjectStorageConfigurationPtr configuration; ObjectStoragePtr object_storage; const std::optional format_settings; @@ -125,6 +123,7 @@ class StorageObjectStorageQueue : public IStorage, WithContext std::shared_ptr createSource( size_t processor_id, const ReadFromFormatInfo & info, + FormatParserGroupPtr parser_group, ProcessingProgressPtr progress_, std::shared_ptr file_iterator, size_t max_block_size, diff --git a/src/Storages/ObjectStorageQueue/registerQueueStorage.cpp b/src/Storages/ObjectStorageQueue/registerQueueStorage.cpp index da6cba2d7ea4..436f59fea3cc 100644 --- a/src/Storages/ObjectStorageQueue/registerQueueStorage.cpp +++ b/src/Storages/ObjectStorageQueue/registerQueueStorage.cpp @@ -34,7 +34,7 @@ StoragePtr createQueueStorage(const StorageFactory::Arguments & args) throw Exception(ErrorCodes::BAD_ARGUMENTS, "External data source must have arguments"); auto configuration = std::make_shared(); - StorageObjectStorage::Configuration::initialize(*configuration, args.engine_args, args.getContext(), false); + StorageObjectStorageConfiguration::initialize(*configuration, args.engine_args, args.getContext(), false); // Use format settings from global server context + settings from // the SETTINGS clause of the create query. Settings from current diff --git a/src/Storages/RabbitMQ/RabbitMQSource.cpp b/src/Storages/RabbitMQ/RabbitMQSource.cpp index 2a31374af951..f87e734300dc 100644 --- a/src/Storages/RabbitMQ/RabbitMQSource.cpp +++ b/src/Storages/RabbitMQ/RabbitMQSource.cpp @@ -161,7 +161,7 @@ Chunk RabbitMQSource::generateImpl() MutableColumns virtual_columns = virtual_header.cloneEmptyColumns(); EmptyReadBuffer empty_buf; auto input_format = FormatFactory::instance().getInput( - storage.getFormatName(), empty_buf, non_virtual_header, context, max_block_size, std::nullopt, 1); + storage.getFormatName(), empty_buf, non_virtual_header, context, max_block_size, std::nullopt, FormatParserGroup::singleThreaded(context->getSettingsRef())); std::optional exception_message; size_t total_rows = 0; diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index 6a2df5ae5897..fb6741063cab 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -6,6 +6,8 @@ #include #include +#include +#include #include @@ -29,9 +31,9 @@ #include #include #include +#include #include -#include -#include + #include #include #include @@ -566,6 +568,8 @@ class ReadFromEmbeddedRocksDB : public SourceStepWithFilter std::string getName() const override { return "ReadFromEmbeddedRocksDB"; } void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; void applyFilters(ActionDAGNodes added_filter_nodes) override; + void describeActions(FormatSettings & format_settings) const override; + void describeActions(JSONBuilder::JSONMap & map) const override; ReadFromEmbeddedRocksDB( const Names & column_names_, @@ -663,7 +667,30 @@ void ReadFromEmbeddedRocksDB::applyFilters(ActionDAGNodes added_filter_nodes) const auto & sample_block = getOutputHeader(); auto primary_key_data_type = sample_block.getByName(storage.primary_key).type; - std::tie(keys, all_scan) = getFilterKeys(storage.primary_key, primary_key_data_type, filter_actions_dag, context); + std::tie(keys, all_scan) = getFilterKeys(storage.primary_key, primary_key_data_type, filter_actions_dag.get(), context); +} + +void ReadFromEmbeddedRocksDB::describeActions(FormatSettings & format_settings) const +{ + std::string prefix(format_settings.offset, format_settings.indent_char); + if (!all_scan) + { + format_settings.out << prefix << "ReadType: GetKeys\n"; + format_settings.out << prefix << "Keys: " << keys->size() << '\n'; + } + else + format_settings.out << prefix << "ReadType: FullScan\n"; +} + +void ReadFromEmbeddedRocksDB::describeActions(JSONBuilder::JSONMap & map) const +{ + if (!all_scan) + { + map.add("Read Type", "GetKeys"); + map.add("Keys", keys->size()); + } + else + map.add("Read Type", "FullScan"); } SinkToStoragePtr StorageEmbeddedRocksDB::write( diff --git a/src/Storages/SelectQueryInfo.h b/src/Storages/SelectQueryInfo.h index feadaf0290c1..0740f0ff8770 100644 --- a/src/Storages/SelectQueryInfo.h +++ b/src/Storages/SelectQueryInfo.h @@ -182,7 +182,11 @@ struct SelectQueryInfo /// It is needed for PK analysis based on row_level_policy and additional_filters. ASTs filter_asts; - /// Filter actions dag for current storage + /// Filter actions dag for current storage. + /// NOTE: Currently we store two copies of the filter DAGs: + /// (1) SourceStepWithFilter::filter_actions_dag, (2) SelectQueryInfo::filter_actions_dag. + /// Prefer to use the one in SourceStepWithFilter, not this one. + /// (See comment in ReadFromMergeTree::applyFilters.) std::shared_ptr filter_actions_dag; ReadInOrderOptimizerPtr order_optimizer; @@ -190,6 +194,7 @@ struct SelectQueryInfo InputOrderInfoPtr input_order_info; /// Prepared sets are used for indices by storage engine. + /// New analyzer stores prepared sets in planner_context and hashes computed of QueryTree instead of AST. /// Example: x IN (1, 2, 3) PreparedSetsPtr prepared_sets; diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index d3f6407c88b1..71fabaa79006 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -1229,7 +1229,7 @@ std::optional StorageDistributed::distributedWriteBetweenDistribu return pipeline; } -static std::optional getFilterFromQuery(const ASTPtr & ast, ContextPtr context) +static std::shared_ptr getFilterFromQuery(const ASTPtr & ast, ContextPtr context) { QueryPlan plan; SelectQueryOptions options; @@ -1273,7 +1273,7 @@ static std::optional getFilterFromQuery(const ASTPtr & ast, ContextP } if (!source) - return {}; + return nullptr; return source->detachFilterActionsDAG(); } @@ -1318,7 +1318,8 @@ std::optional StorageDistributed::distributedWriteFromClusterStor /// Select query is needed for pruining on virtual columns auto number_of_replicas = static_cast(cluster->getShardsInfo().size()); - auto extension = src_storage_cluster.getTaskIteratorExtension(predicate, local_context, number_of_replicas); + auto extension = src_storage_cluster.getTaskIteratorExtension( + predicate, filter.get(), local_context, number_of_replicas); /// Here we take addresses from destination cluster and assume source table exists on these nodes size_t replica_index = 0; diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 48c11e8d8e9e..86d66f5d40e4 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -41,7 +41,6 @@ #include #include #include -#include #include #include #include @@ -1221,11 +1220,13 @@ StorageFileSource::StorageFileSource( UInt64 max_block_size_, FilesIteratorPtr files_iterator_, std::unique_ptr read_buf_, - bool need_only_count_) - : SourceWithKeyCondition(info.source_header, false), WithContext(context_) + bool need_only_count_, + FormatParserGroupPtr parser_group_) + : ISource(info.source_header, false), WithContext(context_) , storage(std::move(storage_)) , files_iterator(std::move(files_iterator_)) , read_buf(std::move(read_buf_)) + , parser_group(std::move(parser_group_)) , columns_description(info.columns_description) , requested_columns(info.requested_columns) , requested_virtual_columns(info.requested_virtual_columns) @@ -1298,11 +1299,6 @@ StorageFileSource::~StorageFileSource() beforeDestroy(); } -void StorageFileSource::setKeyCondition(const std::optional & filter_actions_dag, ContextPtr context_) -{ - setKeyConditionImpl(filter_actions_dag, context_, block_for_format); -} - bool StorageFileSource::tryGetCountFromCache(const struct stat & file_stat) { @@ -1447,8 +1443,6 @@ Chunk StorageFileSource::generate() read_buf = createReadBuffer(current_path, file_stat, storage->use_table_fd, storage->table_fd, storage->compression_method, getContext()); } - const Settings & settings = getContext()->getSettingsRef(); - size_t file_num = 0; if (storage->archive_info) file_num = storage->archive_info->paths_to_archives.size(); @@ -1457,16 +1451,13 @@ Chunk StorageFileSource::generate() chassert(file_num > 0); - const auto max_parsing_threads = std::max(settings[Setting::max_parsing_threads] / file_num, 1UL); input_format = FormatFactory::instance().getInput( - storage->format_name, *read_buf, block_for_format, getContext(), max_block_size, storage->format_settings, - max_parsing_threads, std::nullopt, /*is_remote_fs*/ false, CompressionMethod::None, need_only_count); + storage->format_name, *read_buf, block_for_format, getContext(), max_block_size, + storage->format_settings, parser_group, + /*is_remote_fs=*/ false, CompressionMethod::None, need_only_count); input_format->setSerializationHints(serialization_hints); - if (key_condition) - input_format->setKeyCondition(key_condition); - if (need_only_count) input_format->needOnlyCount(); @@ -1531,8 +1522,7 @@ Chunk StorageFileSource::generate() if (storage->use_table_fd) finished_generate = true; - if (input_format && storage->format_name != "Distributed" && getContext()->getSettingsRef()[Setting::use_cache_for_count_from_files] && - (!key_condition || key_condition->alwaysUnknownOrTrue())) + if (input_format && storage->format_name != "Distributed" && getContext()->getSettingsRef()[Setting::use_cache_for_count_from_files] && !parser_group->hasFilter()) addNumRowsToCache(current_path, total_rows_in_file); total_rows_in_file = 0; @@ -1732,6 +1722,8 @@ void ReadFromFile::initializePipeline(QueryPipelineBuilder & pipeline, const Bui if (progress_callback && !storage->archive_info) progress_callback(FileProgress(0, storage->total_bytes_to_read)); + auto parser_group = std::make_shared(ctx->getSettingsRef(), num_streams, filter_actions_dag, ctx); + for (size_t i = 0; i < num_streams; ++i) { /// In case of reading from fd we have to check whether we have already created @@ -1749,9 +1741,9 @@ void ReadFromFile::initializePipeline(QueryPipelineBuilder & pipeline, const Bui max_block_size, files_iterator, std::move(read_buffer), - need_only_count); + need_only_count, + parser_group); - source->setKeyCondition(filter_actions_dag, ctx); pipes.emplace_back(std::move(source)); } diff --git a/src/Storages/StorageFile.h b/src/Storages/StorageFile.h index 14a67dfa1634..c747db116983 100644 --- a/src/Storages/StorageFile.h +++ b/src/Storages/StorageFile.h @@ -5,9 +5,10 @@ #include #include #include +#include #include -#include #include +#include #include #include @@ -205,7 +206,7 @@ class StorageFile final : public IStorage NamesAndTypesList hive_partition_columns_to_read_from_file_path; }; -class StorageFileSource : public SourceWithKeyCondition, WithContext +class StorageFileSource : public ISource, WithContext { public: class FilesIterator : WithContext @@ -257,7 +258,8 @@ class StorageFileSource : public SourceWithKeyCondition, WithContext UInt64 max_block_size_, FilesIteratorPtr files_iterator_, std::unique_ptr read_buf_, - bool need_only_count_); + bool need_only_count_, + FormatParserGroupPtr); /** @@ -273,12 +275,12 @@ class StorageFileSource : public SourceWithKeyCondition, WithContext return storage->getName(); } - void setKeyCondition(const std::optional & filter_actions_dag, ContextPtr context_) override; - bool tryGetCountFromCache(const struct stat & file_stat); Chunk generate() override; + void onFinish() override { parser_group->finishStream(); } + void addNumRowsToCache(const String & path, size_t num_rows) const; std::optional tryGetNumRowsFromCache(const String & path, time_t last_mod_time) const; @@ -295,6 +297,7 @@ class StorageFileSource : public SourceWithKeyCondition, WithContext InputFormatPtr input_format; std::unique_ptr pipeline; std::unique_ptr reader; + FormatParserGroupPtr parser_group; std::shared_ptr archive_reader; std::unique_ptr file_enumerator; diff --git a/src/Storages/StorageFileCluster.cpp b/src/Storages/StorageFileCluster.cpp index cd661c10d1da..398b33997c56 100644 --- a/src/Storages/StorageFileCluster.cpp +++ b/src/Storages/StorageFileCluster.cpp @@ -94,7 +94,11 @@ void StorageFileCluster::updateQueryToSendIfNeeded(DB::ASTPtr & query, const Sto ); } -RemoteQueryExecutor::Extension StorageFileCluster::getTaskIteratorExtension(const ActionsDAG::Node * predicate, const ContextPtr & context, const size_t) const +RemoteQueryExecutor::Extension StorageFileCluster::getTaskIteratorExtension( + const ActionsDAG::Node * predicate, + const ActionsDAG * /* filter */, + const ContextPtr & context, + const size_t) const { auto iterator = std::make_shared(paths, std::nullopt, predicate, getVirtualsList(), hive_partition_columns_to_read_from_file_path, context); auto callback = std::make_shared([iter = std::move(iterator)](size_t) mutable -> String { return iter->next(); }); diff --git a/src/Storages/StorageFileCluster.h b/src/Storages/StorageFileCluster.h index 2cbd82ba4000..f7555a3f6b5e 100644 --- a/src/Storages/StorageFileCluster.h +++ b/src/Storages/StorageFileCluster.h @@ -27,7 +27,11 @@ class StorageFileCluster : public IStorageCluster const ConstraintsDescription & constraints_); std::string getName() const override { return "FileCluster"; } - RemoteQueryExecutor::Extension getTaskIteratorExtension(const ActionsDAG::Node * predicate, const ContextPtr & context, size_t number_of_replicas) const override; + RemoteQueryExecutor::Extension getTaskIteratorExtension( + const ActionsDAG::Node * predicate, + const ActionsDAG * /* filter */, + const ContextPtr & context, + size_t number_of_replicas) const override; private: void updateQueryToSendIfNeeded(ASTPtr & query, const StorageSnapshotPtr & storage_snapshot, const ContextPtr & context) override; diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index e91c6213e996..8c00f3d30eea 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -23,12 +23,12 @@ #include #include -#include -#include -#include #include #include +#include +#include +#include #include #include @@ -47,6 +47,7 @@ #include #include #include +#include #include #include @@ -284,13 +285,8 @@ class StorageKeeperMapSource : public ISource, WithContext bool with_version_column = false; - static Block getHeader(Block header, bool with_version_column) + static Block getHeader(Block header) { - if (with_version_column) - header.insert( - {DataTypeInt32{}.createColumn(), - std::make_shared(), std::string{version_column_name}}); - return header; } @@ -304,7 +300,7 @@ class StorageKeeperMapSource : public ISource, WithContext KeyContainerIter end_, bool with_version_column_, ContextPtr context_) - : ISource(getHeader(header, with_version_column_)) + : ISource(getHeader(header)) , WithContext(std::move(context_)) , storage(storage_) , max_block_size(max_block_size_) @@ -601,25 +597,62 @@ StorageKeeperMap::StorageKeeperMap( zk_root_path); } +class ReadFromKeeperMap : public SourceStepWithFilter +{ +public: + std::string getName() const override { return "ReadFromKeeperMap"; } + void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; + void applyFilters(ActionDAGNodes added_filter_nodes) override; + void describeActions(FormatSettings & format_settings) const override; + void describeActions(JSONBuilder::JSONMap & map) const override; + + ReadFromKeeperMap( + const Names & column_names_, + const SelectQueryInfo & query_info_, + const StorageSnapshotPtr & storage_snapshot_, + const ContextPtr & context_, + Block sample_block, + const StorageKeeperMap & storage_, + size_t max_block_size_, + size_t num_streams_, + bool with_version_column_) + : SourceStepWithFilter(std::move(sample_block), column_names_, query_info_, storage_snapshot_, context_) + , storage(storage_) + , max_block_size(max_block_size_) + , num_streams(num_streams_) + , with_version_column(with_version_column_) + { + } -Pipe StorageKeeperMap::read( - const Names & column_names, - const StorageSnapshotPtr & storage_snapshot, - SelectQueryInfo & query_info, - ContextPtr context_, - QueryProcessingStage::Enum /*processed_stage*/, - size_t max_block_size, - size_t num_streams) +private: + const StorageKeeperMap & storage; + + size_t max_block_size; + size_t num_streams; + bool with_version_column; + + FieldVectorPtr keys; + bool all_scan = false; + + template + void initializePipelineImpl(QueryPipelineBuilder & pipeline, KeyContainerPtr key_container); + + Strings getAllKeys() const; +}; + +void StorageKeeperMap::read( + QueryPlan & query_plan, + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr context_, + QueryProcessingStage::Enum /*processed_stage*/, + size_t max_block_size, + size_t num_streams) { checkTable(context_); storage_snapshot->check(column_names); - - FieldVectorPtr filtered_keys; - bool all_scan; - Block sample_block = storage_snapshot->metadata->getSampleBlock(); - auto primary_key_type = sample_block.getByName(primary_key).type; - std::tie(filtered_keys, all_scan) = getFilterKeys(primary_key, primary_key_type, query_info, context_); bool with_version_column = false; for (const auto & column : column_names) @@ -631,56 +664,110 @@ Pipe StorageKeeperMap::read( } } - const auto process_keys = [&](KeyContainerPtr keys) -> Pipe + if (with_version_column) + sample_block.insert({DataTypeInt32{}.createColumn(), + std::make_shared(), std::string{version_column_name}}); + + auto reading = std::make_unique( + column_names, query_info, storage_snapshot, context_, std::move(sample_block), *this, max_block_size, num_streams, with_version_column); + + query_plan.addStep(std::move(reading)); +} + +void ReadFromKeeperMap::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) +{ + if (all_scan) + initializePipelineImpl(pipeline, std::make_shared(getAllKeys())); + else + initializePipelineImpl(pipeline, keys); +} + +void ReadFromKeeperMap::applyFilters(ActionDAGNodes added_filter_nodes) +{ + SourceStepWithFilter::applyFilters(std::move(added_filter_nodes)); + + const auto & sample_block = getOutputHeader(); + auto primary_key_data_type = sample_block.getByName(storage.primary_key).type; + std::tie(keys, all_scan) = getFilterKeys(storage.primary_key, primary_key_data_type, filter_actions_dag.get(), context); +} + +template +void ReadFromKeeperMap::initializePipelineImpl(QueryPipelineBuilder & pipeline, KeyContainerPtr key_container) +{ + const auto & sample_block = getOutputHeader(); + + if (key_container->empty()) { - if (keys->empty()) - return {}; + pipeline.init(Pipe(std::make_shared(sample_block))); + return; + } - ::sort(keys->begin(), keys->end()); - keys->erase(std::unique(keys->begin(), keys->end()), keys->end()); + ::sort(key_container->begin(), key_container->end()); + key_container->erase(std::unique(key_container->begin(), key_container->end()), key_container->end()); - Pipes pipes; + Pipes pipes; - size_t num_keys = keys->size(); - size_t num_threads = std::min(num_streams, keys->size()); + size_t num_keys = key_container->size(); + size_t num_threads = std::min(num_streams, key_container->size()); - chassert(num_keys <= std::numeric_limits::max()); - chassert(num_threads <= std::numeric_limits::max()); + chassert(num_keys <= std::numeric_limits::max()); + chassert(num_threads <= std::numeric_limits::max()); - for (size_t thread_idx = 0; thread_idx < num_threads; ++thread_idx) - { - size_t begin = num_keys * thread_idx / num_threads; - size_t end = num_keys * (thread_idx + 1) / num_threads; + for (size_t thread_idx = 0; thread_idx < num_threads; ++thread_idx) + { + size_t begin = num_keys * thread_idx / num_threads; + size_t end = num_keys * (thread_idx + 1) / num_threads; - using KeyContainer = typename KeyContainerPtr::element_type; - pipes.emplace_back(std::make_shared>( - *this, sample_block, max_block_size, keys, keys->begin() + begin, keys->begin() + end, with_version_column, context_)); - } - return Pipe::unitePipes(std::move(pipes)); - }; + using KeyContainer = typename KeyContainerPtr::element_type; + pipes.emplace_back(std::make_shared>( + storage, sample_block, max_block_size, key_container, key_container->begin() + begin, key_container->begin() + end, with_version_column, context)); + } + pipeline.init(Pipe::unitePipes(std::move(pipes))); +} - if (all_scan) +Strings ReadFromKeeperMap::getAllKeys() const +{ + const auto & settings = context->getSettingsRef(); + ZooKeeperRetriesControl zk_retry{ + getName(), + getLogger(getName()), + ZooKeeperRetriesInfo{ + settings[Setting::keeper_max_retries], + settings[Setting::keeper_retry_initial_backoff_ms], + settings[Setting::keeper_retry_max_backoff_ms], + context->getProcessListElement()}}; + + Strings children; + zk_retry.retryLoop([&] { - const auto & settings = context_->getSettingsRef(); - ZooKeeperRetriesControl zk_retry{ - getName(), - getLogger(getName()), - ZooKeeperRetriesInfo{ - settings[Setting::keeper_max_retries], - settings[Setting::keeper_retry_initial_backoff_ms], - settings[Setting::keeper_retry_max_backoff_ms], - context_->getProcessListElement()}}; + auto client = storage.getClient(); + children = client->getChildren(storage.zk_data_path); + }); - std::vector children; - zk_retry.retryLoop([&] - { - auto client = getClient(); - children = client->getChildren(zk_data_path); - }); - return process_keys(std::make_shared>(std::move(children))); + return children; +} + +void ReadFromKeeperMap::describeActions(FormatSettings & format_settings) const +{ + std::string prefix(format_settings.offset, format_settings.indent_char); + if (!all_scan) + { + format_settings.out << prefix << "ReadType: GetKeys\n"; + format_settings.out << prefix << "Keys: " << keys->size() << '\n'; } + else + format_settings.out << prefix << "ReadType: FullScan\n"; +} - return process_keys(std::move(filtered_keys)); +void ReadFromKeeperMap::describeActions(JSONBuilder::JSONMap & map) const +{ + if (!all_scan) + { + map.add("Read Type", "GetKeys"); + map.add("Keys", keys->size()); + } + else + map.add("Read Type", "FullScan"); } SinkToStoragePtr StorageKeeperMap::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, bool /*async_insert*/) diff --git a/src/Storages/StorageKeeperMap.h b/src/Storages/StorageKeeperMap.h index a2c6d4cfcb03..ec920c974cea 100644 --- a/src/Storages/StorageKeeperMap.h +++ b/src/Storages/StorageKeeperMap.h @@ -26,6 +26,7 @@ namespace ErrorCodes // KV store using (Zoo|CH)Keeper class StorageKeeperMap final : public IStorage, public IKeyValueEntity, WithContext { + friend class ReadFromKeeperMap; public: StorageKeeperMap( ContextPtr context_, @@ -36,12 +37,13 @@ class StorageKeeperMap final : public IStorage, public IKeyValueEntity, WithCont const std::string & root_path_, UInt64 keys_limit_); - Pipe read( + void read( + QueryPlan & query_plan, const Names & column_names, const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info, - ContextPtr context, - QueryProcessingStage::Enum processed_stage, + ContextPtr context_, + QueryProcessingStage::Enum /*processed_stage*/, size_t max_block_size, size_t num_streams) override; diff --git a/src/Storages/StorageRedis.cpp b/src/Storages/StorageRedis.cpp index fa0be4364827..db294a2702af 100644 --- a/src/Storages/StorageRedis.cpp +++ b/src/Storages/StorageRedis.cpp @@ -8,7 +8,10 @@ #include #include #include +#include +#include #include +#include #include #include @@ -25,6 +28,7 @@ #include #include #include +#include namespace DB { @@ -215,53 +219,134 @@ StorageRedis::StorageRedis( setInMemoryMetadata(storage_metadata); } -Pipe StorageRedis::read( - const Names & column_names, - const StorageSnapshotPtr & storage_snapshot, - SelectQueryInfo & query_info, - ContextPtr context_, - QueryProcessingStage::Enum /*processed_stage*/, - size_t max_block_size, - size_t num_streams) +class ReadFromRedis : public SourceStepWithFilter { - storage_snapshot->check(column_names); +public: + std::string getName() const override { return "ReadFromRedis"; } + void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; + void applyFilters(ActionDAGNodes added_filter_nodes) override; + void describeActions(FormatSettings & format_settings) const override; + void describeActions(JSONBuilder::JSONMap & map) const override; + + ReadFromRedis( + const Names & column_names_, + const SelectQueryInfo & query_info_, + const StorageSnapshotPtr & storage_snapshot_, + const ContextPtr & context_, + Block sample_block, + StorageRedis & storage_, + size_t max_block_size_, + size_t num_streams_) + : SourceStepWithFilter(std::move(sample_block), column_names_, query_info_, storage_snapshot_, context_) + , storage(storage_) + , max_block_size(max_block_size_) + , num_streams(num_streams_) + { + } + +private: + StorageRedis & storage; + + size_t max_block_size; + size_t num_streams; FieldVectorPtr keys; bool all_scan = false; +}; - Block header = storage_snapshot->metadata->getSampleBlock(); - auto primary_key_data_type = header.getByName(primary_key).type; +void StorageRedis::read( + QueryPlan & query_plan, + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr context_, + QueryProcessingStage::Enum /*processed_stage*/, + size_t max_block_size, + size_t num_streams) +{ + storage_snapshot->check(column_names); + Block sample_block = storage_snapshot->metadata->getSampleBlock(); - std::tie(keys, all_scan) = getFilterKeys(primary_key, primary_key_data_type, query_info, context_); + auto reading = std::make_unique( + column_names, query_info, storage_snapshot, context_, std::move(sample_block), *this, max_block_size, num_streams); + + query_plan.addStep(std::move(reading)); +} + +void ReadFromRedis::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) +{ + const auto & sample_block = getOutputHeader(); if (all_scan) { - return Pipe(std::make_shared(*this, header, max_block_size)); + auto source = std::make_shared(storage, sample_block, max_block_size); + source->setStorageLimits(query_info.storage_limits); + pipeline.init(Pipe(std::move(source))); } + else + { + if (keys->empty()) + { + pipeline.init(Pipe(std::make_shared(sample_block))); + return; + } - if (keys->empty()) - return {}; + ::sort(keys->begin(), keys->end()); + keys->erase(std::unique(keys->begin(), keys->end()), keys->end()); + + Pipes pipes; - Pipes pipes; + size_t num_keys = keys->size(); + size_t num_threads = std::min(num_streams, keys->size()); + num_threads = std::min(num_threads, storage.configuration.pool_size); - ::sort(keys->begin(), keys->end()); - keys->erase(std::unique(keys->begin(), keys->end()), keys->end()); + assert(num_keys <= std::numeric_limits::max()); + assert(num_threads <= std::numeric_limits::max()); + + for (size_t thread_idx = 0; thread_idx < num_threads; ++thread_idx) + { + size_t begin = num_keys * thread_idx / num_threads; + size_t end = num_keys * (thread_idx + 1) / num_threads; + + auto source = std::make_shared( + storage, sample_block, keys, keys->begin() + begin, keys->begin() + end, max_block_size); + source->setStorageLimits(query_info.storage_limits); + pipes.emplace_back(std::move(source)); + } + pipeline.init(Pipe::unitePipes(std::move(pipes))); + } +} - size_t num_keys = keys->size(); - size_t num_threads = std::min(num_streams, keys->size()); +void ReadFromRedis::applyFilters(ActionDAGNodes added_filter_nodes) +{ + SourceStepWithFilter::applyFilters(std::move(added_filter_nodes)); - num_threads = std::min(num_threads, configuration.pool_size); - assert(num_keys <= std::numeric_limits::max()); + const auto & sample_block = getOutputHeader(); + auto primary_key_data_type = sample_block.getByName(storage.primary_key).type; + std::tie(keys, all_scan) = getFilterKeys(storage.primary_key, primary_key_data_type, filter_actions_dag.get(), context); +} - for (size_t thread_idx = 0; thread_idx < num_threads; ++thread_idx) +void ReadFromRedis::describeActions(FormatSettings & format_settings) const +{ + std::string prefix(format_settings.offset, format_settings.indent_char); + if (!all_scan) { - size_t begin = num_keys * thread_idx / num_threads; - size_t end = num_keys * (thread_idx + 1) / num_threads; + format_settings.out << prefix << "ReadType: GetKeys\n"; + format_settings.out << prefix << "Keys: " << keys->size() << '\n'; + } + else + format_settings.out << prefix << "ReadType: FullScan\n"; +} - pipes.emplace_back( - std::make_shared(*this, header, keys, keys->begin() + begin, keys->begin() + end, max_block_size)); +void ReadFromRedis::describeActions(JSONBuilder::JSONMap & map) const +{ + if (!all_scan) + { + map.add("Read Type", "GetKeys"); + map.add("Keys", keys->size()); } - return Pipe::unitePipes(std::move(pipes)); + else + map.add("Read Type", "FullScan"); } namespace diff --git a/src/Storages/StorageRedis.h b/src/Storages/StorageRedis.h index e91cd1023f3f..8fdf2c2c2f34 100644 --- a/src/Storages/StorageRedis.h +++ b/src/Storages/StorageRedis.h @@ -14,6 +14,7 @@ namespace DB */ class StorageRedis : public IStorage, public IKeyValueEntity, WithContext { + friend class ReadFromRedis; public: StorageRedis( const StorageID & table_id_, @@ -24,12 +25,13 @@ class StorageRedis : public IStorage, public IKeyValueEntity, WithContext std::string getName() const override { return "Redis"; } - Pipe read( + void read( + QueryPlan & query_plan, const Names & column_names, const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info, ContextPtr context_, - QueryProcessingStage::Enum processed_stage, + QueryProcessingStage::Enum /*processed_stage*/, size_t max_block_size, size_t num_streams) override; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index f349e9317db0..d2e3b84c2040 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -6087,7 +6087,7 @@ std::optional StorageReplicatedMergeTree::distributedWriteFromClu query_context->increaseDistributedDepth(); auto number_of_replicas = static_cast(src_cluster->getShardsAddresses().size()); - auto extension = src_storage_cluster->getTaskIteratorExtension(nullptr, local_context, number_of_replicas); + auto extension = src_storage_cluster->getTaskIteratorExtension(nullptr, nullptr, local_context, number_of_replicas); size_t replica_index = 0; for (const auto & replicas : src_cluster->getShardsAddresses()) diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 4bd77a879d68..1bbd7409f9c8 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -321,12 +321,12 @@ StorageURLSource::StorageURLSource( UInt64 max_block_size, const ConnectionTimeouts & timeouts, CompressionMethod compression_method, - size_t max_parsing_threads, + FormatParserGroupPtr parser_group_, const HTTPHeaderEntries & headers_, const URIParams & params, bool glob_url, bool need_only_count_) - : SourceWithKeyCondition(info.source_header, false) + : ISource(info.source_header, false) , WithContext(context_) , name(std::move(name_)) , columns_description(info.columns_description) @@ -337,6 +337,7 @@ StorageURLSource::StorageURLSource( , uri_iterator(uri_iterator_) , format(format_) , format_settings(format_settings_) + , parser_group(std::move(parser_group_)) , headers(getHeaders(headers_)) , need_only_count(need_only_count_) , hive_partition_columns_to_read_from_file_path(info.hive_partition_columns_to_read_from_file_path) @@ -403,17 +404,13 @@ StorageURLSource::StorageURLSource( getContext(), max_block_size, format_settings, - max_parsing_threads, - /*max_download_threads*/ std::nullopt, + parser_group, /* is_remote_ fs */ true, compression_method, need_only_count); input_format->setSerializationHints(info.serialization_hints); - if (key_condition) - input_format->setKeyCondition(key_condition); - if (need_only_count) input_format->needOnlyCount(); @@ -509,7 +506,7 @@ Chunk StorageURLSource::generate() } if (input_format && getContext()->getSettingsRef()[Setting::use_cache_for_count_from_files] && - (!key_condition || key_condition->alwaysUnknownOrTrue())) + !parser_group->hasFilter()) addNumRowsToCache(curr_uri.toString(), total_rows_in_file); pipeline->reset(); @@ -1286,7 +1283,7 @@ void ReadFromURL::initializePipeline(QueryPipelineBuilder & pipeline, const Buil Pipes pipes; pipes.reserve(num_streams); - const size_t max_parsing_threads = num_streams >= settings[Setting::max_parsing_threads] ? 1 : (settings[Setting::max_parsing_threads] / num_streams); + auto parser_group = std::make_shared(settings, num_streams, filter_actions_dag, context); for (size_t i = 0; i < num_streams; ++i) { @@ -1302,13 +1299,12 @@ void ReadFromURL::initializePipeline(QueryPipelineBuilder & pipeline, const Buil max_block_size, getHTTPTimeouts(context), storage->compression_method, - max_parsing_threads, + parser_group, storage->headers, read_uri_params, is_url_with_globs, need_only_count); - source->setKeyCondition(filter_actions_dag, context); pipes.emplace_back(std::move(source)); } diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index cc274594ce8c..d7bc68448a87 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -1,12 +1,13 @@ #pragma once #include +#include #include #include #include #include -#include #include +#include #include #include #include @@ -148,7 +149,7 @@ bool urlWithGlobs(const String & uri); String getSampleURI(String uri, ContextPtr context); -class StorageURLSource : public SourceWithKeyCondition, WithContext +class StorageURLSource : public ISource, WithContext { using URIParams = std::vector>; @@ -181,7 +182,7 @@ class StorageURLSource : public SourceWithKeyCondition, WithContext UInt64 max_block_size, const ConnectionTimeouts & timeouts, CompressionMethod compression_method, - size_t max_parsing_threads, + FormatParserGroupPtr parser_group_, const HTTPHeaderEntries & headers_ = {}, const URIParams & params = {}, bool glob_url = false, @@ -191,13 +192,10 @@ class StorageURLSource : public SourceWithKeyCondition, WithContext String getName() const override { return name; } - void setKeyCondition(const std::optional & filter_actions_dag, ContextPtr context_) override - { - setKeyConditionImpl(filter_actions_dag, context_, block_for_format); - } - Chunk generate() override; + void onFinish() override { parser_group->finishStream(); } + static void setCredentials(Poco::Net::HTTPBasicCredentials & credentials, const Poco::URI & request_uri); static std::pair> getFirstAvailableURIAndReadBuffer( @@ -231,6 +229,7 @@ class StorageURLSource : public SourceWithKeyCondition, WithContext std::optional current_file_size; String format; const std::optional & format_settings; + FormatParserGroupPtr parser_group; HTTPHeaderEntries headers; bool need_only_count; size_t total_rows_in_file = 0; diff --git a/src/Storages/StorageURLCluster.cpp b/src/Storages/StorageURLCluster.cpp index 32011140558c..710241a8f394 100644 --- a/src/Storages/StorageURLCluster.cpp +++ b/src/Storages/StorageURLCluster.cpp @@ -126,7 +126,11 @@ void StorageURLCluster::updateQueryToSendIfNeeded(ASTPtr & query, const StorageS ); } -RemoteQueryExecutor::Extension StorageURLCluster::getTaskIteratorExtension(const ActionsDAG::Node * predicate, const ContextPtr & context, size_t) const +RemoteQueryExecutor::Extension StorageURLCluster::getTaskIteratorExtension( + const ActionsDAG::Node * predicate, + const ActionsDAG * /* filter */, + const ContextPtr & context, + size_t) const { auto iterator = std::make_shared( uri, context->getSettingsRef()[Setting::glob_expansion_max_elements], predicate, getVirtualsList(), hive_partition_columns_to_read_from_file_path, context); diff --git a/src/Storages/StorageURLCluster.h b/src/Storages/StorageURLCluster.h index 8349f7594294..bebba232bd51 100644 --- a/src/Storages/StorageURLCluster.h +++ b/src/Storages/StorageURLCluster.h @@ -30,7 +30,12 @@ class StorageURLCluster : public IStorageCluster const StorageURL::Configuration & configuration_); std::string getName() const override { return "URLCluster"; } - RemoteQueryExecutor::Extension getTaskIteratorExtension(const ActionsDAG::Node * predicate, const ContextPtr & context, size_t number_of_replicas) const override; + + RemoteQueryExecutor::Extension getTaskIteratorExtension( + const ActionsDAG::Node * predicate, + const ActionsDAG * /* filter */, + const ContextPtr & context, + size_t number_of_replicas) const override; private: void updateQueryToSendIfNeeded(ASTPtr & query, const StorageSnapshotPtr & storage_snapshot, const ContextPtr & context) override; diff --git a/src/Storages/System/StorageSystemStackTrace.cpp b/src/Storages/System/StorageSystemStackTrace.cpp index 5341950c9b85..f7487044e783 100644 --- a/src/Storages/System/StorageSystemStackTrace.cpp +++ b/src/Storages/System/StorageSystemStackTrace.cpp @@ -281,7 +281,7 @@ class StackTraceSource : public ISource StackTraceSource( const Names & column_names, Block header_, - std::optional filter_dag_, + std::shared_ptr filter_dag_, ContextPtr context_, UInt64 max_block_size_, LoggerPtr log_) @@ -427,7 +427,7 @@ class StackTraceSource : public ISource private: ContextPtr context; Block header; - const std::optional filter_dag; + const std::shared_ptr filter_dag; const ActionsDAG::Node * predicate; const size_t max_block_size; @@ -475,7 +475,7 @@ class ReadFromSystemStackTrace : public SourceStepWithFilter Pipe pipe(std::make_shared( column_names, getOutputHeader(), - std::move(filter_actions_dag), + filter_actions_dag, context, max_block_size, log)); diff --git a/src/TableFunctions/TableFunctionObjectStorage.cpp b/src/TableFunctions/TableFunctionObjectStorage.cpp index 11a6f18f92e0..70f56934bfc1 100644 --- a/src/TableFunctions/TableFunctionObjectStorage.cpp +++ b/src/TableFunctions/TableFunctionObjectStorage.cpp @@ -52,7 +52,7 @@ ObjectStoragePtr TableFunctionObjectStorage -StorageObjectStorage::ConfigurationPtr TableFunctionObjectStorage::getConfiguration() const +StorageObjectStorageConfigurationPtr TableFunctionObjectStorage::getConfiguration() const { if (!configuration) { diff --git a/src/TableFunctions/TableFunctionObjectStorage.h b/src/TableFunctions/TableFunctionObjectStorage.h index b4090093dae9..c1b6003da005 100644 --- a/src/TableFunctions/TableFunctionObjectStorage.h +++ b/src/TableFunctions/TableFunctionObjectStorage.h @@ -156,7 +156,7 @@ class TableFunctionObjectStorage : public ITableFunction virtual void parseArgumentsImpl(ASTs & args, const ContextPtr & context) { - StorageObjectStorage::Configuration::initialize(*getConfiguration(), args, context, true); + StorageObjectStorageConfiguration::initialize(*getConfiguration(), args, context, true); } static void updateStructureAndFormatArgumentsIfNeeded( @@ -183,8 +183,6 @@ class TableFunctionObjectStorage : public ITableFunction } protected: - using ConfigurationPtr = StorageObjectStorage::ConfigurationPtr; - StoragePtr executeImpl( const ASTPtr & ast_function, ContextPtr context, @@ -198,11 +196,11 @@ class TableFunctionObjectStorage : public ITableFunction void parseArguments(const ASTPtr & ast_function, ContextPtr context) override; ObjectStoragePtr getObjectStorage(const ContextPtr & context, bool create_readonly) const; - ConfigurationPtr getConfiguration() const; + StorageObjectStorageConfigurationPtr getConfiguration() const; static std::shared_ptr createEmptySettings(); - mutable ConfigurationPtr configuration; + mutable StorageObjectStorageConfigurationPtr configuration; mutable ObjectStoragePtr object_storage; ColumnsDescription structure_hint; std::shared_ptr settings; diff --git a/tests/integration/test_format_cannot_allocate_thread/__init__.py b/tests/integration/test_format_cannot_allocate_thread/__init__.py new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/tests/integration/test_format_cannot_allocate_thread/configs/config.xml b/tests/integration/test_format_cannot_allocate_thread/configs/config.xml new file mode 100644 index 000000000000..a910debb62e9 --- /dev/null +++ b/tests/integration/test_format_cannot_allocate_thread/configs/config.xml @@ -0,0 +1,3 @@ + + 0.07 + \ No newline at end of file diff --git a/tests/integration/test_format_cannot_allocate_thread/test.py b/tests/integration/test_format_cannot_allocate_thread/test.py new file mode 100644 index 000000000000..9f48d33a0746 --- /dev/null +++ b/tests/integration/test_format_cannot_allocate_thread/test.py @@ -0,0 +1,59 @@ +import logging + +import pytest + +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import assert_eq_with_retry + +cluster = ClickHouseCluster(__file__) +node = cluster.add_instance( + "node1", + main_configs=[ + "configs/config.xml", + ], +) +path_to_data = "/var/lib/clickhouse/" + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + yield cluster + + finally: + cluster.shutdown() + + +def test_it(started_cluster): + if node.is_built_with_sanitizer(): + # Sanitizer builds might be too slow to run even 100 queries in a reasonable time. + pytest.skip("Disabled for sanitizers") + + for i in range(2): + assert_eq_with_retry( + node, f"INSERT INTO FUNCTION file('t{i}.parquet') SELECT * FROM numbers(10) SETTINGS max_threads=1, engine_file_truncate_on_insert=1", "", retry_count=100, sleep_time=0.0) + + # Parquet read with max_threads=2, max_parsing_threads=2 should grab ~4 threads from the pool, + # checking cannot_allocate_thread_fault_injection_probability (0.07) 4 times. + # So probability of success is 0.93^4 ~= 0.75. Probability of 100/100 successes is 2e-13. + # There might be more cannot_allocate_thread_fault_injection_probability checks in other parts + # of query processing, bringing success probability down, but hopefully not so many that all + # queries fail. + errors, successes = 0, 0 + for attempt in range(100): + try: + res = node.query("SELECT sum(number) FROM file('t{0,1}.parquet') SETTINGS max_threads=2, max_parsing_threads=2, max_download_threads=1") + assert res == "90\n" + successes += 1 + except Exception as ex: + if "Cannot schedule a task" in str(ex): + errors += 1 + else: + raise + # (As of the time of writing, this reports ~48 successes on average, suggesting that + # cannot_allocate_thread_fault_injection_probability is actually checked 10 times by the query.) + logging.info(f"{successes} successes, {errors} errors") + assert successes > 0 + assert errors > 0 diff --git a/tests/integration/test_storage_iceberg/test.py b/tests/integration/test_storage_iceberg/test.py index e0e957b70653..5bb8b53008de 100644 --- a/tests/integration/test_storage_iceberg/test.py +++ b/tests/integration/test_storage_iceberg/test.py @@ -666,6 +666,7 @@ def add_df(mode): # write 3 times assert int(instance.query(f"SELECT count() FROM {table_function_expr_cluster}")) == 100 * 3 + @pytest.mark.parametrize("format_version", ["1", "2"]) @pytest.mark.parametrize("storage_type", ["s3", "azure", "local"]) def test_delete_files(started_cluster, format_version, storage_type): @@ -2045,8 +2046,11 @@ def check_validity_and_get_prunned_files_general(instance, table_name, settings1 ) -@pytest.mark.parametrize("storage_type", ["s3", "azure", "local"]) -def test_partition_pruning(started_cluster, storage_type): +@pytest.mark.parametrize( + "storage_type, run_on_cluster", + [("s3", False), ("s3", True), ("azure", False), ("local", False)], +) +def test_partition_pruning(started_cluster, storage_type, run_on_cluster): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session TABLE_NAME = "test_partition_pruning_" + storage_type + "_" + get_uuid_str() @@ -2093,7 +2097,7 @@ def execute_spark_query(query: str): ) creation_expression = get_creation_expression( - storage_type, TABLE_NAME, started_cluster, table_function=True + storage_type, TABLE_NAME, started_cluster, table_function=True, run_on_cluster=run_on_cluster ) def check_validity_and_get_prunned_files(select_expression): diff --git a/tests/integration/test_storage_redis/test.py b/tests/integration/test_storage_redis/test.py index d54718531707..5a931aa2f4ee 100644 --- a/tests/integration/test_storage_redis/test.py +++ b/tests/integration/test_storage_redis/test.py @@ -1,4 +1,5 @@ ## sudo -H pip install redis +import json import struct import sys @@ -430,3 +431,33 @@ def test_direct_join(started_cluster): "test_direct_join.k = test_mt.k FORMAT TSV")) assert len(response) == 1 assert response[0] == ["1", "1"] + + +def test_get_keys(started_cluster): + """ + Checks that ClickHouse reads by key instead of full scan if possible. + """ + address = get_address_for_ch() + + # clean all + drop_table("test_get_keys") + + # create table + node.query(f""" + CREATE TABLE test_get_keys(k Int) Engine=Redis('{address}', 2, 'clickhouse') PRIMARY KEY (k); + INSERT INTO test_get_keys VALUES (1), (2), (3); + """) + + def check_query(query, read_type, keys_count, rows_read): + plan = node.query(f'EXPLAIN actions=1 {query}') + assert 'ReadFromRedis' in plan + assert f'ReadType: {read_type}' in plan + if read_type == 'GetKeys': + assert f'Keys: {keys_count}' in plan + + res = node.query(f'{query} FORMAT JSON') + assert json.loads(res)['statistics']['rows_read'] == rows_read, res + + check_query("SELECT * FROM test_get_keys", "FullScan", 0, 3) + check_query("SELECT * FROM test_get_keys WHERE k = 1", "GetKeys", 1, 1) + check_query("SELECT * FROM test_get_keys WHERE k in (3, 5)", "GetKeys", 2, 1) diff --git a/tests/queries/0_stateless/02375_rocksdb_with_filters.reference b/tests/queries/0_stateless/02375_rocksdb_with_filters.reference index 2d6a0e0ad78a..77201bc4c43d 100644 --- a/tests/queries/0_stateless/02375_rocksdb_with_filters.reference +++ b/tests/queries/0_stateless/02375_rocksdb_with_filters.reference @@ -1,5 +1,10 @@ + ReadFromEmbeddedRocksDB + ReadType: FullScan 1 "rows_read":1, + ReadFromEmbeddedRocksDB + ReadType: GetKeys + Keys: 1 2 "rows_read":2, 1 diff --git a/tests/queries/0_stateless/02375_rocksdb_with_filters.sh b/tests/queries/0_stateless/02375_rocksdb_with_filters.sh index 1f9922100cb0..3a03fb89accb 100755 --- a/tests/queries/0_stateless/02375_rocksdb_with_filters.sh +++ b/tests/queries/0_stateless/02375_rocksdb_with_filters.sh @@ -10,8 +10,11 @@ $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS rocksdb_with_filter;" $CLICKHOUSE_CLIENT --query="CREATE TABLE rocksdb_with_filter (key String, value String) ENGINE=EmbeddedRocksDB PRIMARY KEY key;" $CLICKHOUSE_CLIENT --query="INSERT INTO rocksdb_with_filter (*) SELECT n.number, n.number*10 FROM numbers(10000) n;" +$CLICKHOUSE_CLIENT --query "EXPLAIN actions=1 SELECT value FROM rocksdb_with_filter LIMIT 1" | grep -A 2 "ReadFromEmbeddedRocksDB" + $CLICKHOUSE_CLIENT --query "SELECT count() FROM rocksdb_with_filter WHERE key = '5000'" $CLICKHOUSE_CLIENT --query "SELECT value FROM rocksdb_with_filter WHERE key = '5000' FORMAT JSON" | grep "rows_read" | tr -d "[:blank:]" +$CLICKHOUSE_CLIENT --query "EXPLAIN actions=1 SELECT value FROM rocksdb_with_filter WHERE key = '5000'" | grep -A 3 "ReadFromEmbeddedRocksDB" $CLICKHOUSE_CLIENT --query "SELECT count() FROM rocksdb_with_filter WHERE key = '5000' OR key = '6000'" $CLICKHOUSE_CLIENT --query "SELECT value FROM rocksdb_with_filter WHERE key = '5000' OR key = '6000' FORMAT JSON" | grep "rows_read" | tr -d "[:blank:]" diff --git a/tests/queries/0_stateless/03541_keeper_map_filter_keys.reference b/tests/queries/0_stateless/03541_keeper_map_filter_keys.reference new file mode 100644 index 000000000000..12a3bbcf41a9 --- /dev/null +++ b/tests/queries/0_stateless/03541_keeper_map_filter_keys.reference @@ -0,0 +1,24 @@ + ReadFromKeeperMap + ReadType: FullScan +1 +"rows_read":1, + ReadFromKeeperMap + ReadType: GetKeys + Keys: 1 +2 +"rows_read":2, +1 +"rows_read":1, +2 +"rows_read":2, +1 +"rows_read":1, + ReadFromKeeperMap + ReadType: GetKeys + Keys: 1 +2 +"rows_read":2, +1 +"rows_read":1, +2 +"rows_read":2, diff --git a/tests/queries/0_stateless/03541_keeper_map_filter_keys.sh b/tests/queries/0_stateless/03541_keeper_map_filter_keys.sh new file mode 100755 index 000000000000..b862bda95799 --- /dev/null +++ b/tests/queries/0_stateless/03541_keeper_map_filter_keys.sh @@ -0,0 +1,48 @@ +#!/usr/bin/env bash +# Tags: no-fasttest, no-ordinary-database +# Tag no-ordinary-database: KeeperMap doesn't support Ordinary database + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS keeper_map_with_filter;" +$CLICKHOUSE_CLIENT --query="CREATE TABLE keeper_map_with_filter (key String, value String) ENGINE=KeeperMap(concat(currentDatabase(), '_simple')) PRIMARY KEY key;" +$CLICKHOUSE_CLIENT --query="INSERT INTO keeper_map_with_filter (*) SELECT n.number, n.number*10 FROM numbers(10) n;" + +$CLICKHOUSE_CLIENT --query "EXPLAIN actions=1 SELECT value FROM keeper_map_with_filter LIMIT 1" | grep -A 2 "ReadFromKeeperMap" + +$CLICKHOUSE_CLIENT --query "SELECT count() FROM keeper_map_with_filter WHERE key = '5'" +$CLICKHOUSE_CLIENT --query "SELECT value FROM keeper_map_with_filter WHERE key = '5' FORMAT JSON" | grep "rows_read" | tr -d "[:blank:]" +$CLICKHOUSE_CLIENT --query "EXPLAIN actions=1 SELECT value FROM keeper_map_with_filter WHERE key = '5'" | grep -A 3 "ReadFromKeeperMap" + +$CLICKHOUSE_CLIENT --query "SELECT count() FROM keeper_map_with_filter WHERE key = '5' OR key = '6'" +$CLICKHOUSE_CLIENT --query "SELECT value FROM keeper_map_with_filter WHERE key = '5' OR key = '6' FORMAT JSON" | grep "rows_read" | tr -d "[:blank:]" + +$CLICKHOUSE_CLIENT "--param_key=5" --query "SELECT count() FROM keeper_map_with_filter WHERE key = {key:String}" +$CLICKHOUSE_CLIENT "--param_key=5" --query "SELECT value FROM keeper_map_with_filter WHERE key = {key:String} FORMAT JSON" | grep "rows_read" | tr -d "[:blank:]" + +$CLICKHOUSE_CLIENT --query "SELECT count() FROM keeper_map_with_filter WHERE key IN ('5', '6')" +$CLICKHOUSE_CLIENT --query "SELECT value FROM keeper_map_with_filter WHERE key IN ('5', '6') FORMAT JSON" | grep "rows_read" | tr -d "[:blank:]" + +$CLICKHOUSE_CLIENT --query="DROP TABLE keeper_map_with_filter;" + +# Same test, but with complex key +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS keeper_map_with_filter_and_complex_key;" +$CLICKHOUSE_CLIENT --query="CREATE TABLE keeper_map_with_filter_and_complex_key (key String, value String) ENGINE=KeeperMap(concat(currentDatabase(), '_complex')) PRIMARY KEY hex(toFloat32(key));" +$CLICKHOUSE_CLIENT --query="INSERT INTO keeper_map_with_filter_and_complex_key (*) SELECT n.number, n.number*10 FROM numbers(10) n;" + +$CLICKHOUSE_CLIENT --query "SELECT count() FROM keeper_map_with_filter_and_complex_key WHERE key = '5'" +$CLICKHOUSE_CLIENT --query "SELECT value FROM keeper_map_with_filter_and_complex_key WHERE key = '5' FORMAT JSON" | grep "rows_read" | tr -d "[:blank:]" +$CLICKHOUSE_CLIENT --query "EXPLAIN actions=1 SELECT value FROM keeper_map_with_filter_and_complex_key WHERE key = '5'" | grep -A 3 "ReadFromKeeperMap" + +$CLICKHOUSE_CLIENT --query "SELECT count() FROM keeper_map_with_filter_and_complex_key WHERE key = '5' OR key = '6'" +$CLICKHOUSE_CLIENT --query "SELECT value FROM keeper_map_with_filter_and_complex_key WHERE key = '5' OR key = '6' FORMAT JSON" | grep "rows_read" | tr -d "[:blank:]" + +$CLICKHOUSE_CLIENT "--param_key=5" --query "SELECT count() FROM keeper_map_with_filter_and_complex_key WHERE key = {key:String}" +$CLICKHOUSE_CLIENT "--param_key=5" --query "SELECT value FROM keeper_map_with_filter_and_complex_key WHERE key = {key:String} FORMAT JSON" | grep "rows_read" | tr -d "[:blank:]" + +$CLICKHOUSE_CLIENT --query "SELECT count() FROM keeper_map_with_filter_and_complex_key WHERE key IN ('5', '6')" +$CLICKHOUSE_CLIENT --query "SELECT value FROM keeper_map_with_filter_and_complex_key WHERE key IN ('5', '6') FORMAT JSON" | grep "rows_read" | tr -d "[:blank:]" + +$CLICKHOUSE_CLIENT --query="DROP TABLE keeper_map_with_filter_and_complex_key;" diff --git a/tests/queries/0_stateless/03565_iceberg_field_ids_table.reference b/tests/queries/0_stateless/03565_iceberg_field_ids_table.reference new file mode 100644 index 000000000000..b4de39476753 --- /dev/null +++ b/tests/queries/0_stateless/03565_iceberg_field_ids_table.reference @@ -0,0 +1 @@ +11 diff --git a/tests/queries/0_stateless/03565_iceberg_field_ids_table.sql b/tests/queries/0_stateless/03565_iceberg_field_ids_table.sql new file mode 100644 index 000000000000..fc7042225afa --- /dev/null +++ b/tests/queries/0_stateless/03565_iceberg_field_ids_table.sql @@ -0,0 +1,4 @@ +-- Tags: no-fasttest +-- Tag no-fasttest: Depends on AWS + +SELECT * FROM icebergS3(s3_conn, filename='field_ids_table_test', SETTINGS iceberg_metadata_table_uuid = '8f1f9ae2-18bb-421e-b640-ec2f85e67bce') ORDER BY ALL; diff --git a/tests/queries/0_stateless/data_minio/field_ids_table_test/data/00000-1-cd905c52-8bdd-4d31-9dd5-c1566d06fd5f-0-00001.parquet b/tests/queries/0_stateless/data_minio/field_ids_table_test/data/00000-1-cd905c52-8bdd-4d31-9dd5-c1566d06fd5f-0-00001.parquet new file mode 100644 index 000000000000..8847fde8d15e Binary files /dev/null and b/tests/queries/0_stateless/data_minio/field_ids_table_test/data/00000-1-cd905c52-8bdd-4d31-9dd5-c1566d06fd5f-0-00001.parquet differ diff --git a/tests/queries/0_stateless/data_minio/field_ids_table_test/metadata/3b002f99-b012-4041-9a97-db477fcc7115-m0.avro b/tests/queries/0_stateless/data_minio/field_ids_table_test/metadata/3b002f99-b012-4041-9a97-db477fcc7115-m0.avro new file mode 100644 index 000000000000..2355cb4d229c Binary files /dev/null and b/tests/queries/0_stateless/data_minio/field_ids_table_test/metadata/3b002f99-b012-4041-9a97-db477fcc7115-m0.avro differ diff --git a/tests/queries/0_stateless/data_minio/field_ids_table_test/metadata/snap-2811410366534688344-1-3b002f99-b012-4041-9a97-db477fcc7115.avro b/tests/queries/0_stateless/data_minio/field_ids_table_test/metadata/snap-2811410366534688344-1-3b002f99-b012-4041-9a97-db477fcc7115.avro new file mode 100644 index 000000000000..9952188539d6 Binary files /dev/null and b/tests/queries/0_stateless/data_minio/field_ids_table_test/metadata/snap-2811410366534688344-1-3b002f99-b012-4041-9a97-db477fcc7115.avro differ diff --git a/tests/queries/0_stateless/data_minio/field_ids_table_test/metadata/v1.metadata.json b/tests/queries/0_stateless/data_minio/field_ids_table_test/metadata/v1.metadata.json new file mode 100644 index 000000000000..32225eb618ad --- /dev/null +++ b/tests/queries/0_stateless/data_minio/field_ids_table_test/metadata/v1.metadata.json @@ -0,0 +1,69 @@ +{ + "format-version" : 2, + "table-uuid" : "8f1f9ae2-18bb-421e-b640-ec2f85e67bce", + "location" : "s3a://test/field_ids_table_test/metadata/field_ids_table_test", + "last-sequence-number" : 1, + "last-updated-ms" : 1752481476160, + "last-column-id" : 1, + "current-schema-id" : 0, + "schemas" : [ { + "type" : "struct", + "schema-id" : 0, + "fields" : [ { + "id" : 1, + "name" : "test", + "required" : false, + "type" : "int" + } ] + } ], + "default-spec-id" : 0, + "partition-specs" : [ { + "spec-id" : 0, + "fields" : [ ] + } ], + "last-partition-id" : 999, + "default-sort-order-id" : 0, + "sort-orders" : [ { + "order-id" : 0, + "fields" : [ ] + } ], + "properties" : { + "owner" : "scanhex12", + "write.parquet.compression-codec" : "zstd" + }, + "current-snapshot-id" : 2811410366534688344, + "refs" : { + "main" : { + "snapshot-id" : 2811410366534688344, + "type" : "branch" + } + }, + "snapshots" : [ { + "sequence-number" : 1, + "snapshot-id" : 2811410366534688344, + "timestamp-ms" : 1752481476160, + "summary" : { + "operation" : "append", + "spark.app.id" : "local-1752481338404", + "added-data-files" : "1", + "added-records" : "1", + "added-files-size" : "415", + "changed-partition-count" : "1", + "total-records" : "1", + "total-files-size" : "415", + "total-data-files" : "1", + "total-delete-files" : "0", + "total-position-deletes" : "0", + "total-equality-deletes" : "0" + }, + "manifest-list" : "s3a://test/field_ids_table_test/metadata/field_ids_table_test/metadata/snap-2811410366534688344-1-3b002f99-b012-4041-9a97-db477fcc7115.avro", + "schema-id" : 0 + } ], + "statistics" : [ ], + "partition-statistics" : [ ], + "snapshot-log" : [ { + "timestamp-ms" : 1752481476160, + "snapshot-id" : 2811410366534688344 + } ], + "metadata-log" : [ ] +} \ No newline at end of file