From e335fc9726f0e5d93bf6b30f2d24400d33287a55 Mon Sep 17 00:00:00 2001 From: Chang chen Date: Fri, 6 Sep 2024 11:08:51 +0800 Subject: [PATCH] [GLUTEN-7028][CH][Part-1] Using `PushingPipelineExecutor` to write merge tree (#7029) * 1. Rename Storages/Mergetree to Storages/MergeTree 2. Move MergeTreeTool.cpp/.h from Common to Storages/MergeTree 3. Move CustomStorageMergeTree.cpp/.h and StorageMergeTreeFactory.cpp/.h to MergeTree folderMove CustomStorageMergeTree.cpp/.h and StorageMergeTreeFactory.cpp/.h to MergeTree folder 4. Add CustomMergeTreeDataWriter 5. Remove TempStorageFreer 6. Add SubstraitParserUtils * Make query_map_ as QueryContextManager member * EMBEDDED_PLAN and create_plan_and_executor * minor refactor * tmp * SparkStorageMergeTree CustomMergeTreeDataWriter => SparkMergeTreeDataWriter * Add SparkMergeTreeSink * use SparkStorageMergeTree and SparkMergeTreeSink * Introduce GlutenSettings.h * GlutenMergeTreeWriteSettings * Fix Test Build * typo * ContextPtr => const ContextPtr & * minor refactor * fix style * using GlutenMergeTreeWriteSettings * [TMP] GlutenMergeTreeWriteSettings refactor * [TMP] StorageMergeTreeWrapper * [TMP] StorageMergeTreeWrapper::commitPartToRemoteStorageIfNeeded * [TMP] StorageMergeTreeWrapper::saveMetadata * move thread pool * tmp * rename * move to sparkmergetreesink.h/cpp * MergeTreeTableInstance * sameStructWith => sameTable * parseStorageAndRestore => restoreStorage parseStorage => getStorage * Sink with MergeTreeTable table; * remvoe SparkMergeTreeWriter::writeTempPartAndFinalize * refactor SinkHelper::writeTempPart * Remove write_setting of SparkMergeTreeWriter * SparkMergeTreeWriter using PushingPipelineExecutor * SparkMergeTreeWriteSettings * tmp * GlutenMergeTreeWriteSettings => SparkMergeTreeWriteSettings * make CustomStorageMergeTree constructor protected * MergeTreeTool.cpp/.h => SparkMergeTreeMeta.cpp/.h * CustomStorageMergeTree.cpp/.h => SparkStorageMergeTree.cpp/.h * CustomStorageMergeTree => SparkStorageMergeTree SparkStorageMergeTree => SparkWriteStorageMergeTree * Refactor move codes from MergeTreeRelParser to MergeTreeTable and MergeTreeTableInstance * Refactor Make static member to normal member --- .../Builder/SerializedPlanBuilder.h | 3 +- cpp-ch/local-engine/CMakeLists.txt | 2 +- cpp-ch/local-engine/Common/CHUtil.cpp | 16 +- cpp-ch/local-engine/Common/CHUtil.h | 64 +- cpp-ch/local-engine/Common/GlutenConfig.h | 14 +- cpp-ch/local-engine/Common/GlutenSettings.h | 63 ++ cpp-ch/local-engine/Common/QueryContext.cpp | 39 +- cpp-ch/local-engine/Common/QueryContext.h | 8 +- .../Parser/MergeTreeRelParser.cpp | 108 +--- .../local-engine/Parser/MergeTreeRelParser.h | 21 +- .../Parser/SerializedPlanParser.cpp | 21 +- .../Parser/SerializedPlanParser.h | 2 +- .../Parser/SubstraitParserUtils.cpp | 47 ++ .../Parser/SubstraitParserUtils.h | 4 + cpp-ch/local-engine/Parser/WriteRelParser.cpp | 89 +-- cpp-ch/local-engine/Parser/WriteRelParser.h | 9 +- .../Storages/Cache/CacheManager.cpp | 18 +- .../Storages/Cache/CacheManager.h | 18 +- .../MergeSparkMergeTreeTask.cpp | 6 +- .../MergeSparkMergeTreeTask.h | 11 +- .../MetaDataHelper.cpp | 21 +- .../{Mergetree => MergeTree}/MetaDataHelper.h | 13 +- .../MergeTree/SparkMergeTreeMeta.cpp} | 204 ++++--- .../MergeTree/SparkMergeTreeMeta.h} | 57 +- .../Storages/MergeTree/SparkMergeTreeSink.cpp | 307 ++++++++++ .../Storages/MergeTree/SparkMergeTreeSink.h | 183 ++++++ .../MergeTree/SparkMergeTreeWriteSettings.cpp | 41 ++ .../MergeTree/SparkMergeTreeWriteSettings.h | 40 ++ .../MergeTree/SparkMergeTreeWriter.cpp | 151 +++++ .../Storages/MergeTree/SparkMergeTreeWriter.h | 75 +++ .../SparkStorageMergeTree.cpp} | 239 ++++++-- .../SparkStorageMergeTree.h} | 91 ++- .../StorageMergeTreeFactory.cpp | 25 +- .../{ => MergeTree}/StorageMergeTreeFactory.h | 43 +- .../Mergetree/SparkMergeTreeWriter.cpp | 547 ------------------ .../Storages/Mergetree/SparkMergeTreeWriter.h | 108 ---- cpp-ch/local-engine/local_engine_jni.cpp | 123 ++-- .../tests/benchmark_local_engine.cpp | 6 +- .../tests/benchmark_spark_divide_function.cpp | 13 +- .../18_2.json | 0 .../18_2_flba.snappy.parquet | Bin .../local-engine/tests/gluten_test_util.cpp | 13 +- cpp-ch/local-engine/tests/gluten_test_util.h | 29 +- cpp-ch/local-engine/tests/gtest_ch_join.cpp | 2 +- .../local-engine/tests/gtest_ch_storages.cpp | 3 +- .../tests/gtest_clickhouse_pr_verify.cpp | 54 +- .../local-engine/tests/gtest_local_engine.cpp | 2 +- .../tests/gtest_parquet_columnindex_bug.cpp | 20 +- cpp-ch/local-engine/tests/gtest_parser.cpp | 10 +- .../tests/gtest_write_pipeline.cpp | 255 +++++++- .../tests/json/gtest_local_engine_config.json | 88 +++ .../tests/json/mergetree/1_mergetree.json | 6 + .../json/mergetree/1_mergetree_hdfs.json | 6 + .../tests/json/mergetree/1_plan.json | 246 ++++++++ cpp-ch/local-engine/tests/testConfig.h.in | 3 +- 55 files changed, 2239 insertions(+), 1348 deletions(-) create mode 100644 cpp-ch/local-engine/Common/GlutenSettings.h create mode 100644 cpp-ch/local-engine/Parser/SubstraitParserUtils.cpp rename cpp-ch/local-engine/Storages/{Mergetree => MergeTree}/MergeSparkMergeTreeTask.cpp (99%) rename cpp-ch/local-engine/Storages/{Mergetree => MergeTree}/MergeSparkMergeTreeTask.h (96%) rename cpp-ch/local-engine/Storages/{Mergetree => MergeTree}/MetaDataHelper.cpp (89%) rename cpp-ch/local-engine/Storages/{Mergetree => MergeTree}/MetaDataHelper.h (74%) rename cpp-ch/local-engine/{Common/MergeTreeTool.cpp => Storages/MergeTree/SparkMergeTreeMeta.cpp} (64%) rename cpp-ch/local-engine/{Common/MergeTreeTool.h => Storages/MergeTree/SparkMergeTreeMeta.h} (58%) create mode 100644 cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeSink.cpp create mode 100644 cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeSink.h create mode 100644 cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriteSettings.cpp create mode 100644 cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriteSettings.h create mode 100644 cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriter.cpp create mode 100644 cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriter.h rename cpp-ch/local-engine/Storages/{CustomStorageMergeTree.cpp => MergeTree/SparkStorageMergeTree.cpp} (52%) rename cpp-ch/local-engine/Storages/{CustomStorageMergeTree.h => MergeTree/SparkStorageMergeTree.h} (60%) rename cpp-ch/local-engine/Storages/{ => MergeTree}/StorageMergeTreeFactory.cpp (80%) rename cpp-ch/local-engine/Storages/{ => MergeTree}/StorageMergeTreeFactory.h (69%) delete mode 100644 cpp-ch/local-engine/Storages/Mergetree/SparkMergeTreeWriter.cpp delete mode 100644 cpp-ch/local-engine/Storages/Mergetree/SparkMergeTreeWriter.h rename cpp-ch/local-engine/tests/{decmial_filter_push_down => decimal_filter_push_down}/18_2.json (100%) rename cpp-ch/local-engine/tests/{decmial_filter_push_down => decimal_filter_push_down}/18_2_flba.snappy.parquet (100%) create mode 100644 cpp-ch/local-engine/tests/json/mergetree/1_mergetree.json create mode 100644 cpp-ch/local-engine/tests/json/mergetree/1_mergetree_hdfs.json create mode 100644 cpp-ch/local-engine/tests/json/mergetree/1_plan.json diff --git a/cpp-ch/local-engine/Builder/SerializedPlanBuilder.h b/cpp-ch/local-engine/Builder/SerializedPlanBuilder.h index 4b05ae24e3b7..0cf6be1a3afa 100644 --- a/cpp-ch/local-engine/Builder/SerializedPlanBuilder.h +++ b/cpp-ch/local-engine/Builder/SerializedPlanBuilder.h @@ -16,9 +16,8 @@ */ #pragma once +#include #include -#include - namespace dbms { diff --git a/cpp-ch/local-engine/CMakeLists.txt b/cpp-ch/local-engine/CMakeLists.txt index 2bf99a4949fc..85045941cdee 100644 --- a/cpp-ch/local-engine/CMakeLists.txt +++ b/cpp-ch/local-engine/CMakeLists.txt @@ -52,7 +52,7 @@ add_headers_and_sources(storages Storages) add_headers_and_sources(storages Storages/Output) add_headers_and_sources(storages Storages/Serializations) add_headers_and_sources(storages Storages/IO) -add_headers_and_sources(storages Storages/Mergetree) +add_headers_and_sources(storages Storages/MergeTree) add_headers_and_sources(storages Storages/Cache) add_headers_and_sources(common Common) add_headers_and_sources(external External) diff --git a/cpp-ch/local-engine/Common/CHUtil.cpp b/cpp-ch/local-engine/Common/CHUtil.cpp index 9558bf957d4a..6a4643ad0057 100644 --- a/cpp-ch/local-engine/Common/CHUtil.cpp +++ b/cpp-ch/local-engine/Common/CHUtil.cpp @@ -51,6 +51,7 @@ #include #include #include +#include #include #include #include @@ -58,8 +59,8 @@ #include #include #include +#include #include -#include #include #include #include @@ -519,16 +520,9 @@ std::map BackendInitializerUtil::getBackendConfMap(std if (!success) break; - if (logger && logger->debug()) - { - namespace pb_util = google::protobuf::util; - pb_util::JsonOptions options; - std::string json; - auto s = pb_util::MessageToJsonString(sPlan, &json, options); - if (!s.ok()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Can not convert Substrait Plan to Json"); - LOG_DEBUG(&Poco::Logger::get("CHUtil"), "Update Config Map Plan:\n{}", json); - } + /// see initLoggers, logger == nullptr which meanas initLoggers is not called. + if (logger != nullptr) + logDebugMessage(sPlan, "Update Config Map Plan"); if (!sPlan.has_advanced_extensions() || !sPlan.advanced_extensions().has_enhancement()) break; diff --git a/cpp-ch/local-engine/Common/CHUtil.h b/cpp-ch/local-engine/Common/CHUtil.h index a92155d14ea1..3c741c7ffa22 100644 --- a/cpp-ch/local-engine/Common/CHUtil.h +++ b/cpp-ch/local-engine/Common/CHUtil.h @@ -42,8 +42,6 @@ namespace local_engine static const String MERGETREE_INSERT_WITHOUT_LOCAL_STORAGE = "mergetree.insert_without_local_storage"; static const String MERGETREE_MERGE_AFTER_INSERT = "mergetree.merge_after_insert"; static const std::string DECIMAL_OPERATIONS_ALLOW_PREC_LOSS = "spark.sql.decimalOperations.allowPrecisionLoss"; -static const std::string SPARK_TASK_WRITE_TMEP_DIR = "gluten.write.temp.dir"; -static const std::string SPARK_TASK_WRITE_FILENAME = "gluten.write.file.name"; static const std::unordered_set BOOL_VALUE_SETTINGS{ MERGETREE_MERGE_AFTER_INSERT, MERGETREE_INSERT_WITHOUT_LOCAL_STORAGE, DECIMAL_OPERATIONS_ALLOW_PREC_LOSS}; @@ -84,7 +82,8 @@ class BlockUtil /// The column names may be different in two blocks. /// and the nullability also could be different, with TPCDS-Q1 as an example. - static DB::ColumnWithTypeAndName convertColumnAsNecessary(const DB::ColumnWithTypeAndName & column, const DB::ColumnWithTypeAndName & sample_column); + static DB::ColumnWithTypeAndName + convertColumnAsNecessary(const DB::ColumnWithTypeAndName & column, const DB::ColumnWithTypeAndName & sample_column); }; class PODArrayUtil @@ -216,7 +215,8 @@ class BackendInitializerUtil static void registerAllFactories(); static void applyGlobalConfigAndSettings(DB::Context::ConfigurationPtr, DB::Settings &); static void updateNewSettings(const DB::ContextMutablePtr &, const DB::Settings &); - static std::vector wrapDiskPathConfig(const String & path_prefix, const String & path_suffix, Poco::Util::AbstractConfiguration & config); + static std::vector + wrapDiskPathConfig(const String & path_prefix, const String & path_suffix, Poco::Util::AbstractConfiguration & config); static std::map getBackendConfMap(std::string_view plan); @@ -262,64 +262,12 @@ class MemoryUtil static UInt64 getMemoryRSS(); }; -template -class ConcurrentDeque -{ -public: - std::optional pop_front() - { - std::lock_guard lock(mtx); - - if (deq.empty()) - return {}; - - T t = deq.front(); - deq.pop_front(); - return t; - } - - void emplace_back(T value) - { - std::lock_guard lock(mtx); - deq.emplace_back(value); - } - - void emplace_back(std::vector values) - { - std::lock_guard lock(mtx); - deq.insert(deq.end(), values.begin(), values.end()); - } - - void emplace_front(T value) - { - std::lock_guard lock(mtx); - deq.emplace_front(value); - } - - size_t size() - { - std::lock_guard lock(mtx); - return deq.size(); - } - - bool empty() - { - std::lock_guard lock(mtx); - return deq.empty(); - } - - std::deque unsafeGet() { return deq; } - -private: - std::deque deq; - mutable std::mutex mtx; -}; - class JoinUtil { public: static void reorderJoinOutput(DB::QueryPlan & plan, DB::Names cols); - static std::pair getJoinKindAndStrictness(substrait::JoinRel_JoinType join_type, bool is_existence_join); + static std::pair + getJoinKindAndStrictness(substrait::JoinRel_JoinType join_type, bool is_existence_join); static std::pair getCrossJoinKindAndStrictness(substrait::CrossRel_JoinType join_type); }; diff --git a/cpp-ch/local-engine/Common/GlutenConfig.h b/cpp-ch/local-engine/Common/GlutenConfig.h index 38c4ce162138..feded16e4667 100644 --- a/cpp-ch/local-engine/Common/GlutenConfig.h +++ b/cpp-ch/local-engine/Common/GlutenConfig.h @@ -34,7 +34,7 @@ struct MemoryConfig size_t off_heap_per_task = 0; double spill_mem_ratio = 0.9; - static MemoryConfig loadFromContext(DB::ContextPtr context) + static MemoryConfig loadFromContext(const DB::ContextPtr & context) { MemoryConfig config; config.extra_memory_hard_limit = context->getConfigRef().getUInt64(EXTRA_MEMORY_HARD_LIMIT, 0); @@ -58,7 +58,7 @@ struct GraceMergingAggregateConfig size_t max_pending_flush_blocks_per_grace_aggregate_merging_bucket = 1_MiB; double max_allowed_memory_usage_ratio_for_aggregate_merging = 0.9; - static GraceMergingAggregateConfig loadFromContext(DB::ContextPtr context) + static GraceMergingAggregateConfig loadFromContext(const DB::ContextPtr & context) { GraceMergingAggregateConfig config; config.max_grace_aggregate_merging_buckets = context->getConfigRef().getUInt64(MAX_GRACE_AGGREGATE_MERGING_BUCKETS, 32); @@ -82,7 +82,7 @@ struct StreamingAggregateConfig double high_cardinality_threshold_for_streaming_aggregating = 0.8; bool enable_streaming_aggregating = true; - static StreamingAggregateConfig loadFromContext(DB::ContextPtr context) + static StreamingAggregateConfig loadFromContext(const DB::ContextPtr & context) { StreamingAggregateConfig config; config.aggregated_keys_before_streaming_aggregating_evict = context->getConfigRef().getUInt64(AGGREGATED_KEYS_BEFORE_STREAMING_AGGREGATING_EVICT, 1024); @@ -122,7 +122,7 @@ struct ExecutorConfig bool dump_pipeline = false; bool use_local_format = false; - static ExecutorConfig loadFromContext(DB::ContextPtr context) + static ExecutorConfig loadFromContext(const DB::ContextPtr & context) { ExecutorConfig config; config.dump_pipeline = context->getConfigRef().getBool(DUMP_PIPELINE, false); @@ -161,7 +161,7 @@ struct S3Config String s3_local_cache_cache_path = ""; bool s3_gcs_issue_compose_request = false; - static S3Config loadFromContext(DB::ContextPtr context) + static S3Config loadFromContext(const DB::ContextPtr & context) { S3Config config; @@ -187,7 +187,7 @@ struct MergeTreeConfig size_t table_part_metadata_cache_max_count = 5000; size_t table_metadata_cache_max_count = 500; - static MergeTreeConfig loadFromContext(DB::ContextPtr context) + static MergeTreeConfig loadFromContext(const DB::ContextPtr & context) { MergeTreeConfig config; config.table_part_metadata_cache_max_count = context->getConfigRef().getUInt64(TABLE_PART_METADATA_CACHE_MAX_COUNT, 5000); @@ -202,7 +202,7 @@ struct GlutenJobSchedulerConfig size_t job_scheduler_max_threads = 10; - static GlutenJobSchedulerConfig loadFromContext(DB::ContextPtr context) + static GlutenJobSchedulerConfig loadFromContext(const DB::ContextPtr & context) { GlutenJobSchedulerConfig config; config.job_scheduler_max_threads = context->getConfigRef().getUInt64(JOB_SCHEDULER_MAX_THREADS, 10); diff --git a/cpp-ch/local-engine/Common/GlutenSettings.h b/cpp-ch/local-engine/Common/GlutenSettings.h new file mode 100644 index 000000000000..13c0a1327b64 --- /dev/null +++ b/cpp-ch/local-engine/Common/GlutenSettings.h @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#pragma once +#include + +namespace local_engine +{ + +#define SKIP_ALIAS(ALIAS_NAME) + +#define DECLARE_GLUTEN_SETTING_STATIC_MEMBER_(TYPE, NAME, DEFAULT, DESCRIPTION, UNIQ) TYPE NAME{DEFAULT}; + +#define GLUTEN_SETTING_STATIC_MEMBER_(NAME) s_##NAME##_ + +#define INITIALIZE_GLUTEN_SETTING_STATIC_MEMBER_(TYPE, NAME, DEFAULT, DESCRIPTION, UNIQ) \ + static constexpr std::string_view GLUTEN_SETTING_STATIC_MEMBER_(NAME) = "g." #UNIQ "." #NAME; + +#define DECLARE_GLUTEN_SETTINGS(SETTINGS_CLASS_NAME, LIST_OF_SETTINGS_MACRO) \ + struct SETTINGS_CLASS_NAME \ + { \ + LIST_OF_SETTINGS_MACRO(DECLARE_GLUTEN_SETTING_STATIC_MEMBER_, SKIP_ALIAS, _) \ + static SETTINGS_CLASS_NAME get(const DB::ContextPtr & context); \ + void set(const DB::ContextMutablePtr & context) const; \ +\ + private: \ + LIST_OF_SETTINGS_MACRO(INITIALIZE_GLUTEN_SETTING_STATIC_MEMBER_, SKIP_ALIAS, __COUNTER__) \ + }; + +#define IMPLEMENT_GLUTEN_GET_(TYPE, NAME, DEFAULT, DESCRIPTION, UNIQ) \ + if (DB::Field field_##NAME; settings.tryGet(GLUTEN_SETTING_STATIC_MEMBER_(NAME), field_##NAME)) \ + result.NAME = field_##NAME.safeGet(); + +#define IMPLEMENT_GLUTEN_SET_(TYPE, NAME, DEFAULT, DESCRIPTION, UNIQ) context->setSetting(GLUTEN_SETTING_STATIC_MEMBER_(NAME), NAME); + +#define IMPLEMENT_GLUTEN_SETTINGS(SETTINGS_CLASS_NAME, LIST_OF_SETTINGS_MACRO) \ + SETTINGS_CLASS_NAME SETTINGS_CLASS_NAME::get(const DB::ContextPtr & context) \ + { \ + SETTINGS_CLASS_NAME result; \ + const DB::Settings & settings = context->getSettingsRef(); \ + LIST_OF_SETTINGS_MACRO(IMPLEMENT_GLUTEN_GET_, SKIP_ALIAS, _) \ + return result; \ + } \ + void SETTINGS_CLASS_NAME::SETTINGS_CLASS_NAME::set(const DB::ContextMutablePtr & context) const \ + { \ + LIST_OF_SETTINGS_MACRO(IMPLEMENT_GLUTEN_SET_, SKIP_ALIAS, _) \ + } + + +} \ No newline at end of file diff --git a/cpp-ch/local-engine/Common/QueryContext.cpp b/cpp-ch/local-engine/Common/QueryContext.cpp index 0abff2fc143d..7cd96f4b1c07 100644 --- a/cpp-ch/local-engine/Common/QueryContext.cpp +++ b/cpp-ch/local-engine/Common/QueryContext.cpp @@ -16,17 +16,16 @@ */ #include "QueryContext.h" +#include +#include #include #include -#include -#include +#include #include -#include #include -#include -#include -#include - +#include +#include +#include namespace DB { @@ -47,8 +46,6 @@ struct QueryContext ContextMutablePtr query_context; }; -ConcurrentMap> query_map; - int64_t QueryContextManager::initializeQuery() { std::shared_ptr query_context = std::make_shared(); @@ -71,15 +68,15 @@ int64_t QueryContextManager::initializeQuery() query_context->thread_group->memory_tracker.setSoftLimit(memory_limit); query_context->thread_group->memory_tracker.setHardLimit(memory_limit + config.extra_memory_hard_limit); int64_t id = reinterpret_cast(query_context->thread_group.get()); - query_map.insert(id, query_context); + query_map_.insert(id, query_context); return id; } DB::ContextMutablePtr QueryContextManager::currentQueryContext() { auto thread_group = currentThreadGroup(); - int64_t id = reinterpret_cast(CurrentThread::getGroup().get()); - return query_map.get(id)->query_context; + const int64_t id = reinterpret_cast(CurrentThread::getGroup().get()); + return query_map_.get(id)->query_context; } std::shared_ptr QueryContextManager::currentThreadGroup() @@ -90,13 +87,13 @@ std::shared_ptr QueryContextManager::currentThreadGroup() throw Exception(ErrorCodes::LOGICAL_ERROR, "Thread group not found."); } -void QueryContextManager::logCurrentPerformanceCounters(ProfileEvents::Counters & counters) +void QueryContextManager::logCurrentPerformanceCounters(ProfileEvents::Counters & counters) const { if (!CurrentThread::getGroup()) { return; } - if (logger->information()) + if (logger_->information()) { std::ostringstream msg; msg << "\n---------------------Task Performance Counters-----------------------------\n"; @@ -111,15 +108,15 @@ void QueryContextManager::logCurrentPerformanceCounters(ProfileEvents::Counters << std::setw(20) << std::setfill(' ') << std::left << count.load() << " | (" << doc << ")\n"; } - LOG_INFO(logger, "{}", msg.str()); + LOG_INFO(logger_, "{}", msg.str()); } } size_t QueryContextManager::currentPeakMemory(int64_t id) { - if (!query_map.contains(id)) + if (!query_map_.contains(id)) throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "context released {}", id); - return query_map.get(id)->thread_group->memory_tracker.getPeak(); + return query_map_.get(id)->thread_group->memory_tracker.getPeak(); } void QueryContextManager::finalizeQuery(int64_t id) @@ -130,7 +127,7 @@ void QueryContextManager::finalizeQuery(int64_t id) } std::shared_ptr context; { - context = query_map.get(id); + context = query_map_.get(id); } auto query_context = context->thread_status->getQueryContext(); if (!query_context) @@ -139,11 +136,11 @@ void QueryContextManager::finalizeQuery(int64_t id) } context->thread_status->flushUntrackedMemory(); context->thread_status->finalizePerformanceCounters(); - LOG_INFO(logger, "Task finished, peak memory usage: {} bytes", currentPeakMemory(id)); + LOG_INFO(logger_, "Task finished, peak memory usage: {} bytes", currentPeakMemory(id)); if (currentThreadGroupMemoryUsage() > 1_MiB) { - LOG_WARNING(logger, "{} bytes memory didn't release, There may be a memory leak!", currentThreadGroupMemoryUsage()); + LOG_WARNING(logger_, "{} bytes memory didn't release, There may be a memory leak!", currentThreadGroupMemoryUsage()); } logCurrentPerformanceCounters(context->thread_group->performance_counters); context->thread_status->detachFromGroup(); @@ -151,7 +148,7 @@ void QueryContextManager::finalizeQuery(int64_t id) context->thread_status.reset(); query_context.reset(); { - query_map.erase(id); + query_map_.erase(id); } } diff --git a/cpp-ch/local-engine/Common/QueryContext.h b/cpp-ch/local-engine/Common/QueryContext.h index 4770327d1715..5079589f46b0 100644 --- a/cpp-ch/local-engine/Common/QueryContext.h +++ b/cpp-ch/local-engine/Common/QueryContext.h @@ -16,10 +16,13 @@ */ #pragma once #include +#include #include namespace local_engine { +struct QueryContext; + class QueryContextManager { public: @@ -31,13 +34,14 @@ class QueryContextManager int64_t initializeQuery(); DB::ContextMutablePtr currentQueryContext(); static std::shared_ptr currentThreadGroup(); - void logCurrentPerformanceCounters(ProfileEvents::Counters& counters); + void logCurrentPerformanceCounters(ProfileEvents::Counters & counters) const; size_t currentPeakMemory(int64_t id); void finalizeQuery(int64_t id); private: QueryContextManager() = default; - LoggerPtr logger = getLogger("QueryContextManager"); + LoggerPtr logger_ = getLogger("QueryContextManager"); + ConcurrentMap> query_map_{}; }; size_t currentThreadGroupMemoryUsage(); diff --git a/cpp-ch/local-engine/Parser/MergeTreeRelParser.cpp b/cpp-ch/local-engine/Parser/MergeTreeRelParser.cpp index b1b024ce5e6d..b85ed25e15dc 100644 --- a/cpp-ch/local-engine/Parser/MergeTreeRelParser.cpp +++ b/cpp-ch/local-engine/Parser/MergeTreeRelParser.cpp @@ -15,20 +15,15 @@ * limitations under the License. */ - -#include - +#include "MergeTreeRelParser.h" #include +#include #include -#include -#include -#include +#include +#include #include #include -#include "MergeTreeRelParser.h" - - namespace DB { namespace ErrorCodes @@ -44,7 +39,7 @@ namespace local_engine { using namespace DB; -/// Find minimal position of any of the column in primary key. +/// Find minimal position of the column in primary key. static Int64 findMinPosition(const NameSet & condition_table_columns, const NameToIndexMap & primary_key_positions) { Int64 min_position = std::numeric_limits::max() - 1; @@ -59,75 +54,13 @@ static Int64 findMinPosition(const NameSet & condition_table_columns, const Name return min_position; } -MergeTreeTable MergeTreeRelParser::parseMergeTreeTable(const substrait::ReadRel::ExtensionTable & extension_table) -{ - google::protobuf::StringValue table; - table.ParseFromString(extension_table.detail().value()); - return parseMergeTreeTableString(table.value()); -} - -CustomStorageMergeTreePtr -MergeTreeRelParser::parseStorage(const MergeTreeTable & merge_tree_table, ContextMutablePtr context, bool restore) -{ - DB::Block header = TypeParser::buildBlockFromNamedStruct(merge_tree_table.schema, merge_tree_table.low_card_key); - auto names_and_types_list = header.getNamesAndTypesList(); - auto metadata = buildMetaData(names_and_types_list, context, merge_tree_table); - - // use instance global table (without uuid) to restore metadata folder on current instance - // we need its lock - auto global_storage = StorageMergeTreeFactory::getStorage( - StorageID(merge_tree_table.database, merge_tree_table.table), - merge_tree_table.snapshot_id, - merge_tree_table, - [&]() -> CustomStorageMergeTreePtr - { - auto custom_storage_merge_tree = std::make_shared( - StorageID(merge_tree_table.database, merge_tree_table.table), - merge_tree_table.relative_path, - *metadata, - false, - context, - "", - MergeTreeData::MergingParams(), - buildMergeTreeSettings(merge_tree_table.table_configs)); - return custom_storage_merge_tree; - }); - - if (restore) - restoreMetaData(global_storage, merge_tree_table, *context); - - return global_storage; -} - -CustomStorageMergeTreePtr -MergeTreeRelParser::copyToDefaultPolicyStorage(MergeTreeTable merge_tree_table, ContextMutablePtr context) -{ - auto temp_uuid = UUIDHelpers::generateV4(); - String temp_uuid_str = toString(temp_uuid); - merge_tree_table.table = merge_tree_table.table + "_" + temp_uuid_str; - merge_tree_table.snapshot_id = ""; - merge_tree_table.table_configs.storage_policy = ""; - merge_tree_table.relative_path = merge_tree_table.relative_path + "_" + temp_uuid_str; - return parseStorage(merge_tree_table, context); -} - -CustomStorageMergeTreePtr -MergeTreeRelParser::copyToVirtualStorage(MergeTreeTable merge_tree_table, ContextMutablePtr context) -{ - auto temp_uuid = UUIDHelpers::generateV4(); - String temp_uuid_str = toString(temp_uuid); - merge_tree_table.table = merge_tree_table.table + "_" + temp_uuid_str; - merge_tree_table.snapshot_id = ""; - return parseStorage(merge_tree_table, context); -} - DB::QueryPlanPtr MergeTreeRelParser::parseReadRel( DB::QueryPlanPtr query_plan, const substrait::ReadRel & rel, const substrait::ReadRel::ExtensionTable & extension_table) { - auto merge_tree_table = parseMergeTreeTable(extension_table); + MergeTreeTableInstance merge_tree_table(extension_table); // ignore snapshot id for query merge_tree_table.snapshot_id = ""; - auto storage = parseStorage(merge_tree_table, global_context, true); + auto storage = merge_tree_table.restoreStorage(global_context); DB::Block input; if (rel.has_base_schema() && rel.base_schema().names_size()) @@ -158,8 +91,8 @@ DB::QueryPlanPtr MergeTreeRelParser::parseReadRel( query_info->prewhere_info = parsePreWhereInfo(rel.filter(), input); } - std::vector selected_parts - = StorageMergeTreeFactory::getDataPartsByNames(storage->getStorageID(), merge_tree_table.snapshot_id, merge_tree_table.getPartNames()); + std::vector selected_parts = StorageMergeTreeFactory::getDataPartsByNames( + storage->getStorageID(), merge_tree_table.snapshot_id, merge_tree_table.getPartNames()); auto read_step = storage->reader.readFromParts( selected_parts, @@ -182,9 +115,9 @@ DB::QueryPlanPtr MergeTreeRelParser::parseReadRel( auto ranges = merge_tree_table.extractRange(selected_parts); std::string ret; if (context->getSettingsRef().tryGetString("enabled_driver_filter_mergetree_index", ret) && ret == "'true'") - storage->analysisPartsByRanges(*reinterpret_cast(read_step.get()), ranges); + SparkStorageMergeTree::analysisPartsByRanges(*reinterpret_cast(read_step.get()), ranges); else - storage->wrapRangesInDataParts(*reinterpret_cast(read_step.get()), ranges); + SparkStorageMergeTree::wrapRangesInDataParts(*reinterpret_cast(read_step.get()), ranges); steps.emplace_back(read_step.get()); query_plan->addStep(std::move(read_step)); @@ -375,22 +308,18 @@ void MergeTreeRelParser::collectColumns(const substrait::Expression & rel, NameS } } - -String MergeTreeRelParser::getCHFunctionName(const substrait::Expression_ScalarFunction & substrait_func) +String MergeTreeRelParser::getCHFunctionName(const substrait::Expression_ScalarFunction & substrait_func) const { auto func_signature = getPlanParser()->function_mapping.at(std::to_string(substrait_func.function_reference())); return getPlanParser()->getFunctionName(func_signature, substrait_func); } - String MergeTreeRelParser::filterRangesOnDriver(const substrait::ReadRel & read_rel) { - google::protobuf::StringValue table; - table.ParseFromString(read_rel.advanced_extension().enhancement().value()); - auto merge_tree_table = parseMergeTreeTableString(table.value()); + MergeTreeTableInstance merge_tree_table(read_rel.advanced_extension().enhancement()); // ignore snapshot id for query merge_tree_table.snapshot_id = ""; - auto custom_storage_mergetree = parseStorage(merge_tree_table, global_context, true); + auto storage = merge_tree_table.restoreStorage(global_context); auto input = TypeParser::buildBlockFromNamedStruct(read_rel.base_schema()); auto names_and_types_list = input.getNamesAndTypesList(); @@ -398,14 +327,13 @@ String MergeTreeRelParser::filterRangesOnDriver(const substrait::ReadRel & read_ query_info->prewhere_info = parsePreWhereInfo(read_rel.filter(), input); - auto storage_factory = StorageMergeTreeFactory::instance(); - std::vector selected_parts - = storage_factory.getDataPartsByNames(StorageID(merge_tree_table.database, merge_tree_table.table), merge_tree_table.snapshot_id, merge_tree_table.getPartNames()); + std::vector selected_parts = StorageMergeTreeFactory::getDataPartsByNames( + StorageID(merge_tree_table.database, merge_tree_table.table), merge_tree_table.snapshot_id, merge_tree_table.getPartNames()); - auto storage_snapshot = std::make_shared(*custom_storage_mergetree, custom_storage_mergetree->getInMemoryMetadataPtr()); + auto storage_snapshot = std::make_shared(*storage, storage->getInMemoryMetadataPtr()); if (selected_parts.empty()) throw Exception(ErrorCodes::NO_SUCH_DATA_PART, "no data part found."); - auto read_step = custom_storage_mergetree->reader.readFromParts( + auto read_step = storage->reader.readFromParts( selected_parts, /* alter_conversions = */ {}, diff --git a/cpp-ch/local-engine/Parser/MergeTreeRelParser.h b/cpp-ch/local-engine/Parser/MergeTreeRelParser.h index b26239dc4d36..91c335a686a0 100644 --- a/cpp-ch/local-engine/Parser/MergeTreeRelParser.h +++ b/cpp-ch/local-engine/Parser/MergeTreeRelParser.h @@ -21,9 +21,6 @@ #include #include -#include -#include - namespace DB { @@ -40,20 +37,6 @@ using namespace DB; class MergeTreeRelParser : public RelParser { public: - static CustomStorageMergeTreePtr parseStorage( - const MergeTreeTable & merge_tree_table, ContextMutablePtr context, bool restore = false); - - // Create random table name and table path and use default storage policy. - // In insert case, mergetree data can be upload after merges in default storage(Local Disk). - static CustomStorageMergeTreePtr - copyToDefaultPolicyStorage(MergeTreeTable merge_tree_table, ContextMutablePtr context); - - // Use same table path and data path as the originial table. - static CustomStorageMergeTreePtr - copyToVirtualStorage(MergeTreeTable merge_tree_table, ContextMutablePtr context); - - static MergeTreeTable parseMergeTreeTable(const substrait::ReadRel::ExtensionTable & extension_table); - explicit MergeTreeRelParser(SerializedPlanParser * plan_paser_, const ContextPtr & context_) : RelParser(plan_paser_), context(context_), global_context(plan_paser_->global_context) { @@ -101,8 +84,8 @@ class MergeTreeRelParser : public RelParser void parseToAction(ActionsDAG & filter_action, const substrait::Expression & rel, std::string & filter_name); PrewhereInfoPtr parsePreWhereInfo(const substrait::Expression & rel, Block & input); ActionsDAG optimizePrewhereAction(const substrait::Expression & rel, std::string & filter_name, Block & block); - String getCHFunctionName(const substrait::Expression_ScalarFunction & substrait_func); - void collectColumns(const substrait::Expression & rel, NameSet & columns, Block & block); + String getCHFunctionName(const substrait::Expression_ScalarFunction & substrait_func) const; + static void collectColumns(const substrait::Expression & rel, NameSet & columns, Block & block); UInt64 getColumnsSize(const NameSet & columns); const ContextPtr & context; diff --git a/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp b/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp index 297551bcccc2..718656ac8a55 100644 --- a/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp +++ b/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp @@ -42,7 +42,6 @@ #include #include #include -#include #include #include #include @@ -60,7 +59,6 @@ #include #include #include -#include #include #include #include @@ -71,7 +69,6 @@ #include #include #include -#include #include #include #include @@ -84,7 +81,6 @@ #include #include #include -#include #include #include @@ -119,21 +115,6 @@ std::string join(const ActionsDAG::NodeRawConstPtrs & v, char c) return res; } -void logDebugMessage(const google::protobuf::Message & message, const char * type) -{ - auto * logger = &Poco::Logger::get("SerializedPlanParser"); - if (logger->debug()) - { - namespace pb_util = google::protobuf::util; - pb_util::JsonOptions options; - std::string json; - auto s = pb_util::MessageToJsonString(message, &json, options); - if (!s.ok()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Can not convert {} to Json", type); - LOG_DEBUG(logger, "{}:\n{}", type, json); - } -} - const ActionsDAG::Node * SerializedPlanParser::addColumn(ActionsDAG& actions_dag, const DataTypePtr & type, const Field & field) { return &actions_dag.addColumn( @@ -1325,7 +1306,7 @@ std::unique_ptr SerializedPlanParser::createExecutor(DB::QueryPla const substrait::PlanRel & root_rel = s_plan.relations().at(0); assert(root_rel.has_root()); if (root_rel.root().input().has_write()) - addSinkTransfrom(context, root_rel.root().input().write(), builder); + addSinkTransform(context, root_rel.root().input().write(), builder); /// QueryPipeline pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); diff --git a/cpp-ch/local-engine/Parser/SerializedPlanParser.h b/cpp-ch/local-engine/Parser/SerializedPlanParser.h index e44a7f657a20..85150c099ba9 100644 --- a/cpp-ch/local-engine/Parser/SerializedPlanParser.h +++ b/cpp-ch/local-engine/Parser/SerializedPlanParser.h @@ -26,7 +26,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/cpp-ch/local-engine/Parser/SubstraitParserUtils.cpp b/cpp-ch/local-engine/Parser/SubstraitParserUtils.cpp new file mode 100644 index 000000000000..e72454ba6fb2 --- /dev/null +++ b/cpp-ch/local-engine/Parser/SubstraitParserUtils.cpp @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +#include "SubstraitParserUtils.h" +#include +#include + +using namespace DB; + +namespace local_engine +{ +void logDebugMessage(const google::protobuf::Message & message, const char * type) +{ + auto * logger = &Poco::Logger::get("SubstraitPlan"); + if (logger->debug()) + { + namespace pb_util = google::protobuf::util; + pb_util::JsonOptions options; + std::string json; + auto s = pb_util::MessageToJsonString(message, &json, options); + if (!s.ok()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Can not convert {} to Json", type); + LOG_DEBUG(logger, "{}:\n{}", type, json); + } +} +std::string toString(const google::protobuf::Any & any) +{ + google::protobuf::StringValue sv; + sv.ParseFromString(any.value()); + return sv.value(); +} +} \ No newline at end of file diff --git a/cpp-ch/local-engine/Parser/SubstraitParserUtils.h b/cpp-ch/local-engine/Parser/SubstraitParserUtils.h index f247a3bddc09..d93b80cdacaf 100644 --- a/cpp-ch/local-engine/Parser/SubstraitParserUtils.h +++ b/cpp-ch/local-engine/Parser/SubstraitParserUtils.h @@ -23,6 +23,7 @@ namespace DB::ErrorCodes { extern const int CANNOT_PARSE_PROTOBUF_SCHEMA; +extern const int LOGICAL_ERROR; } namespace local_engine @@ -66,4 +67,7 @@ Message BinaryToMessage(const std::string_view binary) return message; } +void logDebugMessage(const google::protobuf::Message & message, const char * type); + +std::string toString(const google::protobuf::Any & any); } // namespace local_engine diff --git a/cpp-ch/local-engine/Parser/WriteRelParser.cpp b/cpp-ch/local-engine/Parser/WriteRelParser.cpp index 1a468a41eef2..ecae3f16d00a 100644 --- a/cpp-ch/local-engine/Parser/WriteRelParser.cpp +++ b/cpp-ch/local-engine/Parser/WriteRelParser.cpp @@ -27,10 +27,11 @@ #include #include #include +#include using namespace local_engine; -DB::ProcessorPtr makeSink( +DB::ProcessorPtr make_sink( const DB::ContextPtr & context, const DB::Names & partition_by, const DB::Block & input_header, @@ -53,7 +54,7 @@ DB::ProcessorPtr makeSink( return file_sink; } -bool need_fix_tuple(const DB::DataTypePtr& input, const DB::DataTypePtr& output) +bool need_fix_tuple(const DB::DataTypePtr & input, const DB::DataTypePtr & output) { const auto orgial = typeid_cast(input.get()); const auto output_type = typeid_cast(output.get()); @@ -75,74 +76,80 @@ DB::ExpressionActionsPtr create_project_action(const DB::Block & input, const DB { DB::ColumnsWithTypeAndName final_cols; std::ranges::transform( - output, std::back_inserter(final_cols), [](const DB::ColumnWithTypeAndName& out_ocl) { - const auto out_type = out_ocl.type; - return DB::ColumnWithTypeAndName(out_type->createColumn(), out_type, out_ocl.name); - }); + output, + std::back_inserter(final_cols), + [](const DB::ColumnWithTypeAndName & out_ocl) + { + const auto out_type = out_ocl.type; + return DB::ColumnWithTypeAndName(out_type->createColumn(), out_type, out_ocl.name); + }); assert(final_cols.size() == output.columns()); const auto & original_cols = input.getColumnsWithTypeAndName(); ActionsDAG final_project = ActionsDAG::makeConvertingActions(original_cols, final_cols, ActionsDAG::MatchColumnsMode::Position); - return std::make_shared(std::move(final_project)); + return std::make_shared(std::move(final_project)); } -void adjust_output(const DB::QueryPipelineBuilderPtr & builder, const DB::Block& output) +void adjust_output(const DB::QueryPipelineBuilderPtr & builder, const DB::Block & output) { const auto input = builder->getHeader(); if (input.columns() != output.columns()) { - throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, - "Missmatch result columns size, input size is {}, but output size is {}", input.columns(), output.columns()); + throw DB::Exception( + DB::ErrorCodes::LOGICAL_ERROR, + "Missmatch result columns size, input size is {}, but output size is {}", + input.columns(), + output.columns()); } - auto mismatch_pair = - std::mismatch(input.begin(), input.end(), output.begin(), - [](const DB::ColumnWithTypeAndName& lhs, const DB::ColumnWithTypeAndName& rhs) {return lhs.name == rhs.name;}); - bool name_is_diffient = mismatch_pair.first != input.end(); + auto mismatch_pair = std::mismatch( + input.begin(), + input.end(), + output.begin(), + [](const DB::ColumnWithTypeAndName & lhs, const DB::ColumnWithTypeAndName & rhs) { return lhs.name == rhs.name; }); + bool name_is_different = mismatch_pair.first != input.end(); - mismatch_pair = std::mismatch(input.begin(), input.end(), output.begin(), - [](const DB::ColumnWithTypeAndName& lhs, const DB::ColumnWithTypeAndName& rhs) - { - return lhs.type->equals(*rhs.type); - }); - bool type_is_diffient = mismatch_pair.first != input.end(); + mismatch_pair = std::mismatch( + input.begin(), + input.end(), + output.begin(), + [](const DB::ColumnWithTypeAndName & lhs, const DB::ColumnWithTypeAndName & rhs) { return lhs.type->equals(*rhs.type); }); + bool type_is_different = mismatch_pair.first != input.end(); DB::ExpressionActionsPtr convert_action; - if (type_is_diffient) + if (type_is_different) convert_action = create_project_action(input, output); - if(name_is_diffient && !convert_action) + if (name_is_different && !convert_action) convert_action = create_rename_action(input, output); if (!convert_action) return; builder->addSimpleTransform( - [&](const DB::Block & cur_header, const DB::QueryPipelineBuilder::StreamType stream_type) -> DB::ProcessorPtr - { - if (stream_type != DB::QueryPipelineBuilder::StreamType::Main) - return nullptr; - return std::make_shared(cur_header, convert_action); - }); + [&](const DB::Block & cur_header, const DB::QueryPipelineBuilder::StreamType stream_type) -> DB::ProcessorPtr + { + if (stream_type != DB::QueryPipelineBuilder::StreamType::Main) + return nullptr; + return std::make_shared(cur_header, convert_action); + }); } namespace local_engine { -void addSinkTransfrom(const DB::ContextPtr & context, const substrait::WriteRel & write_rel, const DB::QueryPipelineBuilderPtr & builder) +IMPLEMENT_GLUTEN_SETTINGS(GlutenWriteSettings, WRITE_RELATED_SETTINGS) + +void addSinkTransform(const DB::ContextPtr & context, const substrait::WriteRel & write_rel, const DB::QueryPipelineBuilderPtr & builder) { - const DB::Settings & settings = context->getSettingsRef(); + GlutenWriteSettings write_settings = GlutenWriteSettings::get(context); - DB::Field field_tmp_dir; - if (!settings.tryGet(SPARK_TASK_WRITE_TMEP_DIR, field_tmp_dir)) + if (write_settings.task_write_tmp_dir.empty()) throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Write Pipeline need inject temp directory."); - const auto & tmp_dir = field_tmp_dir.safeGet(); - DB::Field field_filename; - if (!settings.tryGet(SPARK_TASK_WRITE_FILENAME, field_filename)) + if (write_settings.task_write_filename.empty()) throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Write Pipeline need inject file name."); - const auto & filename = field_filename.safeGet(); assert(write_rel.has_named_table()); const substrait::NamedObjectWrite & named_table = write_rel.named_table(); @@ -168,7 +175,15 @@ void addSinkTransfrom(const DB::ContextPtr & context, const substrait::WriteRel { if (stream_type != QueryPipelineBuilder::StreamType::Main) return nullptr; - return makeSink(context, partitionCols, cur_header, blockHeader, tmp_dir, filename, config["format"], stats); + return make_sink( + context, + partitionCols, + cur_header, + blockHeader, + write_settings.task_write_tmp_dir, + write_settings.task_write_filename, + config["format"], + stats); }); builder->addSimpleTransform( [&](const Block &, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr diff --git a/cpp-ch/local-engine/Parser/WriteRelParser.h b/cpp-ch/local-engine/Parser/WriteRelParser.h index 9d896e7ca53a..b083d2f637d2 100644 --- a/cpp-ch/local-engine/Parser/WriteRelParser.h +++ b/cpp-ch/local-engine/Parser/WriteRelParser.h @@ -21,6 +21,7 @@ #include #include #include +#include namespace substrait { @@ -37,10 +38,16 @@ using QueryPipelineBuilderPtr = std::unique_ptr; namespace local_engine { -void addSinkTransfrom(const DB::ContextPtr & context, const substrait::WriteRel & write_rel, const DB::QueryPipelineBuilderPtr & builder); +void addSinkTransform(const DB::ContextPtr & context, const substrait::WriteRel & write_rel, const DB::QueryPipelineBuilderPtr & builder); /// Visible for UTs std::map parse_write_parameter(const std::string & input); DB::Names collect_partition_cols(const DB::Block & header, const substrait::NamedStruct & struct_); +#define WRITE_RELATED_SETTINGS(M, ALIAS, UNIQ) \ + M(String, task_write_tmp_dir, , "The temporary directory for writing data", UNIQ) \ + M(String, task_write_filename, , "The filename for writing data", UNIQ) + +DECLARE_GLUTEN_SETTINGS(GlutenWriteSettings, WRITE_RELATED_SETTINGS) + } diff --git a/cpp-ch/local-engine/Storages/Cache/CacheManager.cpp b/cpp-ch/local-engine/Storages/Cache/CacheManager.cpp index 0dc852a90110..e74a74ae0fa6 100644 --- a/cpp-ch/local-engine/Storages/Cache/CacheManager.cpp +++ b/cpp-ch/local-engine/Storages/Cache/CacheManager.cpp @@ -23,17 +23,15 @@ #include #include #include -#include #include #include #include -#include +#include +#include #include #include #include -#include - namespace DB { namespace ErrorCodes @@ -75,10 +73,10 @@ void CacheManager::initialize(DB::ContextMutablePtr context_) struct CacheJobContext { - MergeTreeTable table; + MergeTreeTableInstance table; }; -Task CacheManager::cachePart(const MergeTreeTable& table, const MergeTreePart& part, const std::unordered_set & columns) +Task CacheManager::cachePart(const MergeTreeTableInstance & table, const MergeTreePart & part, const std::unordered_set & columns) { CacheJobContext job_context{table}; job_context.table.parts.clear(); @@ -88,7 +86,8 @@ Task CacheManager::cachePart(const MergeTreeTable& table, const MergeTreePart& p { try { - auto storage = MergeTreeRelParser::parseStorage(job_detail.table, context, true); + auto storage = job_detail.table.restoreStorage(context); + auto storage_snapshot = std::make_shared(*storage, storage->getInMemoryMetadataPtr()); NamesAndTypesList names_and_types_list; auto meta_columns = storage->getInMemoryMetadata().getColumns(); @@ -132,9 +131,8 @@ Task CacheManager::cachePart(const MergeTreeTable& table, const MergeTreePart& p return std::move(task); } -JobId CacheManager::cacheParts(const String& table_def, const std::unordered_set& columns) +JobId CacheManager::cacheParts(const MergeTreeTableInstance & table, const std::unordered_set& columns) { - auto table = parseMergeTreeTableString(table_def); JobId id = toString(UUIDHelpers::generateV4()); Job job(id); for (const auto & part : table.parts) @@ -148,7 +146,7 @@ JobId CacheManager::cacheParts(const String& table_def, const std::unordered_set jobject CacheManager::getCacheStatus(JNIEnv * env, const String & jobId) { - auto& scheduler = JobScheduler::instance(); + auto & scheduler = JobScheduler::instance(); auto job_status = scheduler.getJobSatus(jobId); int status = 0; String message; diff --git a/cpp-ch/local-engine/Storages/Cache/CacheManager.h b/cpp-ch/local-engine/Storages/Cache/CacheManager.h index 6335f86bb162..2c1c010432dd 100644 --- a/cpp-ch/local-engine/Storages/Cache/CacheManager.h +++ b/cpp-ch/local-engine/Storages/Cache/CacheManager.h @@ -18,37 +18,37 @@ #include +#include #include #include -#include #include namespace local_engine { struct MergeTreePart; -struct MergeTreeTable; - - +struct MergeTreeTableInstance; /*** * Manage the cache of the MergeTree, mainly including meta.bin, data.bin, metadata.gluten */ -class CacheManager { +class CacheManager +{ public: static jclass cache_result_class; static jmethodID cache_result_constructor; - static void initJNI(JNIEnv* env); + static void initJNI(JNIEnv * env); static CacheManager & instance(); static void initialize(DB::ContextMutablePtr context); - Task cachePart(const MergeTreeTable& table, const MergeTreePart& part, const std::unordered_set& columns); - JobId cacheParts(const String& table_def, const std::unordered_set& columns); - static jobject getCacheStatus(JNIEnv * env, const String& jobId); + JobId cacheParts(const MergeTreeTableInstance & table, const std::unordered_set & columns); + static jobject getCacheStatus(JNIEnv * env, const String & jobId); Task cacheFile(const substrait::ReadRel::LocalFiles::FileOrFiles & file, ReadBufferBuilderPtr read_buffer_builder); JobId cacheFiles(substrait::ReadRel::LocalFiles file_infos); static void removeFiles(String file, String cache_name); + private: + Task cachePart(const MergeTreeTableInstance & table, const MergeTreePart & part, const std::unordered_set & columns); CacheManager() = default; DB::ContextMutablePtr context; }; diff --git a/cpp-ch/local-engine/Storages/Mergetree/MergeSparkMergeTreeTask.cpp b/cpp-ch/local-engine/Storages/MergeTree/MergeSparkMergeTreeTask.cpp similarity index 99% rename from cpp-ch/local-engine/Storages/Mergetree/MergeSparkMergeTreeTask.cpp rename to cpp-ch/local-engine/Storages/MergeTree/MergeSparkMergeTreeTask.cpp index 05b2623b4d16..cecb6308745c 100644 --- a/cpp-ch/local-engine/Storages/Mergetree/MergeSparkMergeTreeTask.cpp +++ b/cpp-ch/local-engine/Storages/MergeTree/MergeSparkMergeTreeTask.cpp @@ -15,13 +15,13 @@ * limitations under the License. */ #include "MergeSparkMergeTreeTask.h" -#include +#include +#include #include #include -#include -#include #include +#include #include using namespace DB; diff --git a/cpp-ch/local-engine/Storages/Mergetree/MergeSparkMergeTreeTask.h b/cpp-ch/local-engine/Storages/MergeTree/MergeSparkMergeTreeTask.h similarity index 96% rename from cpp-ch/local-engine/Storages/Mergetree/MergeSparkMergeTreeTask.h rename to cpp-ch/local-engine/Storages/MergeTree/MergeSparkMergeTreeTask.h index 4dd2d51527c1..ac167da3fb49 100644 --- a/cpp-ch/local-engine/Storages/Mergetree/MergeSparkMergeTreeTask.h +++ b/cpp-ch/local-engine/Storages/MergeTree/MergeSparkMergeTreeTask.h @@ -16,25 +16,24 @@ */ #pragma once +#include #include -#include -#include #include -#include +#include using namespace DB; namespace local_engine { -class CustomStorageMergeTree; +class SparkStorageMergeTree; class MergeSparkMergeTreeTask : public IExecutableTask { public: MergeSparkMergeTreeTask( - CustomStorageMergeTree & storage_, + SparkStorageMergeTree & storage_, StorageMetadataPtr metadata_snapshot_, bool deduplicate_, Names deduplicate_by_columns_, @@ -82,7 +81,7 @@ class MergeSparkMergeTreeTask : public IExecutableTask State state{State::NEED_PREPARE}; - CustomStorageMergeTree & storage; + SparkStorageMergeTree & storage; StorageMetadataPtr metadata_snapshot; bool deduplicate; diff --git a/cpp-ch/local-engine/Storages/Mergetree/MetaDataHelper.cpp b/cpp-ch/local-engine/Storages/MergeTree/MetaDataHelper.cpp similarity index 89% rename from cpp-ch/local-engine/Storages/Mergetree/MetaDataHelper.cpp rename to cpp-ch/local-engine/Storages/MergeTree/MetaDataHelper.cpp index affdce2b97c8..5cdeaf7a0e4b 100644 --- a/cpp-ch/local-engine/Storages/Mergetree/MetaDataHelper.cpp +++ b/cpp-ch/local-engine/Storages/MergeTree/MetaDataHelper.cpp @@ -19,8 +19,7 @@ #include #include #include -#include -#include +#include #include namespace CurrentMetrics @@ -54,16 +53,16 @@ std::unordered_map extractPartMetaData(ReadBuffer & in) return result; } -void restoreMetaData(CustomStorageMergeTreePtr & storage, const MergeTreeTable & mergeTreeTable, const Context & context) +void restoreMetaData(const SparkStorageMergeTreePtr & storage, const MergeTreeTableInstance & mergeTreeTable, const Context & context) { - auto data_disk = storage->getStoragePolicy()->getAnyDisk(); + const auto data_disk = storage->getStoragePolicy()->getAnyDisk(); if (!data_disk->isRemote()) return; std::unordered_set not_exists_part; - DB::MetadataStorageFromDisk * metadata_storage = static_cast(data_disk->getMetadataStorage().get()); - auto metadata_disk = metadata_storage->getDisk(); - auto table_path = std::filesystem::path(mergeTreeTable.relative_path); + const DB::MetadataStorageFromDisk * metadata_storage = static_cast(data_disk->getMetadataStorage().get()); + const auto metadata_disk = metadata_storage->getDisk(); + const auto table_path = std::filesystem::path(mergeTreeTable.relative_path); for (const auto & part : mergeTreeTable.getPartNames()) { auto part_path = table_path / part; @@ -71,7 +70,6 @@ void restoreMetaData(CustomStorageMergeTreePtr & storage, const MergeTreeTable & not_exists_part.emplace(part); } - if (auto lock = storage->lockForAlter(context.getSettingsRef().lock_acquire_timeout)) { // put this return clause in lockForAlter @@ -121,7 +119,6 @@ void restoreMetaData(CustomStorageMergeTreePtr & storage, const MergeTreeTable & } } - void saveFileStatus( const DB::MergeTreeData & storage, const DB::ContextPtr& context, @@ -154,7 +151,7 @@ std::vector mergeParts( std::vector selected_parts, std::unordered_map & partition_values, const String & new_part_uuid, - CustomStorageMergeTreePtr storage, + SparkStorageMergeTree & storage, const String & partition_dir, const String & bucket_dir) { @@ -181,7 +178,7 @@ std::vector mergeParts( // Copying a vector of columns `deduplicate by columns. DB::IExecutableTask::TaskResultCallback f = [](bool) {}; auto task = std::make_shared( - *storage, storage->getInMemoryMetadataPtr(), false, std::vector{}, false, entry, + storage, storage.getInMemoryMetadataPtr(), false, std::vector{}, false, entry, DB::TableLockHolder{}, f); task->setCurrentTransaction(DB::MergeTreeTransactionHolder{}, DB::MergeTreeTransactionPtr{}); @@ -189,7 +186,7 @@ std::vector mergeParts( executeHere(task); std::unordered_set to_load{future_part->name}; - std::vector merged = storage->loadDataPartsWithNames(to_load); + std::vector merged = storage.loadDataPartsWithNames(to_load); return merged; } diff --git a/cpp-ch/local-engine/Storages/Mergetree/MetaDataHelper.h b/cpp-ch/local-engine/Storages/MergeTree/MetaDataHelper.h similarity index 74% rename from cpp-ch/local-engine/Storages/Mergetree/MetaDataHelper.h rename to cpp-ch/local-engine/Storages/MergeTree/MetaDataHelper.h index 7163ee02c1e2..3297208179a0 100644 --- a/cpp-ch/local-engine/Storages/Mergetree/MetaDataHelper.h +++ b/cpp-ch/local-engine/Storages/MergeTree/MetaDataHelper.h @@ -17,25 +17,22 @@ #pragma once -#include -#include +#include +#include namespace local_engine { -void restoreMetaData(CustomStorageMergeTreePtr & storage, const MergeTreeTable & mergeTreeTable, const Context & context); +void restoreMetaData(const SparkStorageMergeTreePtr & storage, const MergeTreeTableInstance & mergeTreeTable, const Context & context); void saveFileStatus( - const DB::MergeTreeData & storage, - const DB::ContextPtr& context, - const String & part_name, - IDataPartStorage & data_part_storage); + const DB::MergeTreeData & storage, const DB::ContextPtr & context, const String & part_name, IDataPartStorage & data_part_storage); std::vector mergeParts( std::vector selected_parts, std::unordered_map & partition_values, const String & new_part_uuid, - CustomStorageMergeTreePtr storage, + SparkStorageMergeTree & storage, const String & partition_dir, const String & bucket_dir); diff --git a/cpp-ch/local-engine/Common/MergeTreeTool.cpp b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeMeta.cpp similarity index 64% rename from cpp-ch/local-engine/Common/MergeTreeTool.cpp rename to cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeMeta.cpp index 31994170f76d..788b1b39fc20 100644 --- a/cpp-ch/local-engine/Common/MergeTreeTool.cpp +++ b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeMeta.cpp @@ -14,27 +14,29 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -#include "MergeTreeTool.h" - -#include -#include +#include "SparkMergeTreeMeta.h" #include #include -#include -#include +#include +#include +#include +#include #include +#include +#include +#include +#include #include using namespace DB; - -namespace local_engine +using namespace local_engine; +namespace { - // set skip index for each column if specified void setSecondaryIndex( const DB::NamesAndTypesList & columns, - ContextPtr context, + const ContextPtr & context, const MergeTreeTable & table, std::shared_ptr metadata) { @@ -67,7 +69,7 @@ void setSecondaryIndex( ss << ", "; else first = false; - ss << "_minmax_" << column.name << " " << column.name << " TYPE minmax GRANULARITY 1"; + ss << "_minmax_" << column.name << " " << column.name << " TYPE minmax GRANULARITY 1"; } if (bf_index_cols.contains(column.name)) @@ -76,7 +78,7 @@ void setSecondaryIndex( ss << ", "; else first = false; - ss << "_bloomfilter_" << column.name << " " << column.name << " TYPE bloom_filter GRANULARITY 1"; + ss << "_bloomfilter_" << column.name << " " << column.name << " TYPE bloom_filter GRANULARITY 1"; } if (set_index_cols.contains(column.name)) @@ -91,17 +93,21 @@ void setSecondaryIndex( metadata->setSecondaryIndices(IndicesDescription::parse(ss.str(), metadata->getColumns(), context)); } -std::shared_ptr buildMetaData( - const DB::NamesAndTypesList & columns, - ContextPtr context, - const MergeTreeTable & table) +void parseTableConfig(MergeTreeTableSettings & settings, const String & config_json) +{ + rapidjson::Document doc; + doc.Parse(config_json.c_str()); + if (doc.HasMember("storage_policy")) + settings.storage_policy = doc["storage_policy"].GetString(); +} + +std::shared_ptr +doBuildMetadata(const DB::NamesAndTypesList & columns, const ContextPtr & context, const MergeTreeTable & table) { std::shared_ptr metadata = std::make_shared(); ColumnsDescription columns_description; for (const auto & item : columns) - { columns_description.add(ColumnDescription(item.name, item.type)); - } metadata->setColumns(std::move(columns_description)); setSecondaryIndex(columns, context, table, metadata); @@ -109,53 +115,18 @@ std::shared_ptr buildMetaData( metadata->partition_key.expression_list_ast = std::make_shared(); metadata->sorting_key = KeyDescription::parse(table.order_by_key, metadata->getColumns(), context); if (table.primary_key.empty()) - { - if (table.order_by_key != MergeTreeTable::TUPLE) - metadata->primary_key = KeyDescription::parse(table.order_by_key, metadata->getColumns(), context); - else + if (table.order_by_key != MergeTreeTable::TUPLE) + metadata->primary_key = KeyDescription::parse(table.order_by_key, metadata->getColumns(), context); + else metadata->primary_key.expression = std::make_shared(ActionsDAG{}); - } else - { metadata->primary_key = KeyDescription::parse(table.primary_key, metadata->getColumns(), context); - } return metadata; } -std::unique_ptr buildMergeTreeSettings(const MergeTreeTableSettings & config) +void doParseMergeTreeTableString(MergeTreeTable & table, ReadBufferFromString & in) { - auto settings = std::make_unique(); - settings->set("allow_nullable_key", Field(1)); - if (!config.storage_policy.empty()) - settings->set("storage_policy", Field(config.storage_policy)); - return settings; -} - -std::unique_ptr buildQueryInfo(NamesAndTypesList & names_and_types_list) -{ - std::unique_ptr query_info = std::make_unique(); - query_info->query = std::make_shared(); - auto syntax_analyzer_result = std::make_shared(names_and_types_list); - syntax_analyzer_result->analyzed_join = std::make_shared(); - query_info->syntax_analyzer_result = syntax_analyzer_result; - return query_info; -} - - -void parseTableConfig(MergeTreeTableSettings & settings, String config_json) -{ - rapidjson::Document doc; - doc.Parse(config_json.c_str()); - if (doc.HasMember("storage_policy")) - settings.storage_policy = doc["storage_policy"].GetString(); - -} - -MergeTreeTable parseMergeTreeTableString(const std::string & info) -{ - ReadBufferFromString in(info); assertString("MergeTree;", in); - MergeTreeTable table; readString(table.database, in); assertChar('\n', in); readString(table.table, in); @@ -189,6 +160,55 @@ MergeTreeTable parseMergeTreeTableString(const std::string & info) readString(json, in); parseTableConfig(table.table_configs, json); assertChar('\n', in); +} + +} +namespace local_engine +{ + +SparkStorageMergeTreePtr MergeTreeTable::getStorage(ContextMutablePtr context) const +{ + const DB::Block header = TypeParser::buildBlockFromNamedStruct(schema, low_card_key); + const auto metadata = buildMetaData(header, context); + + return StorageMergeTreeFactory::getStorage( + StorageID(database, table), + snapshot_id, + *this, + [&]() -> SparkStorageMergeTreePtr + { + auto custom_storage_merge_tree = std::make_shared(*this, *metadata, context); + return custom_storage_merge_tree; + }); +} + +SparkStorageMergeTreePtr MergeTreeTable::copyToDefaultPolicyStorage(const ContextMutablePtr & context) const +{ + MergeTreeTable merge_tree_table{*this}; + auto temp_uuid = UUIDHelpers::generateV4(); + String temp_uuid_str = toString(temp_uuid); + merge_tree_table.table = merge_tree_table.table + "_" + temp_uuid_str; + merge_tree_table.snapshot_id = ""; + merge_tree_table.table_configs.storage_policy = ""; + merge_tree_table.relative_path = merge_tree_table.relative_path + "_" + temp_uuid_str; + return merge_tree_table.getStorage(context); +} + +SparkStorageMergeTreePtr MergeTreeTable::copyToVirtualStorage(const ContextMutablePtr & context) const +{ + MergeTreeTable merge_tree_table{*this}; + auto temp_uuid = UUIDHelpers::generateV4(); + String temp_uuid_str = toString(temp_uuid); + merge_tree_table.table = merge_tree_table.table + "_" + temp_uuid_str; + merge_tree_table.snapshot_id = ""; + return merge_tree_table.getStorage(context); +} + +MergeTreeTableInstance::MergeTreeTableInstance(const std::string & info) +{ + ReadBufferFromString in(info); + doParseMergeTreeTableString(*this, in); + while (!in.eof()) { MergeTreePart part; @@ -198,12 +218,52 @@ MergeTreeTable parseMergeTreeTableString(const std::string & info) assertChar('\n', in); readIntText(part.end, in); assertChar('\n', in); - table.parts.emplace_back(part); + parts.emplace_back(part); } - return table; } -std::unordered_set MergeTreeTable::getPartNames() const +MergeTreeTableInstance::MergeTreeTableInstance(const google::protobuf::Any & any) : MergeTreeTableInstance(toString(any)) +{ +} + +MergeTreeTableInstance::MergeTreeTableInstance(const substrait::ReadRel::ExtensionTable & extension_table) + : MergeTreeTableInstance(extension_table.detail()) +{ + logDebugMessage(extension_table, "merge_tree_table"); +} + +SparkStorageMergeTreePtr MergeTreeTableInstance::restoreStorage(const ContextMutablePtr & context) const +{ + auto result = getStorage(context); + restoreMetaData(result, *this, *context); + return result; +} + +std::shared_ptr MergeTreeTable::buildMetaData(const DB::Block & header, const ContextPtr & context) const +{ + return doBuildMetadata(header.getNamesAndTypesList(), context, *this); +} + +std::unique_ptr buildMergeTreeSettings(const MergeTreeTableSettings & config) +{ + auto settings = std::make_unique(); + settings->set("allow_nullable_key", Field(1)); + if (!config.storage_policy.empty()) + settings->set("storage_policy", Field(config.storage_policy)); + return settings; +} + +std::unique_ptr buildQueryInfo(NamesAndTypesList & names_and_types_list) +{ + std::unique_ptr query_info = std::make_unique(); + query_info->query = std::make_shared(); + auto syntax_analyzer_result = std::make_shared(names_and_types_list); + syntax_analyzer_result->analyzed_join = std::make_shared(); + query_info->syntax_analyzer_result = syntax_analyzer_result; + return query_info; +} + +std::unordered_set MergeTreeTableInstance::getPartNames() const { std::unordered_set names; for (const auto & part : parts) @@ -211,7 +271,7 @@ std::unordered_set MergeTreeTable::getPartNames() const return names; } -RangesInDataParts MergeTreeTable::extractRange(DataPartsVector parts_vector) const +RangesInDataParts MergeTreeTableInstance::extractRange(DataPartsVector parts_vector) const { std::unordered_map name_index; std::ranges::for_each(parts_vector, [&](const DataPartPtr & part) { name_index.emplace(part->name, part); }); @@ -240,27 +300,17 @@ bool sameColumns(const substrait::NamedStruct & left, const substrait::NamedStru for (size_t i = 0; i < left.names_size(); i++) map.emplace(left.names(i), left.struct_().types(i).kind_case()); for (size_t i = 0; i < right.names_size(); i++) - { if (!map.contains(right.names(i)) || map[right.names(i)] != right.struct_().types(i).kind_case()) return false; - } return true; } -bool MergeTreeTable::sameStructWith(const MergeTreeTable & other) +bool MergeTreeTable::sameTable(const MergeTreeTable & other) const { - return database == other.database && - table == other.table && - snapshot_id == other.snapshot_id && - sameColumns(schema, other.schema) && - order_by_key == other.order_by_key && - low_card_key == other.low_card_key && - minmax_index_key == other.minmax_index_key && - bf_index_key == other.bf_index_key && - set_index_key == other.set_index_key && - primary_key == other.primary_key && - relative_path == other.relative_path && - absolute_path == other.absolute_path && - table_configs.storage_policy == other.table_configs.storage_policy; + return database == other.database && table == other.table && snapshot_id == other.snapshot_id && sameColumns(schema, other.schema) + && order_by_key == other.order_by_key && low_card_key == other.low_card_key && minmax_index_key == other.minmax_index_key + && bf_index_key == other.bf_index_key && set_index_key == other.set_index_key && primary_key == other.primary_key + && relative_path == other.relative_path && absolute_path == other.absolute_path + && table_configs.storage_policy == other.table_configs.storage_policy; } } \ No newline at end of file diff --git a/cpp-ch/local-engine/Common/MergeTreeTool.h b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeMeta.h similarity index 58% rename from cpp-ch/local-engine/Common/MergeTreeTool.h rename to cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeMeta.h index fc312eba9e3c..2a20430205a6 100644 --- a/cpp-ch/local-engine/Common/MergeTreeTool.h +++ b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeMeta.h @@ -15,42 +15,42 @@ * limitations under the License. */ #pragma once -#include -#include -#include -#include + #include #include -#include -#include -#include + +#include #include #include #include #include -#include #include +namespace DB +{ +class ReadBufferFromString; +} namespace local_engine { +class SparkStorageMergeTree; +using SparkStorageMergeTreePtr = std::shared_ptr; using namespace DB; - struct MergeTreePart { - String name; + std::string name; size_t begin; size_t end; }; struct MergeTreeTableSettings { - String storage_policy = ""; + std::string storage_policy{}; }; struct MergeTreeTable { - inline static const String TUPLE = "tuple()"; + static constexpr std::string_view TUPLE = "tuple()"; std::string database; std::string table; std::string snapshot_id; @@ -60,22 +60,39 @@ struct MergeTreeTable std::string minmax_index_key; std::string bf_index_key; std::string set_index_key; - std::string primary_key = ""; + std::string primary_key{}; std::string relative_path; std::string absolute_path; MergeTreeTableSettings table_configs; + + bool sameTable(const MergeTreeTable & other) const; + + SparkStorageMergeTreePtr getStorage(ContextMutablePtr context) const; + + /// Create random table name and table path and use default storage policy. + /// In insert case, mergetree data can be uploaded after merges in default storage(Local Disk). + SparkStorageMergeTreePtr copyToDefaultPolicyStorage(const ContextMutablePtr & context) const; + + /// Use same table path and data path as the original table. + SparkStorageMergeTreePtr copyToVirtualStorage(const ContextMutablePtr & context) const; + + std::shared_ptr buildMetaData(const DB::Block & header, const ContextPtr & context) const; +}; + +struct MergeTreeTableInstance : MergeTreeTable +{ std::vector parts; - std::unordered_set getPartNames() const; + std::unordered_set getPartNames() const; RangesInDataParts extractRange(DataPartsVector parts_vector) const; - bool sameStructWith(const MergeTreeTable& other); -}; -std::shared_ptr buildMetaData(const DB::NamesAndTypesList &columns, ContextPtr context, const MergeTreeTable &); + SparkStorageMergeTreePtr restoreStorage(const ContextMutablePtr & context) const; + + explicit MergeTreeTableInstance(const google::protobuf::Any & any); + explicit MergeTreeTableInstance(const substrait::ReadRel::ExtensionTable & extension_table); + explicit MergeTreeTableInstance(const std::string & info); +}; std::unique_ptr buildMergeTreeSettings(const MergeTreeTableSettings & config); std::unique_ptr buildQueryInfo(NamesAndTypesList & names_and_types_list); - -MergeTreeTable parseMergeTreeTableString(const std::string & info); - } diff --git a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeSink.cpp b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeSink.cpp new file mode 100644 index 000000000000..32662a5abb67 --- /dev/null +++ b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeSink.cpp @@ -0,0 +1,307 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#include "SparkMergeTreeSink.h" + +#include +#include +#include +#include + +namespace CurrentMetrics +{ +extern const Metric LocalThread; +extern const Metric LocalThreadActive; +extern const Metric LocalThreadScheduled; +extern const Metric GlobalThread; +extern const Metric GlobalThreadActive; +extern const Metric GlobalThreadScheduled; +} + +namespace local_engine +{ + +void SparkMergeTreeSink::consume(Chunk & chunk) +{ + assert(!sink_helper->metadata_snapshot->hasPartitionKey()); + + auto block = getHeader().cloneWithColumns(chunk.getColumns()); + auto blocks_with_partition = MergeTreeDataWriter::splitBlockIntoParts(std::move(block), 10, sink_helper->metadata_snapshot, context); + + for (auto & item : blocks_with_partition) + { + size_t before_write_memory = 0; + if (auto * memory_tracker = CurrentThread::getMemoryTracker()) + { + CurrentThread::flushUntrackedMemory(); + before_write_memory = memory_tracker->get(); + } + sink_helper->writeTempPart(item, context, part_num); + part_num++; + /// Reset earlier to free memory + item.block.clear(); + item.partition.clear(); + } + sink_helper->checkAndMerge(); +} + +void SparkMergeTreeSink::onStart() +{ + // DO NOTHING +} + +void SparkMergeTreeSink::onFinish() +{ + sink_helper->finish(context); +} + +///// +SinkHelperPtr SparkMergeTreeSink::create( + const MergeTreeTable & merge_tree_table, const SparkMergeTreeWriteSettings & write_settings_, const DB::ContextMutablePtr & context) +{ + auto dest_storage = merge_tree_table.getStorage(context); + bool isRemoteStorage = dest_storage->getStoragePolicy()->getAnyDisk()->isRemote(); + bool insert_with_local_storage = !write_settings_.insert_without_local_storage; + if (insert_with_local_storage && isRemoteStorage) + { + auto temp = merge_tree_table.copyToDefaultPolicyStorage(context); + LOG_DEBUG( + &Poco::Logger::get("SparkMergeTreeWriter"), + "Create temp table {} for local merge.", + temp->getStorageID().getFullNameNotQuoted()); + return std::make_shared(temp, dest_storage, write_settings_); + } + + return std::make_shared(dest_storage, write_settings_, isRemoteStorage); +} + +SinkHelper::SinkHelper(const SparkStorageMergeTreePtr & data_, const SparkMergeTreeWriteSettings & write_settings_, bool isRemoteStorage_) + : data(data_) + , isRemoteStorage(isRemoteStorage_) + , thread_pool(CurrentMetrics::LocalThread, CurrentMetrics::LocalThreadActive, CurrentMetrics::LocalThreadScheduled, 1, 1, 100000) + , write_settings(write_settings_) + , metadata_snapshot(data->getInMemoryMetadataPtr()) +{ +} + +void SinkHelper::saveMetadata(const DB::ContextPtr & context) +{ + if (!isRemoteStorage) + return; + + const std::deque & parts = new_parts.unsafeGet(); + for (const auto & merge_tree_data_part : parts) + { + auto part = dest_storage().loadDataPartsWithNames({merge_tree_data_part->name}); + if (part.empty()) + { + LOG_WARNING( + &Poco::Logger::get("SparkMergeTreeWriter"), + "Save metadata failed because dest storage load part name {} empty.", + merge_tree_data_part->name); + continue; + } + + saveFileStatus( + dest_storage(), context, merge_tree_data_part->name, const_cast(part.at(0)->getDataPartStorage())); + } +} + +void SinkHelper::doMergePartsAsync(const std::vector & prepare_merge_parts) +{ + for (const auto & selected_part : prepare_merge_parts) + tmp_parts.emplace(selected_part->name); + + // check thread group initialized in task thread + currentThreadGroupMemoryUsage(); + thread_pool.scheduleOrThrow( + [this, prepare_merge_parts, thread_group = CurrentThread::getGroup()]() -> void + { + Stopwatch watch; + CurrentThread::detachFromGroupIfNotDetached(); + CurrentThread::attachToGroup(thread_group); + size_t before_size = 0; + size_t after_size = 0; + for (const auto & prepare_merge_part : prepare_merge_parts) + before_size += prepare_merge_part->getBytesOnDisk(); + + std::unordered_map partition_values; + const auto merged_parts = mergeParts( + prepare_merge_parts, + partition_values, + toString(UUIDHelpers::generateV4()), + dataRef(), + write_settings.partition_settings.partition_dir, + write_settings.partition_settings.bucket_dir); + for (const auto & merge_tree_data_part : merged_parts) + after_size += merge_tree_data_part->getBytesOnDisk(); + + new_parts.emplace_back(merged_parts); + watch.stop(); + LOG_INFO( + &Poco::Logger::get("SparkMergeTreeWriter"), + "Merge success. Before merge part size {}, part count {}, after part size {}, part count {}, " + "total elapsed {} ms", + before_size, + prepare_merge_parts.size(), + after_size, + merged_parts.size(), + watch.elapsedMilliseconds()); + }); +} +void SinkHelper::writeTempPart(DB::BlockWithPartition & block_with_partition, const ContextPtr & context, int part_num) +{ + auto tmp = dataRef().getWriter().writeTempPart( + block_with_partition, metadata_snapshot, context, write_settings.partition_settings, part_num); + new_parts.emplace_back(tmp.part); +} + +void SinkHelper::checkAndMerge(bool force) +{ + if (!write_settings.merge_after_insert) + return; + // Only finalize should force merge. + if (!force && new_parts.size() < write_settings.merge_limit_parts) + return; + + std::vector selected_parts; + selected_parts.reserve(write_settings.merge_limit_parts); + size_t total_size = 0; + std::vector skip_parts; + + while (const auto merge_tree_data_part_option = new_parts.pop_front()) + { + auto merge_tree_data_part = merge_tree_data_part_option.value(); + if (merge_tree_data_part->getBytesOnDisk() >= write_settings.merge_min_size) + { + skip_parts.emplace_back(merge_tree_data_part); + continue; + } + + selected_parts.emplace_back(merge_tree_data_part); + total_size += merge_tree_data_part->getBytesOnDisk(); + if (write_settings.merge_min_size > total_size && write_settings.merge_limit_parts > selected_parts.size()) + continue; + + doMergePartsAsync(selected_parts); + selected_parts.clear(); + total_size = 0; + } + + if (!selected_parts.empty()) + { + if (force && selected_parts.size() > 1) + doMergePartsAsync(selected_parts); + else + new_parts.emplace_back(selected_parts); + } + + new_parts.emplace_back(skip_parts); +} +void SinkHelper::finish(const DB::ContextPtr & context) +{ + if (write_settings.merge_after_insert) + finalizeMerge(); + commit(context->getReadSettings(), context->getWriteSettings()); + saveMetadata(context); +} + +void SinkHelper::finalizeMerge() +{ + LOG_DEBUG(&Poco::Logger::get("SparkMergeTreeWriter"), "Waiting all merge task end and do final merge"); + // waiting all merge task end and do final merge + thread_pool.wait(); + + size_t before_merge_size; + do + { + before_merge_size = new_parts.size(); + checkAndMerge(true); + thread_pool.wait(); + } while (before_merge_size != new_parts.size()); + cleanup(); +} + +void CopyToRemoteSinkHelper::commit(const ReadSettings & read_settings, const WriteSettings & write_settings) +{ + LOG_DEBUG( + &Poco::Logger::get("SparkMergeTreeWriter"), "Begin upload to disk {}.", dest_storage().getStoragePolicy()->getAnyDisk()->getName()); + + const std::deque & parts = new_parts.unsafeGet(); + + Stopwatch watch; + for (const auto & merge_tree_data_part : parts) + { + String local_relative_path = dataRef().getRelativeDataPath() + "/" + merge_tree_data_part->name; + String remote_relative_path = dest_storage().getRelativeDataPath() + "/" + merge_tree_data_part->name; + + std::vector files; + dataRef().getStoragePolicy()->getAnyDisk()->listFiles(local_relative_path, files); + auto src_disk = dataRef().getStoragePolicy()->getAnyDisk(); + auto dest_disk = dest_storage().getStoragePolicy()->getAnyDisk(); + auto tx = dest_disk->createTransaction(); + for (const auto & file : files) + { + auto read_buffer = src_disk->readFile(local_relative_path + "/" + file, read_settings); + auto write_buffer + = tx->writeFile(remote_relative_path + "/" + file, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite, write_settings); + copyData(*read_buffer, *write_buffer); + write_buffer->finalize(); + } + tx->commit(); + LOG_DEBUG( + &Poco::Logger::get("SparkMergeTreeWriter"), + "Upload part {} to disk {} success.", + merge_tree_data_part->name, + dest_storage().getStoragePolicy()->getAnyDisk()->getName()); + } + watch.stop(); + LOG_INFO( + &Poco::Logger::get("SparkMergeTreeWriter"), + "Upload to disk {} finished, total elapsed {} ms", + dest_storage().getStoragePolicy()->getAnyDisk()->getName(), + watch.elapsedMilliseconds()); + StorageMergeTreeFactory::freeStorage(temp_storage()->getStorageID()); + temp_storage()->dropAllData(); + LOG_DEBUG( + &Poco::Logger::get("SparkMergeTreeWriter"), "Clean temp table {} success.", temp_storage()->getStorageID().getFullNameNotQuoted()); +} + +void DirectSinkHelper::cleanup() +{ + // default storage need clean temp. + std::unordered_set final_parts; + for (const auto & merge_tree_data_part : new_parts.unsafeGet()) + final_parts.emplace(merge_tree_data_part->name); + + for (const auto & tmp_part : tmp_parts) + { + if (final_parts.contains(tmp_part)) + continue; + + GlobalThreadPool::instance().scheduleOrThrow( + [storage_ = data, tmp = tmp_part]() -> void + { + for (const auto & disk : storage_->getDisks()) + { + auto rel_path = storage_->getRelativeDataPath() + "/" + tmp; + disk->removeRecursive(rel_path); + } + }); + } +} + +} \ No newline at end of file diff --git a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeSink.h b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeSink.h new file mode 100644 index 000000000000..111bf9a661e8 --- /dev/null +++ b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeSink.h @@ -0,0 +1,183 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#pragma once + +#include +#include +#include +#include +#include +#include + +namespace local_engine +{ + +struct MergeTreeTable; +using SparkStorageMergeTreePtr = std::shared_ptr; +class SinkHelper; +using SinkHelperPtr = std::shared_ptr; + +// TODO: Remove ConcurrentDeque +template +class ConcurrentDeque +{ +public: + std::optional pop_front() + { + std::lock_guard lock(mtx); + + if (deq.empty()) + return {}; + + T t = deq.front(); + deq.pop_front(); + return t; + } + + void emplace_back(T value) + { + std::lock_guard lock(mtx); + deq.emplace_back(value); + } + + void emplace_back(std::vector values) + { + std::lock_guard lock(mtx); + deq.insert(deq.end(), values.begin(), values.end()); + } + + void emplace_front(T value) + { + std::lock_guard lock(mtx); + deq.emplace_front(value); + } + + size_t size() + { + std::lock_guard lock(mtx); + return deq.size(); + } + + bool empty() + { + std::lock_guard lock(mtx); + return deq.empty(); + } + + /// !!! unsafe get, only called when background tasks are finished + const std::deque & unsafeGet() const { return deq; } + +private: + std::deque deq; + mutable std::mutex mtx; +}; + +class SinkHelper +{ +protected: + SparkStorageMergeTreePtr data; + bool isRemoteStorage; + + ConcurrentDeque new_parts; + std::unordered_set tmp_parts{}; + ThreadPool thread_pool; + +public: + const SparkMergeTreeWriteSettings write_settings; + const DB::StorageMetadataPtr metadata_snapshot; + +protected: + virtual SparkStorageMergeTree & dest_storage() { return *data; } + + void doMergePartsAsync(const std::vector & prepare_merge_parts); + void finalizeMerge(); + virtual void cleanup() { } + virtual void commit(const ReadSettings & read_settings, const WriteSettings & write_settings) { } + void saveMetadata(const DB::ContextPtr & context); + SparkWriteStorageMergeTree & dataRef() const { return assert_cast(*data); } + +public: + const std::deque & unsafeGet() const { return new_parts.unsafeGet(); } + + void writeTempPart(DB::BlockWithPartition & block_with_partition, const ContextPtr & context, int part_num); + void checkAndMerge(bool force = false); + void finish(const DB::ContextPtr & context); + + virtual ~SinkHelper() = default; + SinkHelper(const SparkStorageMergeTreePtr & data_, const SparkMergeTreeWriteSettings & write_settings_, bool isRemoteStorage_); +}; + +class DirectSinkHelper : public SinkHelper +{ +protected: + void cleanup() override; + +public: + explicit DirectSinkHelper( + const SparkStorageMergeTreePtr & data_, const SparkMergeTreeWriteSettings & write_settings_, bool isRemoteStorage_) + : SinkHelper(data_, write_settings_, isRemoteStorage_) + { + } +}; + +class CopyToRemoteSinkHelper : public SinkHelper +{ + SparkStorageMergeTreePtr dest; + +protected: + void commit(const ReadSettings & read_settings, const WriteSettings & write_settings) override; + SparkStorageMergeTree & dest_storage() override { return *dest; } + const SparkStorageMergeTreePtr & temp_storage() const { return data; } + +public: + explicit CopyToRemoteSinkHelper( + const SparkStorageMergeTreePtr & temp, const SparkStorageMergeTreePtr & dest_, const SparkMergeTreeWriteSettings & write_settings_) + : SinkHelper(temp, write_settings_, true), dest(dest_) + { + assert(data != dest); + } +}; + +class SparkMergeTreeSink : public DB::SinkToStorage +{ +public: + static SinkHelperPtr create( + const MergeTreeTable & merge_tree_table, + const SparkMergeTreeWriteSettings & write_settings_, + const DB::ContextMutablePtr & context); + + explicit SparkMergeTreeSink(const SinkHelperPtr & sink_helper_, const ContextPtr & context_) + : SinkToStorage(sink_helper_->metadata_snapshot->getSampleBlock()), context(context_), sink_helper(sink_helper_) + { + } + ~SparkMergeTreeSink() override = default; + + String getName() const override { return "SparkMergeTreeSink"; } + void consume(Chunk & chunk) override; + void onStart() override; + void onFinish() override; + + const SinkHelper & sinkHelper() const { return *sink_helper; } + +private: + ContextPtr context; + SinkHelperPtr sink_helper; + + int part_num = 1; +}; + +} diff --git a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriteSettings.cpp b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriteSettings.cpp new file mode 100644 index 000000000000..84cb002e72d9 --- /dev/null +++ b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriteSettings.cpp @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include "SparkMergeTreeWriteSettings.h" + +#include +#include + +namespace local_engine +{ + +IMPLEMENT_GLUTEN_SETTINGS(SparkMergeTreeWritePartitionSettings, MERGE_TREE_WRITE_RELATED_SETTINGS) + +void SparkMergeTreeWriteSettings::load(const DB::ContextPtr & context) +{ + const DB::Settings & settings = context->getSettingsRef(); + merge_after_insert = settings.get(MERGETREE_MERGE_AFTER_INSERT).safeGet(); + insert_without_local_storage = settings.get(MERGETREE_INSERT_WITHOUT_LOCAL_STORAGE).safeGet(); + + if (DB::Field limit_size_field; settings.tryGet("optimize.minFileSize", limit_size_field)) + merge_min_size = limit_size_field.safeGet() <= 0 ? merge_min_size : limit_size_field.safeGet(); + + if (DB::Field limit_cnt_field; settings.tryGet("mergetree.max_num_part_per_merge_task", limit_cnt_field)) + merge_limit_parts = limit_cnt_field.safeGet() <= 0 ? merge_limit_parts : limit_cnt_field.safeGet(); +} + +} diff --git a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriteSettings.h b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriteSettings.h new file mode 100644 index 000000000000..3554a092a470 --- /dev/null +++ b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriteSettings.h @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#pragma once + +#include + +namespace local_engine +{ +#define MERGE_TREE_WRITE_RELATED_SETTINGS(M, ALIAS, UNIQ) \ + M(String, part_name_prefix, , "The part name prefix for writing data", UNIQ) \ + M(String, partition_dir, , "The parition directory for writing data", UNIQ) \ + M(String, bucket_dir, , "The bucket directory for writing data", UNIQ) + +DECLARE_GLUTEN_SETTINGS(SparkMergeTreeWritePartitionSettings, MERGE_TREE_WRITE_RELATED_SETTINGS) + +struct SparkMergeTreeWriteSettings +{ + SparkMergeTreeWritePartitionSettings partition_settings; + bool merge_after_insert{true}; + bool insert_without_local_storage{false}; + size_t merge_min_size = 1024 * 1024 * 1024; + size_t merge_limit_parts = 10; + + void load(const DB::ContextPtr & context); +}; +} \ No newline at end of file diff --git a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriter.cpp b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriter.cpp new file mode 100644 index 000000000000..1ede4960aafe --- /dev/null +++ b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriter.cpp @@ -0,0 +1,151 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#include "SparkMergeTreeWriter.h" + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +using namespace DB; +namespace +{ +Block removeColumnSuffix(const Block & block) +{ + ColumnsWithTypeAndName columns; + for (int i = 0; i < block.columns(); ++i) + { + auto name = block.getByPosition(i).name; + Poco::StringTokenizer splits(name, "#"); + auto column = block.getByPosition(i); + column.name = splits[0]; + columns.emplace_back(column); + } + return Block(columns); +} +} + +namespace local_engine +{ + +std::unique_ptr SparkMergeTreeWriter::create( + const MergeTreeTable & merge_tree_table, + const SparkMergeTreeWritePartitionSettings & write_settings_, + const DB::ContextMutablePtr & context) +{ + const DB::Settings & settings = context->getSettingsRef(); + const auto dest_storage = merge_tree_table.getStorage(context); + StorageMetadataPtr metadata_snapshot = dest_storage->getInMemoryMetadataPtr(); + Block header = metadata_snapshot->getSampleBlock(); + ASTPtr none; + Chain chain; + auto sink = dest_storage->write(none, metadata_snapshot, context, false); + chain.addSink(sink); + chain.addSource( + std::make_shared(header, settings.min_insert_block_size_rows, settings.min_insert_block_size_bytes)); + chain.addSource( + std::make_shared(header, settings.min_insert_block_size_rows, settings.min_insert_block_size_bytes)); + + std::unordered_map partition_values; + if (!write_settings_.partition_dir.empty()) + extractPartitionValues(write_settings_.partition_dir, partition_values); + return std::make_unique( + header, assert_cast(*sink).sinkHelper(), QueryPipeline{std::move(chain)}, std::move(partition_values)); +} + +SparkMergeTreeWriter::SparkMergeTreeWriter( + const DB::Block & header_, + const SinkHelper & sink_helper_, + DB::QueryPipeline && pipeline_, + std::unordered_map && partition_values_) + : header{header_}, sink_helper{sink_helper_}, pipeline{std::move(pipeline_)}, executor{pipeline}, partition_values{partition_values_} +{ +} + +void SparkMergeTreeWriter::write(const DB::Block & block) +{ + auto new_block = removeColumnSuffix(block); + auto converter = ActionsDAG::makeConvertingActions( + new_block.getColumnsWithTypeAndName(), header.getColumnsWithTypeAndName(), DB::ActionsDAG::MatchColumnsMode::Position); + const ExpressionActions expression_actions{std::move(converter)}; + expression_actions.execute(new_block); + executor.push(new_block); +} + +void SparkMergeTreeWriter::finalize() +{ + executor.finish(); +} + +std::vector SparkMergeTreeWriter::getAllPartInfo() const +{ + std::vector res; + auto parts = sink_helper.unsafeGet(); + res.reserve(parts.size()); + + for (const auto & part : parts) + { + res.emplace_back(PartInfo{ + part->name, + part->getMarksCount(), + part->getBytesOnDisk(), + part->rows_count, + partition_values, + sink_helper.write_settings.partition_settings.bucket_dir}); + } + return res; +} + +String SparkMergeTreeWriter::partInfosToJson(const std::vector & part_infos) +{ + rapidjson::StringBuffer result; + rapidjson::Writer writer(result); + writer.StartArray(); + for (const auto & item : part_infos) + { + writer.StartObject(); + writer.Key("part_name"); + writer.String(item.part_name.c_str()); + writer.Key("mark_count"); + writer.Uint(item.mark_count); + writer.Key("disk_size"); + writer.Uint(item.disk_size); + writer.Key("row_count"); + writer.Uint(item.row_count); + writer.Key("bucket_id"); + writer.String(item.bucket_id.c_str()); + writer.Key("partition_values"); + writer.StartObject(); + for (const auto & key_value : item.partition_values) + { + writer.Key(key_value.first.c_str()); + writer.String(key_value.second.c_str()); + } + writer.EndObject(); + writer.EndObject(); + } + writer.EndArray(); + return result.GetString(); +} + +} \ No newline at end of file diff --git a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriter.h b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriter.h new file mode 100644 index 000000000000..699fd3d80b5b --- /dev/null +++ b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriter.h @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#pragma once + +#include +#include +#include +#include +#include + +namespace DB +{ +struct BlockWithPartition; +class MergeTreeData; +struct StorageSnapshot; +using StorageSnapshotPtr = std::shared_ptr; +} + +namespace local_engine +{ +class SinkHelper; + +struct PartInfo +{ + String part_name; + size_t mark_count; + size_t disk_size; + size_t row_count; + std::unordered_map partition_values; + String bucket_id; + + bool operator<(const PartInfo & rhs) const { return disk_size < rhs.disk_size; } +}; + +class SparkMergeTreeWriter +{ +public: + static String partInfosToJson(const std::vector & part_infos); + static std::unique_ptr create( + const MergeTreeTable & merge_tree_table, + const SparkMergeTreeWritePartitionSettings & write_settings_, + const DB::ContextMutablePtr & context); + + SparkMergeTreeWriter( + const DB::Block & header_, + const SinkHelper & sink_helper_, + DB::QueryPipeline && pipeline_, + std::unordered_map && partition_values_); + + void write(const DB::Block & block); + void finalize(); + std::vector getAllPartInfo() const; + +private: + DB::Block header; + const SinkHelper & sink_helper; + DB::QueryPipeline pipeline; + DB::PushingPipelineExecutor executor; + std::unordered_map partition_values; +}; +} diff --git a/cpp-ch/local-engine/Storages/CustomStorageMergeTree.cpp b/cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.cpp similarity index 52% rename from cpp-ch/local-engine/Storages/CustomStorageMergeTree.cpp rename to cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.cpp index 7336d7db5190..b5f8ac048fb6 100644 --- a/cpp-ch/local-engine/Storages/CustomStorageMergeTree.cpp +++ b/cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.cpp @@ -14,11 +14,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -#include "CustomStorageMergeTree.h" +#include "SparkStorageMergeTree.h" #include #include #include +#include #include namespace DB @@ -34,8 +35,7 @@ extern const int NO_SUCH_DATA_PART; namespace local_engine { - -void CustomStorageMergeTree::analysisPartsByRanges(DB::ReadFromMergeTree & source, DB::RangesInDataParts ranges_in_data_parts) +void SparkStorageMergeTree::analysisPartsByRanges(DB::ReadFromMergeTree & source, const DB::RangesInDataParts & ranges_in_data_parts) { ReadFromMergeTree::AnalysisResult result; result.column_names_to_read = source.getAllColumnNames(); @@ -76,14 +76,13 @@ void CustomStorageMergeTree::analysisPartsByRanges(DB::ReadFromMergeTree & sourc result.selected_rows = sum_rows; if (source.getQueryInfo().input_order_info) - result.read_type = (source.getQueryInfo().input_order_info->direction > 0) - ? MergeTreeReadType::InOrder - : MergeTreeReadType::InReverseOrder; + result.read_type + = (source.getQueryInfo().input_order_info->direction > 0) ? MergeTreeReadType::InOrder : MergeTreeReadType::InReverseOrder; source.setAnalyzedResult(std::make_shared(std::move(result))); } -void CustomStorageMergeTree::wrapRangesInDataParts(DB::ReadFromMergeTree & source, DB::RangesInDataParts ranges) +void SparkStorageMergeTree::wrapRangesInDataParts(DB::ReadFromMergeTree & source, const DB::RangesInDataParts & ranges) { auto result = source.getAnalysisResult(); std::unordered_map> range_index; @@ -107,7 +106,7 @@ void CustomStorageMergeTree::wrapRangesInDataParts(DB::ReadFromMergeTree & sourc const size_t end = std::min(range.end, std::get<1>(expected_range)); // [1, 1) or [5, 2) are invalid. if (begin >= end) - continue ; + continue; MarkRange final_range(begin, end); final_ranges.emplace_back(final_range); } @@ -119,12 +118,12 @@ void CustomStorageMergeTree::wrapRangesInDataParts(DB::ReadFromMergeTree & sourc source.setAnalyzedResult(std::make_shared(result)); } -CustomStorageMergeTree::CustomStorageMergeTree( +SparkStorageMergeTree::SparkStorageMergeTree( const StorageID & table_id_, const String & relative_data_path_, const StorageInMemoryMetadata & metadata_, bool attach, - ContextMutablePtr context_, + const ContextMutablePtr & context_, const String & date_column_name, const MergingParams & merging_params_, std::unique_ptr storage_settings_, @@ -138,7 +137,6 @@ CustomStorageMergeTree::CustomStorageMergeTree( std::move(storage_settings_), false, /// require_part_metadata attach ? LoadingStrictnessLevel::ATTACH : LoadingStrictnessLevel::FORCE_RESTORE) - , writer(*this) , reader(*this) , merger_mutator(*this) { @@ -146,31 +144,31 @@ CustomStorageMergeTree::CustomStorageMergeTree( format_version = 1; } -std::atomic CustomStorageMergeTree::part_num; - +std::atomic SparkStorageMergeTree::part_num; - -void CustomStorageMergeTree::prefectchMetaDataFile(std::unordered_set parts) +void SparkStorageMergeTree::prefetchMetaDataFile(std::unordered_set parts) const { auto disk = getDisks().front(); - if (!disk->isRemote()) return; + if (!disk->isRemote()) + return; std::vector meta_paths; std::ranges::for_each(parts, [&](const String & name) { meta_paths.emplace_back(fs::path(relative_data_path) / name / "meta.bin"); }); - for (const auto & meta_path: meta_paths) + for (const auto & meta_path : meta_paths) { - if (!disk->exists(meta_path)) continue; + if (!disk->exists(meta_path)) + continue; auto in = disk->readFile(meta_path); String ignore_data; readStringUntilEOF(ignore_data, *in); } } -std::vector CustomStorageMergeTree::loadDataPartsWithNames(std::unordered_set parts) +std::vector SparkStorageMergeTree::loadDataPartsWithNames(const std::unordered_set & parts) { - prefectchMetaDataFile(parts); + prefetchMetaDataFile(parts); std::vector data_parts; const auto disk = getStoragePolicy()->getDisks().at(0); - for (const auto& name : parts) + for (const auto & name : parts) { const auto num = part_num.fetch_add(1); MergeTreePartInfo part_info = {"all", num, num, 0}; @@ -181,11 +179,8 @@ std::vector CustomStorageMergeTree::loadDataPartsWithNames return data_parts; } -MergeTreeData::LoadPartResult CustomStorageMergeTree::loadDataPart( - const MergeTreePartInfo & part_info, - const String & part_name, - const DiskPtr & part_disk_ptr, - MergeTreeDataPartState to_state) +MergeTreeData::LoadPartResult SparkStorageMergeTree::loadDataPart( + const MergeTreePartInfo & part_info, const String & part_name, const DiskPtr & part_disk_ptr, MergeTreeDataPartState to_state) { LOG_TRACE(log, "Loading {} part {} from disk {}", magic_enum::enum_name(to_state), part_name, part_disk_ptr->getName()); @@ -262,7 +257,7 @@ MergeTreeData::LoadPartResult CustomStorageMergeTree::loadDataPart( return res; } -void CustomStorageMergeTree::removePartFromMemory(const MergeTreeData::DataPart & part_to_detach) +void SparkStorageMergeTree::removePartFromMemory(const MergeTreeData::DataPart & part_to_detach) { auto lock = lockParts(); bool removed_active_part = false; @@ -294,58 +289,222 @@ void CustomStorageMergeTree::removePartFromMemory(const MergeTreeData::DataPart resetObjectColumnsFromActiveParts(lock); } -void CustomStorageMergeTree::dropPartNoWaitNoThrow(const String & /*part_name*/) +void SparkStorageMergeTree::dropPartNoWaitNoThrow(const String & /*part_name*/) { throw std::runtime_error("not implement"); } -void CustomStorageMergeTree::dropPart(const String & /*part_name*/, bool /*detach*/, ContextPtr /*context*/) +void SparkStorageMergeTree::dropPart(const String & /*part_name*/, bool /*detach*/, ContextPtr /*context*/) { throw std::runtime_error("not implement"); } -void CustomStorageMergeTree::dropPartition(const ASTPtr & /*partition*/, bool /*detach*/, ContextPtr /*context*/) +void SparkStorageMergeTree::dropPartition(const ASTPtr & /*partition*/, bool /*detach*/, ContextPtr /*context*/) { } -PartitionCommandsResultInfo CustomStorageMergeTree::attachPartition( +PartitionCommandsResultInfo SparkStorageMergeTree::attachPartition( const ASTPtr & /*partition*/, const StorageMetadataPtr & /*metadata_snapshot*/, bool /*part*/, ContextPtr /*context*/) { throw std::runtime_error("not implement"); } -void CustomStorageMergeTree::replacePartitionFrom( +void SparkStorageMergeTree::replacePartitionFrom( const StoragePtr & /*source_table*/, const ASTPtr & /*partition*/, bool /*replace*/, ContextPtr /*context*/) { throw std::runtime_error("not implement"); } -void CustomStorageMergeTree::movePartitionToTable(const StoragePtr & /*dest_table*/, const ASTPtr & /*partition*/, ContextPtr /*context*/) +void SparkStorageMergeTree::movePartitionToTable(const StoragePtr & /*dest_table*/, const ASTPtr & /*partition*/, ContextPtr /*context*/) { throw std::runtime_error("not implement"); } -bool CustomStorageMergeTree::partIsAssignedToBackgroundOperation(const MergeTreeData::DataPartPtr & /*part*/) const +bool SparkStorageMergeTree::partIsAssignedToBackgroundOperation(const MergeTreeData::DataPartPtr & /*part*/) const { throw std::runtime_error("not implement"); } -std::string CustomStorageMergeTree::getName() const +std::string SparkStorageMergeTree::getName() const { throw std::runtime_error("not implement"); } -std::vector CustomStorageMergeTree::getMutationsStatus() const +std::vector SparkStorageMergeTree::getMutationsStatus() const { throw std::runtime_error("not implement"); } -bool CustomStorageMergeTree::scheduleDataProcessingJob(BackgroundJobsAssignee & /*executor*/) +bool SparkStorageMergeTree::scheduleDataProcessingJob(BackgroundJobsAssignee & /*executor*/) { throw std::runtime_error("not implement"); } -void CustomStorageMergeTree::startBackgroundMovesIfNeeded() +void SparkStorageMergeTree::startBackgroundMovesIfNeeded() { throw std::runtime_error("not implement"); } -std::unique_ptr CustomStorageMergeTree::getDefaultSettings() const +std::unique_ptr SparkStorageMergeTree::getDefaultSettings() const { throw std::runtime_error("not implement"); } -std::map CustomStorageMergeTree::getUnfinishedMutationCommands() const +std::map SparkStorageMergeTree::getUnfinishedMutationCommands() const { throw std::runtime_error("not implement"); } + +MergeTreeDataWriter::TemporaryPart SparkMergeTreeDataWriter::writeTempPart( + BlockWithPartition & block_with_partition, + const StorageMetadataPtr & metadata_snapshot, + const ContextPtr & context, + const SparkMergeTreeWritePartitionSettings & write_settings, + int part_num) const +{ + const std::string & part_name_prefix = write_settings.part_name_prefix; + const std::string & partition_dir = write_settings.partition_dir; + const std::string & bucket_dir = write_settings.bucket_dir; + + MergeTreeDataWriter::TemporaryPart temp_part; + + Block & block = block_with_partition.block; + + auto columns = metadata_snapshot->getColumns().getAllPhysical().filter(block.getNames()); + + for (auto & column : columns) + if (column.type->hasDynamicSubcolumns()) + column.type = block.getByName(column.name).type; + + auto minmax_idx = std::make_shared(); + minmax_idx->update(block, MergeTreeData::getMinMaxColumnsNames(metadata_snapshot->getPartitionKey())); + + MergeTreePartition partition(block_with_partition.partition); + + MergeTreePartInfo new_part_info(partition.getID(metadata_snapshot->getPartitionKey().sample_block), 1, 1, 0); + + std::string part_dir; + if (!partition_dir.empty() && !bucket_dir.empty()) + part_dir = fmt::format("{}/{}/{}_{:03d}", partition_dir, bucket_dir, part_name_prefix, part_num); + else if (!partition_dir.empty()) + part_dir = fmt::format("{}/{}_{:03d}", partition_dir, part_name_prefix, part_num); + else if (!bucket_dir.empty()) + part_dir = fmt::format("{}/{}_{:03d}", bucket_dir, part_name_prefix, part_num); + else + part_dir = fmt::format("{}_{:03d}", part_name_prefix, part_num); + + // assert(part_num > 0 && !part_name_prefix.empty()); + + String part_name = part_dir; + + temp_part.temporary_directory_lock = data.getTemporaryPartDirectoryHolder(part_dir); + + auto indices = MergeTreeIndexFactory::instance().getMany(metadata_snapshot->getSecondaryIndices()); + + /// If we need to calculate some columns to sort. + if (metadata_snapshot->hasSortingKey() || metadata_snapshot->hasSecondaryIndices()) + data.getSortingKeyAndSkipIndicesExpression(metadata_snapshot, indices)->execute(block); + + Names sort_columns = metadata_snapshot->getSortingKeyColumns(); + SortDescription sort_description; + size_t sort_columns_size = sort_columns.size(); + sort_description.reserve(sort_columns_size); + + for (size_t i = 0; i < sort_columns_size; ++i) + sort_description.emplace_back(sort_columns[i], 1, 1); + + /// Sort + IColumn::Permutation * perm_ptr = nullptr; + IColumn::Permutation perm; + if (!sort_description.empty()) + { + if (!isAlreadySorted(block, sort_description)) + { + stableGetPermutation(block, sort_description, perm); + perm_ptr = &perm; + } + } + + Names partition_key_columns = metadata_snapshot->getPartitionKey().column_names; + + /// Size of part would not be greater than block.bytes() + epsilon + size_t expected_size = block.bytes(); + + /// If optimize_on_insert is true, block may become empty after merge. + /// There is no need to create empty part. + if (expected_size == 0) + return temp_part; + + VolumePtr volume = data.getStoragePolicy()->getVolume(0); + VolumePtr data_part_volume = std::make_shared(volume->getName(), volume->getDisk(), volume->max_data_part_size); + auto new_data_part = data.getDataPartBuilder(part_name, data_part_volume, part_dir) + .withPartFormat(data.choosePartFormat(expected_size, block.rows())) + .withPartInfo(new_part_info) + .build(); + + auto data_part_storage = new_data_part->getDataPartStoragePtr(); + + + const auto & data_settings = data.getSettings(); + + SerializationInfo::Settings settings{data_settings->ratio_of_defaults_for_sparse_serialization, true}; + SerializationInfoByName infos(columns, settings); + infos.add(block); + + new_data_part->setColumns(columns, infos, metadata_snapshot->getMetadataVersion()); + new_data_part->rows_count = block.rows(); + new_data_part->partition = std::move(partition); + new_data_part->minmax_idx = std::move(minmax_idx); + + data_part_storage->beginTransaction(); + SyncGuardPtr sync_guard; + if (new_data_part->isStoredOnDisk()) + { + /// The name could be non-unique in case of stale files from previous runs. + String full_path = new_data_part->getDataPartStorage().getFullPath(); + + if (new_data_part->getDataPartStorage().exists()) + { + // LOG_WARNING(log, "Removing old temporary directory {}", full_path); + data_part_storage->removeRecursive(); + } + + data_part_storage->createDirectories(); + + if (data.getSettings()->fsync_part_directory) + { + const auto disk = data_part_volume->getDisk(); + sync_guard = disk->getDirectorySyncGuard(full_path); + } + } + + /// This effectively chooses minimal compression method: + /// either default lz4 or compression method with zero thresholds on absolute and relative part size. + auto compression_codec = data.getContext()->chooseCompressionCodec(0, 0); + auto txn = context->getCurrentTransaction(); + auto out = std::make_unique( + new_data_part, + metadata_snapshot, + columns, + indices, + MergeTreeStatisticsFactory::instance().getMany(metadata_snapshot->getColumns()), + compression_codec, + txn ? txn->tid : Tx::PrehistoricTID, + false, + false, + context->getWriteSettings()); + + out->writeWithPermutation(block, perm_ptr); + auto finalizer = out->finalizePartAsync(new_data_part, data_settings->fsync_after_insert, nullptr, nullptr); + + temp_part.part = new_data_part; + temp_part.streams.emplace_back(MergeTreeDataWriter::TemporaryPart::Stream{.stream = std::move(out), .finalizer = std::move(finalizer)}); + temp_part.finalize(); + data_part_storage->commitTransaction(); + return temp_part; +} + +SinkToStoragePtr SparkWriteStorageMergeTree::write( + const ASTPtr &, const StorageMetadataPtr & /*storage_in_memory_metadata*/, ContextPtr context, bool /*async_insert*/) +{ + SparkMergeTreeWriteSettings settings{.partition_settings{SparkMergeTreeWritePartitionSettings::get(context)}}; + settings.load(context); + SinkHelperPtr sink_helper = SparkMergeTreeSink::create(table, settings, getContext()); +#ifndef NDEBUG + auto dest_storage = table.getStorage(getContext()); + assert(dest_storage.get() == this); +#endif + + return std::make_shared(sink_helper, context); +} + } diff --git a/cpp-ch/local-engine/Storages/CustomStorageMergeTree.h b/cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.h similarity index 60% rename from cpp-ch/local-engine/Storages/CustomStorageMergeTree.h rename to cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.h index 773e5858c24f..fbd19de643ea 100644 --- a/cpp-ch/local-engine/Storages/CustomStorageMergeTree.h +++ b/cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.h @@ -16,63 +16,74 @@ */ #pragma once +#include #include #include #include -#include #include -#include +#include +#include #include namespace local_engine { +struct SparkMergeTreeWritePartitionSettings; using namespace DB; -class CustomMergeTreeSink; - -class CustomStorageMergeTree final : public MergeTreeData +class SparkMergeTreeDataWriter { - friend class CustomMergeTreeSink; +public: + explicit SparkMergeTreeDataWriter(MergeTreeData & data_) : data(data_), log(getLogger(data.getLogName() + " (Writer)")) { } + MergeTreeDataWriter::TemporaryPart writeTempPart( + DB::BlockWithPartition & block_with_partition, + const DB::StorageMetadataPtr & metadata_snapshot, + const ContextPtr & context, + const SparkMergeTreeWritePartitionSettings & write_settings, + int part_num) const; + +private: + MergeTreeData & data; + LoggerPtr log; +}; +class SparkStorageMergeTree : public MergeTreeData +{ friend class MergeSparkMergeTreeTask; public: - static void wrapRangesInDataParts(DB::ReadFromMergeTree & source, DB::RangesInDataParts ranges); - void analysisPartsByRanges(DB::ReadFromMergeTree & source, DB::RangesInDataParts ranges_in_data_parts); - CustomStorageMergeTree( - const StorageID & table_id_, - const String & relative_data_path_, - const StorageInMemoryMetadata & metadata, - bool attach, - ContextMutablePtr context_, - const String & date_column_name, - const MergingParams & merging_params_, - std::unique_ptr settings_, - bool has_force_restore_data_flag = false); + static void wrapRangesInDataParts(DB::ReadFromMergeTree & source, const DB::RangesInDataParts & ranges); + static void analysisPartsByRanges(DB::ReadFromMergeTree & source, const DB::RangesInDataParts & ranges_in_data_parts); std::string getName() const override; std::vector getMutationsStatus() const override; bool scheduleDataProcessingJob(BackgroundJobsAssignee & executor) override; std::map getUnfinishedMutationCommands() const override; - std::vector loadDataPartsWithNames(std::unordered_set parts); + std::vector loadDataPartsWithNames(const std::unordered_set & parts); void removePartFromMemory(const MergeTreeData::DataPart & part_to_detach); - MergeTreeDataWriter writer; MergeTreeDataSelectExecutor reader; MergeTreeDataMergerMutator merger_mutator; - static std::atomic part_num; +protected: + SparkStorageMergeTree( + const StorageID & table_id_, + const String & relative_data_path_, + const StorageInMemoryMetadata & metadata, + bool attach, + const ContextMutablePtr & context_, + const String & date_column_name, + const MergingParams & merging_params_, + std::unique_ptr settings_, + bool has_force_restore_data_flag = false); private: + static std::atomic part_num; SimpleIncrement increment; - void prefectchMetaDataFile(std::unordered_set parts); + void prefetchMetaDataFile(std::unordered_set parts) const; void startBackgroundMovesIfNeeded() override; std::unique_ptr getDefaultSettings() const override; LoadPartResult loadDataPart( - const MergeTreePartInfo & part_info, - const String & part_name, - const DiskPtr & part_disk_ptr, - MergeTreeDataPartState to_state); + const MergeTreePartInfo & part_info, const String & part_name, const DiskPtr & part_disk_ptr, MergeTreeDataPartState to_state); protected: void dropPartNoWaitNoThrow(const String & part_name) override; @@ -85,7 +96,35 @@ class CustomStorageMergeTree final : public MergeTreeData bool partIsAssignedToBackgroundOperation(const DataPartPtr & part) const override; MutationCommands getAlterMutationCommandsForPart(const DataPartPtr & /*part*/) const override { return {}; } void attachRestoredParts(MutableDataPartsVector && /*parts*/) override { throw std::runtime_error("not implement"); } +}; +class SparkWriteStorageMergeTree final : public SparkStorageMergeTree +{ +public: + SparkWriteStorageMergeTree(const MergeTreeTable & table_, const StorageInMemoryMetadata & metadata, const ContextMutablePtr & context_) + : SparkStorageMergeTree( + StorageID(table_.database, table_.table), + table_.relative_path, + metadata, + false, + context_, + "", + MergingParams(), + buildMergeTreeSettings(table_.table_configs), + false /*has_force_restore_data_flag*/) + , table(table_) + , writer(*this) + { + } + + SinkToStoragePtr + write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context, bool async_insert) override; + + SparkMergeTreeDataWriter & getWriter() { return writer; } + +private: + MergeTreeTable table; + SparkMergeTreeDataWriter writer; }; } diff --git a/cpp-ch/local-engine/Storages/StorageMergeTreeFactory.cpp b/cpp-ch/local-engine/Storages/MergeTree/StorageMergeTreeFactory.cpp similarity index 80% rename from cpp-ch/local-engine/Storages/StorageMergeTreeFactory.cpp rename to cpp-ch/local-engine/Storages/MergeTree/StorageMergeTreeFactory.cpp index eefd1c5fd1ec..ed41633d3d40 100644 --- a/cpp-ch/local-engine/Storages/StorageMergeTreeFactory.cpp +++ b/cpp-ch/local-engine/Storages/MergeTree/StorageMergeTreeFactory.cpp @@ -16,11 +16,10 @@ */ #include "StorageMergeTreeFactory.h" +#include +#include #include -#include -#include - namespace local_engine { @@ -59,14 +58,15 @@ void StorageMergeTreeFactory::freeStorage(const StorageID & id, const String & s } -CustomStorageMergeTreePtr -StorageMergeTreeFactory::getStorage(const StorageID& id, const String & snapshot_id, MergeTreeTable merge_tree_table, std::function creator) +SparkStorageMergeTreePtr StorageMergeTreeFactory::getStorage( + const StorageID & id, const String & snapshot_id, MergeTreeTable merge_tree_table, + const std::function & creator) { - auto table_name = getTableName(id, snapshot_id); + const auto table_name = getTableName(id, snapshot_id); std::lock_guard lock(storage_map_mutex); - merge_tree_table.parts.clear(); - if (storage_map->has(table_name) && !storage_map->get(table_name)->second.sameStructWith(merge_tree_table)) + // merge_tree_table.parts.clear(); + if (storage_map->has(table_name) && !storage_map->get(table_name)->second.sameTable(merge_tree_table)) { freeStorage(id); std::lock_guard lock_datapart(datapart_mutex); @@ -78,7 +78,8 @@ StorageMergeTreeFactory::getStorage(const StorageID& id, const String & snapshot return storage_map->get(table_name)->first; } -DataPartsVector StorageMergeTreeFactory::getDataPartsByNames(const StorageID & id, const String & snapshot_id, std::unordered_set part_name) +DataPartsVector +StorageMergeTreeFactory::getDataPartsByNames(const StorageID & id, const String & snapshot_id, std::unordered_set part_name) { DataPartsVector res; auto table_name = getTableName(id, snapshot_id); @@ -106,7 +107,7 @@ DataPartsVector StorageMergeTreeFactory::getDataPartsByNames(const StorageID & i if (!missing_names.empty()) { - CustomStorageMergeTreePtr storage_merge_tree; + SparkStorageMergeTreePtr storage_merge_tree; { std::lock_guard storage_lock(storage_map_mutex); storage_merge_tree = storage_map->get(table_name)->first; @@ -121,8 +122,8 @@ DataPartsVector StorageMergeTreeFactory::getDataPartsByNames(const StorageID & i return res; } // will be inited in native init phase -std::unique_ptr>> StorageMergeTreeFactory::storage_map = nullptr; -std::unique_ptr>>> StorageMergeTreeFactory::datapart_map = nullptr; +std::unique_ptr StorageMergeTreeFactory::storage_map = nullptr; +std::unique_ptr StorageMergeTreeFactory::datapart_map = nullptr; std::recursive_mutex StorageMergeTreeFactory::storage_map_mutex; std::recursive_mutex StorageMergeTreeFactory::datapart_mutex; diff --git a/cpp-ch/local-engine/Storages/StorageMergeTreeFactory.h b/cpp-ch/local-engine/Storages/MergeTree/StorageMergeTreeFactory.h similarity index 69% rename from cpp-ch/local-engine/Storages/StorageMergeTreeFactory.h rename to cpp-ch/local-engine/Storages/MergeTree/StorageMergeTreeFactory.h index 71e4da6bb696..9418198583ec 100644 --- a/cpp-ch/local-engine/Storages/StorageMergeTreeFactory.h +++ b/cpp-ch/local-engine/Storages/MergeTree/StorageMergeTreeFactory.h @@ -15,22 +15,20 @@ * limitations under the License. */ #pragma once -#include -#include -#include -#include -#include #include - +#include +#include +#include +#include namespace local_engine { -using CustomStorageMergeTreePtr = std::shared_ptr; +using SparkStorageMergeTreePtr = std::shared_ptr; class DataPartStorageHolder { public: - DataPartStorageHolder(const DataPartPtr& data_part, const CustomStorageMergeTreePtr& storage) + DataPartStorageHolder(const DataPartPtr& data_part, const SparkStorageMergeTreePtr& storage) : data_part_(data_part), storage_(storage) { @@ -41,7 +39,7 @@ class DataPartStorageHolder return data_part_; } - [[nodiscard]] CustomStorageMergeTreePtr storage() const + [[nodiscard]] SparkStorageMergeTreePtr storage() const { return storage_; } @@ -54,17 +52,21 @@ class DataPartStorageHolder private: DataPartPtr data_part_; - CustomStorageMergeTreePtr storage_; + SparkStorageMergeTreePtr storage_; }; + using DataPartStorageHolderPtr = std::shared_ptr; +using storage_map_cache = Poco::LRUCache>; +using datapart_map_cache = Poco::LRUCache>>; class StorageMergeTreeFactory { public: static StorageMergeTreeFactory & instance(); static void freeStorage(const StorageID & id, const String & snapshot_id = ""); - static CustomStorageMergeTreePtr - getStorage(const StorageID& id, const String & snapshot_id, MergeTreeTable merge_tree_table, std::function creator); + static SparkStorageMergeTreePtr + getStorage(const StorageID& id, const String & snapshot_id, MergeTreeTable merge_tree_table, + const std::function & creator); static DataPartsVector getDataPartsByNames(const StorageID & id, const String & snapshot_id, std::unordered_set part_name); static void init_cache_map() { @@ -72,7 +74,7 @@ class StorageMergeTreeFactory auto & storage_map_v = storage_map; if (!storage_map_v) { - storage_map_v = std::make_unique>>(config.table_metadata_cache_max_count); + storage_map_v = std::make_unique(config.table_metadata_cache_max_count); } else { @@ -81,8 +83,7 @@ class StorageMergeTreeFactory auto & datapart_map_v = datapart_map; if (!datapart_map_v) { - datapart_map_v = std::make_unique>>>( - config.table_metadata_cache_max_count); + datapart_map_v = std::make_unique(config.table_metadata_cache_max_count); } else { @@ -98,19 +99,11 @@ class StorageMergeTreeFactory static String getTableName(const StorageID & id, const String & snapshot_id); private: - static std::unique_ptr>> storage_map; - static std::unique_ptr>>> datapart_map; + static std::unique_ptr storage_map; + static std::unique_ptr datapart_map; static std::recursive_mutex storage_map_mutex; static std::recursive_mutex datapart_mutex; }; -struct TempStorageFreer -{ - StorageID id; - ~TempStorageFreer() - { - StorageMergeTreeFactory::instance().freeStorage(id); - } -}; } diff --git a/cpp-ch/local-engine/Storages/Mergetree/SparkMergeTreeWriter.cpp b/cpp-ch/local-engine/Storages/Mergetree/SparkMergeTreeWriter.cpp deleted file mode 100644 index e7d3be0db509..000000000000 --- a/cpp-ch/local-engine/Storages/Mergetree/SparkMergeTreeWriter.cpp +++ /dev/null @@ -1,547 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -#include "SparkMergeTreeWriter.h" - -#include - -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -namespace CurrentMetrics -{ -extern const Metric LocalThread; -extern const Metric LocalThreadActive; -extern const Metric LocalThreadScheduled; -extern const Metric GlobalThread; -extern const Metric GlobalThreadActive; -extern const Metric GlobalThreadScheduled; -} - -using namespace DB; - -namespace local_engine -{ - -Block removeColumnSuffix(const Block & block) -{ - ColumnsWithTypeAndName columns; - for (int i = 0; i < block.columns(); ++i) - { - auto name = block.getByPosition(i).name; - Poco::StringTokenizer splits(name, "#"); - auto column = block.getByPosition(i); - column.name = splits[0]; - columns.emplace_back(column); - } - return Block(columns); -} - -SparkMergeTreeWriter::SparkMergeTreeWriter( - const MergeTreeTable & merge_tree_table, - const DB::ContextPtr & context_, - const String & part_name_prefix_, - const String & partition_dir_, - const String & bucket_dir_) - : context(context_) - , part_name_prefix(part_name_prefix_) - , partition_dir(partition_dir_) - , bucket_dir(bucket_dir_) - , thread_pool(CurrentMetrics::LocalThread, CurrentMetrics::LocalThreadActive, CurrentMetrics::LocalThreadScheduled, 1, 1, 100000) -{ - const DB::Settings & settings = context->getSettingsRef(); - merge_after_insert = settings.get(MERGETREE_MERGE_AFTER_INSERT).safeGet(); - insert_without_local_storage = settings.get(MERGETREE_INSERT_WITHOUT_LOCAL_STORAGE).safeGet(); - - Field limit_size_field; - if (settings.tryGet("optimize.minFileSize", limit_size_field)) - merge_min_size = limit_size_field.safeGet() <= 0 ? merge_min_size : limit_size_field.safeGet(); - - Field limit_cnt_field; - if (settings.tryGet("mergetree.max_num_part_per_merge_task", limit_cnt_field)) - merge_limit_parts = limit_cnt_field.safeGet() <= 0 ? merge_limit_parts : limit_cnt_field.safeGet(); - - dest_storage = MergeTreeRelParser::parseStorage(merge_tree_table, SerializedPlanParser::global_context); - isRemoteStorage = dest_storage->getStoragePolicy()->getAnyDisk()->isRemote(); - - if (useLocalStorage()) - { - temp_storage = MergeTreeRelParser::copyToDefaultPolicyStorage(merge_tree_table, SerializedPlanParser::global_context); - storage = temp_storage; - LOG_DEBUG( - &Poco::Logger::get("SparkMergeTreeWriter"), - "Create temp table {} for local merge.", - temp_storage->getStorageID().getFullNameNotQuoted()); - } - else - storage = dest_storage; - - metadata_snapshot = storage->getInMemoryMetadataPtr(); - header = metadata_snapshot->getSampleBlock(); - squashing = std::make_unique(header, settings.min_insert_block_size_rows, settings.min_insert_block_size_bytes); - if (!partition_dir.empty()) - extractPartitionValues(partition_dir, partition_values); -} - -bool SparkMergeTreeWriter::useLocalStorage() const -{ - return !insert_without_local_storage && isRemoteStorage; -} - -void SparkMergeTreeWriter::write(const DB::Block & block) -{ - auto new_block = removeColumnSuffix(block); - auto converter = ActionsDAG::makeConvertingActions( - new_block.getColumnsWithTypeAndName(), header.getColumnsWithTypeAndName(), DB::ActionsDAG::MatchColumnsMode::Position); - const ExpressionActions expression_actions{std::move(converter)}; - expression_actions.execute(new_block); - - bool has_part = chunkToPart(squashing->add({new_block.getColumns(), new_block.rows()})); - - if (has_part && merge_after_insert) - checkAndMerge(); -} - -bool SparkMergeTreeWriter::chunkToPart(Chunk && plan_chunk) -{ - if (Chunk result_chunk = DB::Squashing::squash(std::move(plan_chunk))) - { - auto result = squashing->getHeader().cloneWithColumns(result_chunk.detachColumns()); - return blockToPart(result); - } - return false; -} - -bool SparkMergeTreeWriter::blockToPart(Block & block) -{ - auto blocks_with_partition = MergeTreeDataWriter::splitBlockIntoParts(std::move(block), 10, metadata_snapshot, context); - - if (blocks_with_partition.empty()) - return false; - - for (auto & item : blocks_with_partition) - { - size_t before_write_memory = 0; - if (auto * memory_tracker = CurrentThread::getMemoryTracker()) - { - CurrentThread::flushUntrackedMemory(); - before_write_memory = memory_tracker->get(); - } - - new_parts.emplace_back(writeTempPartAndFinalize(item, metadata_snapshot).part); - part_num++; - /// Reset earlier to free memory - item.block.clear(); - item.partition.clear(); - } - - return true; -} - -void SparkMergeTreeWriter::finalize() -{ - chunkToPart(squashing->flush()); - if (merge_after_insert) - finalizeMerge(); - - commitPartToRemoteStorageIfNeeded(); - saveMetadata(); -} - -void SparkMergeTreeWriter::saveMetadata() -{ - if (!isRemoteStorage) - return; - - for (const auto & merge_tree_data_part : new_parts.unsafeGet()) - { - auto part = dest_storage->loadDataPartsWithNames({merge_tree_data_part->name}); - if (part.empty()) - { - LOG_WARNING( - &Poco::Logger::get("SparkMergeTreeWriter"), - "Save metadata failed because dest storage load part name {} empty.", - merge_tree_data_part->name); - continue; - } - - saveFileStatus( - *dest_storage, context, merge_tree_data_part->name, const_cast(part.at(0)->getDataPartStorage())); - } -} - -void SparkMergeTreeWriter::commitPartToRemoteStorageIfNeeded() -{ - if (!useLocalStorage()) - return; - - LOG_DEBUG( - &Poco::Logger::get("SparkMergeTreeWriter"), "Begin upload to disk {}.", dest_storage->getStoragePolicy()->getAnyDisk()->getName()); - - auto read_settings = context->getReadSettings(); - auto write_settings = context->getWriteSettings(); - Stopwatch watch; - for (const auto & merge_tree_data_part : new_parts.unsafeGet()) - { - String local_relative_path = storage->getRelativeDataPath() + "/" + merge_tree_data_part->name; - String remote_relative_path = dest_storage->getRelativeDataPath() + "/" + merge_tree_data_part->name; - - std::vector files; - storage->getStoragePolicy()->getAnyDisk()->listFiles(local_relative_path, files); - auto src_disk = storage->getStoragePolicy()->getAnyDisk(); - auto dest_disk = dest_storage->getStoragePolicy()->getAnyDisk(); - auto tx = dest_disk->createTransaction(); - for (const auto & file : files) - { - auto read_buffer = src_disk->readFile(local_relative_path + "/" + file, read_settings); - auto write_buffer = tx->writeFile(remote_relative_path + "/" + file, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite, write_settings); - copyData(*read_buffer, *write_buffer); - write_buffer->finalize(); - } - tx->commit(); - LOG_DEBUG( - &Poco::Logger::get("SparkMergeTreeWriter"), - "Upload part {} to disk {} success.", - merge_tree_data_part->name, - dest_storage->getStoragePolicy()->getAnyDisk()->getName()); - } - watch.stop(); - LOG_INFO( - &Poco::Logger::get("SparkMergeTreeWriter"), - "Upload to disk {} finished, total elapsed {} ms", - dest_storage->getStoragePolicy()->getAnyDisk()->getName(), - watch.elapsedMilliseconds()); - StorageMergeTreeFactory::freeStorage(temp_storage->getStorageID()); - temp_storage->dropAllData(); - LOG_DEBUG( - &Poco::Logger::get("SparkMergeTreeWriter"), "Clean temp table {} success.", temp_storage->getStorageID().getFullNameNotQuoted()); -} - -void SparkMergeTreeWriter::finalizeMerge() -{ - LOG_DEBUG(&Poco::Logger::get("SparkMergeTreeWriter"), "Waiting all merge task end and do final merge"); - // waiting all merge task end and do final merge - thread_pool.wait(); - - size_t before_merge_size; - do - { - before_merge_size = new_parts.size(); - checkAndMerge(true); - thread_pool.wait(); - } while (before_merge_size != new_parts.size()); - - std::unordered_set final_parts; - for (const auto & merge_tree_data_part : new_parts.unsafeGet()) - final_parts.emplace(merge_tree_data_part->name); - - // default storage need clean temp. - if (!temp_storage) - { - for (const auto & tmp_part : tmp_parts) - { - if (final_parts.contains(tmp_part)) - continue; - - GlobalThreadPool::instance().scheduleOrThrow( - [storage_ = storage, tmp = tmp_part]() -> void - { - for (const auto & disk : storage_->getDisks()) - { - auto rel_path = storage_->getRelativeDataPath() + "/" + tmp; - disk->removeRecursive(rel_path); - } - }); - } - } -} - -DB::MergeTreeDataWriter::TemporaryPart SparkMergeTreeWriter::writeTempPartAndFinalize( - DB::BlockWithPartition & block_with_partition, const DB::StorageMetadataPtr & metadata_snapshot) -{ - MergeTreeDataWriter::TemporaryPart temp_part; - writeTempPart(temp_part, block_with_partition, metadata_snapshot); - return temp_part; -} - -void SparkMergeTreeWriter::writeTempPart( - MergeTreeDataWriter::TemporaryPart & temp_part, BlockWithPartition & block_with_partition, const StorageMetadataPtr & metadata_snapshot) -{ - Block & block = block_with_partition.block; - - auto columns = metadata_snapshot->getColumns().getAllPhysical().filter(block.getNames()); - - for (auto & column : columns) - if (column.type->hasDynamicSubcolumns()) - column.type = block.getByName(column.name).type; - - auto minmax_idx = std::make_shared(); - minmax_idx->update(block, storage->getMinMaxColumnsNames(metadata_snapshot->getPartitionKey())); - - MergeTreePartition partition(block_with_partition.partition); - - MergeTreePartInfo new_part_info(partition.getID(metadata_snapshot->getPartitionKey().sample_block), 1, 1, 0); - - std::string part_dir; - if (!partition_dir.empty() && !bucket_dir.empty()) - part_dir = fmt::format("{}/{}/{}_{:03d}", partition_dir, bucket_dir, part_name_prefix, part_num); - else if (!partition_dir.empty()) - part_dir = fmt::format("{}/{}_{:03d}", partition_dir, part_name_prefix, part_num); - else if (!bucket_dir.empty()) - part_dir = fmt::format("{}/{}_{:03d}", bucket_dir, part_name_prefix, part_num); - else - part_dir = fmt::format("{}_{:03d}", part_name_prefix, part_num); - - String part_name = part_dir; - - temp_part.temporary_directory_lock = storage->getTemporaryPartDirectoryHolder(part_dir); - - auto indices = MergeTreeIndexFactory::instance().getMany(metadata_snapshot->getSecondaryIndices()); - - /// If we need to calculate some columns to sort. - if (metadata_snapshot->hasSortingKey() || metadata_snapshot->hasSecondaryIndices()) - storage->getSortingKeyAndSkipIndicesExpression(metadata_snapshot, indices)->execute(block); - - Names sort_columns = metadata_snapshot->getSortingKeyColumns(); - SortDescription sort_description; - size_t sort_columns_size = sort_columns.size(); - sort_description.reserve(sort_columns_size); - - for (size_t i = 0; i < sort_columns_size; ++i) - sort_description.emplace_back(sort_columns[i], 1, 1); - - /// Sort - IColumn::Permutation * perm_ptr = nullptr; - IColumn::Permutation perm; - if (!sort_description.empty()) - { - if (!isAlreadySorted(block, sort_description)) - { - stableGetPermutation(block, sort_description, perm); - perm_ptr = &perm; - } - } - - Names partition_key_columns = metadata_snapshot->getPartitionKey().column_names; - - /// Size of part would not be greater than block.bytes() + epsilon - size_t expected_size = block.bytes(); - - /// If optimize_on_insert is true, block may become empty after merge. - /// There is no need to create empty part. - if (expected_size == 0) - return; - - VolumePtr volume = storage->getStoragePolicy()->getVolume(0); - VolumePtr data_part_volume = std::make_shared(volume->getName(), volume->getDisk(), volume->max_data_part_size); - auto new_data_part = storage->getDataPartBuilder(part_name, data_part_volume, part_dir) - .withPartFormat(storage->choosePartFormat(expected_size, block.rows())) - .withPartInfo(new_part_info) - .build(); - - auto data_part_storage = new_data_part->getDataPartStoragePtr(); - - - const auto & data_settings = storage->getSettings(); - - SerializationInfo::Settings settings{data_settings->ratio_of_defaults_for_sparse_serialization, true}; - SerializationInfoByName infos(columns, settings); - infos.add(block); - - new_data_part->setColumns(columns, infos, metadata_snapshot->getMetadataVersion()); - new_data_part->rows_count = block.rows(); - new_data_part->partition = std::move(partition); - new_data_part->minmax_idx = std::move(minmax_idx); - - data_part_storage->beginTransaction(); - SyncGuardPtr sync_guard; - if (new_data_part->isStoredOnDisk()) - { - /// The name could be non-unique in case of stale files from previous runs. - String full_path = new_data_part->getDataPartStorage().getFullPath(); - - if (new_data_part->getDataPartStorage().exists()) - { - // LOG_WARNING(log, "Removing old temporary directory {}", full_path); - data_part_storage->removeRecursive(); - } - - data_part_storage->createDirectories(); - - if (storage->getSettings()->fsync_part_directory) - { - const auto disk = data_part_volume->getDisk(); - sync_guard = disk->getDirectorySyncGuard(full_path); - } - } - - /// This effectively chooses minimal compression method: - /// either default lz4 or compression method with zero thresholds on absolute and relative part size. - auto compression_codec = storage->getContext()->chooseCompressionCodec(0, 0); - auto txn = context->getCurrentTransaction(); - auto out = std::make_unique( - new_data_part, - metadata_snapshot, - columns, - indices, - MergeTreeStatisticsFactory::instance().getMany(metadata_snapshot->getColumns()), - compression_codec, - txn ? txn->tid : Tx::PrehistoricTID, - false, - false, - context->getWriteSettings()); - - out->writeWithPermutation(block, perm_ptr); - auto finalizer = out->finalizePartAsync(new_data_part, data_settings->fsync_after_insert, nullptr, nullptr); - - temp_part.part = new_data_part; - temp_part.streams.emplace_back(MergeTreeDataWriter::TemporaryPart::Stream{.stream = std::move(out), .finalizer = std::move(finalizer)}); - temp_part.finalize(); - data_part_storage->commitTransaction(); -} - -std::vector SparkMergeTreeWriter::getAllPartInfo() -{ - std::vector res; - res.reserve(new_parts.size()); - - for (const auto & part : new_parts.unsafeGet()) - { - res.emplace_back( - PartInfo{part->name, part->getMarksCount(), part->getBytesOnDisk(), part->rows_count, partition_values, bucket_dir}); - } - return res; -} - -String SparkMergeTreeWriter::partInfosToJson(const std::vector & part_infos) -{ - rapidjson::StringBuffer result; - rapidjson::Writer writer(result); - writer.StartArray(); - for (const auto & item : part_infos) - { - writer.StartObject(); - writer.Key("part_name"); - writer.String(item.part_name.c_str()); - writer.Key("mark_count"); - writer.Uint(item.mark_count); - writer.Key("disk_size"); - writer.Uint(item.disk_size); - writer.Key("row_count"); - writer.Uint(item.row_count); - writer.Key("bucket_id"); - writer.String(item.bucket_id.c_str()); - writer.Key("partition_values"); - writer.StartObject(); - for (const auto & key_value : item.partition_values) - { - writer.Key(key_value.first.c_str()); - writer.String(key_value.second.c_str()); - } - writer.EndObject(); - writer.EndObject(); - } - writer.EndArray(); - return result.GetString(); -} - -void SparkMergeTreeWriter::checkAndMerge(bool force) -{ - // Only finalize should force merge. - if (!force && new_parts.size() < merge_limit_parts) - return; - - auto doMergeTask = [this](const std::vector & prepare_merge_parts) - { - for (const auto & selected_part : prepare_merge_parts) - tmp_parts.emplace(selected_part->name); - // check thread group initailized in task thread - currentThreadGroupMemoryUsage(); - thread_pool.scheduleOrThrow( - [this, prepare_merge_parts, thread_group = CurrentThread::getGroup()]() -> void - { - Stopwatch watch; - CurrentThread::detachFromGroupIfNotDetached(); - CurrentThread::attachToGroup(thread_group); - size_t before_size = 0; - size_t after_size = 0; - for (const auto & prepare_merge_part : prepare_merge_parts) - before_size += prepare_merge_part->getBytesOnDisk(); - - std::unordered_map partition_values; - const auto merged_parts = mergeParts( - prepare_merge_parts, partition_values, toString(UUIDHelpers::generateV4()), storage, partition_dir, bucket_dir); - for (const auto & merge_tree_data_part : merged_parts) - after_size += merge_tree_data_part->getBytesOnDisk(); - - new_parts.emplace_back(merged_parts); - watch.stop(); - LOG_INFO( - &Poco::Logger::get("SparkMergeTreeWriter"), - "Merge success. Before merge part size {}, part count {}, after part size {}, part count {}, " - "total elapsed {} ms", - before_size, - prepare_merge_parts.size(), - after_size, - merged_parts.size(), - watch.elapsedMilliseconds()); - }); - }; - - std::vector selected_parts; - selected_parts.reserve(merge_limit_parts); - size_t totol_size = 0; - std::vector skip_parts; - - while (const auto merge_tree_data_part_option = new_parts.pop_front()) - { - auto merge_tree_data_part = merge_tree_data_part_option.value(); - if (merge_tree_data_part->getBytesOnDisk() >= merge_min_size) - { - skip_parts.emplace_back(merge_tree_data_part); - continue; - } - - selected_parts.emplace_back(merge_tree_data_part); - totol_size += merge_tree_data_part->getBytesOnDisk(); - if (merge_min_size > totol_size && merge_limit_parts > selected_parts.size()) - continue; - - doMergeTask(selected_parts); - selected_parts.clear(); - totol_size = 0; - } - - if (!selected_parts.empty()) - { - if (force && selected_parts.size() > 1) - doMergeTask(selected_parts); - else - new_parts.emplace_back(selected_parts); - } - - new_parts.emplace_back(skip_parts); -} - -} \ No newline at end of file diff --git a/cpp-ch/local-engine/Storages/Mergetree/SparkMergeTreeWriter.h b/cpp-ch/local-engine/Storages/Mergetree/SparkMergeTreeWriter.h deleted file mode 100644 index 3fd1491ea389..000000000000 --- a/cpp-ch/local-engine/Storages/Mergetree/SparkMergeTreeWriter.h +++ /dev/null @@ -1,108 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -#pragma once - -#include -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ -struct BlockWithPartition; -class MergeTreeData; -struct StorageSnapshot; -using StorageSnapshotPtr = std::shared_ptr; -} - -namespace local_engine -{ - -struct PartInfo -{ - String part_name; - size_t mark_count; - size_t disk_size; - size_t row_count; - std::unordered_map partition_values; - String bucket_id; - - bool operator<(const PartInfo & rhs) const { return disk_size < rhs.disk_size; } -}; - -class SparkMergeTreeWriter -{ -public: - static String partInfosToJson(const std::vector & part_infos); - SparkMergeTreeWriter( - const MergeTreeTable & merge_tree_table, - const DB::ContextPtr & context_, - const String & part_name_prefix_, - const String & partition_dir_ = "", - const String & bucket_dir_ = ""); - - void write(const DB::Block & block); - void finalize(); - std::vector getAllPartInfo(); - -private: - void writeTempPart( - MergeTreeDataWriter::TemporaryPart & temp_part, - DB::BlockWithPartition & block_with_partition, - const DB::StorageMetadataPtr & metadata_snapshot); - DB::MergeTreeDataWriter::TemporaryPart - writeTempPartAndFinalize(DB::BlockWithPartition & block_with_partition, const DB::StorageMetadataPtr & metadata_snapshot); - void checkAndMerge(bool force = false); - void safeEmplaceBackPart(DB::MergeTreeDataPartPtr); - void safeAddPart(DB::MergeTreeDataPartPtr); - void saveMetadata(); - void commitPartToRemoteStorageIfNeeded(); - void finalizeMerge(); - bool chunkToPart(Chunk && plan_chunk); - bool blockToPart(Block & block); - bool useLocalStorage() const; - - CustomStorageMergeTreePtr storage = nullptr; - CustomStorageMergeTreePtr dest_storage = nullptr; - CustomStorageMergeTreePtr temp_storage = nullptr; - DB::StorageMetadataPtr metadata_snapshot = nullptr; - - String part_name_prefix; - String partition_dir; - String bucket_dir; - - DB::ContextPtr context; - std::unique_ptr squashing; - int part_num = 1; - ConcurrentDeque new_parts; - std::unordered_map partition_values; - std::unordered_set tmp_parts; - DB::Block header; - bool merge_after_insert; - bool insert_without_local_storage; - ThreadPool thread_pool; - size_t merge_min_size = 1024 * 1024 * 1024; - size_t merge_limit_parts = 10; - std::mutex memory_mutex; - bool isRemoteStorage = false; -}; - -} diff --git a/cpp-ch/local-engine/local_engine_jni.cpp b/cpp-ch/local-engine/local_engine_jni.cpp index 6ed9b4a43c44..5e862040ca72 100644 --- a/cpp-ch/local-engine/local_engine_jni.cpp +++ b/cpp-ch/local-engine/local_engine_jni.cpp @@ -28,20 +28,23 @@ #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 @@ -50,12 +53,10 @@ #include #include #include -#include +#include #include #include #include -#include -#include #ifdef __cplusplus @@ -80,15 +81,15 @@ static std::string jstring2string(JNIEnv * env, jstring jStr) const jclass string_class = env->GetObjectClass(jStr); const jmethodID get_bytes = env->GetMethodID(string_class, "getBytes", "(Ljava/lang/String;)[B"); - const auto string_jbytes + auto * const string_jbytes = static_cast(local_engine::safeCallObjectMethod(env, jStr, get_bytes, env->NewStringUTF("UTF-8"))); + SCOPE_EXIT({ + env->DeleteLocalRef(string_jbytes); + env->DeleteLocalRef(string_class); + }); const auto string_jbytes_a = local_engine::getByteArrayElementsSafe(env, string_jbytes); - std::string ret{reinterpret_cast(string_jbytes_a.elems()), static_cast(string_jbytes_a.length())}; - - env->DeleteLocalRef(string_jbytes); - env->DeleteLocalRef(string_class); - return ret; + return {reinterpret_cast(string_jbytes_a.elems()), static_cast(string_jbytes_a.length())}; } extern "C" { @@ -213,19 +214,20 @@ JNIEXPORT void Java_org_apache_gluten_vectorized_ExpressionEvaluatorJniWrapper_i const auto query_context = local_engine::QueryContextManager::instance().currentQueryContext(); const auto path_array = local_engine::getByteArrayElementsSafe(env, temp_path); - const std::string c_path{reinterpret_cast(path_array.elems()), static_cast(path_array.length())}; - query_context->setSetting(local_engine::SPARK_TASK_WRITE_TMEP_DIR, c_path); - const auto filename_array = local_engine::getByteArrayElementsSafe(env, filename); - const std::string c_filename{reinterpret_cast(filename_array.elems()), static_cast(filename_array.length())}; - query_context->setSetting(local_engine::SPARK_TASK_WRITE_FILENAME, c_filename); + + local_engine::GlutenWriteSettings settings{ + .task_write_tmp_dir = {reinterpret_cast(path_array.elems()), static_cast(path_array.length())}, + .task_write_filename = {reinterpret_cast(filename_array.elems()), static_cast(filename_array.length())}, + }; + settings.set(query_context); LOCAL_ENGINE_JNI_METHOD_END(env, ) } JNIEXPORT jlong Java_org_apache_gluten_vectorized_ExpressionEvaluatorJniWrapper_nativeCreateKernelWithIterator( JNIEnv * env, - jclass , + jclass, jbyteArray plan, jobjectArray split_infos, jobjectArray iter_arr, @@ -593,7 +595,8 @@ JNIEXPORT jlong Java_org_apache_gluten_vectorized_CHShuffleSplitterJniWrapper_na .max_sort_buffer_size = static_cast(max_sort_buffer_size), .force_memory_sort = static_cast(force_memory_sort)}; auto name = jstring2string(env, short_name); - local_engine::SplitterHolder * splitter = new local_engine::SplitterHolder{.splitter = std::make_unique(name, options)}; + local_engine::SplitterHolder * splitter + = new local_engine::SplitterHolder{.splitter = std::make_unique(name, options)}; return reinterpret_cast(splitter); LOCAL_ENGINE_JNI_METHOD_END(env, -1) } @@ -890,18 +893,16 @@ JNIEXPORT jlong Java_org_apache_spark_sql_execution_datasources_CHDatasourceJniW const auto task_id = jstring2string(env, task_id_); const auto partition_dir = jstring2string(env, partition_dir_); const auto bucket_dir = jstring2string(env, bucket_dir_); - - const auto plan_a = local_engine::getByteArrayElementsSafe(env, plan_); - auto plan_ptr = local_engine::BinaryToMessage( - {reinterpret_cast(plan_a.elems()), static_cast(plan_a.length())}); + auto uuid = uuid_str + "_" + task_id; + local_engine::SparkMergeTreeWritePartitionSettings settings{ + .part_name_prefix{uuid}, .partition_dir{partition_dir}, .bucket_dir{bucket_dir}}; + settings.set(query_context); const auto split_info_a = local_engine::getByteArrayElementsSafe(env, split_info_); auto extension_table = local_engine::BinaryToMessage( {reinterpret_cast(split_info_a.elems()), static_cast(split_info_a.length())}); - - auto merge_tree_table = local_engine::MergeTreeRelParser::parseMergeTreeTable(extension_table); - auto uuid = uuid_str + "_" + task_id; - auto * writer = new local_engine::SparkMergeTreeWriter(merge_tree_table, query_context, uuid, partition_dir, bucket_dir); + local_engine::MergeTreeTableInstance merge_tree_table(extension_table); + auto * writer = local_engine::SparkMergeTreeWriter::create(merge_tree_table, settings, query_context).release(); return reinterpret_cast(writer); LOCAL_ENGINE_JNI_METHOD_END(env, 0) @@ -944,8 +945,8 @@ JNIEXPORT void Java_org_apache_spark_sql_execution_datasources_CHDatasourceJniWr { LOCAL_ENGINE_JNI_METHOD_START auto * writer = reinterpret_cast(instanceId); + SCOPE_EXIT({ delete writer; }); writer->close(); - delete writer; LOCAL_ENGINE_JNI_METHOD_END(env, ) } @@ -954,7 +955,7 @@ JNIEXPORT void Java_org_apache_spark_sql_execution_datasources_CHDatasourceJniWr { LOCAL_ENGINE_JNI_METHOD_START auto * writer = reinterpret_cast(instanceId); - auto * block = reinterpret_cast(block_address); + const auto * block = reinterpret_cast(block_address); writer->write(*block); LOCAL_ENGINE_JNI_METHOD_END(env, ) } @@ -964,10 +965,11 @@ Java_org_apache_spark_sql_execution_datasources_CHDatasourceJniWrapper_closeMerg { LOCAL_ENGINE_JNI_METHOD_START auto * writer = reinterpret_cast(instanceId); + SCOPE_EXIT({ delete writer; }); + writer->finalize(); - auto part_infos = writer->getAllPartInfo(); - auto json_info = local_engine::SparkMergeTreeWriter::partInfosToJson(part_infos); - delete writer; + const auto part_infos = writer->getAllPartInfo(); + const auto json_info = local_engine::SparkMergeTreeWriter::partInfosToJson(part_infos); return local_engine::charTojstring(env, json_info.c_str()); LOCAL_ENGINE_JNI_METHOD_END(env, nullptr) } @@ -985,45 +987,36 @@ JNIEXPORT jstring Java_org_apache_spark_sql_execution_datasources_CHDatasourceJn LOCAL_ENGINE_JNI_METHOD_START const auto uuid_str = jstring2string(env, uuid_); - const auto task_id = jstring2string(env, task_id_); + const auto partition_dir = jstring2string(env, partition_dir_); const auto bucket_dir = jstring2string(env, bucket_dir_); - const auto plan_a = local_engine::getByteArrayElementsSafe(env, plan_); - auto plan_ptr = local_engine::BinaryToMessage( - {reinterpret_cast(plan_a.elems()), static_cast(plan_a.length())}); - const auto split_info_a = local_engine::getByteArrayElementsSafe(env, split_info_); auto extension_table = local_engine::BinaryToMessage( {reinterpret_cast(split_info_a.elems()), static_cast(split_info_a.length())}); - google::protobuf::StringValue table; - table.ParseFromString(extension_table.detail().value()); - auto merge_tree_table = local_engine::parseMergeTreeTableString(table.value()); - + local_engine::MergeTreeTableInstance merge_tree_table(extension_table); auto context = local_engine::QueryContextManager::instance().currentQueryContext(); // each task using its own CustomStorageMergeTree, don't reuse - auto temp_storage - = local_engine::MergeTreeRelParser::copyToVirtualStorage(merge_tree_table, context); + auto temp_storage = merge_tree_table.copyToVirtualStorage(context); // prefetch all needed parts metadata before merge local_engine::restoreMetaData(temp_storage, merge_tree_table, *context); - local_engine::TempStorageFreer freer{temp_storage->getStorageID()}; // to release temp CustomStorageMergeTree with RAII + // to release temp CustomStorageMergeTree with RAII + DB::StorageID storage_id = temp_storage->getStorageID(); + SCOPE_EXIT({ local_engine::StorageMergeTreeFactory::freeStorage(storage_id); }); + std::vector selected_parts = local_engine::StorageMergeTreeFactory::instance().getDataPartsByNames( temp_storage->getStorageID(), "", merge_tree_table.getPartNames()); std::unordered_map partition_values; std::vector loaded - = local_engine::mergeParts(selected_parts, partition_values, uuid_str, temp_storage, partition_dir, bucket_dir); + = local_engine::mergeParts(selected_parts, partition_values, uuid_str, *temp_storage, partition_dir, bucket_dir); std::vector res; for (auto & partPtr : loaded) { - saveFileStatus( - *temp_storage, - context, - partPtr->name, - const_cast(partPtr->getDataPartStorage())); + saveFileStatus(*temp_storage, context, partPtr->name, const_cast(partPtr->getDataPartStorage())); res.emplace_back(local_engine::PartInfo{ partPtr->name, partPtr->getMarksCount(), partPtr->getBytesOnDisk(), partPtr->rows_count, partition_values, bucket_dir}); } @@ -1085,7 +1078,15 @@ JNIEXPORT jlong Java_org_apache_gluten_vectorized_StorageJoinBuilder_nativeBuild DB::CompressedReadBuffer input(read_buffer_from_java_array); local_engine::configureCompressedReadBuffer(input); const auto * obj = make_wrapper(local_engine::BroadCastJoinBuilder::buildJoin( - hash_table_id, input, row_count_, join_key, join_type_, has_mixed_join_condition, is_existence_join, struct_string, is_null_aware_anti_join)); + hash_table_id, + input, + row_count_, + join_key, + join_type_, + has_mixed_join_condition, + is_existence_join, + struct_string, + is_null_aware_anti_join)); return obj->instance(); LOCAL_ENGINE_JNI_METHOD_END(env, 0) } @@ -1171,7 +1172,14 @@ JNIEXPORT jint Java_org_apache_gluten_vectorized_BlockSplitIterator_nativeNextPa } JNIEXPORT jlong Java_org_apache_gluten_vectorized_BlockOutputStream_nativeCreate( - JNIEnv * env, jobject, jobject output_stream, jbyteArray buffer, jstring codec, jint level, jboolean compressed, jint customize_buffer_size) + JNIEnv * env, + jobject, + jobject output_stream, + jbyteArray buffer, + jstring codec, + jint level, + jboolean compressed, + jint customize_buffer_size) { LOCAL_ENGINE_JNI_METHOD_START local_engine::ShuffleWriter * writer @@ -1275,8 +1283,8 @@ JNIEXPORT void Java_org_apache_gluten_utils_TestExceptionUtils_generateNativeExc } - -JNIEXPORT jstring Java_org_apache_gluten_execution_CHNativeCacheManager_nativeCacheParts(JNIEnv * env, jobject, jstring table_, jstring columns_) +JNIEXPORT jstring +Java_org_apache_gluten_execution_CHNativeCacheManager_nativeCacheParts(JNIEnv * env, jobject, jstring table_, jstring columns_) { LOCAL_ENGINE_JNI_METHOD_START auto table_def = jstring2string(env, table_); @@ -1284,10 +1292,9 @@ JNIEXPORT jstring Java_org_apache_gluten_execution_CHNativeCacheManager_nativeCa Poco::StringTokenizer tokenizer(columns, ","); std::unordered_set column_set; for (const auto & col : tokenizer) - { column_set.insert(col); - } - auto id = local_engine::CacheManager::instance().cacheParts(table_def, column_set); + local_engine::MergeTreeTableInstance table(table_def); + auto id = local_engine::CacheManager::instance().cacheParts(table, column_set); return local_engine::charTojstring(env, id.c_str()); LOCAL_ENGINE_JNI_METHOD_END(env, nullptr); } diff --git a/cpp-ch/local-engine/tests/benchmark_local_engine.cpp b/cpp-ch/local-engine/tests/benchmark_local_engine.cpp index 839f3ea529c1..bc3e95c2dadc 100644 --- a/cpp-ch/local-engine/tests/benchmark_local_engine.cpp +++ b/cpp-ch/local-engine/tests/benchmark_local_engine.cpp @@ -35,8 +35,9 @@ #include #include #include -#include #include +#include +#include #include #include #include @@ -45,7 +46,6 @@ #include #include #include -#include #include #include #include @@ -787,7 +787,7 @@ class FasterCompressedReadBuffer : public FasterCompressedReadBufferBase, public struct MergeTreeWithSnapshot { - std::shared_ptr merge_tree; + std::shared_ptr merge_tree; std::shared_ptr snapshot; NamesAndTypesList columns; }; diff --git a/cpp-ch/local-engine/tests/benchmark_spark_divide_function.cpp b/cpp-ch/local-engine/tests/benchmark_spark_divide_function.cpp index 7f1a7309e7d4..2a824174bcf5 100644 --- a/cpp-ch/local-engine/tests/benchmark_spark_divide_function.cpp +++ b/cpp-ch/local-engine/tests/benchmark_spark_divide_function.cpp @@ -14,22 +14,16 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -#include -#include #include #include -#include #include +#include #include #include +#include #include -#include -#include -#include #include #include -#include -#include #include using namespace DB; @@ -66,7 +60,8 @@ static std::string join(const ActionsDAG::NodeRawConstPtrs & v, char c) return res; } -static const ActionsDAG::Node * addFunction(ActionsDAG & actions_dag, const String & function, const DB::ActionsDAG::NodeRawConstPtrs & args) +static const ActionsDAG::Node * +addFunction(ActionsDAG & actions_dag, const String & function, const DB::ActionsDAG::NodeRawConstPtrs & args) { auto function_builder = FunctionFactory::instance().get(function, local_engine::SerializedPlanParser::global_context); std::string args_name = join(args, ','); diff --git a/cpp-ch/local-engine/tests/decmial_filter_push_down/18_2.json b/cpp-ch/local-engine/tests/decimal_filter_push_down/18_2.json similarity index 100% rename from cpp-ch/local-engine/tests/decmial_filter_push_down/18_2.json rename to cpp-ch/local-engine/tests/decimal_filter_push_down/18_2.json diff --git a/cpp-ch/local-engine/tests/decmial_filter_push_down/18_2_flba.snappy.parquet b/cpp-ch/local-engine/tests/decimal_filter_push_down/18_2_flba.snappy.parquet similarity index 100% rename from cpp-ch/local-engine/tests/decmial_filter_push_down/18_2_flba.snappy.parquet rename to cpp-ch/local-engine/tests/decimal_filter_push_down/18_2_flba.snappy.parquet diff --git a/cpp-ch/local-engine/tests/gluten_test_util.cpp b/cpp-ch/local-engine/tests/gluten_test_util.cpp index 2d558ebe4744..7dbc7206dbb7 100644 --- a/cpp-ch/local-engine/tests/gluten_test_util.cpp +++ b/cpp-ch/local-engine/tests/gluten_test_util.cpp @@ -21,13 +21,14 @@ #include #include #include - #include #include +#include #include #include #include #include +#include #include #include @@ -73,6 +74,16 @@ std::optional parseFilter(const std::string & filter, const AnotherR return ActionsDAG::buildFilterActionsDAG({visitor_data.getActions().getOutputs().back()}, node_name_to_input_column); } +std::pair> create_plan_and_executor( + std::string_view json_plan, std::string_view split_template, std::string_view file, const std::optional & context) +{ + const std::string split = replaceLocalFilesWildcards(split_template, file); + SerializedPlanParser parser(context.value_or(SerializedPlanParser::global_context)); + parser.addSplitInfo(local_engine::JsonStringToBinary(split)); + const auto plan = local_engine::JsonStringToMessage(json_plan); + return {plan, parser.createExecutor(plan)}; +} + const char * get_data_dir() { const auto * const result = std::getenv("PARQUET_TEST_DATA"); diff --git a/cpp-ch/local-engine/tests/gluten_test_util.h b/cpp-ch/local-engine/tests/gluten_test_util.h index 996b27bf884d..9f7380cf5446 100644 --- a/cpp-ch/local-engine/tests/gluten_test_util.h +++ b/cpp-ch/local-engine/tests/gluten_test_util.h @@ -21,13 +21,16 @@ #include #include #include -#include -#include -#include + #include +#include #include #include +namespace local_engine +{ +class LocalExecutor; +} using BlockRowType = DB::ColumnsWithTypeAndName; using BlockFieldType = DB::ColumnWithTypeAndName; using AnotherRowType = DB::NamesAndTypesList; @@ -65,12 +68,18 @@ AnotherRowType readParquetSchema(const std::string & file); std::optional parseFilter(const std::string & filter, const AnotherRowType & name_and_types); +std::pair> create_plan_and_executor( + std::string_view json_plan, + std::string_view split_template, + std::string_view file, + const std::optional & context = std::nullopt); + } -inline std::string replaceLocalFilesWildcards(const String & haystack, const String & replaced) +inline std::string replaceLocalFilesWildcards(const std::string_view haystack, const std::string_view replaced) { - static constexpr auto _WILDCARD_ = "{replace_local_files}"; - return boost::replace_all_copy(haystack, _WILDCARD_, replaced); + static constexpr auto wildcard = "{replace_local_files}"; + return boost::replace_all_copy(std::string{haystack}, wildcard, replaced); } inline BlockFieldType toBlockFieldType(const AnotherFieldType & type) @@ -122,4 +131,10 @@ inline parquet::ByteArray ByteArrayFromString(const std::string & s) { const auto * const ptr = reinterpret_cast(s.data()); return parquet::ByteArray(static_cast(s.size()), ptr); -} \ No newline at end of file +} + +#define EMBEDDED_PLAN(res) \ + std::string_view \ + { \ + reinterpret_cast(g##res##Data), g##res##Size \ + } \ No newline at end of file diff --git a/cpp-ch/local-engine/tests/gtest_ch_join.cpp b/cpp-ch/local-engine/tests/gtest_ch_join.cpp index e130d82b9c7c..67775dd1a4a6 100644 --- a/cpp-ch/local-engine/tests/gtest_ch_join.cpp +++ b/cpp-ch/local-engine/tests/gtest_ch_join.cpp @@ -26,10 +26,10 @@ #include #include +#include #include #include #include -#include #include #include diff --git a/cpp-ch/local-engine/tests/gtest_ch_storages.cpp b/cpp-ch/local-engine/tests/gtest_ch_storages.cpp index 7de21a26df72..a32e7c476dcb 100644 --- a/cpp-ch/local-engine/tests/gtest_ch_storages.cpp +++ b/cpp-ch/local-engine/tests/gtest_ch_storages.cpp @@ -17,16 +17,15 @@ #include #include #include -#include #include #include +#include #include #include #include #include #include #include -#include using namespace DB; using namespace local_engine; diff --git a/cpp-ch/local-engine/tests/gtest_clickhouse_pr_verify.cpp b/cpp-ch/local-engine/tests/gtest_clickhouse_pr_verify.cpp index 5b5797ed7d21..1e2525b33a5b 100644 --- a/cpp-ch/local-engine/tests/gtest_clickhouse_pr_verify.cpp +++ b/cpp-ch/local-engine/tests/gtest_clickhouse_pr_verify.cpp @@ -28,8 +28,7 @@ using namespace local_engine; using namespace DB; // Plan for https://github.com/ClickHouse/ClickHouse/pull/54881 -INCBIN(resource_embedded_pr_54881_json, SOURCE_DIR "/utils/extern-local-engine/tests/json/clickhouse_pr_54881.json"); - +INCBIN(_pr_54881_, SOURCE_DIR "/utils/extern-local-engine/tests/json/clickhouse_pr_54881.json"); TEST(Clickhouse, PR54881) { const auto context1 = DB::Context::createCopy(SerializedPlanParser::global_context); @@ -37,18 +36,10 @@ TEST(Clickhouse, PR54881) auto settings = context1->getSettingsRef(); EXPECT_FALSE(settings.enable_named_columns_in_function_tuple) << "GLUTEN NEED set enable_named_columns_in_function_tuple to false"; - const std::string split_template + constexpr std::string_view split_template = R"({"items":[{"uriFile":"{replace_local_files}","partitionIndex":"0","length":"1529","parquet":{},"schema":{},"metadataColumns":[{}]}]})"; - const std::string split - = replaceLocalFilesWildcards(split_template, GLUTEN_DATA_DIR("/utils/extern-local-engine/tests/data/54881.snappy.parquet")); - - SerializedPlanParser parser(context1); - parser.addSplitInfo(local_engine::JsonStringToBinary(split)); - - const auto plan = local_engine::JsonStringToMessage( - {reinterpret_cast(gresource_embedded_pr_54881_jsonData), gresource_embedded_pr_54881_jsonSize}); - - auto local_executor = parser.createExecutor(plan); + constexpr std::string_view file{GLUTEN_DATA_DIR("/utils/extern-local-engine/tests/data/54881.snappy.parquet")}; + auto [_, local_executor] = test::create_plan_and_executor(EMBEDDED_PLAN(_pr_54881_), split_template, file, context1); EXPECT_TRUE(local_executor->hasNext()); const Block & block = *local_executor->nextColumnar(); @@ -86,53 +77,36 @@ TEST(Clickhouse, PR54881) } // Plan for https://github.com/ClickHouse/ClickHouse/pull/65234 -INCBIN(resource_embedded_pr_65234_json, SOURCE_DIR "/utils/extern-local-engine/tests/json/clickhouse_pr_65234.json"); - +INCBIN(_pr_65234_, SOURCE_DIR "/utils/extern-local-engine/tests/json/clickhouse_pr_65234.json"); TEST(Clickhouse, PR65234) { const std::string split = R"({"items":[{"uriFile":"file:///foo","length":"84633","parquet":{},"schema":{},"metadataColumns":[{}]}]})"; SerializedPlanParser parser(SerializedPlanParser::global_context); parser.addSplitInfo(local_engine::JsonStringToBinary(split)); - const auto plan = local_engine::JsonStringToMessage( - {reinterpret_cast(gresource_embedded_pr_65234_jsonData), gresource_embedded_pr_65234_jsonSize}); + const auto plan = local_engine::JsonStringToMessage(EMBEDDED_PLAN(_pr_65234_)); auto query_plan = parser.parse(plan); } -INCBIN(resource_embedded_pr_68135_json, SOURCE_DIR "/utils/extern-local-engine/tests/json/clickhouse_pr_68135.json"); +INCBIN(_pr_68135_, SOURCE_DIR "/utils/extern-local-engine/tests/json/clickhouse_pr_68135.json"); TEST(Clickhouse, PR68135) { - const std::string split_template + constexpr std::string_view split_template = R"({"items":[{"uriFile":"{replace_local_files}","partitionIndex":"0","length":"461","parquet":{},"schema":{},"metadataColumns":[{}]}]})"; - const std::string split - = replaceLocalFilesWildcards(split_template, GLUTEN_DATA_DIR("/utils/extern-local-engine/tests/data/68135.snappy.parquet")); + constexpr std::string_view file{GLUTEN_DATA_DIR("/utils/extern-local-engine/tests/data/68135.snappy.parquet")}; + auto [_, local_executor] = test::create_plan_and_executor(EMBEDDED_PLAN(_pr_68135_), split_template, file); - SerializedPlanParser parser(SerializedPlanParser::global_context); - parser.addSplitInfo(local_engine::JsonStringToBinary(split)); - - const auto plan = local_engine::JsonStringToMessage( - {reinterpret_cast(gresource_embedded_pr_68135_jsonData), gresource_embedded_pr_68135_jsonSize}); - - auto local_executor = parser.createExecutor(plan); EXPECT_TRUE(local_executor->hasNext()); const Block & x = *local_executor->nextColumnar(); debug::headBlock(x); } -INCBIN(resource_embedded_pr_68131_json, SOURCE_DIR "/utils/extern-local-engine/tests/json/clickhouse_pr_68131.json"); +INCBIN(_pr_68131_, SOURCE_DIR "/utils/extern-local-engine/tests/json/clickhouse_pr_68131.json"); TEST(Clickhouse, PR68131) { - const std::string split_template + constexpr std::string_view split_template = R"({"items":[{"uriFile":"{replace_local_files}","partitionIndex":"0","length":"289","parquet":{},"schema":{},"metadataColumns":[{}]}]})"; - const std::string split - = replaceLocalFilesWildcards(split_template, GLUTEN_DATA_DIR("/utils/extern-local-engine/tests/data/68131.parquet")); - - SerializedPlanParser parser(SerializedPlanParser::global_context); - parser.addSplitInfo(local_engine::JsonStringToBinary(split)); - - const auto plan = local_engine::JsonStringToMessage( - {reinterpret_cast(gresource_embedded_pr_68131_jsonData), gresource_embedded_pr_68131_jsonSize}); - - auto local_executor = parser.createExecutor(plan); + auto [_, local_executor] = test::create_plan_and_executor( + EMBEDDED_PLAN(_pr_68131_), split_template, GLUTEN_DATA_DIR("/utils/extern-local-engine/tests/data/68131.parquet")); EXPECT_TRUE(local_executor->hasNext()); const Block & x = *local_executor->nextColumnar(); debug::headBlock(x); diff --git a/cpp-ch/local-engine/tests/gtest_local_engine.cpp b/cpp-ch/local-engine/tests/gtest_local_engine.cpp index 86cea39e5eb4..558f0df7a764 100644 --- a/cpp-ch/local-engine/tests/gtest_local_engine.cpp +++ b/cpp-ch/local-engine/tests/gtest_local_engine.cpp @@ -29,8 +29,8 @@ #include #include #include -#include #include +#include #include #include #include diff --git a/cpp-ch/local-engine/tests/gtest_parquet_columnindex_bug.cpp b/cpp-ch/local-engine/tests/gtest_parquet_columnindex_bug.cpp index ee6e70305b27..4436bf0cd0ac 100644 --- a/cpp-ch/local-engine/tests/gtest_parquet_columnindex_bug.cpp +++ b/cpp-ch/local-engine/tests/gtest_parquet_columnindex_bug.cpp @@ -28,27 +28,19 @@ using namespace local_engine; using namespace DB; -INCBIN(resource_embedded_pr_18_2_json, SOURCE_DIR "/utils/extern-local-engine/tests/decmial_filter_push_down/18_2.json"); -TEST(ColumnIndex, Deciaml182) +INCBIN(_pr_18_2, SOURCE_DIR "/utils/extern-local-engine/tests/decimal_filter_push_down/18_2.json"); +TEST(ColumnIndex, Decimal182) { // [precision,scale] = [18,2] const auto context1 = DB::Context::createCopy(SerializedPlanParser::global_context); - - auto config = ExecutorConfig::loadFromContext(context1); + const auto config = ExecutorConfig::loadFromContext(context1); EXPECT_TRUE(config.use_local_format) << "gtest need set use_local_format to true"; - const std::string split_template + constexpr std::string_view split_template = R"({"items":[{"uriFile":"{replace_local_files}","partitionIndex":"0","length":"488","parquet":{},"schema":{},"metadataColumns":[{}]}]})"; - const std::string split = replaceLocalFilesWildcards( - split_template, GLUTEN_DATA_DIR("/utils/extern-local-engine/tests/decmial_filter_push_down/18_2_flba.snappy.parquet")); - - SerializedPlanParser parser(context1); - parser.addSplitInfo(local_engine::JsonStringToBinary(split)); - - const auto plan = local_engine::JsonStringToMessage( - {reinterpret_cast(gresource_embedded_pr_18_2_jsonData), gresource_embedded_pr_18_2_jsonSize}); + constexpr std::string_view file{GLUTEN_DATA_DIR("/utils/extern-local-engine/tests/decimal_filter_push_down/18_2_flba.snappy.parquet")}; + auto [_, local_executor] = test::create_plan_and_executor(EMBEDDED_PLAN(_pr_18_2), split_template, file, context1); - auto local_executor = parser.createExecutor(plan); EXPECT_TRUE(local_executor->hasNext()); const Block & x = *local_executor->nextColumnar(); debug::headBlock(x); diff --git a/cpp-ch/local-engine/tests/gtest_parser.cpp b/cpp-ch/local-engine/tests/gtest_parser.cpp index 135f81a9149e..bf52bd54ccee 100644 --- a/cpp-ch/local-engine/tests/gtest_parser.cpp +++ b/cpp-ch/local-engine/tests/gtest_parser.cpp @@ -32,21 +32,21 @@ using namespace local_engine; using namespace DB; -INCBIN(resource_embedded_readcsv_json, SOURCE_DIR "/utils/extern-local-engine/tests/json/read_student_option_schema.csv.json"); +INCBIN(_readcsv_plan, SOURCE_DIR "/utils/extern-local-engine/tests/json/read_student_option_schema.csv.json"); TEST(LocalExecutor, ReadCSV) { - const std::string split_template + constexpr std::string_view split_template = R"({"items":[{"uriFile":"{replace_local_files}","length":"56","text":{"fieldDelimiter":",","maxBlockSize":"8192","header":"1"},"schema":{"names":["id","name","language"],"struct":{"types":[{"string":{"nullability":"NULLABILITY_NULLABLE"}},{"string":{"nullability":"NULLABILITY_NULLABLE"}},{"string":{"nullability":"NULLABILITY_NULLABLE"}}]}},"metadataColumns":[{}]}]})"; const std::string split = replaceLocalFilesWildcards( split_template, GLUTEN_SOURCE_DIR("/backends-velox/src/test/resources/datasource/csv/student_option_schema.csv")); SerializedPlanParser parser(SerializedPlanParser::global_context); parser.addSplitInfo(local_engine::JsonStringToBinary(split)); - auto plan = local_engine::JsonStringToMessage( - {reinterpret_cast(gresource_embedded_readcsv_jsonData), gresource_embedded_readcsv_jsonSize}); + auto plan = local_engine::JsonStringToMessage(EMBEDDED_PLAN(_readcsv_plan)); auto query_plan = parser.parse(plan); const auto pipeline = parser.buildQueryPipeline(*query_plan); LocalExecutor local_executor{std::move(query_plan), QueryPipelineBuilder::getPipeline(std::move(*pipeline))}; + EXPECT_TRUE(local_executor.hasNext()); const Block & x = *local_executor.nextColumnar(); EXPECT_EQ(4, x.rows()); @@ -56,12 +56,10 @@ size_t count(const substrait::Type_Struct & type) { size_t ret = 0; for (const auto & t : type.types()) - { if (t.has_struct_()) ret += 1 + count(t.struct_()); else ret++; - } return ret; } diff --git a/cpp-ch/local-engine/tests/gtest_write_pipeline.cpp b/cpp-ch/local-engine/tests/gtest_write_pipeline.cpp index e56184a1417c..69b6fa0f6d44 100644 --- a/cpp-ch/local-engine/tests/gtest_write_pipeline.cpp +++ b/cpp-ch/local-engine/tests/gtest_write_pipeline.cpp @@ -17,16 +17,25 @@ #include #include + #include #include #include +#include +#include #include #include #include #include #include #include +#include #include +#include +#include +#include +#include +#include #include #include #include @@ -34,7 +43,6 @@ #include #include - using namespace local_engine; using namespace DB; @@ -103,24 +111,20 @@ TEST(LocalExecutor, StorageObjectStorageSink) } -INCBIN(resource_embedded_write_json, SOURCE_DIR "/utils/extern-local-engine/tests/json/native_write_plan.json"); +INCBIN(native_write, SOURCE_DIR "/utils/extern-local-engine/tests/json/native_write_plan.json"); TEST(WritePipeline, SubstraitFileSink) { - const auto tmpdir = std::string{"file:///tmp/test_table/test"}; - const auto filename = std::string{"data.parquet"}; - const std::string split_template - = R"({"items":[{"uriFile":"{replace_local_files}","length":"1399183","text":{"fieldDelimiter":"|","maxBlockSize":"8192"},"schema":{"names":["s_suppkey","s_name","s_address","s_nationkey","s_phone","s_acctbal","s_comment"],"struct":{"types":[{"i64":{"nullability":"NULLABILITY_NULLABLE"}},{"string":{"nullability":"NULLABILITY_NULLABLE"}},{"string":{"nullability":"NULLABILITY_NULLABLE"}},{"i64":{"nullability":"NULLABILITY_NULLABLE"}},{"string":{"nullability":"NULLABILITY_NULLABLE"}},{"decimal":{"scale":2,"precision":15,"nullability":"NULLABILITY_NULLABLE"}},{"string":{"nullability":"NULLABILITY_NULLABLE"}}]}},"metadataColumns":[{}]}]})"; - const std::string split - = replaceLocalFilesWildcards(split_template, GLUTEN_SOURCE_DIR("/backends-clickhouse/src/test/resources/csv-data/supplier.csv")); - const auto context = DB::Context::createCopy(SerializedPlanParser::global_context); - context->setSetting(local_engine::SPARK_TASK_WRITE_TMEP_DIR, tmpdir); - context->setSetting(local_engine::SPARK_TASK_WRITE_FILENAME, filename); - SerializedPlanParser parser(context); - parser.addSplitInfo(local_engine::JsonStringToBinary(split)); + GlutenWriteSettings settings{ + .task_write_tmp_dir = "file:///tmp/test_table/test", + .task_write_filename = "data.parquet", + }; + settings.set(context); - const auto plan = local_engine::JsonStringToMessage( - {reinterpret_cast(gresource_embedded_write_jsonData), gresource_embedded_write_jsonSize}); + constexpr std::string_view split_template + = R"({"items":[{"uriFile":"{replace_local_files}","length":"1399183","text":{"fieldDelimiter":"|","maxBlockSize":"8192"},"schema":{"names":["s_suppkey","s_name","s_address","s_nationkey","s_phone","s_acctbal","s_comment"],"struct":{"types":[{"i64":{"nullability":"NULLABILITY_NULLABLE"}},{"string":{"nullability":"NULLABILITY_NULLABLE"}},{"string":{"nullability":"NULLABILITY_NULLABLE"}},{"i64":{"nullability":"NULLABILITY_NULLABLE"}},{"string":{"nullability":"NULLABILITY_NULLABLE"}},{"decimal":{"scale":2,"precision":15,"nullability":"NULLABILITY_NULLABLE"}},{"string":{"nullability":"NULLABILITY_NULLABLE"}}]}},"metadataColumns":[{}]}]})"; + constexpr std::string_view file{GLUTEN_SOURCE_DIR("/backends-clickhouse/src/test/resources/csv-data/supplier.csv")}; + auto [plan, local_executor] = test::create_plan_and_executor(EMBEDDED_PLAN(native_write), split_template, file, context); EXPECT_EQ(1, plan.relations_size()); const substrait::PlanRel & root_rel = plan.relations().at(0); @@ -139,7 +143,6 @@ TEST(WritePipeline, SubstraitFileSink) EXPECT_EQ("parquet", config["format"]); EXPECT_EQ("1", config["isSnappy"]); - EXPECT_TRUE(write_rel.has_table_schema()); const substrait::NamedStruct & table_schema = write_rel.table_schema(); auto block = TypeParser::buildBlockFromNamedStruct(table_schema); @@ -151,37 +154,33 @@ TEST(WritePipeline, SubstraitFileSink) DB::Names expected_partition_cols; EXPECT_EQ(expected_partition_cols, partitionCols); - - auto local_executor = parser.createExecutor(plan); EXPECT_TRUE(local_executor->hasNext()); const Block & x = *local_executor->nextColumnar(); debug::headBlock(x); EXPECT_EQ(1, x.rows()); const auto & col_a = *(x.getColumns()[0]); - EXPECT_EQ(filename, col_a.getDataAt(0)); + EXPECT_EQ(settings.task_write_filename, col_a.getDataAt(0)); const auto & col_b = *(x.getColumns()[1]); EXPECT_EQ(SubstraitFileSink::NO_PARTITION_ID, col_b.getDataAt(0)); const auto & col_c = *(x.getColumns()[2]); EXPECT_EQ(10000, col_c.getInt(0)); } -INCBIN(resource_embedded_write_one_partition_json, SOURCE_DIR "/utils/extern-local-engine/tests/json/native_write_one_partition.json"); +INCBIN(native_write_one_partition, SOURCE_DIR "/utils/extern-local-engine/tests/json/native_write_one_partition.json"); TEST(WritePipeline, SubstraitPartitionedFileSink) { - const std::string split_template - = R"({"items":[{"uriFile":"{replace_local_files}","length":"1399183","text":{"fieldDelimiter":"|","maxBlockSize":"8192"},"schema":{"names":["s_suppkey","s_name","s_address","s_nationkey","s_phone","s_acctbal","s_comment"],"struct":{"types":[{"i64":{"nullability":"NULLABILITY_NULLABLE"}},{"string":{"nullability":"NULLABILITY_NULLABLE"}},{"string":{"nullability":"NULLABILITY_NULLABLE"}},{"i64":{"nullability":"NULLABILITY_NULLABLE"}},{"string":{"nullability":"NULLABILITY_NULLABLE"}},{"decimal":{"scale":2,"precision":15,"nullability":"NULLABILITY_NULLABLE"}},{"string":{"nullability":"NULLABILITY_NULLABLE"}}]}},"metadataColumns":[{}]}]})"; - const std::string split - = replaceLocalFilesWildcards(split_template, GLUTEN_SOURCE_DIR("/backends-clickhouse/src/test/resources/csv-data/supplier.csv")); - const auto context = DB::Context::createCopy(SerializedPlanParser::global_context); - context->setSetting(local_engine::SPARK_TASK_WRITE_TMEP_DIR, std::string{"file:///tmp/test_table/test_partition"}); - context->setSetting(local_engine::SPARK_TASK_WRITE_FILENAME, std::string{"data.parquet"}); - SerializedPlanParser parser(context); - parser.addSplitInfo(local_engine::JsonStringToBinary(split)); + GlutenWriteSettings settings{ + .task_write_tmp_dir = "file:///tmp/test_table/test_partition", + .task_write_filename = "data.parquet", + }; + settings.set(context); - const auto plan = local_engine::JsonStringToMessage( - {reinterpret_cast(gresource_embedded_write_one_partition_jsonData), gresource_embedded_write_one_partition_jsonSize}); + constexpr std::string_view split_template + = R"({"items":[{"uriFile":"{replace_local_files}","length":"1399183","text":{"fieldDelimiter":"|","maxBlockSize":"8192"},"schema":{"names":["s_suppkey","s_name","s_address","s_nationkey","s_phone","s_acctbal","s_comment"],"struct":{"types":[{"i64":{"nullability":"NULLABILITY_NULLABLE"}},{"string":{"nullability":"NULLABILITY_NULLABLE"}},{"string":{"nullability":"NULLABILITY_NULLABLE"}},{"i64":{"nullability":"NULLABILITY_NULLABLE"}},{"string":{"nullability":"NULLABILITY_NULLABLE"}},{"decimal":{"scale":2,"precision":15,"nullability":"NULLABILITY_NULLABLE"}},{"string":{"nullability":"NULLABILITY_NULLABLE"}}]}},"metadataColumns":[{}]}]})"; + constexpr std::string_view file{GLUTEN_SOURCE_DIR("/backends-clickhouse/src/test/resources/csv-data/supplier.csv")}; + auto [plan, local_executor] = test::create_plan_and_executor(EMBEDDED_PLAN(native_write_one_partition), split_template, file, context); EXPECT_EQ(1, plan.relations_size()); const substrait::PlanRel & root_rel = plan.relations().at(0); @@ -212,7 +211,6 @@ TEST(WritePipeline, SubstraitPartitionedFileSink) DB::Names expected_partition_cols{"s_nationkey"}; EXPECT_EQ(expected_partition_cols, partitionCols); - auto local_executor = parser.createExecutor(plan); EXPECT_TRUE(local_executor->hasNext()); const Block & x = *local_executor->nextColumnar(); debug::headBlock(x, 25); @@ -250,4 +248,197 @@ TEST(WritePipeline, ComputePartitionedExpression) EXPECT_EQ("s_nationkey=1/name=one", partition_by_result_column->getDataAt(0)); EXPECT_EQ("s_nationkey=2/name=two", partition_by_result_column->getDataAt(1)); EXPECT_EQ("s_nationkey=3/name=three", partition_by_result_column->getDataAt(2)); +} + +void do_remove(const std::string & folder) +{ + namespace fs = std::filesystem; + if (const std::filesystem::path ph(folder); fs::exists(ph)) + fs::remove_all(ph); +} + +Chunk person_chunk() +{ + auto id = INT()->createColumn(); + id->insert(100); + id->insert(200); + id->insert(300); + id->insert(400); + id->insert(500); + id->insert(600); + id->insert(700); + + auto name = STRING()->createColumn(); + name->insert("Joe"); + name->insert("Marry"); + name->insert("Mike"); + name->insert("Fred"); + name->insert("Albert"); + name->insert("Michelle"); + name->insert("Dan"); + + auto age = makeNullable(INT())->createColumn(); + Field null_field; + age->insert(30); + age->insert(null_field); + age->insert(18); + age->insert(50); + age->insert(null_field); + age->insert(30); + age->insert(50); + + + MutableColumns x; + x.push_back(std::move(id)); + x.push_back(std::move(name)); + x.push_back(std::move(age)); + return {std::move(x), 7}; +} + +TEST(WritePipeline, MergeTree) +{ + ThreadStatus thread_status; + + const auto context = DB::Context::createCopy(SerializedPlanParser::global_context); + context->setPath("./"); + const Settings & settings = context->getSettingsRef(); + + const std::string query + = R"(create table if not exists person (id Int32, Name String, Age Nullable(Int32)) engine = MergeTree() ORDER BY id)"; + + const char * begin = query.data(); + const char * end = query.data() + query.size(); + ParserQuery parser(end, settings.allow_settings_after_format_in_insert); + + ASTPtr ast = parseQuery(parser, begin, end, "", settings.max_query_size, settings.max_parser_depth, settings.max_parser_backtracks); + + EXPECT_TRUE(ast->as()); + auto & create = ast->as(); + + ColumnsDescription column_descriptions + = InterpreterCreateQuery::getColumnsDescription(*create.columns_list->columns, context, LoadingStrictnessLevel::CREATE); + + StorageInMemoryMetadata metadata; + metadata.setColumns(column_descriptions); + metadata.setComment("args.comment"); + ASTPtr partition_by_key; + metadata.partition_key = KeyDescription::getKeyFromAST(partition_by_key, metadata.columns, context); + + MergeTreeData::MergingParams merging_params; + merging_params.mode = MergeTreeData::MergingParams::Ordinary; + + + /// This merging param maybe used as part of sorting key + std::optional merging_param_key_arg; + /// Get sorting key from engine arguments. + /// + /// NOTE: store merging_param_key_arg as additional key column. We do it + /// before storage creation. After that storage will just copy this + /// column if sorting key will be changed. + metadata.sorting_key + = KeyDescription::getSortingKeyFromAST(create.storage->order_by->ptr(), metadata.columns, context, merging_param_key_arg); + + std::unique_ptr storage_settings = std::make_unique(context->getMergeTreeSettings()); + + UUID uuid; + UUIDHelpers::getHighBytes(uuid) = 0xffffffffffff0fffull | 0x0000000000004000ull; + UUIDHelpers::getLowBytes(uuid) = 0x3fffffffffffffffull | 0x8000000000000000ull; + + SCOPE_EXIT({ do_remove("WritePipeline_MergeTree"); }); + + auto merge_tree = std::make_shared( + StorageID("", "", uuid), + "WritePipeline_MergeTree", + metadata, + LoadingStrictnessLevel::CREATE, + context, + "", + merging_params, + std::move(storage_settings)); + + Block header{{INT(), "id"}, {STRING(), "Name"}, {makeNullable(INT()), "Age"}}; + DB::Squashing squashing(header, settings.min_insert_block_size_rows, settings.min_insert_block_size_bytes); + squashing.add(person_chunk()); + auto x = Squashing::squash(squashing.flush()); + x.getChunkInfos().add(std::make_shared()); + + ASSERT_EQ(7, x.getNumRows()); + ASSERT_EQ(3, x.getNumColumns()); + + + auto metadata_snapshot = std::make_shared(metadata); + ASTPtr none; + auto sink = std::static_pointer_cast(merge_tree->write(none, metadata_snapshot, context, false)); + + sink->consume(x); + sink->onFinish(); +} + +INCBIN(_1_mergetree_, SOURCE_DIR "/utils/extern-local-engine/tests/json/mergetree/1_mergetree.json"); +INCBIN(_1_mergetree_hdfs_, SOURCE_DIR "/utils/extern-local-engine/tests/json/mergetree/1_mergetree_hdfs.json"); +INCBIN(_1_read_, SOURCE_DIR "/utils/extern-local-engine/tests/json/mergetree/1_plan.json"); + +TEST(WritePipeline, SparkMergeTree) +{ + ThreadStatus thread_status; + + const auto context = DB::Context::createCopy(SerializedPlanParser::global_context); + context->setPath("./"); + const Settings & settings = context->getSettingsRef(); + + const auto extension_table = local_engine::JsonStringToMessage(EMBEDDED_PLAN(_1_mergetree_)); + MergeTreeTableInstance merge_tree_table(extension_table); + + EXPECT_EQ(merge_tree_table.database, "default"); + EXPECT_EQ(merge_tree_table.table, "lineitem_mergetree"); + EXPECT_EQ(merge_tree_table.relative_path, "lineitem_mergetree"); + EXPECT_EQ(merge_tree_table.table_configs.storage_policy, "default"); + + do_remove(merge_tree_table.relative_path); + + const auto dest_storage = merge_tree_table.getStorage(SerializedPlanParser::global_context); + EXPECT_TRUE(dest_storage); + EXPECT_FALSE(dest_storage->getStoragePolicy()->getAnyDisk()->isRemote()); + DB::StorageMetadataPtr metadata_snapshot = dest_storage->getInMemoryMetadataPtr(); + Block header = metadata_snapshot->getSampleBlock(); + + constexpr std::string_view split_template + = R"({"items":[{"uriFile":"{replace_local_files}","length":"19230111","parquet":{},"schema":{},"metadataColumns":[{}],"properties":{"fileSize":"19230111","modificationTime":"1722330598029"}}]})"; + constexpr std::string_view file{GLUTEN_SOURCE_TPCH_DIR("lineitem/part-00000-d08071cb-0dfa-42dc-9198-83cb334ccda3-c000.snappy.parquet")}; + + SparkMergeTreeWritePartitionSettings gm_write_settings{ + .part_name_prefix{"this_is_prefix"}, + }; + gm_write_settings.set(context); + + auto writer = local_engine::SparkMergeTreeWriter::create(merge_tree_table, gm_write_settings, context); + SparkMergeTreeWriter & spark_merge_tree_writer = *writer; + + auto [_, local_executor] = test::create_plan_and_executor(EMBEDDED_PLAN(_1_read_), split_template, file); + EXPECT_TRUE(local_executor->hasNext()); + + do + { + spark_merge_tree_writer.write(*local_executor->nextColumnar()); + } while (local_executor->hasNext()); + + spark_merge_tree_writer.finalize(); + auto part_infos = spark_merge_tree_writer.getAllPartInfo(); + auto json_info = local_engine::SparkMergeTreeWriter::partInfosToJson(part_infos); + std::cerr << json_info << std::endl; + + /// + { + const auto extension_table_hdfs + = local_engine::JsonStringToMessage(EMBEDDED_PLAN(_1_mergetree_hdfs_)); + MergeTreeTableInstance merge_tree_table_hdfs(extension_table_hdfs); + EXPECT_EQ(merge_tree_table_hdfs.database, "default"); + EXPECT_EQ(merge_tree_table_hdfs.table, "lineitem_mergetree_hdfs"); + EXPECT_EQ(merge_tree_table_hdfs.relative_path, "3.5/test/lineitem_mergetree_hdfs"); + EXPECT_EQ(merge_tree_table_hdfs.table_configs.storage_policy, "__hdfs_main"); + + const auto dest_storage_hdfs = merge_tree_table_hdfs.getStorage(SerializedPlanParser::global_context); + EXPECT_TRUE(dest_storage_hdfs); + EXPECT_TRUE(dest_storage_hdfs->getStoragePolicy()->getAnyDisk()->isRemote()); + } } \ No newline at end of file diff --git a/cpp-ch/local-engine/tests/json/gtest_local_engine_config.json b/cpp-ch/local-engine/tests/json/gtest_local_engine_config.json index 8ada07819bb6..ecd86faa549a 100644 --- a/cpp-ch/local-engine/tests/json/gtest_local_engine_config.json +++ b/cpp-ch/local-engine/tests/json/gtest_local_engine_config.json @@ -268,6 +268,94 @@ "value": { "string": "true" } + }, + { + "key": { + "string": "spark.gluten.sql.columnar.backend.ch.runtime_config.storage_configuration.disks.hdfs.type" + }, + "value": { + "string": "hdfs_gluten" + } + }, + { + "key": { + "string": "spark.gluten.sql.columnar.backend.ch.runtime_config.storage_configuration.disks.hdfs_cache.path" + }, + "value": { + "string": "/tmp/hdfs_cache/3.5/" + } + }, + { + "key": { + "string": "spark.gluten.sql.columnar.backend.ch.runtime_config.storage_configuration.disks.hdfs_cache.type" + }, + "value": { + "string": "cache" + } + }, + { + "key": { + "string": "spark.gluten.sql.columnar.backend.ch.runtime_config.storage_configuration.disks.hdfs_cache.max_size" + }, + "value": { + "string": "10Gi" + } + }, + { + "key": { + "string": "spark.gluten.sql.columnar.backend.ch.runtime_config.storage_configuration.policies.__hdfs_main.volumes.main.disk" + }, + "value": { + "string": "hdfs_cache" + } + }, + { + "key": { + "string": "spark.gluten.sql.columnar.backend.ch.runtime_config.hdfs.dfs_client_read_shortcircuit" + }, + "value": { + "string": "false" + } + }, + { + "key": { + "string": "spark.gluten.sql.columnar.backend.ch.runtime_config.hdfs.dfs_default_replica" + }, + "value": { + "string": "1" + } + }, + { + "key": { + "string": "spark.gluten.sql.columnar.backend.ch.runtime_config.storage_configuration.disks.hdfs.metadata_path" + }, + "value": { + "string": "/tmp/metadata/hdfs/3.5/" + } + }, + { + "key": { + "string": "spark.gluten.sql.columnar.backend.ch.runtime_config.storage_configuration.disks.hdfs.endpoint" + }, + "value": { + "string": "hdfs://127.0.0.1:8020/" + } + }, + { + "key": { + "string": "spark.gluten.sql.columnar.backend.ch.runtime_config.storage_configuration.policies.__hdfs_main.volumes" + }, + "value": { + "string": "main" + } + }, + { + "key": { + "string": "spark.gluten.sql.columnar.backend.ch.runtime_config.storage_configuration.disks.hdfs_cache.disk" + }, + "value": { + "string": "hdfs" + } } ] } diff --git a/cpp-ch/local-engine/tests/json/mergetree/1_mergetree.json b/cpp-ch/local-engine/tests/json/mergetree/1_mergetree.json new file mode 100644 index 000000000000..eecb8d7484d2 --- /dev/null +++ b/cpp-ch/local-engine/tests/json/mergetree/1_mergetree.json @@ -0,0 +1,6 @@ +{ + "detail": { + "@type": "type.googleapis.com/google.protobuf.StringValue", + "value": "MergeTree;default\nlineitem_mergetree\n1724764750266_0\n{\"names\":[\"l_orderkey\",\"l_partkey\",\"l_suppkey\",\"l_linenumber\",\"l_quantity\",\"l_extendedprice\",\"l_discount\",\"l_tax\",\"l_returnflag\",\"l_linestatus\",\"l_shipdate\",\"l_commitdate\",\"l_receiptdate\",\"l_shipinstruct\",\"l_shipmode\",\"l_comment\"],\"struct\":{\"types\":[{\"i64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"i64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"i64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"i64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"fp64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"fp64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"fp64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"fp64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"string\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"string\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"date\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"date\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"date\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"string\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"string\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"string\":{\"nullability\":\"NULLABILITY_NULLABLE\"}}]}}\ntuple()\n\n\n\n\nlineitem_mergetree\n\n{\"storage_policy\":\"default\"}\n" + } +} \ No newline at end of file diff --git a/cpp-ch/local-engine/tests/json/mergetree/1_mergetree_hdfs.json b/cpp-ch/local-engine/tests/json/mergetree/1_mergetree_hdfs.json new file mode 100644 index 000000000000..d1ed674596e5 --- /dev/null +++ b/cpp-ch/local-engine/tests/json/mergetree/1_mergetree_hdfs.json @@ -0,0 +1,6 @@ +{ + "detail": { + "@type": "type.googleapis.com/google.protobuf.StringValue", + "value": "MergeTree;default\nlineitem_mergetree_hdfs\n1724766584676_0\n{\"names\":[\"l_orderkey\",\"l_partkey\",\"l_suppkey\",\"l_linenumber\",\"l_quantity\",\"l_extendedprice\",\"l_discount\",\"l_tax\",\"l_returnflag\",\"l_linestatus\",\"l_shipdate\",\"l_commitdate\",\"l_receiptdate\",\"l_shipinstruct\",\"l_shipmode\",\"l_comment\"],\"struct\":{\"types\":[{\"i64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"i64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"i64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"i64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"fp64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"fp64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"fp64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"fp64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"string\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"string\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"date\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"date\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"date\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"string\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"string\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"string\":{\"nullability\":\"NULLABILITY_NULLABLE\"}}]}}\ntuple()\n\n\n\n\n3.5/test/lineitem_mergetree_hdfs\n\n{\"storage_policy\":\"__hdfs_main\"}\n" + } +} \ No newline at end of file diff --git a/cpp-ch/local-engine/tests/json/mergetree/1_plan.json b/cpp-ch/local-engine/tests/json/mergetree/1_plan.json new file mode 100644 index 000000000000..20d542d81784 --- /dev/null +++ b/cpp-ch/local-engine/tests/json/mergetree/1_plan.json @@ -0,0 +1,246 @@ +{ + "relations": [ + { + "root": { + "input": { + "read": { + "common": { + "direct": {} + }, + "baseSchema": { + "names": [ + "l_orderkey", + "l_partkey", + "l_suppkey", + "l_linenumber", + "l_quantity", + "l_extendedprice", + "l_discount", + "l_tax", + "l_returnflag", + "l_linestatus", + "l_shipdate", + "l_commitdate", + "l_receiptdate", + "l_shipinstruct", + "l_shipmode", + "l_comment" + ], + "struct": { + "types": [ + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "fp64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "fp64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "fp64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "fp64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "date": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "date": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "date": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + } + ] + }, + "columnTypes": [ + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL" + ] + }, + "advancedExtension": { + "optimization": { + "@type": "type.googleapis.com/google.protobuf.StringValue", + "value": "isMergeTree=0\n" + } + } + } + }, + "names": [ + "l_orderkey#84", + "l_partkey#85", + "l_suppkey#86", + "l_linenumber#87", + "l_quantity#88", + "l_extendedprice#89", + "l_discount#90", + "l_tax#91", + "l_returnflag#92", + "l_linestatus#93", + "l_shipdate#94", + "l_commitdate#95", + "l_receiptdate#96", + "l_shipinstruct#97", + "l_shipmode#98", + "l_comment#99" + ], + "outputSchema": { + "types": [ + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "fp64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "fp64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "fp64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "fp64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "date": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "date": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "date": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + } + ], + "nullability": "NULLABILITY_REQUIRED" + } + } + } + ] +} \ No newline at end of file diff --git a/cpp-ch/local-engine/tests/testConfig.h.in b/cpp-ch/local-engine/tests/testConfig.h.in index 8dd3c2cb68d9..98740e48e930 100644 --- a/cpp-ch/local-engine/tests/testConfig.h.in +++ b/cpp-ch/local-engine/tests/testConfig.h.in @@ -5,4 +5,5 @@ #define MERGETREE_DATA(file) "@MERGETREE_DATA_DIR@"#file #define GLUTEN_SOURCE_DIR_ "file://@GLUTEN_REAL_PATH@" -#define GLUTEN_SOURCE_DIR(file) GLUTEN_SOURCE_DIR_ file \ No newline at end of file +#define GLUTEN_SOURCE_DIR(file) GLUTEN_SOURCE_DIR_ file +#define GLUTEN_SOURCE_TPCH_DIR(file) GLUTEN_SOURCE_DIR("/gluten-core/src/test/resources/tpch-data/" file) \ No newline at end of file