From 3ba1ffb627566278fbeda7c41636d18b94b3f5d7 Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Fri, 11 Nov 2022 22:09:36 +0800 Subject: [PATCH 01/93] optimize Signed-off-by: Zhigao Tong --- CMakeLists.txt | 21 +-- cmake/cpu_features.cmake | 3 +- dbms/CMakeLists.txt | 5 + dbms/src/Columns/ColumnsCommon.cpp | 134 +++++++++++------- dbms/src/Functions/CMakeLists.txt | 10 +- .../Storages/Transaction/ReadIndexWorker.cpp | 30 ++-- .../Storages/Transaction/ReadIndexWorker.h | 20 +-- libs/libcommon/CMakeLists.txt | 9 +- .../include/common/avx2_byte_count.h | 74 ++++++++++ .../libcommon/include/common/avx2_mem_utils.h | 38 ++++- libs/libcommon/include/common/avx2_memcpy.h | 11 +- libs/libcommon/include/common/avx2_strstr.h | 13 -- libs/libcommon/include/common/mem_utils_opt.h | 4 + libs/libcommon/include/common/sse2_memcpy.h | 134 ++++++++++++------ libs/libcommon/src/avx2_mem_utils_impl.cpp | 5 + libs/libcommon/src/tests/CMakeLists.txt | 2 +- .../src/tests/gtest_mem_utils_opt.cpp | 25 +++- 17 files changed, 375 insertions(+), 163 deletions(-) create mode 100644 libs/libcommon/include/common/avx2_byte_count.h diff --git a/CMakeLists.txt b/CMakeLists.txt index 2f5428909c8..7bde213e3ce 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -144,16 +144,15 @@ if (USE_STATIC_LIBRARIES) list(REVERSE CMAKE_FIND_LIBRARY_SUFFIXES) endif () -if (CMAKE_LIBRARY_ARCHITECTURE MATCHES "amd64.*|x86_64.*|AMD64.*") - option (USE_INTERNAL_MEMCPY "Use internal implementation of 'memcpy' function instead of provided by libc. Only for x86_64." ON) - - if (OS_LINUX) - option (GLIBC_COMPATIBILITY "Set to TRUE to enable compatibility with older glibc libraries. Only for x86_64, Linux. Implies USE_INTERNAL_MEMCPY." ON) - endif() +if (OS_LINUX AND ARCH_AMD64 AND NOT UNBUNDLED AND MAKE_STATIC_LIBRARIES AND NOT SPLIT_SHARED_LIBRARIES) + # Only for Linux (x86_64). + option(GLIBC_COMPATIBILITY "Enable compatibility with older glibc libraries." ON) +elseif (GLIBC_COMPATIBILITY) + message (STATUS "Glibc compatibility cannot be enabled in current configuration") endif () -if (GLIBC_COMPATIBILITY) - set (USE_INTERNAL_MEMCPY ON) +if (OS_LINUX AND ARCH_AMD64) + option (USE_INTERNAL_MEMCPY "Use internal implementation of 'memcpy' function instead of provided by libc. Only for x86_64." ON) endif () option (PIPE "-pipe compiler option [less /tmp usage, more ram usage]" ON) @@ -380,8 +379,10 @@ if (ARCH_AMD64) else() add_definitions(-DTIFLASH_COMPILER_VPCLMULQDQ_SUPPORT=0) endif() - - check_cxx_compiler_flag("-mmovbe" TIFLASH_COMPILER_MOVBE_SUPPORT) + set (TIFLASH_COMPILER_MOVBE_FLAG "-mmovbe") + check_cxx_compiler_flag("${TIFLASH_COMPILER_MOVBE_FLAG}" TIFLASH_COMPILER_MOVBE_SUPPORT) + set (TIFLASH_COMPILER_BMI2_FLAG "-mbmi2") + check_cxx_compiler_flag("${TIFLASH_COMPILER_BMI2_FLAG}" TIFLASH_COMPILER_BMI2_SUPPORT) else() add_definitions(-DTIFLASH_COMPILER_VPCLMULQDQ_SUPPORT=0) endif() diff --git a/cmake/cpu_features.cmake b/cmake/cpu_features.cmake index 4d09329bf97..cfde43717c7 100644 --- a/cmake/cpu_features.cmake +++ b/cmake/cpu_features.cmake @@ -171,7 +171,8 @@ elseif (ARCH_AMD64) set (COMPILER_FLAGS "${COMPILER_FLAGS} ${TEST_FLAG}") endif () - set (TEST_FLAG "-mavx -mavx2") + set (TIFLASH_COMPILER_AVX2_FLAG "-mavx2") + set (TEST_FLAG "${TIFLASH_COMPILER_AVX2_FLAG}") set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG} -O0") check_cxx_source_compiles(" #include diff --git a/dbms/CMakeLists.txt b/dbms/CMakeLists.txt index be417c0a26e..e9e2275d09b 100644 --- a/dbms/CMakeLists.txt +++ b/dbms/CMakeLists.txt @@ -109,6 +109,11 @@ add_sources_compile_flag_avx2 ( src/Columns/ColumnString.cpp src/Columns/ColumnsCommon.cpp ) +check_then_add_sources_compile_flag ( + TIFLASH_COMPILER_BMI2_SUPPORT + "${TIFLASH_COMPILER_BMI2_FLAG}" + src/Columns/ColumnsCommon.cpp +) list (APPEND clickhouse_common_io_sources ${CONFIG_BUILD}) list (APPEND clickhouse_common_io_headers ${CONFIG_VERSION} ${CONFIG_COMMON}) diff --git a/dbms/src/Columns/ColumnsCommon.cpp b/dbms/src/Columns/ColumnsCommon.cpp index da468c86505..4fb521b510f 100644 --- a/dbms/src/Columns/ColumnsCommon.cpp +++ b/dbms/src/Columns/ColumnsCommon.cpp @@ -16,25 +16,47 @@ #include #include +#ifdef TIFLASH_ENABLE_AVX_SUPPORT +ASSERT_USE_AVX2_COMPILE_FLAG +#endif + namespace DB { -#if defined(__SSE2__) && defined(__POPCNT__) +#if defined(__AVX2__) || defined(__SSE2__) /// Transform 64-byte mask to 64-bit mask. -static UInt64 toBits64(const Int8 * bytes64) +inline UInt64 ToBits64(const Int8 * bytes64) { - static const __m128i zero16 = _mm_setzero_si128(); +#if defined(__AVX2__) + const auto check_block = _mm256_setzero_si256(); + uint64_t mask0 = mem_utils::details::get_block32_cmp_eq_mask(bytes64, check_block); + uint64_t mask1 = mem_utils::details::get_block32_cmp_eq_mask(bytes64 + mem_utils::details::BLOCK32_SIZE, check_block); + auto res = mask0 | (mask1 << mem_utils::details::BLOCK32_SIZE); + return ~res; +#elif defined(__SSE2__) + const auto zero16 = _mm_setzero_si128(); UInt64 res = static_cast(_mm_movemask_epi8(_mm_cmpeq_epi8(_mm_loadu_si128(reinterpret_cast(bytes64)), zero16))) | (static_cast(_mm_movemask_epi8(_mm_cmpeq_epi8(_mm_loadu_si128(reinterpret_cast(bytes64 + 16)), zero16))) << 16) | (static_cast(_mm_movemask_epi8(_mm_cmpeq_epi8(_mm_loadu_si128(reinterpret_cast(bytes64 + 32)), zero16))) << 32) | (static_cast(_mm_movemask_epi8(_mm_cmpeq_epi8(_mm_loadu_si128(reinterpret_cast(bytes64 + 48)), zero16))) << 48); - return ~res; +#endif } #endif -size_t countBytesInFilter(const UInt8 * filt, size_t sz) +#if defined(ADDRESS_SANITIZER) || defined(THREAD_SANITIZER) +NO_INLINE NO_SANITIZE_ADDRESS NO_SANITIZE_THREAD +#else +ALWAYS_INLINE inline +#endif + static size_t + CountBytesInFilter(const UInt8 * filt, size_t start, size_t end) { +#if defined(__AVX2__) + size_t size = end - start; + auto zero_cnt = mem_utils::details::avx2_byte_count(reinterpret_cast(filt + start), size, 0); + return size - zero_cnt; +#else size_t count = 0; /** NOTE: In theory, `filt` should only contain zeros and ones. @@ -42,31 +64,38 @@ size_t countBytesInFilter(const UInt8 * filt, size_t sz) * It would be better to use != 0, then this does not allow SSE2. */ - const Int8 * pos = reinterpret_cast(filt); - const Int8 * end = pos + sz; - -#if defined(__SSE2__) && defined(__POPCNT__) - const Int8 * end64 = pos + sz / 64 * 64; + const char * pos = reinterpret_cast(filt); + pos += start; - for (; pos < end64; pos += 64) - { - count += __builtin_popcountll(toBits64(pos)); - } - /// TODO Add duff device for tail? -#endif - - for (; pos < end; ++pos) + const char * end_pos = pos + (end - start); + for (; pos < end_pos; ++pos) count += *pos != 0; return count; +#endif +} + +size_t countBytesInFilter(const UInt8 * filt, size_t sz) +{ + return CountBytesInFilter(filt, 0, sz); } size_t countBytesInFilter(const IColumn::Filter & filt) { - return countBytesInFilter(filt.data(), filt.size()); + return CountBytesInFilter(filt.data(), 0, filt.size()); } -size_t countBytesInFilterWithNull(const IColumn::Filter & filt, const UInt8 * null_map) +static inline size_t CountBytesInFilterWithNull(const Int8 * p1, const Int8 * p2, size_t size) +{ + size_t count = 0; + for (size_t i = 0; i < size; ++i) + { + count += (p1[i] & ~p2[i]) != 0; + } + return count; +} + +static inline size_t CountBytesInFilterWithNull(const IColumn::Filter & filt, const UInt8 * null_map, size_t start, size_t end) { size_t count = 0; @@ -75,25 +104,27 @@ size_t countBytesInFilterWithNull(const IColumn::Filter & filt, const UInt8 * nu * It would be better to use != 0, then this does not allow SSE2. */ - const Int8 * pos = reinterpret_cast(filt.data()); - const Int8 * pos2 = reinterpret_cast(null_map); - const Int8 * end = pos + filt.size(); - -#if defined(__SSE2__) && defined(__POPCNT__) - const Int8 * end64 = pos + filt.size() / 64 * 64; - - for (; pos < end64; pos += 64, pos2 += 64) - count += __builtin_popcountll(toBits64(pos) & ~toBits64(pos2)); + const Int8 * p1 = reinterpret_cast(filt.data()) + start; + const Int8 * p2 = reinterpret_cast(null_map) + start; + size_t size = end - start; - /// TODO Add duff device for tail? +#if defined(__SSE2__) || defined(__AVX2__) + for (; size >= 64;) + { + count += __builtin_popcountll(ToBits64(p1) & ~ToBits64(p2)); + p1 += 64, p2 += 64; + size -= 64; + } #endif - - for (; pos < end; ++pos) - count += (*pos & ~*pos2) != 0; - + count += CountBytesInFilterWithNull(p1, p2, size); return count; } +size_t countBytesInFilterWithNull(const IColumn::Filter & filt, const UInt8 * null_map) +{ + return CountBytesInFilterWithNull(filt, null_map, 0, filt.size()); +} + std::vector countColumnsSizeInSelector(IColumn::ColumnIndex num_columns, const IColumn::Selector & selector) { std::vector counts(num_columns); @@ -151,7 +182,7 @@ struct ResultOffsetsBuilder if (diff_offset > 0) { - auto * const res_offsets_pos = &res_offsets[offsets_size_old]; + auto * res_offsets_pos = &res_offsets[offsets_size_old]; /// adjust offsets for (size_t i = 0; i < SIMD_BYTES; ++i) @@ -178,7 +209,6 @@ struct NoResultOffsetsBuilder } }; - template void filterArraysImplGeneric( const PaddedPODArray & src_elems, @@ -204,40 +234,36 @@ void filterArraysImplGeneric( res_elems.reserve((result_size_hint * src_elems.size() + size - 1) / size); } - const UInt8 * filt_pos = &filt[0]; - const auto * const filt_end = filt_pos + size; + const UInt8 * filt_pos = filt.data(); + const auto * filt_end = filt_pos + size; - const auto * offsets_pos = &src_offsets[0]; - const auto * const offsets_begin = offsets_pos; + const auto * offsets_pos = src_offsets.data(); + const auto * offsets_begin = offsets_pos; /// copy array ending at *end_offset_ptr const auto copy_array = [&](const IColumn::Offset * offset_ptr) { - const auto offset = offset_ptr == offsets_begin ? 0 : offset_ptr[-1]; - const auto size = *offset_ptr - offset; + const auto arr_offset = offset_ptr == offsets_begin ? 0 : offset_ptr[-1]; + const auto arr_size = *offset_ptr - arr_offset; - result_offsets_builder.insertOne(size); + result_offsets_builder.insertOne(arr_size); const auto elems_size_old = res_elems.size(); - res_elems.resize(elems_size_old + size); - inline_memcpy(&res_elems[elems_size_old], &src_elems[offset], size * sizeof(T)); + res_elems.resize(elems_size_old + arr_size); + inline_memcpy(&res_elems[elems_size_old], &src_elems[arr_offset], arr_size * sizeof(T)); }; #if __SSE2__ - const __m128i zero_vec = _mm_setzero_si128(); - static constexpr size_t SIMD_BYTES = 16; - const auto * const filt_end_aligned = filt_pos + size / SIMD_BYTES * SIMD_BYTES; + static constexpr size_t SIMD_BYTES = mem_utils::details::BLOCK16_SIZE; + const auto * filt_end_aligned = filt_pos + size / SIMD_BYTES * SIMD_BYTES; + const auto zero_vec = _mm_setzero_si128(); while (filt_pos < filt_end_aligned) { uint32_t mask = _mm_movemask_epi8(_mm_cmpgt_epi8( - _mm_loadu_si128(reinterpret_cast(filt_pos)), + mem_utils::details::load_block16(filt_pos), zero_vec)); - if (mask == 0) - { - /// SIMD_BYTES consecutive rows do not pass the filter - } - else if (mask == 0xffff) + if (mem_utils::details::Block16Mask == mask) { /// SIMD_BYTES consecutive rows pass the filter const auto first = offsets_pos == offsets_begin; diff --git a/dbms/src/Functions/CMakeLists.txt b/dbms/src/Functions/CMakeLists.txt index defd75d64e5..4a6cedca496 100644 --- a/dbms/src/Functions/CMakeLists.txt +++ b/dbms/src/Functions/CMakeLists.txt @@ -19,9 +19,13 @@ add_headers_and_sources(clickhouse_functions ./GatherUtils) add_headers_and_sources(clickhouse_functions ./Conditional) add_headers_and_sources(clickhouse_functions ${TiFlash_BINARY_DIR}/dbms/src/Functions) -check_then_add_sources_compile_flag (TIFLASH_ENABLE_AVX_SUPPORT "-mavx2" CollationStringOptimized.cpp) -check_then_add_sources_compile_flag (TIFLASH_COMPILER_MOVBE_SUPPORT "-mmovbe" CollationStringOptimized.cpp) - +check_then_add_sources_compile_flag (TIFLASH_ENABLE_AVX_SUPPORT "${TIFLASH_COMPILER_AVX2_FLAG}" CollationStringOptimized.cpp) +check_then_add_sources_compile_flag (TIFLASH_COMPILER_MOVBE_SUPPORT "${TIFLASH_COMPILER_MOVBE_FLAG}" CollationStringOptimized.cpp) +check_then_add_sources_compile_flag ( + TIFLASH_COMPILER_BMI2_SUPPORT + "${TIFLASH_COMPILER_BMI2_FLAG}" + CollationStringOptimized.cpp +) list(REMOVE_ITEM clickhouse_functions_sources IFunction.cpp FunctionFactory.cpp FunctionHelpers.cpp) list(REMOVE_ITEM clickhouse_functions_headers IFunction.h FunctionFactory.h FunctionHelpers.h) diff --git a/dbms/src/Storages/Transaction/ReadIndexWorker.cpp b/dbms/src/Storages/Transaction/ReadIndexWorker.cpp index c8f3f807f7a..374d646ccfc 100644 --- a/dbms/src/Storages/Transaction/ReadIndexWorker.cpp +++ b/dbms/src/Storages/Transaction/ReadIndexWorker.cpp @@ -42,7 +42,7 @@ static std::mutex global_logger_mutex; auto _ = std::lock_guard(global_logger_mutex); \ std::cout << fmt::format( \ "[{}][{}:{}][{}]", \ - Clock::now(), \ + std::chrono::steady_clock::now(), \ &__FILE__[LogFmtDetails::getFileNameOffset(__FILE__)], \ __LINE__, \ formatted_message) \ @@ -156,7 +156,7 @@ struct BlockedReadIndexHelperTrait uint64_t timeout_ms; }; -struct BlockedReadIndexHelper : BlockedReadIndexHelperTrait +struct BlockedReadIndexHelper final : BlockedReadIndexHelperTrait { public: BlockedReadIndexHelper(uint64_t timeout_ms_, AsyncWaker & waker_) @@ -181,7 +181,7 @@ struct BlockedReadIndexHelper : BlockedReadIndexHelperTrait AsyncWaker & waker; }; -struct BlockedReadIndexHelperV3 : BlockedReadIndexHelperTrait +struct BlockedReadIndexHelperV3 final : BlockedReadIndexHelperTrait { BlockedReadIndexHelperV3(uint64_t timeout_ms_, AsyncWaker::Notifier & notifier_) : BlockedReadIndexHelperTrait(timeout_ms_) @@ -335,7 +335,7 @@ struct ReadIndexNotifyCtrl : MutexLockWrap AsyncWaker::NotifierPtr notifier; }; -struct RegionReadIndexNotifier : AsyncNotifier +struct RegionReadIndexNotifier final : AsyncNotifier { void wake() override { @@ -446,7 +446,7 @@ void ReadIndexDataNode::ReadIndexElement::doPoll(const TiFlashRaftProxyHelper & clean_task = true; } - else if (Clock::now() > timeout + start_time) + else if (std::chrono::steady_clock::now() > timeout + start_time) { TEST_LOG_FMT("poll ReadIndexElement timeout for region {}", region_id); @@ -458,7 +458,7 @@ void ReadIndexDataNode::ReadIndexElement::doPoll(const TiFlashRaftProxyHelper & TEST_LOG_FMT( "poll ReadIndexElement failed for region {}, time cost {}, timeout {}, start time {}", region_id, - Clock::now() - start_time, + std::chrono::steady_clock::now() - start_time, timeout, start_time); } @@ -706,7 +706,7 @@ void ReadIndexWorker::consumeReadIndexNotifyCtrl() } } -void ReadIndexWorker::consumeRegionNotifies(Duration min_dur) +void ReadIndexWorker::consumeRegionNotifies(std::chrono::steady_clock::duration min_dur) { if (!lastRunTimeout(min_dur)) { @@ -720,8 +720,8 @@ void ReadIndexWorker::consumeRegionNotifies(Duration min_dur) node->runOneRound(proxy_helper, read_index_notify_ctrl); } - TEST_LOG_FMT("worker {} set last run time {}", getID(), Clock::now()); - last_run_time.store(Clock::now(), std::memory_order_release); + TEST_LOG_FMT("worker {} set last run time {}", getID(), std::chrono::steady_clock::now()); + last_run_time.store(std::chrono::steady_clock::now(), std::memory_order_release); } ReadIndexFuturePtr ReadIndexWorker::genReadIndexFuture(const kvrpcpb::ReadIndexRequest & req) @@ -737,7 +737,7 @@ ReadIndexFuturePtr ReadIndexWorkerManager::genReadIndexFuture(const kvrpcpb::Rea return getWorkerByRegion(req.context().region_id()).genReadIndexFuture(req); } -void ReadIndexWorker::runOneRound(Duration min_dur) +void ReadIndexWorker::runOneRound(std::chrono::steady_clock::duration min_dur) { if (!read_index_notify_ctrl->empty()) { @@ -759,10 +759,10 @@ ReadIndexWorker::ReadIndexWorker( { } -bool ReadIndexWorker::lastRunTimeout(Duration timeout) const +bool ReadIndexWorker::lastRunTimeout(std::chrono::steady_clock::duration timeout) const { TEST_LOG_FMT("worker {}, last run time {}, timeout {}", getID(), last_run_time.load(std::memory_order_relaxed), timeout); - return last_run_time.load(std::memory_order_relaxed) + timeout < Clock::now(); + return last_run_time.load(std::memory_order_relaxed) + timeout < std::chrono::steady_clock::now(); } ReadIndexWorker & ReadIndexWorkerManager::getWorkerByRegion(RegionID region_id) @@ -828,13 +828,13 @@ ReadIndexWorkerManager::~ReadIndexWorkerManager() stop(); } -void ReadIndexWorkerManager::runOneRoundAll(Duration min_dur) +void ReadIndexWorkerManager::runOneRoundAll(std::chrono::steady_clock::duration min_dur) { for (size_t id = 0; id < runners.size(); ++id) runOneRound(min_dur, id); } -void ReadIndexWorkerManager::runOneRound(Duration min_dur, size_t id) +void ReadIndexWorkerManager::runOneRound(std::chrono::steady_clock::duration min_dur, size_t id) { runners[id]->runOneRound(min_dur); } @@ -1003,7 +1003,7 @@ void ReadIndexWorkerManager::ReadIndexRunner::blockedWaitFor(std::chrono::millis global_notifier->blockedWaitFor(timeout); } -void ReadIndexWorkerManager::ReadIndexRunner::runOneRound(Duration min_dur) +void ReadIndexWorkerManager::ReadIndexRunner::runOneRound(std::chrono::steady_clock::duration min_dur) { for (size_t i = id; i < workers.size(); i += runner_cnt) workers[i]->runOneRound(min_dur); diff --git a/dbms/src/Storages/Transaction/ReadIndexWorker.h b/dbms/src/Storages/Transaction/ReadIndexWorker.h index e26671a2303..ebc36874226 100644 --- a/dbms/src/Storages/Transaction/ReadIndexWorker.h +++ b/dbms/src/Storages/Transaction/ReadIndexWorker.h @@ -35,7 +35,7 @@ class ReadIndexTest; struct AsyncWaker { - struct Notifier : AsyncNotifier + struct Notifier final : AsyncNotifier , MutexLockWrap { mutable std::condition_variable cv; @@ -46,7 +46,7 @@ struct AsyncWaker AsyncNotifier::Status blockedWaitFor(std::chrono::milliseconds timeout) override; void wake() override; - virtual ~Notifier() = default; + ~Notifier() override = default; }; using NotifierPtr = std::shared_ptr; @@ -88,7 +88,7 @@ class ReadIndexWorkerManager : boost::noncopyable void wakeAll(); // wake all runners to handle tasks void asyncRun(); - void runOneRound(Duration min_dur, size_t id); + void runOneRound(std::chrono::steady_clock::duration min_dur, size_t id); void stop(); ~ReadIndexWorkerManager(); BatchReadIndexRes batchReadIndex( @@ -104,7 +104,7 @@ class ReadIndexWorkerManager : boost::noncopyable ReadIndexFuturePtr genReadIndexFuture(const kvrpcpb::ReadIndexRequest & req); private: - void runOneRoundAll(Duration min_dur = std::chrono::milliseconds{0}); + void runOneRoundAll(std::chrono::steady_clock::duration min_dur = std::chrono::milliseconds{0}); enum class State : uint8_t { @@ -124,7 +124,7 @@ class ReadIndexWorkerManager : boost::noncopyable void blockedWaitFor(std::chrono::milliseconds timeout) const; /// Traverse its workers and try to execute tasks. - void runOneRound(Duration min_dur); + void runOneRound(std::chrono::steady_clock::duration min_dur); /// Create one thread to run asynchronously. void asyncRun(); @@ -219,7 +219,7 @@ struct ReadIndexDataNode : MutexLockWrap Task task_pair; kvrpcpb::ReadIndexResponse resp; std::deque callbacks; - Timepoint start_time = Clock::now(); + std::chrono::steady_clock::time_point start_time = std::chrono::steady_clock::now(); }; struct WaitingTasks : MutexLockWrap @@ -298,12 +298,12 @@ struct ReadIndexWorker void consumeReadIndexNotifyCtrl(); - void consumeRegionNotifies(Duration min_dur); + void consumeRegionNotifies(std::chrono::steady_clock::duration min_dur); ReadIndexFuturePtr genReadIndexFuture(const kvrpcpb::ReadIndexRequest & req); // try to consume read-index response notifications & region waiting list - void runOneRound(Duration min_dur); + void runOneRound(std::chrono::steady_clock::duration min_dur); explicit ReadIndexWorker( const TiFlashRaftProxyHelper & proxy_helper_, @@ -329,7 +329,7 @@ struct ReadIndexWorker // x = x == 0 ? 1 : x; // max_read_index_history = x; // } - bool lastRunTimeout(Duration timeout) const; + bool lastRunTimeout(std::chrono::steady_clock::duration timeout) const; void removeRegion(uint64_t); @@ -348,7 +348,7 @@ struct ReadIndexWorker RegionNotifyMap region_notify_map; // no need to be protected - std::atomic last_run_time{Timepoint::min()}; + std::atomic last_run_time{std::chrono::steady_clock::time_point::min()}; }; struct MockStressTestCfg diff --git a/libs/libcommon/CMakeLists.txt b/libs/libcommon/CMakeLists.txt index b17867ed662..dbda79e250d 100644 --- a/libs/libcommon/CMakeLists.txt +++ b/libs/libcommon/CMakeLists.txt @@ -160,7 +160,7 @@ endif () check_then_add_sources_compile_flag ( TIFLASH_ENABLE_AVX_SUPPORT - "-mavx2" + "${TIFLASH_COMPILER_AVX2_FLAG}" src/mem_utils_avx2.cpp src/crc64_avx2.cpp src/avx2_mem_utils_impl.cpp @@ -173,7 +173,12 @@ check_then_add_sources_compile_flag ( ) check_then_add_sources_compile_flag ( TIFLASH_COMPILER_MOVBE_SUPPORT - "-mmovbe" + "${TIFLASH_COMPILER_MOVBE_FLAG}" + src/avx2_mem_utils_impl.cpp +) +check_then_add_sources_compile_flag ( + TIFLASH_COMPILER_BMI2_SUPPORT + "${TIFLASH_COMPILER_BMI2_FLAG}" src/avx2_mem_utils_impl.cpp ) diff --git a/libs/libcommon/include/common/avx2_byte_count.h b/libs/libcommon/include/common/avx2_byte_count.h new file mode 100644 index 00000000000..71c0541b7bb --- /dev/null +++ b/libs/libcommon/include/common/avx2_byte_count.h @@ -0,0 +1,74 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#pragma once + +#include + +namespace mem_utils::details +{ + +ALWAYS_INLINE static inline uint64_t avx2_byte_count(const char * src, size_t size, char target) +{ + uint64_t zero_bytes_cnt = 0; + const auto check_block32 = _mm256_set1_epi8(target); + + if (uint8_t right_offset = OFFSET_FROM_ALIGNED(size_t(src), BLOCK32_SIZE); right_offset != 0) + { + // align to 32 + src = reinterpret_cast(ALIGNED_ADDR(size_t(src), BLOCK32_SIZE)); + + // right shift offset to remove useless mask bit + auto mask = get_block32_cmp_eq_mask(src, check_block32); + size_t left_remain = BLOCK32_SIZE - right_offset; + + if unlikely (left_remain >= size) + { + left_remain -= size; + mask <<= left_remain; + mask >>= left_remain; + mask >>= right_offset; + return __builtin_popcount(mask); + } + + mask >>= right_offset; + zero_bytes_cnt += __builtin_popcount(mask); + size -= left_remain; + src += BLOCK32_SIZE; + } + + assert(size_t(src) % BLOCK32_SIZE == 0); + + // clang will unroll by step 4 automatically with flags `-mavx2` if size >= 4 * 32 + // unrolled by step 8 with flags `-march=haswell` + for (; size >= BLOCK32_SIZE;) + { + auto mask = get_block32_cmp_eq_mask(src, check_block32); + zero_bytes_cnt += __builtin_popcount(mask); + size -= BLOCK32_SIZE, src += BLOCK32_SIZE; + } + + if unlikely (size != 0) + { + auto mask = get_block32_cmp_eq_mask(src, check_block32); + uint32_t left_remain = BLOCK32_SIZE - size; + mask <<= left_remain; + mask >>= left_remain; + zero_bytes_cnt += __builtin_popcount(mask); + } + + return zero_bytes_cnt; +} + +} // namespace mem_utils::details diff --git a/libs/libcommon/include/common/avx2_mem_utils.h b/libs/libcommon/include/common/avx2_mem_utils.h index 28051208637..e2b1ac81b0d 100644 --- a/libs/libcommon/include/common/avx2_mem_utils.h +++ b/libs/libcommon/include/common/avx2_mem_utils.h @@ -32,7 +32,7 @@ template ALWAYS_INLINE static inline T clear_rightmost_bit_one(const T value) { assert(value != 0); - + // recommended to use compile flag `-mbmi` under AMD64 platform return value & (value - 1); } @@ -67,9 +67,18 @@ FLATTEN_INLINE static inline void write(void * tar, const S & src) { *reinterpret_cast(tar) = src; } +template FLATTEN_INLINE_PURE static inline Block32 load_block32(const void * p) { - return _mm256_loadu_si256(reinterpret_cast(p)); + if constexpr (aligned) + { + if constexpr (non_temporal) + return _mm256_stream_load_si256(reinterpret_cast(p)); + else + return _mm256_load_si256(reinterpret_cast(p)); + } + else + return _mm256_loadu_si256(reinterpret_cast(p)); } FLATTEN_INLINE_PURE static inline Block16 load_block16(const void * p) { @@ -83,11 +92,16 @@ FLATTEN_INLINE static inline void write_block16(void * p, const Block16 & src) else _mm_storeu_si128(reinterpret_cast(p), src); } -template +template FLATTEN_INLINE static inline void write_block32(void * p, const Block32 & src) { if constexpr (aligned) - _mm256_store_si256(reinterpret_cast(p), src); + { + if constexpr (non_temporal) + _mm256_stream_si256(reinterpret_cast(p), src); + else + _mm256_store_si256(reinterpret_cast(p), src); + } else _mm256_storeu_si256(reinterpret_cast(p), src); } @@ -96,11 +110,27 @@ FLATTEN_INLINE_PURE static inline uint32_t get_block32_cmp_eq_mask(const void * uint32_t mask = _mm256_movemask_epi8(_mm256_cmpeq_epi8(load_block32(p1), load_block32(p2))); return mask; } +FLATTEN_INLINE_PURE static inline uint32_t get_block32_cmp_eq_mask( + const void * s, + const Block32 & check_block) +{ + const auto block = load_block32(s); + uint32_t mask = _mm256_movemask_epi8(_mm256_cmpeq_epi8(block, check_block)); + return mask; +} FLATTEN_INLINE_PURE static inline uint32_t get_block16_cmp_eq_mask(const void * p1, const void * p2) { uint32_t mask = _mm_movemask_epi8(_mm_cmpeq_epi8(load_block16(p1), load_block16(p2))); return mask; } +FLATTEN_INLINE_PURE static inline uint32_t get_block16_cmp_eq_mask( + const void * s, + const Block16 & check_block) +{ + const auto block = load_block16(s); + uint32_t mask = _mm_movemask_epi8(_mm_cmpeq_epi8(block, check_block)); + return mask; +} FLATTEN_INLINE_PURE static inline bool check_block32_eq(const char * a, const char * b) { auto data = _mm256_xor_si256( diff --git a/libs/libcommon/include/common/avx2_memcpy.h b/libs/libcommon/include/common/avx2_memcpy.h index d8937928b5c..d959b7b299c 100644 --- a/libs/libcommon/include/common/avx2_memcpy.h +++ b/libs/libcommon/include/common/avx2_memcpy.h @@ -18,6 +18,7 @@ #include #include +#include #include #include @@ -37,11 +38,7 @@ ALWAYS_INLINE static inline void * avx2_inline_memcpy(void * __restrict dst_, co #define MEM_CP_END(n) tiflash_compiler_builtin_memcpy(dst + size - (n), src + size - (n), (n)); #define MEM_CP_HEAD(n) tiflash_compiler_builtin_memcpy(dst, src, (n)); -#ifdef AVX2_MEMCPY_USE_PREFETCH #define PREFETCH(addr) __builtin_prefetch(addr) -#else -#define PREFETCH(addr) -#endif constexpr int block32_size = details::BLOCK32_SIZE; @@ -58,7 +55,7 @@ ALWAYS_INLINE static inline void * avx2_inline_memcpy(void * __restrict dst_, co } /// No bytes remaining. } - else if (unlikely(size <= 4)) + else if (unlikely(size < 4)) { /// Chunks of 2..3 bytes. details::memcpy_ignore_overlap<2>(dst, src, size); @@ -95,8 +92,12 @@ ALWAYS_INLINE static inline void * avx2_inline_memcpy(void * __restrict dst_, co dst -= offset; src -= offset; size += offset; + assert(size_t(dst) % block32_size == 0); } + // TODO: use non-temporal way(mark data unlikely to be used again soon) to minimize caching for large memory size(bigger than L1 cache size) if necessary. + // TODO: check whether source address is aligned to 32 and use specific aligned instructions if necessary. + /// Aligned unrolled copy. while (size >= block32_size * loop_block32_cnt) { diff --git a/libs/libcommon/include/common/avx2_strstr.h b/libs/libcommon/include/common/avx2_strstr.h index ad0b6715413..d9722dffbb1 100644 --- a/libs/libcommon/include/common/avx2_strstr.h +++ b/libs/libcommon/include/common/avx2_strstr.h @@ -19,19 +19,6 @@ namespace mem_utils::details { -FLATTEN_INLINE_PURE static inline uint32_t get_block32_cmp_eq_mask( - const void * s, - const Block32 & check_block) -{ - /* - vpcmpeqb ymm0, ymm0, ymmword ptr [...] - */ - // `_mm256_loadu_si256` and `_mm256_load_si256` are same in such case - const auto block = load_block32(s); - uint32_t mask = _mm256_movemask_epi8(_mm256_cmpeq_epi8(block, check_block)); - return mask; -} - template ALWAYS_INLINE static inline bool check_aligned_block32_may_exceed(const char * src, ssize_t n, const char *& res, const Block32 & check_block, F && callback) { diff --git a/libs/libcommon/include/common/mem_utils_opt.h b/libs/libcommon/include/common/mem_utils_opt.h index 0b5e86aa6fe..98f47111256 100644 --- a/libs/libcommon/include/common/mem_utils_opt.h +++ b/libs/libcommon/include/common/mem_utils_opt.h @@ -25,6 +25,7 @@ constexpr bool tiflash_use_avx2_compile_flag = true; // if cpp source file is compiled with flag `-mavx2`, it's recommended to use inline function for better performance. +#include #include #include @@ -60,6 +61,9 @@ bool avx2_mem_equal(const char * p1, const char * p2, size_t n); // same function like `std::memcmp` int avx2_mem_cmp(const char * p1, const char * p2, size_t n); +// return count of target byte +uint64_t avx2_byte_count(const char * src, size_t size, char target); + } // namespace mem_utils #endif diff --git a/libs/libcommon/include/common/sse2_memcpy.h b/libs/libcommon/include/common/sse2_memcpy.h index 5c57275b39d..3c75d9c225a 100644 --- a/libs/libcommon/include/common/sse2_memcpy.h +++ b/libs/libcommon/include/common/sse2_memcpy.h @@ -17,14 +17,98 @@ #include #include +#include #include #include -// Custom inline memcpy implementation for TiFlash. -// - it is recommended to use for inline function with `sse2` supported -// - it perform better than `legacy::inline_memcpy`(from clickhouse) according to `libs/libcommon/src/tests/bench_memcpy.cpp` -// - like `std::memcpy`, the behavior is undefined when the source and the destination objects overlap -// - moving data from register to memory costs more than the reversed way, so it's useful to reduce times about memory copying. +template +ALWAYS_INLINE inline void memcpy_ignore_overlap(char * __restrict dst, const char * __restrict src, size_t size); + +template +ALWAYS_INLINE inline void memcpy_block32_ignore_overlap(char * __restrict dst, const char * __restrict src, size_t size); + +template +ALWAYS_INLINE inline void memcpy_ignore_overlap(char * __restrict dst, const char * __restrict src, size_t size) +{ + assert(size >= sizeof(T)); + auto a = *reinterpret_cast(src); + auto b = *reinterpret_cast(src + size - sizeof(T)); + *reinterpret_cast(dst) = a; + *reinterpret_cast(dst + size - sizeof(T)) = b; +} + +template <> +ALWAYS_INLINE inline void memcpy_ignore_overlap<2>(char * __restrict dst, const char * __restrict src, size_t size) +{ + assert(size >= 2 && size <= 4); + using T = uint16_t; + static_assert(sizeof(T) == 2); + memcpy_ignore_overlap(dst, src, size); +} +template <> +ALWAYS_INLINE inline void memcpy_ignore_overlap<4>(char * __restrict dst, const char * __restrict src, size_t size) +{ + assert(size >= 4 && size <= 8); + using T = uint32_t; + static_assert(sizeof(T) == 4); + memcpy_ignore_overlap(dst, src, size); +} +template <> +ALWAYS_INLINE inline void memcpy_ignore_overlap<8>(char * __restrict dst, const char * __restrict src, size_t size) +{ + assert(size >= 8 && size <= 16); + using T = uint64_t; + static_assert(sizeof(T) == 8); + memcpy_ignore_overlap(dst, src, size); +} +template <> +ALWAYS_INLINE inline void memcpy_ignore_overlap<16>(char * __restrict dst, const char * __restrict src, size_t size) +{ + assert(size >= 16 && size <= 32); + auto c0 = _mm_loadu_si128(reinterpret_cast(src)); + auto c1 = _mm_loadu_si128(reinterpret_cast(src + size - 16)); + _mm_storeu_si128(reinterpret_cast<__m128i *>(dst), c0); + _mm_storeu_si128(reinterpret_cast<__m128i *>(dst + size - 16), c1); +} + +ALWAYS_INLINE static inline void sse2_inline_memcpy_small(void * __restrict dst_, const void * __restrict src_, size_t size) +{ + char * __restrict dst = reinterpret_cast(dst_); + const char * __restrict src = reinterpret_cast(src_); + + assert(size <= 32); + + if unlikely (size <= 1) + { + if likely (size == 1) + { + /// A single byte. + *dst = *src; + } + /// No bytes remaining. + } + else if unlikely (size <= 4) // sse2_inline_memcpy(_,_, 4 ) should use 4 bytes register directly + { + /// Chunks of 2..4 bytes. + memcpy_ignore_overlap<2>(dst, src, size); + } + else if unlikely (size <= 8) // sse2_inline_memcpy(_,_, 8 ) should use 8 bytes register directly + { + /// Chunks of 4..8 bytes. + memcpy_ignore_overlap<4>(dst, src, size); + } + else if unlikely (size <= 16) + { + /// Chunks of 8..16 bytes. + memcpy_ignore_overlap<8>(dst, src, size); + } + else + { + /// Chunks of 17..32 bytes. + memcpy_ignore_overlap<16>(dst, src, size); + } +} + ALWAYS_INLINE static inline void * sse2_inline_memcpy(void * __restrict dst_, const void * __restrict src_, size_t size) { char * __restrict dst = reinterpret_cast(dst_); @@ -32,48 +116,12 @@ ALWAYS_INLINE static inline void * sse2_inline_memcpy(void * __restrict dst_, co void * ret = dst; -#if defined(MCP) || defined(MCP_END) - static_assert(false); -#endif - #define MCP_END(n) tiflash_compiler_builtin_memcpy(dst + size - (n), src + size - (n), (n)); #define MCP(n) tiflash_compiler_builtin_memcpy(dst, src, (n)); - if (likely(size <= 32)) + if likely (size <= 32) { - if (unlikely(size <= 1)) - { - if (likely(size == 1)) - { - /// A single byte. - *dst = *src; - } - /// No bytes remaining. - } - else if (unlikely(size < 4)) // sse2_inline_memcpy(_,_, 4 ) should use 4 bytes register directly - { - /// Chunks of 2..3 bytes. - MCP(2); - MCP_END(2); - } - else if (unlikely(size < 8)) // sse2_inline_memcpy(_,_, 8 ) should use 8 bytes register directly - { - /// Chunks of 4..7 bytes. - MCP(4); - MCP_END(4); - } - else if (unlikely(size <= 16)) - { - /// Chunks of 8..16 bytes. - MCP(8); - MCP_END(8); - } - else - { - /// Chunks of 17..32 bytes. - MCP(16); - MCP_END(16); - } + sse2_inline_memcpy_small(dst, src, size); } else { diff --git a/libs/libcommon/src/avx2_mem_utils_impl.cpp b/libs/libcommon/src/avx2_mem_utils_impl.cpp index fad0ee3962e..2b4bfe7ec79 100644 --- a/libs/libcommon/src/avx2_mem_utils_impl.cpp +++ b/libs/libcommon/src/avx2_mem_utils_impl.cpp @@ -44,6 +44,11 @@ const char * avx2_memchr(const char * src, size_t n, char target) return details::avx2_memchr(src, n, target); } +uint64_t avx2_byte_count(const char * src, size_t size, char target) +{ + return details::avx2_byte_count(src, size, target); +} + } // namespace mem_utils #endif diff --git a/libs/libcommon/src/tests/CMakeLists.txt b/libs/libcommon/src/tests/CMakeLists.txt index d220c025015..6d60ab67674 100644 --- a/libs/libcommon/src/tests/CMakeLists.txt +++ b/libs/libcommon/src/tests/CMakeLists.txt @@ -39,7 +39,7 @@ add_executable (gtests_libcommon gtest_crc64.cpp gtest_logger.cpp gtest_arithmetic_overflow.cpp - ) +) add_sources_compile_flag_avx2 (gtest_mem_utils_opt.cpp) diff --git a/libs/libcommon/src/tests/gtest_mem_utils_opt.cpp b/libs/libcommon/src/tests/gtest_mem_utils_opt.cpp index 04941075d32..689a6e94004 100644 --- a/libs/libcommon/src/tests/gtest_mem_utils_opt.cpp +++ b/libs/libcommon/src/tests/gtest_mem_utils_opt.cpp @@ -26,8 +26,6 @@ #include #include -#include "../../libmemcpy/folly/FollyMemcpy.h" - #if defined(TIFLASH_ENABLE_AVX_SUPPORT) void TestFunc(size_t size) @@ -232,4 +230,27 @@ TEST(MemUtilsTestOPT, Memcopy) } } +void TestMemByteCount(size_t size) +{ + char target = 8; + std::string oa(size + 100, target); + char * start = oa.data(); + start = reinterpret_cast(ALIGNED_ADDR(size_t(start), 32)) + 32; + for (auto * pos = start; pos < start + 32; ++pos) + { + ASSERT_EQ(mem_utils::avx2_byte_count(pos, size, target), size); + std::memset(pos, target - 1, size); + ASSERT_EQ(mem_utils::avx2_byte_count(pos, size, target), 0); + std::memset(pos, target, size); + } +} + +TEST(MemUtilsTestOPT, MemByteCount) +{ + for (size_t size = 0; size <= 32 * 6; ++size) + { + TestMemByteCount(size); + } +} + #endif \ No newline at end of file From 2541792d74ff1532ca91edac142c5dedeec9be04 Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Tue, 15 Nov 2022 10:43:53 +0800 Subject: [PATCH 02/93] support haswell(intel 2013) arch Signed-off-by: Zhigao Tong --- CMakeLists.txt | 2 ++ 1 file changed, 2 insertions(+) diff --git a/CMakeLists.txt b/CMakeLists.txt index 7bde213e3ce..4d1e0b47ba3 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -383,6 +383,8 @@ if (ARCH_AMD64) check_cxx_compiler_flag("${TIFLASH_COMPILER_MOVBE_FLAG}" TIFLASH_COMPILER_MOVBE_SUPPORT) set (TIFLASH_COMPILER_BMI2_FLAG "-mbmi2") check_cxx_compiler_flag("${TIFLASH_COMPILER_BMI2_FLAG}" TIFLASH_COMPILER_BMI2_SUPPORT) + set (TIFLASH_COMPILER_ARCH_HASWELL_FLAG "-march=haswell") + check_cxx_compiler_flag("${TIFLASH_COMPILER_ARCH_HASWELL_FLAG}" TIFLASH_COMPILER_ARCH_HASWELL_SUPPORT) else() add_definitions(-DTIFLASH_COMPILER_VPCLMULQDQ_SUPPORT=0) endif() From 7585264639df7567a071b5a4ae1f9ef1b8d6273b Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Tue, 15 Nov 2022 14:19:39 +0800 Subject: [PATCH 03/93] fix some modules std::chrono::system_clock->std::chrono::steady_clock Signed-off-by: Zhigao Tong --- dbms/src/Common/MPMCQueue.h | 12 +++++------- dbms/src/Interpreters/AsynchronousMetrics.cpp | 4 ++-- dbms/src/Server/MetricsTransmitter.cpp | 4 ++-- dbms/src/Storages/Transaction/PDTiKVClient.cpp | 2 +- dbms/src/Storages/Transaction/PDTiKVClient.h | 6 +++--- 5 files changed, 13 insertions(+), 15 deletions(-) diff --git a/dbms/src/Common/MPMCQueue.h b/dbms/src/Common/MPMCQueue.h index e966cd44547..709ce336516 100644 --- a/dbms/src/Common/MPMCQueue.h +++ b/dbms/src/Common/MPMCQueue.h @@ -73,6 +73,7 @@ class MPMCQueue public: using Status = MPMCQueueStatus; using Result = MPMCQueueResult; + using SteadyClock = std::chrono::steady_clock; explicit MPMCQueue(size_t capacity_) : capacity(capacity_) @@ -108,8 +109,7 @@ class MPMCQueue template ALWAYS_INLINE Result popTimeout(T & obj, const Duration & timeout) { - /// std::condition_variable::wait_until will always use system_clock. - auto deadline = std::chrono::system_clock::now() + timeout; + auto deadline = SteadyClock::now() + timeout; return popObj(obj, &deadline); } @@ -138,8 +138,7 @@ class MPMCQueue template ALWAYS_INLINE Result pushTimeout(U && u, const Duration & timeout) { - /// std::condition_variable::wait_until will always use system_clock. - auto deadline = std::chrono::system_clock::now() + timeout; + auto deadline = SteadyClock::now() + timeout; return pushObj(std::forward(u), &deadline); } @@ -162,8 +161,7 @@ class MPMCQueue template ALWAYS_INLINE Result emplaceTimeout(Args &&... args, const Duration & timeout) { - /// std::condition_variable::wait_until will always use system_clock. - auto deadline = std::chrono::system_clock::now() + timeout; + auto deadline = SteadyClock::now() + timeout; return emplaceObj(&deadline, std::forward(args)...); } @@ -222,7 +220,7 @@ class MPMCQueue } private: - using TimePoint = std::chrono::time_point; + using TimePoint = std::chrono::time_point; using WaitingNode = MPMCQueueDetail::WaitingNode; void notifyAll() diff --git a/dbms/src/Interpreters/AsynchronousMetrics.cpp b/dbms/src/Interpreters/AsynchronousMetrics.cpp index d1b074e9141..076b44554fa 100644 --- a/dbms/src/Interpreters/AsynchronousMetrics.cpp +++ b/dbms/src/Interpreters/AsynchronousMetrics.cpp @@ -82,8 +82,8 @@ void AsynchronousMetrics::run() /// Next minute + 30 seconds. To be distant with moment of transmission of metrics, see MetricsTransmitter. const auto get_next_minute = [] { - return std::chrono::time_point_cast( - std::chrono::system_clock::now() + std::chrono::minutes(1)) + return std::chrono::time_point_cast( + std::chrono::steady_clock::now() + std::chrono::minutes(1)) + std::chrono::seconds(30); }; diff --git a/dbms/src/Server/MetricsTransmitter.cpp b/dbms/src/Server/MetricsTransmitter.cpp index e596ae0a95c..0ccd2d9346b 100644 --- a/dbms/src/Server/MetricsTransmitter.cpp +++ b/dbms/src/Server/MetricsTransmitter.cpp @@ -57,8 +57,8 @@ void MetricsTransmitter::run() /// To avoid time drift and transmit values exactly each interval: /// next time aligned to system seconds /// (60s -> every minute at 00 seconds, 5s -> every minute:[00, 05, 15 ... 55]s, 3600 -> every hour:00:00 - return std::chrono::system_clock::time_point( - (std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()) / seconds) * seconds + return std::chrono::steady_clock::time_point( + (std::chrono::duration_cast(std::chrono::steady_clock::now().time_since_epoch()) / seconds) * seconds + std::chrono::seconds(seconds)); }; diff --git a/dbms/src/Storages/Transaction/PDTiKVClient.cpp b/dbms/src/Storages/Transaction/PDTiKVClient.cpp index a06f1a3ae64..ed46aeb18c2 100644 --- a/dbms/src/Storages/Transaction/PDTiKVClient.cpp +++ b/dbms/src/Storages/Transaction/PDTiKVClient.cpp @@ -23,6 +23,6 @@ extern const int LOGICAL_ERROR; } std::atomic PDClientHelper::cached_gc_safe_point = 0; -std::atomic> PDClientHelper::safe_point_last_update_time; +std::atomic> PDClientHelper::safe_point_last_update_time; } // namespace DB diff --git a/dbms/src/Storages/Transaction/PDTiKVClient.h b/dbms/src/Storages/Transaction/PDTiKVClient.h index 111ae9862ed..ad4da83a3fe 100644 --- a/dbms/src/Storages/Transaction/PDTiKVClient.h +++ b/dbms/src/Storages/Transaction/PDTiKVClient.h @@ -45,7 +45,7 @@ struct PDClientHelper if (!ignore_cache) { // In case we cost too much to update safe point from PD. - std::chrono::time_point now = std::chrono::system_clock::now(); + std::chrono::time_point now = std::chrono::steady_clock::now(); const auto duration = std::chrono::duration_cast(now - safe_point_last_update_time.load()); const auto min_interval = std::max(Int64(1), safe_point_update_interval_seconds); // at least one second if (duration.count() < min_interval) @@ -59,7 +59,7 @@ struct PDClientHelper { auto safe_point = pd_client->getGCSafePoint(); cached_gc_safe_point = safe_point; - safe_point_last_update_time = std::chrono::system_clock::now(); + safe_point_last_update_time = std::chrono::steady_clock::now(); return safe_point; } catch (pingcap::Exception & e) @@ -71,7 +71,7 @@ struct PDClientHelper private: static std::atomic cached_gc_safe_point; - static std::atomic> safe_point_last_update_time; + static std::atomic> safe_point_last_update_time; }; From 518b51b93a40103a405d059c5cb66fd76b9b0418 Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Tue, 15 Nov 2022 18:01:49 +0800 Subject: [PATCH 04/93] test compress ch block chunck Signed-off-by: Zhigao Tong --- contrib/lz4-cmake/CMakeLists.txt | 5 + contrib/zstd-cmake/CMakeLists.txt | 5 + dbms/CMakeLists.txt | 11 ++ dbms/src/DataTypes/DataTypeString.cpp | 57 ++------ .../Flash/Coprocessor/CHBlockChunkCodec.cpp | 120 +++++++++++++-- .../Coprocessor/ChunkDecodeAndSquash.cpp | 19 ++- dbms/src/Flash/Coprocessor/tzg-metrics.h | 72 +++++++++ dbms/src/IO/CompressedReadBufferBase.cpp | 13 +- dbms/src/IO/CompressedStream.h | 2 + dbms/src/IO/CompressedWriteBuffer.cpp | 32 +++- dbms/src/IO/CompressedWriteBuffer.h | 2 +- .../Transaction/ProxyFFIStatusService.cpp | 73 ++++++++- libs/CMakeLists.txt | 2 +- libs/libsnappy/CMakeLists.txt | 27 ++++ tiflash-tools/.gitignore | 5 + tiflash-tools/define.py | 18 +++ tiflash-tools/flash_cluster_manager.py | 94 ++++++++++++ tiflash-tools/flash_http_client.py | 37 +++++ tiflash-tools/pd_client.py | 126 ++++++++++++++++ tiflash-tools/util.py | 138 ++++++++++++++++++ 20 files changed, 795 insertions(+), 63 deletions(-) create mode 100644 dbms/src/Flash/Coprocessor/tzg-metrics.h create mode 100644 libs/libsnappy/CMakeLists.txt create mode 100644 tiflash-tools/.gitignore create mode 100644 tiflash-tools/define.py create mode 100644 tiflash-tools/flash_cluster_manager.py create mode 100644 tiflash-tools/flash_http_client.py create mode 100644 tiflash-tools/pd_client.py create mode 100644 tiflash-tools/util.py diff --git a/contrib/lz4-cmake/CMakeLists.txt b/contrib/lz4-cmake/CMakeLists.txt index fd2d3d69004..5f456d65844 100644 --- a/contrib/lz4-cmake/CMakeLists.txt +++ b/contrib/lz4-cmake/CMakeLists.txt @@ -10,3 +10,8 @@ add_library (lz4 target_compile_definitions(lz4 PUBLIC LZ4_DISABLE_DEPRECATE_WARNINGS=1) target_include_directories(lz4 PUBLIC ${LIBRARY_DIR}) + +if (TIFLASH_ENABLE_AVX_SUPPORT) + set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${TIFLASH_COMPILER_AVX2_FLAG}") + set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${TIFLASH_COMPILER_AVX2_FLAG}") +endif () \ No newline at end of file diff --git a/contrib/zstd-cmake/CMakeLists.txt b/contrib/zstd-cmake/CMakeLists.txt index 0505ad60774..4ab932cb88a 100644 --- a/contrib/zstd-cmake/CMakeLists.txt +++ b/contrib/zstd-cmake/CMakeLists.txt @@ -155,3 +155,8 @@ ENABLE_LANGUAGE(ASM) ADD_LIBRARY(zstd ${Sources} ${Headers}) target_include_directories (zstd PUBLIC ${LIBRARY_DIR}) + +if (TIFLASH_ENABLE_AVX_SUPPORT) + set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${TIFLASH_COMPILER_AVX2_FLAG}") + set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${TIFLASH_COMPILER_AVX2_FLAG}") +endif () \ No newline at end of file diff --git a/dbms/CMakeLists.txt b/dbms/CMakeLists.txt index e9e2275d09b..1dc07d4b7cb 100644 --- a/dbms/CMakeLists.txt +++ b/dbms/CMakeLists.txt @@ -108,11 +108,15 @@ add_headers_only(dbms src/Server) add_sources_compile_flag_avx2 ( src/Columns/ColumnString.cpp src/Columns/ColumnsCommon.cpp + src/Columns/ColumnVector.cpp + src/DataTypes/DataTypeString.cpp ) check_then_add_sources_compile_flag ( TIFLASH_COMPILER_BMI2_SUPPORT "${TIFLASH_COMPILER_BMI2_FLAG}" src/Columns/ColumnsCommon.cpp + src/Columns/ColumnVector.cpp + src/DataTypes/DataTypeString.cpp ) list (APPEND clickhouse_common_io_sources ${CONFIG_BUILD}) @@ -206,6 +210,13 @@ target_link_libraries (clickhouse_common_io magic_enum libsymbolization ) + +target_link_libraries ( + clickhouse_common_io + PRIVATE + snappy +) + target_include_directories (clickhouse_common_io BEFORE PRIVATE ${kvClient_SOURCE_DIR}/include) target_compile_definitions(clickhouse_common_io PUBLIC -DTIFLASH_SOURCE_PREFIX=\"${TiFlash_SOURCE_DIR}\") target_link_libraries (dbms diff --git a/dbms/src/DataTypes/DataTypeString.cpp b/dbms/src/DataTypes/DataTypeString.cpp index 45698220c7f..07e008b91d0 100644 --- a/dbms/src/DataTypes/DataTypeString.cpp +++ b/dbms/src/DataTypes/DataTypeString.cpp @@ -32,7 +32,7 @@ namespace DB { void DataTypeString::serializeBinary(const Field & field, WriteBuffer & ostr) const { - const String & s = get(field); + const auto & s = get(field); writeVarUInt(s.size(), ostr); writeString(s, ostr); } @@ -43,7 +43,7 @@ void DataTypeString::deserializeBinary(Field & field, ReadBuffer & istr) const UInt64 size; readVarUInt(size, istr); field = String(); - String & s = get(field); + auto & s = get(field); s.resize(size); istr.readStrict(&s[0], size); } @@ -59,7 +59,7 @@ void DataTypeString::serializeBinary(const IColumn & column, size_t row_num, Wri void DataTypeString::deserializeBinary(IColumn & column, ReadBuffer & istr) const { - ColumnString & column_string = static_cast(column); + auto & column_string = static_cast(column); ColumnString::Chars_t & data = column_string.getChars(); ColumnString::Offsets & offsets = column_string.getOffsets(); @@ -91,7 +91,7 @@ void DataTypeString::serializeBinaryBulk(const IColumn & column, WriteBuffer & o const ColumnString::Chars_t & data = column_string.getChars(); const ColumnString::Offsets & offsets = column_string.getOffsets(); - size_t size = column.size(); + size_t size = column_string.size(); if (!size) return; @@ -103,7 +103,7 @@ void DataTypeString::serializeBinaryBulk(const IColumn & column, WriteBuffer & o { UInt64 str_size = offsets[0] - 1; writeVarUInt(str_size, ostr); - ostr.write(reinterpret_cast(&data[0]), str_size); + ostr.write(reinterpret_cast(data.data()), str_size); ++offset; } @@ -136,44 +136,21 @@ static NO_INLINE void deserializeBinarySSE2(ColumnString::Chars_t & data, Column if (size) { -#if __SSE2__ +#ifdef __SSE2__ /// An optimistic branch in which more efficient copying is possible. if (offset + 16 * UNROLL_TIMES <= data.capacity() && istr.position() + size + 16 * UNROLL_TIMES <= istr.buffer().end()) { - const __m128i * sse_src_pos = reinterpret_cast(istr.position()); + const auto * sse_src_pos = reinterpret_cast(istr.position()); const __m128i * sse_src_end = sse_src_pos + (size + (16 * UNROLL_TIMES - 1)) / 16 / UNROLL_TIMES * UNROLL_TIMES; - __m128i * sse_dst_pos = reinterpret_cast<__m128i *>(&data[offset - size - 1]); + auto * sse_dst_pos = reinterpret_cast<__m128i *>(&data[offset - size - 1]); while (sse_src_pos < sse_src_end) { - /// NOTE gcc 4.9.2 unrolls the loop, but for some reason uses only one xmm register. - /// for (size_t j = 0; j < UNROLL_TIMES; ++j) - /// _mm_storeu_si128(sse_dst_pos + j, _mm_loadu_si128(sse_src_pos + j)); + for (size_t j = 0; j < UNROLL_TIMES; ++j) + _mm_storeu_si128(sse_dst_pos + j, _mm_loadu_si128(sse_src_pos + j)); sse_src_pos += UNROLL_TIMES; sse_dst_pos += UNROLL_TIMES; - - if (UNROLL_TIMES >= 4) - __asm__("movdqu %0, %%xmm0" ::"m"(sse_src_pos[-4])); - if (UNROLL_TIMES >= 3) - __asm__("movdqu %0, %%xmm1" ::"m"(sse_src_pos[-3])); - if (UNROLL_TIMES >= 2) - __asm__("movdqu %0, %%xmm2" ::"m"(sse_src_pos[-2])); - if (UNROLL_TIMES >= 1) - __asm__("movdqu %0, %%xmm3" ::"m"(sse_src_pos[-1])); - - if (UNROLL_TIMES >= 4) - __asm__("movdqu %%xmm0, %0" - : "=m"(sse_dst_pos[-4])); - if (UNROLL_TIMES >= 3) - __asm__("movdqu %%xmm1, %0" - : "=m"(sse_dst_pos[-3])); - if (UNROLL_TIMES >= 2) - __asm__("movdqu %%xmm2, %0" - : "=m"(sse_dst_pos[-2])); - if (UNROLL_TIMES >= 1) - __asm__("movdqu %%xmm3, %0" - : "=m"(sse_dst_pos[-1])); } istr.position() += size; @@ -196,22 +173,18 @@ void DataTypeString::deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, ColumnString::Chars_t & data = column_string.getChars(); ColumnString::Offsets & offsets = column_string.getOffsets(); - double avg_chars_size; + double avg_chars_size = 1; /// By default reserve only for empty strings. - if (avg_value_size_hint && avg_value_size_hint > sizeof(offsets[0])) + if (avg_value_size_hint > 0.0 && avg_value_size_hint > sizeof(offsets[0])) { /// Randomly selected. constexpr auto avg_value_size_hint_reserve_multiplier = 1.2; avg_chars_size = (avg_value_size_hint - sizeof(offsets[0])) * avg_value_size_hint_reserve_multiplier; } - else - { - /// By default reserve only for empty strings. - avg_chars_size = 1; - } - data.reserve(data.size() + std::ceil(limit * avg_chars_size)); + size_t size_to_reserve = data.size() + static_cast(std::ceil(limit * avg_chars_size)); + data.reserve(size_to_reserve); offsets.reserve(offsets.size() + limit); @@ -241,7 +214,7 @@ void DataTypeString::serializeTextEscaped(const IColumn & column, size_t row_num template static inline void read(IColumn & column, Reader && reader) { - ColumnString & column_string = static_cast(column); + auto & column_string = static_cast(column); ColumnString::Chars_t & data = column_string.getChars(); ColumnString::Offsets & offsets = column_string.getOffsets(); diff --git a/dbms/src/Flash/Coprocessor/CHBlockChunkCodec.cpp b/dbms/src/Flash/Coprocessor/CHBlockChunkCodec.cpp index 8ea5c7a4a57..d5de380f3ed 100644 --- a/dbms/src/Flash/Coprocessor/CHBlockChunkCodec.cpp +++ b/dbms/src/Flash/Coprocessor/CHBlockChunkCodec.cpp @@ -18,8 +18,14 @@ #include #include #include +#include +#include #include +#include + +#include "Flash/Coprocessor/tzg-metrics.h" + namespace DB { class CHBlockChunkCodecStream : public ChunkCodecStream @@ -80,6 +86,80 @@ size_t getExtraInfoSize(const Block & block) return size; } +/* +class SnappyCompressWriteBuffer final : public BufferWithOwnMemory +{ + WriteBuffer & out; + PODArray compressed_buffer; + + explicit SnappyCompressWriteBuffer(WriteBuffer & out_) + : out(out_) + { + } + + void nextImpl() override + { + if (!offset()) + return; + size_t uncompressed_size = offset(); + size_t compressed_size = snappy::MaxCompressedLength(uncompressed_size); + compressed_buffer.resize(compressed_size); + snappy::RawCompress(working_buffer.begin(), uncompressed_size, compressed_buffer.data(), &compressed_size); + out.write(compressed_buffer.data(), compressed_size); + }; +}; + +class SnappyUncompressWriteBuffer final : public BufferWithOwnMemory +{ + ReadBuffer * compressed_in; + PODArray own_compressed_buffer; + + explicit SnappyUncompressWriteBuffer(ReadBuffer * in_) + : BufferWithOwnMemory(0) + , in(in_) + {} + + bool nextImpl() override + { + if (compressed_in->eof()) + return false; + own_compressed_buffer.resize(COMPRESSED_BLOCK_HEADER_SIZE); + compressed_in->readStrict(&own_compressed_buffer[0], COMPRESSED_BLOCK_HEADER_SIZE); + auto method = own_compressed_buffer[0]; + auto size_compressed = unalignedLoad(&own_compressed_buffer[1]); + auto size_decompressed = unalignedLoad(&own_compressed_buffer[5]); + + if (size_compressed > DBMS_MAX_COMPRESSED_SIZE) + throw Exception("Too large size_compressed. Most likely corrupted data."); + + if (compressed_in->offset() >= COMPRESSED_BLOCK_HEADER_SIZE + && compressed_in->position() + size_compressed - COMPRESSED_BLOCK_HEADER_SIZE <= compressed_in->buffer().end()) + { + compressed_in->position() -= COMPRESSED_BLOCK_HEADER_SIZE; + compressed_buffer = compressed_in->position(); + compressed_in->position() += size_compressed; + } + else + { + own_compressed_buffer.resize(size_compressed); + compressed_buffer = &own_compressed_buffer[0]; + compressed_in->readStrict(compressed_buffer + COMPRESSED_BLOCK_HEADER_SIZE, size_compressed - COMPRESSED_BLOCK_HEADER_SIZE); + } + + if constexpr (has_checksum) + { + if (!disable_checksum[0] && checksum != CityHash_v1_0_2::CityHash128(compressed_buffer, size_compressed)) + throw Exception("Checksum doesn't match: corrupted data.", ErrorCodes::CHECKSUM_DOESNT_MATCH); + return size_compressed + sizeof(checksum); + } + else + { + return size_compressed; + } + } +}; +*/ + void writeData(const IDataType & type, const ColumnPtr & column, WriteBuffer & ostr, size_t offset, size_t limit) { /** If there are columns-constants - then we materialize them. @@ -117,24 +197,34 @@ void CHBlockChunkCodecStream::encode(const Block & block, size_t start, size_t e throw TiFlashException("CHBlock encode only support encode whole block", Errors::Coprocessor::Internal); assert(output == nullptr); - output = std::make_unique(block.bytes() + getExtraInfoSize(block)); + size_t init_size = block.bytes() + getExtraInfoSize(block); + output = std::make_unique(init_size); + std::unique_ptr compress_buffer; + WriteBuffer * ostr_ptr = output.get(); + auto mm = static_cast(tzg::SnappyStatistic::globalInstance().getMethod()); + if (mm != CompressionMethod::NONE) + { + compress_buffer = std::make_unique>(*output, CompressionSettings(mm), init_size); + ostr_ptr = compress_buffer.get(); + } block.checkNumberOfRows(); size_t columns = block.columns(); size_t rows = block.rows(); - writeVarUInt(columns, *output); - writeVarUInt(rows, *output); + writeVarUInt(columns, *ostr_ptr); + writeVarUInt(rows, *ostr_ptr); + for (size_t i = 0; i < columns; i++) { const ColumnWithTypeAndName & column = block.safeGetByPosition(i); - writeStringBinary(column.name, *output); - writeStringBinary(column.type->getName(), *output); + writeStringBinary(column.name, *ostr_ptr); + writeStringBinary(column.type->getName(), *ostr_ptr); if (rows) - writeData(*column.type, column.column, *output, 0, 0); + writeData(*column.type, column.column, *ostr_ptr, 0, 0); } } @@ -146,7 +236,17 @@ std::unique_ptr CHBlockChunkCodec::newCodecStream(const std::v Block CHBlockChunkCodec::decodeImpl(ReadBuffer & istr, size_t reserve_size) { Block res; - if (istr.eof()) + + std::unique_ptr compress_buffer; + ReadBuffer * istr_ptr = &istr; + auto mm = static_cast(tzg::SnappyStatistic::globalInstance().getMethod()); + if (mm != CompressionMethod::NONE) + { + compress_buffer = std::make_unique>(istr); + istr_ptr = compress_buffer.get(); + } + + if (istr_ptr->eof()) { return res; } @@ -154,12 +254,12 @@ Block CHBlockChunkCodec::decodeImpl(ReadBuffer & istr, size_t reserve_size) /// Dimensions size_t columns = 0; size_t rows = 0; - readBlockMeta(istr, columns, rows); + readBlockMeta(*istr_ptr, columns, rows); for (size_t i = 0; i < columns; ++i) { ColumnWithTypeAndName column; - readColumnMeta(i, istr, column); + readColumnMeta(i, *istr_ptr, column); /// Data MutableColumnPtr read_column = column.type->createColumn(); @@ -169,7 +269,7 @@ Block CHBlockChunkCodec::decodeImpl(ReadBuffer & istr, size_t reserve_size) read_column->reserve(rows); if (rows) /// If no rows, nothing to read. - readData(*column.type, *read_column, istr, rows); + readData(*column.type, *read_column, *istr_ptr, rows); column.column = std::move(read_column); res.insert(std::move(column)); diff --git a/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.cpp b/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.cpp index a61a00cc77b..c975282a584 100644 --- a/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.cpp +++ b/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.cpp @@ -15,6 +15,10 @@ #include #include +#include "Flash/Coprocessor/tzg-metrics.h" +#include "IO/CompressedReadBuffer.h" +#include "IO/CompressedStream.h" + namespace DB { CHBlockChunkDecodeAndSquash::CHBlockChunkDecodeAndSquash( @@ -46,10 +50,19 @@ std::optional CHBlockChunkDecodeAndSquash::decodeAndSquash(const String & } else { + std::unique_ptr compress_buffer; + ReadBuffer * istr_ptr = &istr; + auto mm = static_cast(tzg::SnappyStatistic::globalInstance().getMethod()); + if (mm != CompressionMethod::NONE) + { + compress_buffer = std::make_unique>(istr); + istr_ptr = compress_buffer.get(); + } + /// Dimensions size_t columns = 0; size_t rows = 0; - codec.readBlockMeta(istr, columns, rows); + codec.readBlockMeta(*istr_ptr, columns, rows); if (rows) { @@ -57,8 +70,8 @@ std::optional CHBlockChunkDecodeAndSquash::decodeAndSquash(const String & for (size_t i = 0; i < columns; ++i) { ColumnWithTypeAndName column; - codec.readColumnMeta(i, istr, column); - CHBlockChunkCodec::readData(*column.type, *(mutable_columns[i]), istr, rows); + codec.readColumnMeta(i, *istr_ptr, column); + CHBlockChunkCodec::readData(*column.type, *(mutable_columns[i]), *istr_ptr, rows); } accumulated_block->setColumns(std::move(mutable_columns)); } diff --git a/dbms/src/Flash/Coprocessor/tzg-metrics.h b/dbms/src/Flash/Coprocessor/tzg-metrics.h new file mode 100644 index 00000000000..90374cb1742 --- /dev/null +++ b/dbms/src/Flash/Coprocessor/tzg-metrics.h @@ -0,0 +1,72 @@ +#pragma once + +#include +#include + +namespace tzg +{ +struct SnappyStatistic +{ + enum class Method : int + { + LZ4 = 1, + LZ4HC = 2, /// The format is the same as for LZ4. The difference is only in compression. + ZSTD = 3, /// Experimental algorithm: https://github.com/Cyan4973/zstd + NONE = 4, /// No compression + SNAPPY = 5, /// Snappy: https://github.com/google/snappy + }; + + mutable std::atomic_uint64_t compressed_size; + mutable std::atomic_uint64_t uncompressed_size; + mutable std::atomic_uint64_t package; + mutable Method method = Method::SNAPPY; + + SnappyStatistic(const SnappyStatistic &) = delete; + + void clear() + { + compressed_size = {}; + uncompressed_size = {}; + package = {}; + } + + uint64_t getCompressedSize() const + { + return compressed_size; + } + uint64_t getUncompressedSize() const + { + return uncompressed_size; + } + + Method getMethod() const + { + return method; + } + void setMethod(Method m) + { + method = m; + } + + void update(uint64_t compressed_size_, uint64_t uncompressed_size_) + { + compressed_size += compressed_size_; + uncompressed_size += uncompressed_size_; + ++package; + } + + void load(uint64_t & compressed_size_, uint64_t & uncompressed_size_, uint64_t & package_, Method & m) const + { + compressed_size_ = getCompressedSize(); + uncompressed_size_ = getUncompressedSize(); + package_ = package; + m = getMethod(); + } + + static SnappyStatistic & globalInstance() + { + static SnappyStatistic data{}; + return data; + } +}; +} // namespace tzg \ No newline at end of file diff --git a/dbms/src/IO/CompressedReadBufferBase.cpp b/dbms/src/IO/CompressedReadBufferBase.cpp index f5ce5ed41e7..76103c64c00 100644 --- a/dbms/src/IO/CompressedReadBufferBase.cpp +++ b/dbms/src/IO/CompressedReadBufferBase.cpp @@ -23,6 +23,7 @@ #include #include #include +#include #include #include @@ -63,8 +64,10 @@ size_t CompressedReadBufferBase::readCompressedData(size_t & size_ size_t & size_compressed = size_compressed_without_checksum; - if (method == static_cast(CompressionMethodByte::LZ4) || method == static_cast(CompressionMethodByte::ZSTD) - || method == static_cast(CompressionMethodByte::NONE)) + if (method == static_cast(CompressionMethodByte::LZ4) + || method == static_cast(CompressionMethodByte::ZSTD) + || method == static_cast(CompressionMethodByte::NONE) + || method == static_cast(CompressionMethodByte::SNAPPY)) { size_compressed = unalignedLoad(&own_compressed_buffer[1]); size_decompressed = unalignedLoad(&own_compressed_buffer[5]); @@ -119,6 +122,12 @@ void CompressedReadBufferBase::decompress(char * to, size_t size_d if (ZSTD_isError(res)) throw Exception("Cannot ZSTD_decompress: " + std::string(ZSTD_getErrorName(res)), ErrorCodes::CANNOT_DECOMPRESS); } + else if (method == static_cast(CompressionMethodByte::SNAPPY)) + { + auto res = snappy::RawUncompress(compressed_buffer + COMPRESSED_BLOCK_HEADER_SIZE, size_compressed_without_checksum - COMPRESSED_BLOCK_HEADER_SIZE, to); + if (!res) + throw Exception("Cannot decompress by SNAPPY", ErrorCodes::CANNOT_DECOMPRESS); + } else if (method == static_cast(CompressionMethodByte::NONE)) { memcpy(to, &compressed_buffer[COMPRESSED_BLOCK_HEADER_SIZE], size_decompressed); diff --git a/dbms/src/IO/CompressedStream.h b/dbms/src/IO/CompressedStream.h index 9c4cc9fc7d3..c55c5975370 100644 --- a/dbms/src/IO/CompressedStream.h +++ b/dbms/src/IO/CompressedStream.h @@ -32,6 +32,7 @@ enum class CompressionMethod LZ4HC = 2, /// The format is the same as for LZ4. The difference is only in compression. ZSTD = 3, /// Experimental algorithm: https://github.com/Cyan4973/zstd NONE = 4, /// No compression + SNAPPY = 5, /// Snappy: https://github.com/google/snappy }; /** The compressed block format is as follows: @@ -62,6 +63,7 @@ enum class CompressionMethodByte : uint8_t ZSTD = 0x90, // COL_END is not a compreesion method, but a flag of column end used in compact file. COL_END = 0x66, + SNAPPY = 0x67, }; } // namespace DB diff --git a/dbms/src/IO/CompressedWriteBuffer.cpp b/dbms/src/IO/CompressedWriteBuffer.cpp index 738830217c0..4e385992c3e 100644 --- a/dbms/src/IO/CompressedWriteBuffer.cpp +++ b/dbms/src/IO/CompressedWriteBuffer.cpp @@ -18,11 +18,14 @@ #include #include #include +#include #include #include #include +#include "Flash/Coprocessor/tzg-metrics.h" + namespace DB { @@ -53,10 +56,7 @@ void CompressedWriteBuffer::nextImpl() { static constexpr size_t header_size = 1 + sizeof(UInt32) + sizeof(UInt32); -#pragma GCC diagnostic push -#pragma GCC diagnostic ignored "-Wold-style-cast" compressed_buffer.resize(header_size + LZ4_COMPRESSBOUND(uncompressed_size)); -#pragma GCC diagnostic pop compressed_buffer[0] = static_cast(CompressionMethodByte::LZ4); @@ -122,6 +122,31 @@ void CompressedWriteBuffer::nextImpl() compressed_buffer_ptr = &compressed_buffer[0]; break; } + case CompressionMethod::SNAPPY: + { + static constexpr size_t header_size = 1 + sizeof(UInt32) + sizeof(UInt32); + compressed_size = snappy::MaxCompressedLength(uncompressed_size); + compressed_buffer.resize(header_size + compressed_size); + + compressed_buffer[0] = static_cast(CompressionMethodByte::SNAPPY); + + snappy::RawCompress( + working_buffer.begin(), + uncompressed_size, + &compressed_buffer[header_size], + &compressed_size); + + compressed_size = header_size + compressed_size; + + UInt32 compressed_size_32 = compressed_size; + UInt32 uncompressed_size_32 = uncompressed_size; + + unalignedStore(&compressed_buffer[1], compressed_size_32); + unalignedStore(&compressed_buffer[5], uncompressed_size_32); + + compressed_buffer_ptr = &compressed_buffer[0]; + break; + } default: throw Exception("Unknown compression method", ErrorCodes::UNKNOWN_COMPRESSION_METHOD); } @@ -131,6 +156,7 @@ void CompressedWriteBuffer::nextImpl() CityHash_v1_0_2::uint128 checksum = CityHash_v1_0_2::CityHash128(compressed_buffer_ptr, compressed_size); out.write(reinterpret_cast(&checksum), sizeof(checksum)); } + tzg::SnappyStatistic::globalInstance().update(compressed_size, uncompressed_size); out.write(compressed_buffer_ptr, compressed_size); } diff --git a/dbms/src/IO/CompressedWriteBuffer.h b/dbms/src/IO/CompressedWriteBuffer.h index 69bd87c6956..1fc169cffb4 100644 --- a/dbms/src/IO/CompressedWriteBuffer.h +++ b/dbms/src/IO/CompressedWriteBuffer.h @@ -36,7 +36,7 @@ class CompressedWriteBuffer : public BufferWithOwnMemory void nextImpl() override; public: - CompressedWriteBuffer( + explicit CompressedWriteBuffer( WriteBuffer & out_, CompressionSettings compression_settings = CompressionSettings(), size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE); diff --git a/dbms/src/Storages/Transaction/ProxyFFIStatusService.cpp b/dbms/src/Storages/Transaction/ProxyFFIStatusService.cpp index f3551bed94e..ec8d4ea0fad 100644 --- a/dbms/src/Storages/Transaction/ProxyFFIStatusService.cpp +++ b/dbms/src/Storages/Transaction/ProxyFFIStatusService.cpp @@ -19,6 +19,13 @@ #include #include #include +#include + +#include +#include + +#include "Flash/Coprocessor/tzg-metrics.h" +#include "magic_enum.hpp" namespace DB { @@ -113,11 +120,75 @@ HttpRequestRes HandleHttpRequestStoreStatus( .view = BaseBuffView{name->data(), name->size()}}}; } +HttpRequestRes HandleHttpRequestCompressStatus( + EngineStoreServerWrap *, + std::string_view path, + const std::string &, + std::string_view, + std::string_view) +{ + uint64_t compressed_size; + uint64_t uncompressed_size; + uint64_t package; + tzg::SnappyStatistic::Method method; + tzg::SnappyStatistic::globalInstance().load(compressed_size, uncompressed_size, package, method); + + // double f_compressed_size_mb = compressed_size * 1.0 / 1024 / 1024; + // double f_uncompressed_size_mb = uncompressed_size * 1.0 / 1024 / 1024; + auto * res = RawCppString::New(fmt::format("package: {}, compressed_size: {}, uncompressed_size: {}, compress_rate: {:.2f}, method: {}", + package, + compressed_size, + uncompressed_size, + uncompressed_size ? double(compressed_size) / uncompressed_size : 0.0, + magic_enum::enum_name(method))); + static const std::string_view clean_str = "tzg-compress-and-clean"; + if (path.find(clean_str) != path.npos) + { + tzg::SnappyStatistic::globalInstance().clear(); + res->append(", clean statistic"); + } + return HttpRequestRes{ + .status = HttpRequestStatus::Ok, + .res = CppStrWithView{ + .inner = GenRawCppPtr(res, RawCppPtrTypeImpl::String), + .view = BaseBuffView{res->data(), res->size()}}}; +} + +HttpRequestRes HandleHttpRequestSetCompressMethod( + EngineStoreServerWrap *, + std::string_view path, + const std::string & api_name, + std::string_view, + std::string_view) +{ + auto method_str(path.substr(api_name.size())); + auto method = magic_enum::enum_cast(method_str); + if (method) + { + tzg::SnappyStatistic::globalInstance().setMethod(*method); + auto * res = RawCppString::New(fmt::format("Set compress method to {}", method_str)); + return HttpRequestRes{ + .status = HttpRequestStatus::Ok, + .res = CppStrWithView{ + .inner = GenRawCppPtr(res, RawCppPtrTypeImpl::String), + .view = BaseBuffView{res->data(), res->size()}}}; + } + else + { + return HttpRequestRes{.status = HttpRequestStatus::ErrorParam, .res = CppStrWithView{.inner = GenRawCppPtr(), .view = BaseBuffView{}}}; + } +} + + using HANDLE_HTTP_URI_METHOD = HttpRequestRes (*)(EngineStoreServerWrap *, std::string_view, const std::string &, std::string_view, std::string_view); static const std::map AVAILABLE_HTTP_URI = { {"/tiflash/sync-status/", HandleHttpRequestSyncStatus}, - {"/tiflash/store-status", HandleHttpRequestStoreStatus}}; + {"/tiflash/store-status", HandleHttpRequestStoreStatus}, + {"/tiflash/tzg-compress", HandleHttpRequestCompressStatus}, + {"/tiflash/tzg-compress-and-clean", HandleHttpRequestCompressStatus}, + {"/tiflash/set-tzg-compress-method/", HandleHttpRequestSetCompressMethod}, +}; uint8_t CheckHttpUriAvailable(BaseBuffView path_) { diff --git a/libs/CMakeLists.txt b/libs/CMakeLists.txt index 01f1fa9ac37..e10fc785e97 100644 --- a/libs/CMakeLists.txt +++ b/libs/CMakeLists.txt @@ -20,7 +20,7 @@ endif () add_subdirectory (libcommon) add_subdirectory (libpocoext) add_subdirectory (libdaemon) - +add_subdirectory (libsnappy) add_subdirectory (libmemcpy) if (GLIBC_COMPATIBILITY) diff --git a/libs/libsnappy/CMakeLists.txt b/libs/libsnappy/CMakeLists.txt new file mode 100644 index 00000000000..c7ebbbe3059 --- /dev/null +++ b/libs/libsnappy/CMakeLists.txt @@ -0,0 +1,27 @@ +# Copyright 2022 PingCAP, Ltd. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +if (TIFLASH_ENABLE_AVX_SUPPORT) + set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -mavx2") + set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} mavx2") + set (SNAPPY_REQUIRE_AVX2 ON) +endif () + +#set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -fomit-frame-pointer") +#set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -fomit-frame-pointer") + +set (SNAPPY_BUILD_TESTS OFF CACHE BOOL "Build Snappy's own tests.") +set (SNAPPY_BUILD_BENCHMARKS OFF CACHE BOOL "Build Snappy for fuzzing.") + +add_subdirectory (snappy) diff --git a/tiflash-tools/.gitignore b/tiflash-tools/.gitignore new file mode 100644 index 00000000000..6b1b5d54dad --- /dev/null +++ b/tiflash-tools/.gitignore @@ -0,0 +1,5 @@ +/dist/ +/build/ +*.spec +version.py +/tikv_util/.eggs/ diff --git a/tiflash-tools/define.py b/tiflash-tools/define.py new file mode 100644 index 00000000000..2e01aca50a1 --- /dev/null +++ b/tiflash-tools/define.py @@ -0,0 +1,18 @@ +#!/usr/bin/python3 + +TIFLASH = 'tiflash' +TIFLASH_LABEL = {'key': 'engine', 'value': TIFLASH} +REGION_COUNT = 'region_count' +TIFLASH_REGION_COUNT = 'flash_region_count' +LOCATION_LABELS = 'location_labels' +REPLICA_COUNT = 'replica_count' +LEARNER = 'learner' +AVAILABLE = 'available' +PRIORITY = 'high_priority' +TIFLASH_CLUSTER_MUTEX_KEY = '/{}/cluster/leader'.format(TIFLASH) +TIFLASH_CLUSTER_HTTP_PORT = '/{}/cluster/http_port/'.format(TIFLASH) +LABEL_CONSTRAINTS = 'label_constraints' +DDL_GLOBAL_SCHEMA_VERSION = '/tidb/ddl/global_schema_version' +TIFLASH_LAST_HANDLED_SCHEMA_VERSION = '/{}/cluster/last_handled_schema_version'.format(TIFLASH) +TIFLASH_LAST_HANDLED_SCHEMA_VERSION_TSO_SPLIT = b'_tso_' +TIFLASH_LAST_HANDLED_SCHEMA_TIME_OUT = 300 diff --git a/tiflash-tools/flash_cluster_manager.py b/tiflash-tools/flash_cluster_manager.py new file mode 100644 index 00000000000..a3b90c90b75 --- /dev/null +++ b/tiflash-tools/flash_cluster_manager.py @@ -0,0 +1,94 @@ +#!/usr/bin/python3 +import argparse +import logging + +import define +import flash_http_client +from pd_client import PDClient + + +class Store: + def __eq__(self, other): + return self.inner == other + + def __str__(self): + return str(self.inner) + + def __init__(self, pd_store): + self.inner = pd_store + address = self.inner['address'] + host, port = address.split(':') + self.address = '{}:{}'.format(host, port) + _, status_port = self.inner['status_address'].split(':') + self.tiflash_status_address = '{}:{}'.format(host, status_port) + + @property + def id(self): + return self.inner['id'] + + +class Runner: + def __init__(self): + parser = argparse.ArgumentParser(description="check compress statistic", + formatter_class=argparse.ArgumentDefaultsHelpFormatter) + parser.add_argument( + '--pd-address', default="172.16.4.39:2174") + parser.add_argument( + '--type', help='run type', required=True, choices=('show', 'clean', 'set')) + parser.add_argument( + '--method', choices=('SNAPPY', 'LZ4', 'ZSTD', 'NONE')) + self.args = parser.parse_args() + try: + assert self.args.pd_address + pd_address = self.args.pd_address.split(",") + pd_client = PDClient(pd_address) + self.tiflash_stores = {store_id: Store(store) for store_id, store in pd_client.get_store_by_labels( + define.TIFLASH_LABEL).items()} + except Exception as e: + logging.exception(e) + + def run(self): + if self.args.type == 'show': + self.run_show() + elif self.args.type == 'clean': + self.run_clean() + elif self.args.type == 'set': + self.run_set_method() + + def run_clean(self): + res = {store_id: flash_http_client.clean_compress_info( + store.tiflash_status_address) for store_id, store in self.tiflash_stores.items()} + print(res) + + def run_set_method(self): + assert self.args.method + res = {store_id: flash_http_client.set_compress_method( + store.tiflash_status_address, self.args.method) for store_id, store in self.tiflash_stores.items()} + print(res) + + def run_show(self): + compress_info = {store_id: flash_http_client.get_compress_info( + store.tiflash_status_address) for store_id, store in self.tiflash_stores.items()} + tol_compressed_size = 0 + tol_uncompressed_size = 0 + method = None + for store_id, info in compress_info.items(): + print(info) + tol_compressed_size += int(info['compressed_size']) + tol_uncompressed_size += int(info['uncompressed_size']) + if method: + assert method == info['method'] + else: + method = info['method'] + compress_rate = 0.0 if not tol_uncompressed_size else tol_compressed_size / \ + tol_uncompressed_size + MB = 1024**2 + msg = 'method: {}, compress_rate: {:.3f}, compressed_size: {:.3f}MB, uncompressed_size: {:.3f}MB'.format( + method, compress_rate, + tol_compressed_size/MB, tol_uncompressed_size/MB, + ) + print(msg) + + +if __name__ == '__main__': + Runner().run() diff --git a/tiflash-tools/flash_http_client.py b/tiflash-tools/flash_http_client.py new file mode 100644 index 00000000000..3450f4e28bf --- /dev/null +++ b/tiflash-tools/flash_http_client.py @@ -0,0 +1,37 @@ +#!/usr/bin/python3 + +import util + + +def curl_flash(address, params): + if type(params) != dict: + params = {'query': params} + r = util.curl_http(address, params) + return r + + +def get_compress_info(tiflash_status_address): + res = util.curl_http( + '{}/tiflash/tzg-compress'.format(tiflash_status_address)) + util.check_status_code(res) + data = res.text + data = [e.strip() for e in data.split(',')] + res = {} + for e in data: + a, b = e.split(":") + res[a.strip()] = b.strip() + return res + + +def set_compress_method(tiflash_status_address, method): + res = util.curl_http( + '{}/tiflash/set-tzg-compress-method/{}'.format(tiflash_status_address, method)) + util.check_status_code(res) + return res.text + + +def clean_compress_info(tiflash_status_address, ): + res = util.curl_http( + '{}/tiflash/tzg-compress-and-clean'.format(tiflash_status_address, )) + util.check_status_code(res) + return res.text diff --git a/tiflash-tools/pd_client.py b/tiflash-tools/pd_client.py new file mode 100644 index 00000000000..4f25bc800e5 --- /dev/null +++ b/tiflash-tools/pd_client.py @@ -0,0 +1,126 @@ +#!/usr/bin/python3 +import logging +from typing import Optional + +import uri +import util + + +class PDClient: + PD_API_PREFIX = 'pd/api' + PD_API_VERSION = 'v1' + + def get_all_regions_json(self): + r = util.curl_http('{}/{}/{}/regions'.format(self.leader, + PDClient.PD_API_PREFIX, PDClient.PD_API_VERSION)) + return util.try_get_json(r) + + def get_regions_by_key_json(self, key: str, limit=16): + r = util.curl_http( + '{}/{}/{}/regions/key'.format(self.leader, + PDClient.PD_API_PREFIX, PDClient.PD_API_VERSION), + {'key': key, 'limit': limit}) + return util.try_get_json(r) + + def get_region_by_id_json(self, region_id: int): + r = util.curl_http( + '{}/{}/{}/region/id/{}'.format(self.leader, PDClient.PD_API_PREFIX, PDClient.PD_API_VERSION, region_id)) + return util.try_get_json(r) + + def get_all_stores_json(self): + r = util.curl_http( + '{}/{}/{}/stores'.format(self.leader, PDClient.PD_API_PREFIX, PDClient.PD_API_VERSION)) + return util.try_get_json(r) + + def get_members_json(self, *args): + url = args[0] if args else self.leader + r = util.curl_http( + '{}/{}/{}/members'.format(url, PDClient.PD_API_PREFIX, PDClient.PD_API_VERSION)) + return util.try_get_json(r) + + def get_stats_region_by_range_json(self, start_key, end_key): + r = util.curl_http( + '{}/{}/{}/stats/region'.format(self.leader, + PDClient.PD_API_PREFIX, PDClient.PD_API_VERSION), + {'start_key': start_key, 'end_key': end_key}, + ) + return util.try_get_json(r) + + def get_group_rules(self, group): + r = util.curl_http( + '{}/{}/{}/config/rules/group/{}'.format(self.leader, PDClient.PD_API_PREFIX, PDClient.PD_API_VERSION, + group)) + res = util.try_get_json(r) + res = res if res is not None else {} + for e in res: + if not isinstance(e, dict): + raise Exception('Got placement rules fail: {}'.format(r.text)) + from placement_rule import PlacementRule + return {e['id']: PlacementRule(**e) for e in res} + + def get_all_rules(self): + r = util.curl_http( + '{}/{}/{}/config/rules'.format(self.leader, PDClient.PD_API_PREFIX, PDClient.PD_API_VERSION)) + res = util.try_get_json(r) + return res if res is not None else {} + + def get_rule(self, group, rule_id): + r = util.curl_http( + '{}/{}/{}/config/rule/{}/{}'.format(self.leader, PDClient.PD_API_PREFIX, PDClient.PD_API_VERSION, group, + rule_id)) + return util.try_get_json(r) + + def set_rule(self, rule): + r = util.post_http( + '{}/{}/{}/config/rule'.format(self.leader, PDClient.PD_API_PREFIX, PDClient.PD_API_VERSION), rule) + return r.status_code + + def set_accelerate_schedule(self, start_key, end_key): + r = util.post_http( + '{}/{}/{}/regions/accelerate-schedule'.format( + self.leader, PDClient.PD_API_PREFIX, PDClient.PD_API_VERSION), + {'start_key': start_key, 'end_key': end_key}, ) + if r.status_code != 200: + raise Exception( + "fail to accelerate schedule range [{},{}), error msg: {}".format(start_key, end_key, r.text)) + + def remove_rule(self, group, rule_id): + r = util.delete_http( + '{}/{}/{}/config/rule/{}/{}'.format(self.leader, PDClient.PD_API_PREFIX, PDClient.PD_API_VERSION, group, + rule_id)) + return r.status_code + + def _try_update_leader_etcd(self, url): + resp = self.get_members_json(url) + leader = resp.get('leader', {}) + client_urls = leader.get('client_urls', []) + if client_urls: + _client_urls = [] + for member in resp.get('members', {}): + _client_urls.extend(member.get('client_urls', [])) + self.urls = _client_urls + self.leader = uri.URI(client_urls[0]).authority + + def _update_leader_etcd(self): + errors = [] + for url in self.urls: + try: + return self._try_update_leader_etcd(url) + except Exception as e: + errors.append(e) + raise Exception("can not find pd leader: {}".format(errors)) + + def get_store_by_labels(self, flash_label): + res = {} + all_stores = self.get_all_stores_json() + for store in all_stores['stores']: + store = store['store'] + for label in store.get('labels', []): + if label == flash_label: + res[store['id']] = store + return res + + def __init__(self, urls): + self.urls = urls + self.leader = "" + self._update_leader_etcd() diff --git a/tiflash-tools/util.py b/tiflash-tools/util.py new file mode 100644 index 00000000000..1986782d91c --- /dev/null +++ b/tiflash-tools/util.py @@ -0,0 +1,138 @@ +#!/usr/bin/python3 +import errno +import fcntl +import logging +import os +import socket +import time + +import requests + + +def wrap_run_time(func): + def wrap_func(*args, **kwargs): + bg = time.time() + r = func(*args, **kwargs) + print('time cost {}'.format(time.time() - bg)) + return r + + return wrap_func + + +class FLOCK(object): + def __init__(self, name): + self.obj = open(name, 'w') + self.fd = self.obj.fileno() + + def lock(self): + try: + fcntl.lockf(self.fd, fcntl.LOCK_EX | fcntl.LOCK_NB) + return True + except OSError: + logging.error( + 'Cannot lock file {}. Another instance in same directory is already running'.format(self.obj.name)) + return False + + +def gen_http_kwargs(): + kwargs = {} + http_name = 'http' + return http_name, kwargs + + +def curl_http(uri, params=None): + if params is None: + params = {} + http_name, kwargs = gen_http_kwargs() + r = requests.get('{}://{}'.format(http_name, uri), params, **kwargs) + return r + + +def check_status_code(r): + if r.status_code != 200: + raise Exception( + 'unexpected status code {} from {}'.format(r.status_code, r.url)) + + +def try_get_json(r): + check_status_code(r) + return r.json() + + +def post_http(uri, params): + http_name, kwargs = gen_http_kwargs() + r = requests.post('{}://{}'.format(http_name, uri), json=params, **kwargs) + return r + + +def delete_http(uri): + http_name, kwargs = gen_http_kwargs() + r = requests.delete('{}://{}'.format(http_name, uri), **kwargs) + return r + + +def obj_2_dict(obj): + pr = {} + for name in dir(obj): + value = getattr(obj, name) + if not name.startswith('_') and not callable(value): + pr[name] = value + return pr + + +def make_compare_pd_key(key): + return (1, key) if key else (0, '') + + +def net_is_used(ip, port): + s = socket.socket(socket.AF_INET, socket.SOCK_STREAM) + result = s.connect_ex((ip, port)) + s.close() + return result == 0 + + +def pid_exists(pid): + """Check whether pid exists in the current process table. + UNIX only. + """ + if pid < 0: + return False + if pid == 0: + # According to "man 2 kill" PID 0 refers to every process + # in the process group of the calling process. + # On certain systems 0 is a valid PID but we have no way + # to know that in a portable fashion. + raise ValueError('invalid PID 0') + try: + os.kill(pid, 0) + except OSError as err: + if err.errno == errno.ESRCH: + # ESRCH == No such process + return False + elif err.errno == errno.EPERM: + # EPERM clearly means there's a process to deny access to + return True + else: + # According to "man 2 kill" possible error values are + # (EINVAL, EPERM, ESRCH) + raise + else: + return True + + +def pid_exists2(pid): + if pid == 0: + return True + return pid_exists(pid) + + +def compute_addr_list(addrs): + return [e.strip() for e in addrs.split(',') if e] + + +def main(): + pass + + +if __name__ == '__main__': + main() From d6495b1dc2cdc5f417d48ecd402df7514dc63ef3 Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Tue, 22 Nov 2022 17:48:41 +0800 Subject: [PATCH 05/93] test Signed-off-by: Zhigao Tong --- .gitmodules | 1 + contrib/CMakeLists.txt | 25 +++ contrib/tzg.mpp.proto.patch | 25 +++ dbms/CMakeLists.txt | 6 +- .../ExchangeSenderBlockInputStream.h | 2 +- .../Flash/Coprocessor/CHBlockChunkCodec.cpp | 184 +++++++++--------- .../src/Flash/Coprocessor/CHBlockChunkCodec.h | 6 +- .../Coprocessor/ChunkDecodeAndSquash.cpp | 21 +- .../Flash/Coprocessor/ChunkDecodeAndSquash.h | 4 +- .../Coprocessor/CompressedCHBlockChunkCodec.h | 45 +++++ dbms/src/Flash/Coprocessor/tzg-metrics.h | 61 +++++- dbms/src/Flash/Mpp/ExchangeReceiver.cpp | 5 +- dbms/src/Flash/Mpp/HashPartitionWriter.cpp | 72 ++++++- dbms/src/Flash/Mpp/HashPartitionWriter.h | 24 ++- dbms/src/Flash/Mpp/LocalBlockMPPTunnelSet.h | 96 +++++++++ dbms/src/Flash/Mpp/MPPTunnel.cpp | 22 ++- dbms/src/Flash/Mpp/MPPTunnel.h | 7 +- dbms/src/Flash/Mpp/newMPPExchangeWriter.h | 22 ++- dbms/src/IO/CompressedReadBufferBase.cpp | 14 +- dbms/src/IO/CompressedWriteBuffer.cpp | 40 ++-- .../Transaction/ProxyFFIStatusService.cpp | 69 ++++++- libs/CMakeLists.txt | 1 - libs/libcommon/include/common/avx2_memcpy.h | 2 +- tiflash-tools/flash_cluster_manager.py | 48 ++++- tiflash-tools/flash_http_client.py | 16 +- tiflash-tools/util.py | 2 +- 26 files changed, 653 insertions(+), 167 deletions(-) create mode 100644 contrib/tzg.mpp.proto.patch create mode 100644 dbms/src/Flash/Coprocessor/CompressedCHBlockChunkCodec.h create mode 100644 dbms/src/Flash/Mpp/LocalBlockMPPTunnelSet.h diff --git a/.gitmodules b/.gitmodules index 44bb7d920ab..9665bc2c5da 100644 --- a/.gitmodules +++ b/.gitmodules @@ -25,6 +25,7 @@ [submodule "contrib/kvproto"] path = contrib/kvproto url = https://github.com/pingcap/kvproto.git + ignore = dirty [submodule "contrib/tipb"] path = contrib/tipb url = https://github.com/pingcap/tipb.git diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index cf5c6d5704d..8c84924eaaa 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -111,6 +111,31 @@ if (USE_INTERNAL_GRPC_LIBRARY) unset (CMAKE_POLICY_DEFAULT_CMP0077) endif () +execute_process( + COMMAND grep "CompressMethod compress = 5;" "${TiFlash_SOURCE_DIR}/contrib/kvproto/proto/mpp.proto" + RESULT_VARIABLE HAVE_APPLY_PATCH) + +# grep - Normally, the exit status is 0 if selected lines are found and 1 otherwise. But the exit status is 2 if an error occurred. +if (HAVE_APPLY_PATCH EQUAL 1) + message(STATUS "mpp compress patch not apply: ${HAVE_APPLY_PATCH}, patching...") + set (MPP_PATCH_FILE "${TiFlash_SOURCE_DIR}/contrib/tzg.mpp.proto.patch") + # apply the patch + execute_process( + COMMAND git apply -v "${MPP_PATCH_FILE}" + WORKING_DIRECTORY "${TiFlash_SOURCE_DIR}/contrib/kvproto" + COMMAND_ECHO STDOUT + RESULT_VARIABLE PATCH_SUCC) + if (NOT PATCH_SUCC EQUAL 0) + message(FATAL_ERROR "Can not apply mpp compress patch ${MPP_PATCH_FILE}") + else () + message(STATUS "mpp compress patch done") + endif () +elseif (HAVE_APPLY_PATCH EQUAL 0) + message(STATUS "mpp compress patch have been applied: ${HAVE_APPLY_PATCH}") +else () + message(FATAL_ERROR "Can not check the mpp compress patch status") +endif () + add_subdirectory (kvproto/cpp) target_no_warning(kvproto deprecated-declarations) target_no_warning(kvproto non-c-typedef-for-linkage) diff --git a/contrib/tzg.mpp.proto.patch b/contrib/tzg.mpp.proto.patch new file mode 100644 index 00000000000..79b6d38abc5 --- /dev/null +++ b/contrib/tzg.mpp.proto.patch @@ -0,0 +1,25 @@ +diff --git a/proto/mpp.proto b/proto/mpp.proto +index f8af03b..d9ab839 100644 +--- a/proto/mpp.proto ++++ b/proto/mpp.proto +@@ -60,6 +60,12 @@ message EstablishMPPConnectionRequest { + TaskMeta receiver_meta = 2; // node closer to the tidb mpp gather. + } + ++enum CompressMethod { ++ NONE = 0; ++ LZ4 = 1; ++ ZSTD = 2; ++} ++ + // when TiFlash sends data to TiDB, Data packets wrap tipb.SelectResponse, i.e., serialize tipb.SelectResponse into data; + // when TiFlash sends data to TiFlash, data blocks are serialized into chunks, and the execution_summaries in tipb.SelectResponse are serialized into data only for the last packet. + message MPPDataPacket { +@@ -67,6 +73,7 @@ message MPPDataPacket { + Error error = 2; + repeated bytes chunks = 3; + repeated uint64 stream_ids = 4; ++ CompressMethod compress = 5; + } + + message Error { diff --git a/dbms/CMakeLists.txt b/dbms/CMakeLists.txt index 1dc07d4b7cb..025dda30e48 100644 --- a/dbms/CMakeLists.txt +++ b/dbms/CMakeLists.txt @@ -211,11 +211,7 @@ target_link_libraries (clickhouse_common_io libsymbolization ) -target_link_libraries ( - clickhouse_common_io - PRIVATE - snappy -) +# target_link_libraries (clickhouse_common_io PRIVATE snappy) target_include_directories (clickhouse_common_io BEFORE PRIVATE ${kvClient_SOURCE_DIR}/include) target_compile_definitions(clickhouse_common_io PUBLIC -DTIFLASH_SOURCE_PREFIX=\"${TiFlash_SOURCE_DIR}\") diff --git a/dbms/src/DataStreams/ExchangeSenderBlockInputStream.h b/dbms/src/DataStreams/ExchangeSenderBlockInputStream.h index 9044ff293dc..3cae52adf02 100644 --- a/dbms/src/DataStreams/ExchangeSenderBlockInputStream.h +++ b/dbms/src/DataStreams/ExchangeSenderBlockInputStream.h @@ -47,7 +47,7 @@ class ExchangeSenderBlockInputStream : public IProfilingBlockInputStream void readSuffixImpl() override { writer->finishWrite(); - LOG_DEBUG(log, "finish write with {} rows", total_rows); + LOG_DEBUG(log, "finish write with {} rows, cost time {:.3f}s", total_rows, getProfileInfo().execution_time / 1e9); } private: diff --git a/dbms/src/Flash/Coprocessor/CHBlockChunkCodec.cpp b/dbms/src/Flash/Coprocessor/CHBlockChunkCodec.cpp index d5de380f3ed..b3f892a294a 100644 --- a/dbms/src/Flash/Coprocessor/CHBlockChunkCodec.cpp +++ b/dbms/src/Flash/Coprocessor/CHBlockChunkCodec.cpp @@ -17,14 +17,15 @@ #include #include #include +#include #include #include #include #include -#include - #include "Flash/Coprocessor/tzg-metrics.h" +#include "ext/scope_guard.h" +#include "mpp.pb.h" namespace DB { @@ -48,11 +49,54 @@ class CHBlockChunkCodecStream : public ChunkCodecStream } return output->releaseStr(); } + virtual WriteBuffer * initOutput(size_t init_size) + { + assert(output == nullptr); + output = std::make_unique(init_size); + return output.get(); + } void clear() override { output = nullptr; } void encode(const Block & block, size_t start, size_t end) override; std::unique_ptr output; DataTypes expected_types; + + ~CHBlockChunkCodecStream() override = default; +}; + +class CompressCHBlockChunkCodecStream final : public CHBlockChunkCodecStream +{ + using Base = CHBlockChunkCodecStream; + +public: + explicit CompressCHBlockChunkCodecStream(const std::vector & field_types, CompressionMethod compress_method_ = CompressionMethod::LZ4) + : Base(field_types) + , compress_method(compress_method_) + { + } + WriteBuffer * initOutput(size_t init_size) override + { + assert(compress_write_buffer == nullptr); + compress_write_buffer = std::make_unique>(*Base::initOutput(init_size), CompressionSettings(compress_method), init_size); + return compress_write_buffer.get(); + } + void clear() override + { + compress_write_buffer = nullptr; + Base::clear(); + } + String getString() override + { + if (compress_write_buffer == nullptr) + { + throw Exception("The output should not be null in getString()"); + } + compress_write_buffer->next(); + return Base::getString(); + } + CompressionMethod compress_method; + std::unique_ptr> compress_write_buffer{}; + ~CompressCHBlockChunkCodecStream() override = default; }; CHBlockChunkCodec::CHBlockChunkCodec( @@ -86,80 +130,6 @@ size_t getExtraInfoSize(const Block & block) return size; } -/* -class SnappyCompressWriteBuffer final : public BufferWithOwnMemory -{ - WriteBuffer & out; - PODArray compressed_buffer; - - explicit SnappyCompressWriteBuffer(WriteBuffer & out_) - : out(out_) - { - } - - void nextImpl() override - { - if (!offset()) - return; - size_t uncompressed_size = offset(); - size_t compressed_size = snappy::MaxCompressedLength(uncompressed_size); - compressed_buffer.resize(compressed_size); - snappy::RawCompress(working_buffer.begin(), uncompressed_size, compressed_buffer.data(), &compressed_size); - out.write(compressed_buffer.data(), compressed_size); - }; -}; - -class SnappyUncompressWriteBuffer final : public BufferWithOwnMemory -{ - ReadBuffer * compressed_in; - PODArray own_compressed_buffer; - - explicit SnappyUncompressWriteBuffer(ReadBuffer * in_) - : BufferWithOwnMemory(0) - , in(in_) - {} - - bool nextImpl() override - { - if (compressed_in->eof()) - return false; - own_compressed_buffer.resize(COMPRESSED_BLOCK_HEADER_SIZE); - compressed_in->readStrict(&own_compressed_buffer[0], COMPRESSED_BLOCK_HEADER_SIZE); - auto method = own_compressed_buffer[0]; - auto size_compressed = unalignedLoad(&own_compressed_buffer[1]); - auto size_decompressed = unalignedLoad(&own_compressed_buffer[5]); - - if (size_compressed > DBMS_MAX_COMPRESSED_SIZE) - throw Exception("Too large size_compressed. Most likely corrupted data."); - - if (compressed_in->offset() >= COMPRESSED_BLOCK_HEADER_SIZE - && compressed_in->position() + size_compressed - COMPRESSED_BLOCK_HEADER_SIZE <= compressed_in->buffer().end()) - { - compressed_in->position() -= COMPRESSED_BLOCK_HEADER_SIZE; - compressed_buffer = compressed_in->position(); - compressed_in->position() += size_compressed; - } - else - { - own_compressed_buffer.resize(size_compressed); - compressed_buffer = &own_compressed_buffer[0]; - compressed_in->readStrict(compressed_buffer + COMPRESSED_BLOCK_HEADER_SIZE, size_compressed - COMPRESSED_BLOCK_HEADER_SIZE); - } - - if constexpr (has_checksum) - { - if (!disable_checksum[0] && checksum != CityHash_v1_0_2::CityHash128(compressed_buffer, size_compressed)) - throw Exception("Checksum doesn't match: corrupted data.", ErrorCodes::CHECKSUM_DOESNT_MATCH); - return size_compressed + sizeof(checksum); - } - else - { - return size_compressed; - } - } -}; -*/ - void writeData(const IDataType & type, const ColumnPtr & column, WriteBuffer & ostr, size_t offset, size_t limit) { /** If there are columns-constants - then we materialize them. @@ -196,17 +166,8 @@ void CHBlockChunkCodecStream::encode(const Block & block, size_t start, size_t e if (start != 0 || end != block.rows()) throw TiFlashException("CHBlock encode only support encode whole block", Errors::Coprocessor::Internal); - assert(output == nullptr); size_t init_size = block.bytes() + getExtraInfoSize(block); - output = std::make_unique(init_size); - std::unique_ptr compress_buffer; - WriteBuffer * ostr_ptr = output.get(); - auto mm = static_cast(tzg::SnappyStatistic::globalInstance().getMethod()); - if (mm != CompressionMethod::NONE) - { - compress_buffer = std::make_unique>(*output, CompressionSettings(mm), init_size); - ostr_ptr = compress_buffer.get(); - } + WriteBuffer * ostr_ptr = initOutput(init_size); block.checkNumberOfRows(); size_t columns = block.columns(); @@ -215,7 +176,6 @@ void CHBlockChunkCodecStream::encode(const Block & block, size_t start, size_t e writeVarUInt(columns, *ostr_ptr); writeVarUInt(rows, *ostr_ptr); - for (size_t i = 0; i < columns; i++) { const ColumnWithTypeAndName & column = block.safeGetByPosition(i); @@ -237,14 +197,7 @@ Block CHBlockChunkCodec::decodeImpl(ReadBuffer & istr, size_t reserve_size) { Block res; - std::unique_ptr compress_buffer; ReadBuffer * istr_ptr = &istr; - auto mm = static_cast(tzg::SnappyStatistic::globalInstance().getMethod()); - if (mm != CompressionMethod::NONE) - { - compress_buffer = std::make_unique>(istr); - istr_ptr = compress_buffer.get(); - } if (istr_ptr->eof()) { @@ -323,4 +276,47 @@ Block CHBlockChunkCodec::decode(const String & str, const Block & header) ReadBufferFromString read_buffer(str); return CHBlockChunkCodec(header).decodeImpl(read_buffer); } + +std::unique_ptr CompressedCHBlockChunkCodec::newCodecStream(const std::vector & field_types, CompressionMethod compress_method) +{ + return std::make_unique(field_types, compress_method); +} + +CompressedCHBlockChunkCodec::CompressedCHBlockChunkCodec( + const Block & header_) + : chunk_codec(header_) +{ +} +CompressedCHBlockChunkCodec::CompressedCHBlockChunkCodec(const DAGSchema & schema) + : chunk_codec(schema) +{ +} +Block CompressedCHBlockChunkCodec::decode(const String & str, const DAGSchema & schema) +{ + ReadBufferFromString read_buffer(str); + CompressedReadBuffer compress_read_buffer(read_buffer); + return CHBlockChunkCodec(schema).decodeImpl(compress_read_buffer); +} +Block CompressedCHBlockChunkCodec::decode(const String & str, const Block & header) +{ + ReadBufferFromString read_buffer(str); + CompressedReadBuffer compress_read_buffer(read_buffer); + return CHBlockChunkCodec(header).decodeImpl(compress_read_buffer); +} +Block CompressedCHBlockChunkCodec::decodeImpl(CompressedReadBuffer & istr, size_t reserve_size) +{ + return chunk_codec.decodeImpl(istr, reserve_size); +} +void CompressedCHBlockChunkCodec::readColumnMeta(size_t i, CompressedReadBuffer & istr, ColumnWithTypeAndName & column) +{ + return chunk_codec.readColumnMeta(i, istr, column); +} +void CompressedCHBlockChunkCodec::readBlockMeta(CompressedReadBuffer & istr, size_t & columns, size_t & rows) const +{ + return chunk_codec.readBlockMeta(istr, columns, rows); +} +void CompressedCHBlockChunkCodec::readData(const IDataType & type, IColumn & column, CompressedReadBuffer & istr, size_t rows) +{ + return CHBlockChunkCodec::readData(type, column, istr, rows); +} } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/CHBlockChunkCodec.h b/dbms/src/Flash/Coprocessor/CHBlockChunkCodec.h index 2fa520e19a9..9c91834d264 100644 --- a/dbms/src/Flash/Coprocessor/CHBlockChunkCodec.h +++ b/dbms/src/Flash/Coprocessor/CHBlockChunkCodec.h @@ -20,13 +20,14 @@ namespace DB { class CHBlockChunkDecodeAndSquash; +class CompressedCHBlockChunkCodec; class CHBlockChunkCodec final : public ChunkCodec { public: CHBlockChunkCodec() = default; - CHBlockChunkCodec(const Block & header_); - CHBlockChunkCodec(const DAGSchema & schema); + explicit CHBlockChunkCodec(const Block & header_); + explicit CHBlockChunkCodec(const DAGSchema & schema); Block decode(const String &, const DAGSchema & schema) override; static Block decode(const String &, const Block & header); @@ -34,6 +35,7 @@ class CHBlockChunkCodec final : public ChunkCodec private: friend class CHBlockChunkDecodeAndSquash; + friend class CompressedCHBlockChunkCodec; void readColumnMeta(size_t i, ReadBuffer & istr, ColumnWithTypeAndName & column); void readBlockMeta(ReadBuffer & istr, size_t & columns, size_t & rows) const; static void readData(const IDataType & type, IColumn & column, ReadBuffer & istr, size_t rows); diff --git a/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.cpp b/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.cpp index c975282a584..9d36c10eca0 100644 --- a/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.cpp +++ b/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.cpp @@ -16,6 +16,7 @@ #include #include "Flash/Coprocessor/tzg-metrics.h" +#include "Flash/Mpp/HashPartitionWriter.h" #include "IO/CompressedReadBuffer.h" #include "IO/CompressedStream.h" @@ -29,7 +30,7 @@ CHBlockChunkDecodeAndSquash::CHBlockChunkDecodeAndSquash( { } -std::optional CHBlockChunkDecodeAndSquash::decodeAndSquash(const String & str) +std::optional CHBlockChunkDecodeAndSquash::decodeAndSquash(const String & str, mpp::CompressMethod compress_method) { std::optional res; ReadBufferFromString istr(str); @@ -39,26 +40,24 @@ std::optional CHBlockChunkDecodeAndSquash::decodeAndSquash(const String & res.swap(accumulated_block); return res; } + std::unique_ptr compress_buffer{}; + ReadBuffer * istr_ptr = &istr; + if (ToCompressionMethod(compress_method) != CompressionMethod::NONE) + { + compress_buffer = std::make_unique>(istr); + istr_ptr = compress_buffer.get(); + } if (!accumulated_block) { /// hard-code 1.5 here, since final column size will be more than rows_limit in most situations, /// so it should be larger than 1.0, just use 1.5 here, no special meaning - Block block = codec.decodeImpl(istr, static_cast(rows_limit * 1.5)); + Block block = codec.decodeImpl(*istr_ptr, static_cast(rows_limit * 1.5)); if (block) accumulated_block.emplace(std::move(block)); } else { - std::unique_ptr compress_buffer; - ReadBuffer * istr_ptr = &istr; - auto mm = static_cast(tzg::SnappyStatistic::globalInstance().getMethod()); - if (mm != CompressionMethod::NONE) - { - compress_buffer = std::make_unique>(istr); - istr_ptr = compress_buffer.get(); - } - /// Dimensions size_t columns = 0; size_t rows = 0; diff --git a/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.h b/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.h index b3745f89ffc..0b020d48cd4 100644 --- a/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.h +++ b/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.h @@ -18,6 +18,8 @@ #include #include +#include "mpp.pb.h" + namespace DB { @@ -26,7 +28,7 @@ class CHBlockChunkDecodeAndSquash public: CHBlockChunkDecodeAndSquash(const Block & header, size_t rows_limit_); ~CHBlockChunkDecodeAndSquash() = default; - std::optional decodeAndSquash(const String &); + std::optional decodeAndSquash(const String &, mpp::CompressMethod); std::optional flush(); private: diff --git a/dbms/src/Flash/Coprocessor/CompressedCHBlockChunkCodec.h b/dbms/src/Flash/Coprocessor/CompressedCHBlockChunkCodec.h new file mode 100644 index 00000000000..90975bbe923 --- /dev/null +++ b/dbms/src/Flash/Coprocessor/CompressedCHBlockChunkCodec.h @@ -0,0 +1,45 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#pragma once + +#include +#include +#include + +namespace DB +{ +class CompressedCHBlockChunkCodec final +{ +public: + using CompressedReadBuffer = CompressedReadBuffer; + CompressedCHBlockChunkCodec() = default; + explicit CompressedCHBlockChunkCodec(const Block & header_); + explicit CompressedCHBlockChunkCodec(const DAGSchema & schema); + + static Block decode(const String &, const DAGSchema & schema); + static Block decode(const String &, const Block & header); + static std::unique_ptr newCodecStream(const std::vector & field_types, CompressionMethod compress_method); + +private: + void readColumnMeta(size_t i, CompressedReadBuffer & istr, ColumnWithTypeAndName & column); + void readBlockMeta(CompressedReadBuffer & istr, size_t & columns, size_t & rows) const; + static void readData(const IDataType & type, IColumn & column, CompressedReadBuffer & istr, size_t rows); + /// 'reserve_size' used for Squash usage, and takes effect when 'reserve_size' > 0 + Block decodeImpl(CompressedReadBuffer & istr, size_t reserve_size = 0); + + CHBlockChunkCodec chunk_codec; +}; + +} // namespace DB diff --git a/dbms/src/Flash/Coprocessor/tzg-metrics.h b/dbms/src/Flash/Coprocessor/tzg-metrics.h index 90374cb1742..6e2bd17239f 100644 --- a/dbms/src/Flash/Coprocessor/tzg-metrics.h +++ b/dbms/src/Flash/Coprocessor/tzg-metrics.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include @@ -10,16 +11,17 @@ struct SnappyStatistic enum class Method : int { LZ4 = 1, - LZ4HC = 2, /// The format is the same as for LZ4. The difference is only in compression. ZSTD = 3, /// Experimental algorithm: https://github.com/Cyan4973/zstd NONE = 4, /// No compression - SNAPPY = 5, /// Snappy: https://github.com/google/snappy }; - mutable std::atomic_uint64_t compressed_size; - mutable std::atomic_uint64_t uncompressed_size; - mutable std::atomic_uint64_t package; - mutable Method method = Method::SNAPPY; + mutable std::atomic_uint64_t compressed_size{}; + mutable std::atomic_uint64_t uncompressed_size{}; + mutable std::atomic_uint64_t package{}; + mutable std::atomic_int64_t chunck_stream_cnt{}, max_chunck_stream_cnt{}; + mutable Method method = Method::NONE; + mutable std::atomic durations{}, has_write_dur{}; + mutable std::atomic_uint64_t encode_bytes{}, has_write_rows{}; SnappyStatistic(const SnappyStatistic &) = delete; @@ -47,6 +49,19 @@ struct SnappyStatistic { method = m; } + int64_t getChunckStreamCnt() const + { + return chunck_stream_cnt; + } + int64_t getMaxChunckStreamCnt() const + { + return max_chunck_stream_cnt; + } + void addChunckStreamCnt(int x = 1) const + { + chunck_stream_cnt += x; + max_chunck_stream_cnt = std::max(max_chunck_stream_cnt.load(), chunck_stream_cnt.load()); + } void update(uint64_t compressed_size_, uint64_t uncompressed_size_) { @@ -68,5 +83,39 @@ struct SnappyStatistic static SnappyStatistic data{}; return data; } + + void addEncodeInfo(std::chrono::steady_clock::duration d, uint64_t ec) + { + std::chrono::steady_clock::duration x = durations.load(); + std::chrono::steady_clock::duration y = x + d; + for (; !durations.compare_exchange_strong(x, y);) + { + y = x + d; + } + encode_bytes += ec; + } + void addHashPartitionWriter(std::chrono::steady_clock::duration d, uint64_t rows) + { + std::chrono::steady_clock::duration x = has_write_dur.load(); + std::chrono::steady_clock::duration y = x + d; + for (; !has_write_dur.compare_exchange_strong(x, y);) + { + y = x + d; + } + has_write_rows += rows; + } + void getEncodeInfo(std::chrono::steady_clock::duration & d, uint64_t & ec, std::chrono::steady_clock::duration & hash_dur, uint64_t & hash_rows) + { + hash_dur = has_write_dur, hash_rows = has_write_rows; + d = durations; + ec = encode_bytes; + } + void clearEncodeInfo() + { + durations.store({}); + has_write_dur.store({}); + encode_bytes = 0; + has_write_rows = 0; + } }; } // namespace tzg \ No newline at end of file diff --git a/dbms/src/Flash/Mpp/ExchangeReceiver.cpp b/dbms/src/Flash/Mpp/ExchangeReceiver.cpp index 791ed3d41c2..9ddba8b622d 100644 --- a/dbms/src/Flash/Mpp/ExchangeReceiver.cpp +++ b/dbms/src/Flash/Mpp/ExchangeReceiver.cpp @@ -97,7 +97,7 @@ bool pushPacket(size_t source_index, if (resp_ptr == nullptr && error_ptr == nullptr && chunks[i].empty()) continue; - std::shared_ptr recv_msg = std::make_shared( + auto recv_msg = std::make_shared( source_index, req_info, tracked_packet, @@ -689,12 +689,13 @@ DecodeDetail ExchangeReceiverBase::decodeChunks( if (recv_msg->chunks.empty()) return detail; auto & packet = recv_msg->packet->packet; + auto compress_method = recv_msg->packet->getPacket().compress(); // Record total packet size even if fine grained shuffle is enabled. detail.packet_bytes = packet.ByteSizeLong(); for (const String * chunk : recv_msg->chunks) { - auto result = decoder_ptr->decodeAndSquash(*chunk); + auto result = decoder_ptr->decodeAndSquash(*chunk, compress_method); if (!result) continue; detail.rows += result->rows(); diff --git a/dbms/src/Flash/Mpp/HashPartitionWriter.cpp b/dbms/src/Flash/Mpp/HashPartitionWriter.cpp index f7da45a09b0..6ac89599db3 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriter.cpp +++ b/dbms/src/Flash/Mpp/HashPartitionWriter.cpp @@ -18,6 +18,17 @@ #include #include +#include + +#include "Common/Exception.h" +#include "Common/Stopwatch.h" +#include "Flash/Coprocessor/CompressedCHBlockChunkCodec.h" +#include "Flash/Coprocessor/tzg-metrics.h" +#include "IO/CompressedStream.h" +#include "common/logger_useful.h" +#include "ext/scope_guard.h" +#include "mpp.pb.h" + namespace DB { template @@ -27,19 +38,38 @@ HashPartitionWriter::HashPartitionWriter( TiDB::TiDBCollators collators_, Int64 batch_send_min_limit_, bool should_send_exec_summary_at_last_, - DAGContext & dag_context_) + DAGContext & dag_context_, + mpp::CompressMethod compress_method_) : DAGResponseWriter(/*records_per_chunk=*/-1, dag_context_) , batch_send_min_limit(batch_send_min_limit_) , should_send_exec_summary_at_last(should_send_exec_summary_at_last_) , writer(writer_) , partition_col_ids(std::move(partition_col_ids_)) , collators(std::move(collators_)) + , compress_method(compress_method_) { rows_in_blocks = 0; partition_num = writer_->getPartitionNum(); RUNTIME_CHECK(partition_num > 0); RUNTIME_CHECK(dag_context.encode_type == tipb::EncodeType::TypeCHBlock); + auto method = ToCompressionMethod(compress_method); + if (method != CompressionMethod::NONE) + { + compress_chunk_codec_stream = CompressedCHBlockChunkCodec::newCodecStream(dag_context.result_field_types, method); + } chunk_codec_stream = std::make_unique()->newCodecStream(dag_context.result_field_types); + // LOG_TRACE(&Poco::Logger::get("tzg"), "using mpp CompressMethod {}, partition_num {}", mpp::CompressMethod_Name(compress_method), partition_num); + // { + // size_t local_cnt = 0; + // for (size_t part_id = 0; part_id < partition_num; ++part_id) + // { + // if (writer->getTunnels()[part_id]->isLocal()) + // { + // ++local_cnt; + // } + // } + // LOG_TRACE(&Poco::Logger::get("tzg"), "local_cnt is {}", local_cnt); + // } } template @@ -85,8 +115,28 @@ void HashPartitionWriter::write(const Block & block) template void HashPartitionWriter::partitionAndEncodeThenWriteBlocks() { + // Stopwatch watch{}; + // auto cost = watch.elapsedSeconds(); + // SCOPE_EXIT({ + // if (cost > 0.5) + // { + // LOG_DEBUG(&Poco::Logger::get("tzg"), "time cost {:.3f}s", cost); + // } + // }); + auto tracked_packets = HashBaseWriterHelper::createPackets(partition_num); + for (size_t part_id = 0; part_id < partition_num; ++part_id) + { + auto method = compress_method; + if (writer->getTunnels()[part_id]->isLocal()) + { + method = mpp::CompressMethod::NONE; + } + tracked_packets[part_id]->getPacket().set_compress(method); + } + + if (!blocks.empty()) { assert(rows_in_blocks > 0); @@ -108,9 +158,15 @@ void HashPartitionWriter::partitionAndEncodeThenWriteBlocks() size_t dest_block_rows = dest_block.rows(); if (dest_block_rows > 0) { - chunk_codec_stream->encode(dest_block, 0, dest_block_rows); - tracked_packets[part_id]->addChunk(chunk_codec_stream->getString()); - chunk_codec_stream->clear(); + auto * codec_stream = chunk_codec_stream.get(); + if (compress_chunk_codec_stream && !writer->getTunnels()[part_id]->isLocal()) + { + // no need compress + codec_stream = compress_chunk_codec_stream.get(); + } + codec_stream->encode(dest_block, 0, dest_block_rows); + tracked_packets[part_id]->addChunk(codec_stream->getString()); + codec_stream->clear(); } } } @@ -124,12 +180,20 @@ void HashPartitionWriter::partitionAndEncodeThenWriteBlocks() template void HashPartitionWriter::writePackets(const TrackedMppDataPacketPtrs & packets) { + // auto * logger = &Poco::Logger::get("tzg"); + for (size_t part_id = 0; part_id < packets.size(); ++part_id) { const auto & packet = packets[part_id]; assert(packet); if (likely(packet->getPacket().chunks_size() > 0)) + { + // Stopwatch watch{}; writer->partitionWrite(packet, part_id); + // auto cost = watch.elapsedSeconds(); + // if (cost > 1.001) + // LOG_DEBUG(logger, "finish to write partition {}, chunck cnt {}, time cost {:.3f}s", part_id, packet->getPacket().chunks_size(), cost); + } } } diff --git a/dbms/src/Flash/Mpp/HashPartitionWriter.h b/dbms/src/Flash/Mpp/HashPartitionWriter.h index 564fd66cbfc..dae51bb5560 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriter.h +++ b/dbms/src/Flash/Mpp/HashPartitionWriter.h @@ -15,11 +15,14 @@ #pragma once #include +#include #include #include #include #include +#include "mpp.pb.h" + namespace DB { template @@ -32,7 +35,8 @@ class HashPartitionWriter : public DAGResponseWriter TiDB::TiDBCollators collators_, Int64 batch_send_min_limit_, bool should_send_exec_summary_at_last, - DAGContext & dag_context_); + DAGContext & dag_context_, + mpp::CompressMethod compress_method_); void write(const Block & block) override; void flush() override; void finishWrite() override; @@ -53,7 +57,25 @@ class HashPartitionWriter : public DAGResponseWriter TiDB::TiDBCollators collators; size_t rows_in_blocks; uint16_t partition_num; + mpp::CompressMethod compress_method; + std::unique_ptr chunk_codec_stream; + std::unique_ptr compress_chunk_codec_stream; }; +inline CompressionMethod ToCompressionMethod(mpp::CompressMethod compress_method) +{ + switch (compress_method) + { + case mpp::NONE: + return CompressionMethod::NONE; + case mpp::LZ4: + return CompressionMethod::LZ4; + case mpp::ZSTD: + return CompressionMethod::ZSTD; + default: + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unkown compress method {}", mpp::CompressMethod_Name(compress_method)); + } +} + } // namespace DB diff --git a/dbms/src/Flash/Mpp/LocalBlockMPPTunnelSet.h b/dbms/src/Flash/Mpp/LocalBlockMPPTunnelSet.h new file mode 100644 index 00000000000..3bb90d70617 --- /dev/null +++ b/dbms/src/Flash/Mpp/LocalBlockMPPTunnelSet.h @@ -0,0 +1,96 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#pragma once + +#include +#include +#include + +#include + +namespace DB +{ +template +class LocalBlockMPPTunnelSetBase : private boost::noncopyable +{ +public: + using TunnelPtr = std::shared_ptr; + explicit LocalBlockMPPTunnelSetBase(const String & req_id) + : log(Logger::get(req_id)) + {} + + /// for both broadcast writing and partition writing, only + /// return meaningful execution summary for the first tunnel, + /// because in TiDB, it does not know enough information + /// about the execution details for the mpp query, it just + /// add up all the execution summaries for the same executor, + /// so if return execution summary for all the tunnels, the + /// information in TiDB will be amplified, which may make + /// user confused. + // this is a root mpp writing. + void write(tipb::SelectResponse & response); + // this is a broadcast or pass through writing. + void broadcastOrPassThroughWrite(const TrackedMppDataPacketPtr & packet); + // this is a partition writing. + void partitionWrite(Block && block, int16_t partition_id) + { + } + // this is a execution summary writing. + void sendExecutionSummary(tipb::SelectResponse & response); + + void close(const String & reason, bool wait_sender_finish); + void finishWrite(); + void registerTunnel(const MPPTaskId & receiver_task_id, const TunnelPtr & tunnel) + { + if (receiver_task_id_to_index_map.find(receiver_task_id) != receiver_task_id_to_index_map.end()) + throw Exception(fmt::format("the tunnel {} has been registered", tunnel->id())); + + receiver_task_id_to_index_map[receiver_task_id] = tunnels.size(); + tunnels.push_back(tunnel); + if (!tunnel->isLocal()) + { + remote_tunnel_cnt++; + } + } + + TunnelPtr getTunnelByReceiverTaskId(const MPPTaskId & id); + + uint16_t getPartitionNum() const { return tunnels.size(); } + + int getRemoteTunnelCnt() + { + return remote_tunnel_cnt; + } + + const std::vector & getTunnels() const { return tunnels; } + +private: + std::vector tunnels; + std::unordered_map receiver_task_id_to_index_map; + const LoggerPtr log; + + int remote_tunnel_cnt = 0; +}; + +class LocalBlockMPPTunnelSet : public LocalBlockMPPTunnelSetBase +{ +public: + using Base = LocalBlockMPPTunnelSetBase; + using Base::Base; +}; + +using LocalBlockMPPTunnelSetPtr = std::shared_ptr; + +} // namespace DB diff --git a/dbms/src/Flash/Mpp/MPPTunnel.cpp b/dbms/src/Flash/Mpp/MPPTunnel.cpp index 0568161c695..4897e696f59 100644 --- a/dbms/src/Flash/Mpp/MPPTunnel.cpp +++ b/dbms/src/Flash/Mpp/MPPTunnel.cpp @@ -20,6 +20,9 @@ #include #include +#include "common/logger_useful.h" +#include "ext/scope_guard.h" + namespace DB { namespace FailPoints @@ -67,7 +70,7 @@ MPPTunnel::MPPTunnel( , timeout(timeout_) , tunnel_id(tunnel_id_) , mem_tracker(current_memory_tracker ? current_memory_tracker->shared_from_this() : nullptr) - , queue_size(std::max(5, input_steams_num_ * 5)) // MPMCQueue can benefit from a slightly larger queue size + , queue_size(std::max(5, input_steams_num_ * 10)) // MPMCQueue can benefit from a slightly larger queue size , log(Logger::get(req_id, tunnel_id)) { RUNTIME_ASSERT(!(is_local_ && is_async_), log, "is_local: {}, is_async: {}.", is_local_, is_async_); @@ -133,6 +136,8 @@ void MPPTunnel::close(const String & reason, bool wait_sender_finish) // TODO: consider to hold a buffer void MPPTunnel::write(const TrackedMppDataPacketPtr & data) { + Stopwatch watch{}; + LOG_TRACE(log, "ready to write"); { std::unique_lock lk(mu); @@ -140,7 +145,20 @@ void MPPTunnel::write(const TrackedMppDataPacketPtr & data) if (tunnel_sender == nullptr) throw Exception(fmt::format("write to tunnel which is already closed.")); } + double cost{}; + cost = watch.elapsedSeconds(); + if (cost > 1.0) + { + LOG_INFO(log, "tzg, `{}`, write costs {:.3f}s", tunnelSenderModeToString(mode), cost); + } + SCOPE_EXIT({ + cost = watch.elapsedSeconds(); + if (cost > 1.0) + { + LOG_INFO(log, "tzg, `{}`, `tunnel_sender->push(data) cap {} name {}`, costs {:.3f}s", tunnelSenderModeToString(mode), tunnel_sender->queue_size, tunnel_id, cost); + } + }); if (tunnel_sender->push(data)) { connection_profile_info.bytes += data->getPacket().ByteSizeLong(); @@ -196,7 +214,7 @@ void MPPTunnel::connect(PacketWriter * writer) status = TunnelStatus::Connected; cv_for_status_changed.notify_all(); } - LOG_DEBUG(log, "connected"); + LOG_DEBUG(log, "Tunnel connected in {} mode", tunnelSenderModeToString(mode)); } void MPPTunnel::connectAsync(IAsyncCallData * call_data) diff --git a/dbms/src/Flash/Mpp/MPPTunnel.h b/dbms/src/Flash/Mpp/MPPTunnel.h index f94b254dacd..71364fb99dc 100644 --- a/dbms/src/Flash/Mpp/MPPTunnel.h +++ b/dbms/src/Flash/Mpp/MPPTunnel.h @@ -64,8 +64,9 @@ class TunnelSender : private boost::noncopyable { public: virtual ~TunnelSender() = default; - TunnelSender(size_t queue_size, MemoryTrackerPtr & memory_tracker_, const LoggerPtr & log_, const String & tunnel_id_) - : memory_tracker(memory_tracker_) + TunnelSender(size_t queue_size_, MemoryTrackerPtr & memory_tracker_, const LoggerPtr & log_, const String & tunnel_id_) + : queue_size(queue_size_) + , memory_tracker(memory_tracker_) , send_queue(MPMCQueue(queue_size)) , log(log_) , tunnel_id(tunnel_id_) @@ -106,6 +107,8 @@ class TunnelSender : private boost::noncopyable return memory_tracker != nullptr ? memory_tracker.get() : nullptr; } + size_t queue_size; + protected: /// TunnelSender use consumer state to inform tunnel that whether sender has finished its work class ConsumerState diff --git a/dbms/src/Flash/Mpp/newMPPExchangeWriter.h b/dbms/src/Flash/Mpp/newMPPExchangeWriter.h index 9310caabcf5..447979bacf2 100644 --- a/dbms/src/Flash/Mpp/newMPPExchangeWriter.h +++ b/dbms/src/Flash/Mpp/newMPPExchangeWriter.h @@ -19,6 +19,8 @@ #include #include +#include "Flash/Coprocessor/tzg-metrics.h" + namespace DB { template @@ -33,8 +35,23 @@ std::unique_ptr newMPPExchangeWriter( DAGContext & dag_context, bool enable_fine_grained_shuffle, UInt64 fine_grained_shuffle_stream_count, - UInt64 fine_grained_shuffle_batch_size) + UInt64 fine_grained_shuffle_batch_size, + mpp::CompressMethod compress_method = mpp::CompressMethod::LZ4) { + auto mm = tzg::SnappyStatistic::globalInstance().getMethod(); + switch (mm) + { + case tzg::SnappyStatistic::Method::LZ4: + compress_method = mpp::CompressMethod::LZ4; + break; + case tzg::SnappyStatistic::Method::ZSTD: + compress_method = mpp::CompressMethod::ZSTD; + break; + default: + compress_method = mpp::CompressMethod::NONE; + break; + } + // compress_method = ; RUNTIME_CHECK(dag_context.isMPPTask()); should_send_exec_summary_at_last = dag_context.collect_execution_summaries && should_send_exec_summary_at_last; if (dag_context.isRootMPPTask()) @@ -71,7 +88,8 @@ std::unique_ptr newMPPExchangeWriter( partition_col_collators, batch_send_min_limit, should_send_exec_summary_at_last, - dag_context); + dag_context, + compress_method); } } else diff --git a/dbms/src/IO/CompressedReadBufferBase.cpp b/dbms/src/IO/CompressedReadBufferBase.cpp index 76103c64c00..40efeb1208a 100644 --- a/dbms/src/IO/CompressedReadBufferBase.cpp +++ b/dbms/src/IO/CompressedReadBufferBase.cpp @@ -23,7 +23,7 @@ #include #include #include -#include +// #include #include #include @@ -122,12 +122,12 @@ void CompressedReadBufferBase::decompress(char * to, size_t size_d if (ZSTD_isError(res)) throw Exception("Cannot ZSTD_decompress: " + std::string(ZSTD_getErrorName(res)), ErrorCodes::CANNOT_DECOMPRESS); } - else if (method == static_cast(CompressionMethodByte::SNAPPY)) - { - auto res = snappy::RawUncompress(compressed_buffer + COMPRESSED_BLOCK_HEADER_SIZE, size_compressed_without_checksum - COMPRESSED_BLOCK_HEADER_SIZE, to); - if (!res) - throw Exception("Cannot decompress by SNAPPY", ErrorCodes::CANNOT_DECOMPRESS); - } + // else if (method == static_cast(CompressionMethodByte::SNAPPY)) + // { + // auto res = snappy::RawUncompress(compressed_buffer + COMPRESSED_BLOCK_HEADER_SIZE, size_compressed_without_checksum - COMPRESSED_BLOCK_HEADER_SIZE, to); + // if (!res) + // throw Exception("Cannot decompress by SNAPPY", ErrorCodes::CANNOT_DECOMPRESS); + // } else if (method == static_cast(CompressionMethodByte::NONE)) { memcpy(to, &compressed_buffer[COMPRESSED_BLOCK_HEADER_SIZE], size_decompressed); diff --git a/dbms/src/IO/CompressedWriteBuffer.cpp b/dbms/src/IO/CompressedWriteBuffer.cpp index 4e385992c3e..46f000d63e7 100644 --- a/dbms/src/IO/CompressedWriteBuffer.cpp +++ b/dbms/src/IO/CompressedWriteBuffer.cpp @@ -18,7 +18,7 @@ #include #include #include -#include +// #include #include #include @@ -122,31 +122,31 @@ void CompressedWriteBuffer::nextImpl() compressed_buffer_ptr = &compressed_buffer[0]; break; } - case CompressionMethod::SNAPPY: - { - static constexpr size_t header_size = 1 + sizeof(UInt32) + sizeof(UInt32); - compressed_size = snappy::MaxCompressedLength(uncompressed_size); - compressed_buffer.resize(header_size + compressed_size); + // case CompressionMethod::SNAPPY: + // { + // static constexpr size_t header_size = 1 + sizeof(UInt32) + sizeof(UInt32); + // compressed_size = snappy::MaxCompressedLength(uncompressed_size); + // compressed_buffer.resize(header_size + compressed_size); - compressed_buffer[0] = static_cast(CompressionMethodByte::SNAPPY); + // compressed_buffer[0] = static_cast(CompressionMethodByte::SNAPPY); - snappy::RawCompress( - working_buffer.begin(), - uncompressed_size, - &compressed_buffer[header_size], - &compressed_size); + // snappy::RawCompress( + // working_buffer.begin(), + // uncompressed_size, + // &compressed_buffer[header_size], + // &compressed_size); - compressed_size = header_size + compressed_size; + // compressed_size = header_size + compressed_size; - UInt32 compressed_size_32 = compressed_size; - UInt32 uncompressed_size_32 = uncompressed_size; + // UInt32 compressed_size_32 = compressed_size; + // UInt32 uncompressed_size_32 = uncompressed_size; - unalignedStore(&compressed_buffer[1], compressed_size_32); - unalignedStore(&compressed_buffer[5], uncompressed_size_32); + // unalignedStore(&compressed_buffer[1], compressed_size_32); + // unalignedStore(&compressed_buffer[5], uncompressed_size_32); - compressed_buffer_ptr = &compressed_buffer[0]; - break; - } + // compressed_buffer_ptr = &compressed_buffer[0]; + // break; + // } default: throw Exception("Unknown compression method", ErrorCodes::UNKNOWN_COMPRESSION_METHOD); } diff --git a/dbms/src/Storages/Transaction/ProxyFFIStatusService.cpp b/dbms/src/Storages/Transaction/ProxyFFIStatusService.cpp index ec8d4ea0fad..4323be82adf 100644 --- a/dbms/src/Storages/Transaction/ProxyFFIStatusService.cpp +++ b/dbms/src/Storages/Transaction/ProxyFFIStatusService.cpp @@ -154,6 +154,67 @@ HttpRequestRes HandleHttpRequestCompressStatus( .view = BaseBuffView{res->data(), res->size()}}}; } +static inline void ClearCompress() +{ + tzg::SnappyStatistic::globalInstance().clear(); + tzg::SnappyStatistic::globalInstance().clearEncodeInfo(); +} + +HttpRequestRes HandleHttpRequestCompressClean( + EngineStoreServerWrap *, + std::string_view, + const std::string &, + std::string_view, + std::string_view) +{ + ClearCompress(); + auto * res = RawCppString::New(fmt::format("Clean compress info")); + + return HttpRequestRes{ + .status = HttpRequestStatus::Ok, + .res = CppStrWithView{ + .inner = GenRawCppPtr(res, RawCppPtrTypeImpl::String), + .view = BaseBuffView{res->data(), res->size()}}}; +} + +HttpRequestRes HandleHttpRequestStreamCnt( + EngineStoreServerWrap *, + std::string_view, + const std::string &, + std::string_view, + std::string_view) +{ + auto cnt = tzg::SnappyStatistic::globalInstance().getChunckStreamCnt(); + auto max_cnt = tzg::SnappyStatistic::globalInstance().getMaxChunckStreamCnt(); + auto * res = RawCppString::New(fmt::format("chunck-stream-cnt: {}, max-chunck-stream-cnt: {}", cnt, max_cnt)); + return HttpRequestRes{ + .status = HttpRequestStatus::Ok, + .res = CppStrWithView{ + .inner = GenRawCppPtr(res, RawCppPtrTypeImpl::String), + .view = BaseBuffView{res->data(), res->size()}}}; +} + +HttpRequestRes HandleHttpRequestEncodeInfo( + EngineStoreServerWrap *, + std::string_view, + const std::string &, + std::string_view, + std::string_view) +{ + std::chrono::steady_clock::duration d; + uint64_t ec; + std::chrono::steady_clock::duration hash_dur; + uint64_t hash_rows; + tzg::SnappyStatistic::globalInstance().getEncodeInfo(d, ec, hash_dur, hash_rows); + auto * res = RawCppString::New(fmt::format("uncompress-bytes: {}, time : {}, hash-part-write: {}, hash-part-time: {}", ec, d.count(), hash_rows, hash_dur.count())); + + return HttpRequestRes{ + .status = HttpRequestStatus::Ok, + .res = CppStrWithView{ + .inner = GenRawCppPtr(res, RawCppPtrTypeImpl::String), + .view = BaseBuffView{res->data(), res->size()}}}; +} + HttpRequestRes HandleHttpRequestSetCompressMethod( EngineStoreServerWrap *, std::string_view path, @@ -165,6 +226,10 @@ HttpRequestRes HandleHttpRequestSetCompressMethod( auto method = magic_enum::enum_cast(method_str); if (method) { + if (tzg::SnappyStatistic::globalInstance().getMethod() != *method) + { + ClearCompress(); + } tzg::SnappyStatistic::globalInstance().setMethod(*method); auto * res = RawCppString::New(fmt::format("Set compress method to {}", method_str)); return HttpRequestRes{ @@ -186,8 +251,10 @@ static const std::map AVAILABLE_HTTP_URI = {"/tiflash/sync-status/", HandleHttpRequestSyncStatus}, {"/tiflash/store-status", HandleHttpRequestStoreStatus}, {"/tiflash/tzg-compress", HandleHttpRequestCompressStatus}, - {"/tiflash/tzg-compress-and-clean", HandleHttpRequestCompressStatus}, + {"/tiflash/tzg-clean-compress", HandleHttpRequestCompressClean}, {"/tiflash/set-tzg-compress-method/", HandleHttpRequestSetCompressMethod}, + {"/tiflash/get-tzg-compress-stream-cnt", HandleHttpRequestStreamCnt}, + {"/tiflash/get-tzg-encode-info", HandleHttpRequestEncodeInfo}, }; uint8_t CheckHttpUriAvailable(BaseBuffView path_) diff --git a/libs/CMakeLists.txt b/libs/CMakeLists.txt index e10fc785e97..971d23c2c29 100644 --- a/libs/CMakeLists.txt +++ b/libs/CMakeLists.txt @@ -20,7 +20,6 @@ endif () add_subdirectory (libcommon) add_subdirectory (libpocoext) add_subdirectory (libdaemon) -add_subdirectory (libsnappy) add_subdirectory (libmemcpy) if (GLIBC_COMPATIBILITY) diff --git a/libs/libcommon/include/common/avx2_memcpy.h b/libs/libcommon/include/common/avx2_memcpy.h index d959b7b299c..9e2556dee3d 100644 --- a/libs/libcommon/include/common/avx2_memcpy.h +++ b/libs/libcommon/include/common/avx2_memcpy.h @@ -95,7 +95,7 @@ ALWAYS_INLINE static inline void * avx2_inline_memcpy(void * __restrict dst_, co assert(size_t(dst) % block32_size == 0); } - // TODO: use non-temporal way(mark data unlikely to be used again soon) to minimize caching for large memory size(bigger than L1 cache size) if necessary. + // TODO: use non-temporal way(mark data unlikely to be used again soon) to minimize caching for large memory size(bigger than L3 cache size) if necessary. // TODO: check whether source address is aligned to 32 and use specific aligned instructions if necessary. /// Aligned unrolled copy. diff --git a/tiflash-tools/flash_cluster_manager.py b/tiflash-tools/flash_cluster_manager.py index a3b90c90b75..320b92e3223 100644 --- a/tiflash-tools/flash_cluster_manager.py +++ b/tiflash-tools/flash_cluster_manager.py @@ -34,7 +34,7 @@ def __init__(self): parser.add_argument( '--pd-address', default="172.16.4.39:2174") parser.add_argument( - '--type', help='run type', required=True, choices=('show', 'clean', 'set')) + '--type', help='run type', required=True, choices=('show', 'clean', 'set', )) parser.add_argument( '--method', choices=('SNAPPY', 'LZ4', 'ZSTD', 'NONE')) self.args = parser.parse_args() @@ -54,6 +54,10 @@ def run(self): self.run_clean() elif self.args.type == 'set': self.run_set_method() + elif self.args.type == 'get-cnt': + self.run_show_cnt() + elif self.args.type == 'get-encode': + self.run_get_encode() def run_clean(self): res = {store_id: flash_http_client.clean_compress_info( @@ -73,7 +77,8 @@ def run_show(self): tol_uncompressed_size = 0 method = None for store_id, info in compress_info.items(): - print(info) + print('store {}, addr {}, {}'.format( + store_id, self.tiflash_stores[store_id].tiflash_status_address, info)) tol_compressed_size += int(info['compressed_size']) tol_uncompressed_size += int(info['uncompressed_size']) if method: @@ -89,6 +94,45 @@ def run_show(self): ) print(msg) + def run_show_cnt(self): + compress_info = {store_id: flash_http_client.get_stream_info( + store.tiflash_status_address) for store_id, store in self.tiflash_stores.items()} + for store_id, info in compress_info.items(): + print('store {}: {}'.format(store_id, info)) + + def run_get_encode(self): + compress_info = {store_id: flash_http_client.get_codec_info( + store.tiflash_status_address) for store_id, store in self.tiflash_stores.items()} + all_bytes = {} + all_time = {} + all_hash_row = {} + all_hash_time = {} + for store_id, info in compress_info.items(): + all_bytes[store_id] = 0 + all_time[store_id] = 0 + all_hash_row[store_id] = 0 + all_hash_time[store_id] = 0 + x = [x for x in info.split(",")] + for o in x: + a, b = o.split(':') + a, b = a.strip(), int(b.strip()) + if a == 'uncompress-bytes': + all_bytes[store_id] += b + elif a == 'time': + all_time[store_id] += b + elif a == 'hash-part-write': + all_hash_row[store_id] += b + elif a == 'hash-part-time': + all_hash_time[store_id] += b + + all_time[store_id] = all_time[store_id]/(10**9) + all_hash_time[store_id] = all_hash_time[store_id]/(10**9) + all_bytes[store_id] /= 1024 ** 2 + y = all_bytes[store_id] / \ + all_time[store_id] if all_time[store_id] else 0.0 + print("store {}: uncompress-bytes: {:.4f}MB, time: {:.4f}s, MBPS/core: {:.4f}, hash-part-write: {}rows, hash-part-time: {:.4f}s".format( + store_id, all_bytes[store_id], all_time[store_id], y, all_hash_row[store_id], all_hash_time[store_id])) + if __name__ == '__main__': Runner().run() diff --git a/tiflash-tools/flash_http_client.py b/tiflash-tools/flash_http_client.py index 3450f4e28bf..0870ddf0459 100644 --- a/tiflash-tools/flash_http_client.py +++ b/tiflash-tools/flash_http_client.py @@ -23,6 +23,20 @@ def get_compress_info(tiflash_status_address): return res +def get_stream_info(tiflash_status_address): + res = util.curl_http( + '{}/tiflash/get-tzg-compress-stream-cnt'.format(tiflash_status_address)) + util.check_status_code(res) + return res.text + + +def get_codec_info(tiflash_status_address): + res = util.curl_http( + '{}/tiflash/get-tzg-encode-info'.format(tiflash_status_address)) + util.check_status_code(res) + return res.text + + def set_compress_method(tiflash_status_address, method): res = util.curl_http( '{}/tiflash/set-tzg-compress-method/{}'.format(tiflash_status_address, method)) @@ -32,6 +46,6 @@ def set_compress_method(tiflash_status_address, method): def clean_compress_info(tiflash_status_address, ): res = util.curl_http( - '{}/tiflash/tzg-compress-and-clean'.format(tiflash_status_address, )) + '{}/tiflash/tzg-clean-compress'.format(tiflash_status_address, )) util.check_status_code(res) return res.text diff --git a/tiflash-tools/util.py b/tiflash-tools/util.py index 1986782d91c..5b01da3dbeb 100644 --- a/tiflash-tools/util.py +++ b/tiflash-tools/util.py @@ -51,7 +51,7 @@ def curl_http(uri, params=None): def check_status_code(r): if r.status_code != 200: raise Exception( - 'unexpected status code {} from {}'.format(r.status_code, r.url)) + 'unexpected status code `{}` from `{}` error-msg `{}`'.format(r.status_code, r.url, r.text)) def try_get_json(r): From b1dd8907fc281a82129fc7ae118a278baf480928 Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Wed, 7 Dec 2022 15:29:24 +0800 Subject: [PATCH 06/93] use TIFLASH_COMPILER_ARCH_HASWELL_FLAG Signed-off-by: Zhigao Tong --- CMakeLists.txt | 6 ------ cmake/cpu_features.cmake | 6 ++++++ contrib/lz4-cmake/CMakeLists.txt | 8 ++++---- contrib/zstd-cmake/CMakeLists.txt | 8 ++++---- 4 files changed, 14 insertions(+), 14 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 4d1e0b47ba3..6d4fb10d919 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -379,12 +379,6 @@ if (ARCH_AMD64) else() add_definitions(-DTIFLASH_COMPILER_VPCLMULQDQ_SUPPORT=0) endif() - set (TIFLASH_COMPILER_MOVBE_FLAG "-mmovbe") - check_cxx_compiler_flag("${TIFLASH_COMPILER_MOVBE_FLAG}" TIFLASH_COMPILER_MOVBE_SUPPORT) - set (TIFLASH_COMPILER_BMI2_FLAG "-mbmi2") - check_cxx_compiler_flag("${TIFLASH_COMPILER_BMI2_FLAG}" TIFLASH_COMPILER_BMI2_SUPPORT) - set (TIFLASH_COMPILER_ARCH_HASWELL_FLAG "-march=haswell") - check_cxx_compiler_flag("${TIFLASH_COMPILER_ARCH_HASWELL_FLAG}" TIFLASH_COMPILER_ARCH_HASWELL_SUPPORT) else() add_definitions(-DTIFLASH_COMPILER_VPCLMULQDQ_SUPPORT=0) endif() diff --git a/cmake/cpu_features.cmake b/cmake/cpu_features.cmake index cfde43717c7..603d2c2e878 100644 --- a/cmake/cpu_features.cmake +++ b/cmake/cpu_features.cmake @@ -209,6 +209,12 @@ elseif (ARCH_AMD64) add_definitions(-DTIFLASH_ENABLE_AVX512_SUPPORT=1) endif () + set (TIFLASH_COMPILER_MOVBE_FLAG "-mmovbe") + check_cxx_compiler_flag("${TIFLASH_COMPILER_MOVBE_FLAG}" TIFLASH_COMPILER_MOVBE_SUPPORT) + set (TIFLASH_COMPILER_BMI2_FLAG "-mbmi2") + check_cxx_compiler_flag("${TIFLASH_COMPILER_BMI2_FLAG}" TIFLASH_COMPILER_BMI2_SUPPORT) + set (TIFLASH_COMPILER_ARCH_HASWELL_FLAG "-march=haswell") + check_cxx_compiler_flag("${TIFLASH_COMPILER_ARCH_HASWELL_FLAG}" TIFLASH_COMPILER_ARCH_HASWELL_SUPPORT) else () # ignore all other platforms endif () diff --git a/contrib/lz4-cmake/CMakeLists.txt b/contrib/lz4-cmake/CMakeLists.txt index 5f456d65844..eb757e79668 100644 --- a/contrib/lz4-cmake/CMakeLists.txt +++ b/contrib/lz4-cmake/CMakeLists.txt @@ -11,7 +11,7 @@ target_compile_definitions(lz4 PUBLIC LZ4_DISABLE_DEPRECATE_WARNINGS=1) target_include_directories(lz4 PUBLIC ${LIBRARY_DIR}) -if (TIFLASH_ENABLE_AVX_SUPPORT) - set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${TIFLASH_COMPILER_AVX2_FLAG}") - set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${TIFLASH_COMPILER_AVX2_FLAG}") -endif () \ No newline at end of file +if (TIFLASH_COMPILER_ARCH_HASWELL_SUPPORT) + set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${TIFLASH_COMPILER_ARCH_HASWELL_FLAG}") + set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${TIFLASH_COMPILER_ARCH_HASWELL_FLAG}") +endif () diff --git a/contrib/zstd-cmake/CMakeLists.txt b/contrib/zstd-cmake/CMakeLists.txt index 4ab932cb88a..83b4e5693f6 100644 --- a/contrib/zstd-cmake/CMakeLists.txt +++ b/contrib/zstd-cmake/CMakeLists.txt @@ -156,7 +156,7 @@ ADD_LIBRARY(zstd ${Sources} ${Headers}) target_include_directories (zstd PUBLIC ${LIBRARY_DIR}) -if (TIFLASH_ENABLE_AVX_SUPPORT) - set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${TIFLASH_COMPILER_AVX2_FLAG}") - set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${TIFLASH_COMPILER_AVX2_FLAG}") -endif () \ No newline at end of file +if (TIFLASH_COMPILER_ARCH_HASWELL_SUPPORT) + set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${TIFLASH_COMPILER_ARCH_HASWELL_FLAG}") + set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${TIFLASH_COMPILER_ARCH_HASWELL_FLAG}") +endif () From ae30e4a5fc7a37e1b54dbd9d79093bc2ae6a5811 Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Fri, 2 Dec 2022 12:44:53 +0800 Subject: [PATCH 07/93] 1 Signed-off-by: Zhigao Tong --- dbms/src/Flash/Mpp/HashPartitionWriter.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/dbms/src/Flash/Mpp/HashPartitionWriter.cpp b/dbms/src/Flash/Mpp/HashPartitionWriter.cpp index 6ac89599db3..c9aeb32539f 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriter.cpp +++ b/dbms/src/Flash/Mpp/HashPartitionWriter.cpp @@ -18,6 +18,7 @@ #include #include +#include #include #include "Common/Exception.h" @@ -159,8 +160,9 @@ void HashPartitionWriter::partitionAndEncodeThenWriteBlocks() if (dest_block_rows > 0) { auto * codec_stream = chunk_codec_stream.get(); - if (compress_chunk_codec_stream && !writer->getTunnels()[part_id]->isLocal()) + if (tracked_packets[part_id]->getPacket().compress() != mpp::CompressMethod::NONE) { + assert(compress_chunk_codec_stream); // no need compress codec_stream = compress_chunk_codec_stream.get(); } From 54921da0c5be25f631ebf0b46b3b8c6b6ccbbe2c Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Thu, 8 Dec 2022 10:08:40 +0800 Subject: [PATCH 08/93] 2 --- contrib/tzg.mpp.proto.patch | 22 ++++++++++-- dbms/src/Flash/Coprocessor/DAGContext.cpp | 1 + .../Coprocessor/DAGQueryBlockInterpreter.cpp | 4 +-- dbms/src/Flash/FlashService.cpp | 14 ++++++++ dbms/src/Flash/Mpp/HashPartitionWriter.cpp | 9 ----- dbms/src/Flash/Mpp/MPPTask.cpp | 27 ++++++++++----- dbms/src/Flash/Mpp/MPPTask.h | 2 +- dbms/src/Flash/Mpp/MppVersion.h | 10 ++++++ dbms/src/Flash/Mpp/Utils.cpp | 29 ++++++++++++++++ dbms/src/Flash/Mpp/newMPPExchangeWriter.h | 34 +++++++++++-------- .../Planner/plans/PhysicalExchangeSender.cpp | 2 +- 11 files changed, 114 insertions(+), 40 deletions(-) create mode 100644 dbms/src/Flash/Mpp/MppVersion.h diff --git a/contrib/tzg.mpp.proto.patch b/contrib/tzg.mpp.proto.patch index 79b6d38abc5..5c25c940d3f 100644 --- a/contrib/tzg.mpp.proto.patch +++ b/contrib/tzg.mpp.proto.patch @@ -1,8 +1,24 @@ diff --git a/proto/mpp.proto b/proto/mpp.proto -index f8af03b..d9ab839 100644 +index f8af03b..213caf3 100644 --- a/proto/mpp.proto +++ b/proto/mpp.proto -@@ -60,6 +60,12 @@ message EstablishMPPConnectionRequest { +@@ -17,6 +17,7 @@ message TaskMeta { + int64 task_id = 2; // if task id is -1 , it indicates a tidb task. + int64 partition_id = 3; // Only used for hash partition + string address = 4; // target address of this task. ++ int64 mpp_version = 5; + } + + message IsAliveRequest { +@@ -24,6 +25,7 @@ message IsAliveRequest { + + message IsAliveResponse { + bool available = 1; ++ int64 mpp_version = 2; + } + + // Dipsatch the task request to different tiflash servers. +@@ -60,6 +62,12 @@ message EstablishMPPConnectionRequest { TaskMeta receiver_meta = 2; // node closer to the tidb mpp gather. } @@ -15,7 +31,7 @@ index f8af03b..d9ab839 100644 // when TiFlash sends data to TiDB, Data packets wrap tipb.SelectResponse, i.e., serialize tipb.SelectResponse into data; // when TiFlash sends data to TiFlash, data blocks are serialized into chunks, and the execution_summaries in tipb.SelectResponse are serialized into data only for the last packet. message MPPDataPacket { -@@ -67,6 +73,7 @@ message MPPDataPacket { +@@ -67,6 +75,7 @@ message MPPDataPacket { Error error = 2; repeated bytes chunks = 3; repeated uint64 stream_ids = 4; diff --git a/dbms/src/Flash/Coprocessor/DAGContext.cpp b/dbms/src/Flash/Coprocessor/DAGContext.cpp index 6167090194a..ad57c9abf8a 100644 --- a/dbms/src/Flash/Coprocessor/DAGContext.cpp +++ b/dbms/src/Flash/Coprocessor/DAGContext.cpp @@ -17,6 +17,7 @@ #include #include #include +#include #include #include diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp index b4ba767562b..dd8b2cbbe53 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp @@ -750,8 +750,8 @@ void DAGQueryBlockInterpreter::handleExchangeSender(DAGPipeline & pipeline) } pipeline.transform([&](auto & stream) { // construct writer - std::unique_ptr response_writer = newMPPExchangeWriter( - context.getDAGContext()->tunnel_set, + std::unique_ptr response_writer = NewMPPExchangeWriter( + dagContext().tunnel_set, partition_col_ids, partition_col_collators, exchange_sender.tp(), diff --git a/dbms/src/Flash/FlashService.cpp b/dbms/src/Flash/FlashService.cpp index cb6bf08a0e8..259051d57a5 100644 --- a/dbms/src/Flash/FlashService.cpp +++ b/dbms/src/Flash/FlashService.cpp @@ -24,6 +24,7 @@ #include #include #include +#include #include #include #include @@ -31,6 +32,7 @@ #include #include #include +#include #include @@ -269,6 +271,18 @@ grpc::Status AsyncFlashService::establishMPPConnectionAsync(grpc::ServerContext grpc::Status FlashService::EstablishMPPConnection(grpc::ServerContext * grpc_context, const mpp::EstablishMPPConnectionRequest * request, grpc::ServerWriter * sync_writer) { + { + const auto & sender_mpp_version = request->sender_meta().mpp_version(); + const auto & receiver_mpp_version = request->receiver_meta().mpp_version(); + if (!TiDB::CheckMppVersion(sender_mpp_version) || !TiDB::CheckMppVersion(receiver_mpp_version)) + { + auto && err_msg = fmt::format("Failed to establish MPP connection, sender: {}, receiver: {}", + TiDB::GenMppVersionErrorMessage(sender_mpp_version), + TiDB::GenMppVersionErrorMessage(receiver_mpp_version)); + return grpc::Status(grpc::StatusCode::INTERNAL, std::move(err_msg)); + } + } + CPUAffinityManager::getInstance().bindSelfGrpcThread(); // Establish a pipe for data transferring. The pipes have registered by the task in advance. // We need to find it out and bind the grpc stream with it. diff --git a/dbms/src/Flash/Mpp/HashPartitionWriter.cpp b/dbms/src/Flash/Mpp/HashPartitionWriter.cpp index c9aeb32539f..6effe888e12 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriter.cpp +++ b/dbms/src/Flash/Mpp/HashPartitionWriter.cpp @@ -116,15 +116,6 @@ void HashPartitionWriter::write(const Block & block) template void HashPartitionWriter::partitionAndEncodeThenWriteBlocks() { - // Stopwatch watch{}; - // auto cost = watch.elapsedSeconds(); - // SCOPE_EXIT({ - // if (cost > 0.5) - // { - // LOG_DEBUG(&Poco::Logger::get("tzg"), "time cost {:.3f}s", cost); - // } - // }); - auto tracked_packets = HashBaseWriterHelper::createPackets(partition_num); for (size_t part_id = 0; part_id < partition_num; ++part_id) diff --git a/dbms/src/Flash/Mpp/MPPTask.cpp b/dbms/src/Flash/Mpp/MPPTask.cpp index c0db48d2be4..37ba73cf926 100644 --- a/dbms/src/Flash/Mpp/MPPTask.cpp +++ b/dbms/src/Flash/Mpp/MPPTask.cpp @@ -25,6 +25,7 @@ #include #include #include +#include #include #include #include @@ -32,7 +33,6 @@ #include #include #include -#include #include #include @@ -51,9 +51,8 @@ extern const char exception_during_mpp_register_tunnel_for_non_root_mpp_task[]; extern const char force_no_local_region_for_mpp_task[]; } // namespace FailPoints -MPPTask::MPPTask(const mpp::TaskMeta & meta_, const ContextPtr & context_) - : meta(meta_) - , id(meta.start_ts(), meta.task_id()) +MPPTask::MPPTask(const mpp::TaskMeta & meta, const ContextPtr & context_) + : id(meta.start_ts(), meta.task_id()) , context(context_) , manager(context_->getTMTContext().getMPPTaskManager().get()) , schedule_entry(manager, id) @@ -113,6 +112,8 @@ void MPPTask::run() void MPPTask::registerTunnels(const mpp::DispatchTaskRequest & task_request) { + const mpp::TaskMeta & sender_meta = task_request.meta(); + auto tunnel_set_local = std::make_shared(log->identifier()); std::chrono::seconds timeout(task_request.timeout()); const auto & exchange_sender = dag_req.root_executor().exchange_sender(); @@ -120,16 +121,19 @@ void MPPTask::registerTunnels(const mpp::DispatchTaskRequest & task_request) for (int i = 0; i < exchange_sender.encoded_task_meta_size(); ++i) { // exchange sender will register the tunnels and wait receiver to found a connection. - mpp::TaskMeta task_meta; - if (unlikely(!task_meta.ParseFromString(exchange_sender.encoded_task_meta(i)))) + mpp::TaskMeta receiver_meta; + if (unlikely(!receiver_meta.ParseFromString(exchange_sender.encoded_task_meta(i)))) throw TiFlashException("Failed to decode task meta info in ExchangeSender", Errors::Coprocessor::BadRequest); - bool is_local = context->getSettingsRef().enable_local_tunnel && meta.address() == task_meta.address(); + + bool is_local = context->getSettingsRef().enable_local_tunnel && sender_meta.address() == receiver_meta.address(); + receiver_meta.mpp_version(); bool is_async = !is_local && context->getSettingsRef().enable_async_server; - MPPTunnelPtr tunnel = std::make_shared(task_meta, task_request.meta(), timeout, context->getSettingsRef().max_threads, is_local, is_async, log->identifier()); + + MPPTunnelPtr tunnel = std::make_shared(receiver_meta, sender_meta, timeout, context->getSettingsRef().max_threads, is_local, is_async, log->identifier()); LOG_DEBUG(log, "begin to register the tunnel {}, is_local: {}, is_async: {}", tunnel->id(), is_local, is_async); if (status != INITIALIZING) throw Exception(fmt::format("The tunnel {} can not be registered, because the task is not in initializing state", tunnel->id())); - tunnel_set_local->registerTunnel(MPPTaskId{task_meta.start_ts(), task_meta.task_id()}, tunnel); + tunnel_set_local->registerTunnel(MPPTaskId{receiver_meta.start_ts(), receiver_meta.task_id()}, tunnel); if (!dag_context->isRootMPPTask()) { FAIL_POINT_TRIGGER_EXCEPTION(FailPoints::exception_during_mpp_register_tunnel_for_non_root_mpp_task); @@ -220,6 +224,11 @@ void MPPTask::unregisterTask() void MPPTask::prepare(const mpp::DispatchTaskRequest & task_request) { + if (auto mpp_version = task_request.meta().mpp_version(); !TiDB::CheckMppVersion(mpp_version)) + { + throw TiFlashException(TiDB::GenMppVersionErrorMessage(mpp_version), Errors::MPP::Internal); + } + dag_req = getDAGRequestFromStringWithRetry(task_request.encoded_plan()); TMTContext & tmt_context = context->getTMTContext(); /// MPP task will only use key ranges in mpp::DispatchTaskRequest::regions/mpp::DispatchTaskRequest::table_regions. diff --git a/dbms/src/Flash/Mpp/MPPTask.h b/dbms/src/Flash/Mpp/MPPTask.h index 8bf586073a2..5107cd57ea5 100644 --- a/dbms/src/Flash/Mpp/MPPTask.h +++ b/dbms/src/Flash/Mpp/MPPTask.h @@ -111,7 +111,7 @@ class MPPTask : public std::enable_shared_from_this void initExchangeReceivers(); tipb::DAGRequest dag_req; - mpp::TaskMeta meta; + // mpp::TaskMeta meta; MPPTaskId id; ContextPtr context; diff --git a/dbms/src/Flash/Mpp/MppVersion.h b/dbms/src/Flash/Mpp/MppVersion.h new file mode 100644 index 00000000000..e60cd88bd2f --- /dev/null +++ b/dbms/src/Flash/Mpp/MppVersion.h @@ -0,0 +1,10 @@ +#pragma once + +#include + +namespace TiDB +{ +bool CheckMppVersion(int64_t mpp_version); +std::string GenMppVersionErrorMessage(int64_t mpp_version); +int64_t GetMppVersion(); +} // namespace TiDB \ No newline at end of file diff --git a/dbms/src/Flash/Mpp/Utils.cpp b/dbms/src/Flash/Mpp/Utils.cpp index 21d89b3cd52..3ce15aa274c 100644 --- a/dbms/src/Flash/Mpp/Utils.cpp +++ b/dbms/src/Flash/Mpp/Utils.cpp @@ -39,3 +39,32 @@ void trimStackTrace(String & message) } } // namespace DB + +namespace TiDB +{ + +constexpr int64_t MPP_VERSION = 1; +static const char * MPP_TIFLASH_RELEASE_VERSION = "v6.5.0"; +constexpr int64_t MIN_MPP_VERSION = 0; + +bool CheckMppVersion(int64_t mpp_version) +{ + return mpp_version >= MIN_MPP_VERSION && mpp_version <= MPP_VERSION; +} + +std::string GenMppVersionErrorMessage(int64_t mpp_version) +{ + auto err_msg = fmt::format("Invalid mpp version `{}`, expect version: min `{}`, max `{}` release version `{}`", + mpp_version, + TiDB::MIN_MPP_VERSION, + TiDB::MPP_VERSION, + MPP_TIFLASH_RELEASE_VERSION); + return err_msg; +} + +int64_t GetMppVersion() +{ + return MPP_VERSION; +} + +} // namespace TiDB \ No newline at end of file diff --git a/dbms/src/Flash/Mpp/newMPPExchangeWriter.h b/dbms/src/Flash/Mpp/newMPPExchangeWriter.h index 447979bacf2..0a27c0ad7a9 100644 --- a/dbms/src/Flash/Mpp/newMPPExchangeWriter.h +++ b/dbms/src/Flash/Mpp/newMPPExchangeWriter.h @@ -24,7 +24,7 @@ namespace DB { template -std::unique_ptr newMPPExchangeWriter( +std::unique_ptr NewMPPExchangeWriter( const StreamWriterPtr & writer, const std::vector & partition_col_ids, const TiDB::TiDBCollators & partition_col_collators, @@ -35,23 +35,27 @@ std::unique_ptr newMPPExchangeWriter( DAGContext & dag_context, bool enable_fine_grained_shuffle, UInt64 fine_grained_shuffle_stream_count, - UInt64 fine_grained_shuffle_batch_size, - mpp::CompressMethod compress_method = mpp::CompressMethod::LZ4) + UInt64 fine_grained_shuffle_batch_size) { - auto mm = tzg::SnappyStatistic::globalInstance().getMethod(); - switch (mm) + auto compress_method = dag_context.getMPPTaskMeta().exchange_sender_meta().compress(); + if (true) // nolint { - case tzg::SnappyStatistic::Method::LZ4: - compress_method = mpp::CompressMethod::LZ4; - break; - case tzg::SnappyStatistic::Method::ZSTD: - compress_method = mpp::CompressMethod::ZSTD; - break; - default: - compress_method = mpp::CompressMethod::NONE; - break; + auto mm = tzg::SnappyStatistic::globalInstance().getMethod(); + switch (mm) + { + case tzg::SnappyStatistic::Method::LZ4: + compress_method = mpp::CompressMethod::LZ4; + break; + case tzg::SnappyStatistic::Method::ZSTD: + compress_method = mpp::CompressMethod::ZSTD; + break; + default: + compress_method = mpp::CompressMethod::NONE; + break; + } } - // compress_method = ; + RUNTIME_CHECK(!enable_fine_grained_shuffle); + RUNTIME_CHECK(dag_context.isMPPTask()); should_send_exec_summary_at_last = dag_context.collect_execution_summaries && should_send_exec_summary_at_last; if (dag_context.isRootMPPTask()) diff --git a/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp b/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp index fcb1be3cea8..c123113aef8 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp +++ b/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp @@ -69,7 +69,7 @@ void PhysicalExchangeSender::transformImpl(DAGPipeline & pipeline, Context & con int stream_id = 0; pipeline.transform([&](auto & stream) { // construct writer - std::unique_ptr response_writer = newMPPExchangeWriter( + std::unique_ptr response_writer = NewMPPExchangeWriter( dag_context.tunnel_set, partition_col_ids, partition_col_collators, From adac1810a2a75a31d57462ad71fb981723ebcdb9 Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Mon, 12 Dec 2022 14:26:46 +0800 Subject: [PATCH 09/93] 3 Signed-off-by: Zhigao Tong --- contrib/tzg.mpp.proto.patch | 41 +++++++++++++++++++++---------------- 1 file changed, 23 insertions(+), 18 deletions(-) diff --git a/contrib/tzg.mpp.proto.patch b/contrib/tzg.mpp.proto.patch index 5c25c940d3f..a771b4171c9 100644 --- a/contrib/tzg.mpp.proto.patch +++ b/contrib/tzg.mpp.proto.patch @@ -1,16 +1,34 @@ diff --git a/proto/mpp.proto b/proto/mpp.proto -index f8af03b..213caf3 100644 +index f8af03b..bea4ae9 100644 --- a/proto/mpp.proto +++ b/proto/mpp.proto -@@ -17,6 +17,7 @@ message TaskMeta { +@@ -11,12 +11,25 @@ option (gogoproto.unmarshaler_all) = true; + + option java_package = "org.tikv.kvproto"; + ++// Data compression method ++enum CompressMethod { ++ NONE = 0; ++ LZ4 = 1; ++ ZSTD = 2; ++} ++ ++message ExchangeSenderMeta { ++ CompressMethod compress = 1; ++} ++ + // TaskMeta contains meta of a mpp plan, including query's ts and task address. + message TaskMeta { + uint64 start_ts = 1; // start ts of a query int64 task_id = 2; // if task id is -1 , it indicates a tidb task. int64 partition_id = 3; // Only used for hash partition string address = 4; // target address of this task. -+ int64 mpp_version = 5; ++ int64 mpp_version = 5; // mpp version ++ optional ExchangeSenderMeta exchange_sender_meta = 6; } message IsAliveRequest { -@@ -24,6 +25,7 @@ message IsAliveRequest { +@@ -24,6 +37,7 @@ message IsAliveRequest { message IsAliveResponse { bool available = 1; @@ -18,20 +36,7 @@ index f8af03b..213caf3 100644 } // Dipsatch the task request to different tiflash servers. -@@ -60,6 +62,12 @@ message EstablishMPPConnectionRequest { - TaskMeta receiver_meta = 2; // node closer to the tidb mpp gather. - } - -+enum CompressMethod { -+ NONE = 0; -+ LZ4 = 1; -+ ZSTD = 2; -+} -+ - // when TiFlash sends data to TiDB, Data packets wrap tipb.SelectResponse, i.e., serialize tipb.SelectResponse into data; - // when TiFlash sends data to TiFlash, data blocks are serialized into chunks, and the execution_summaries in tipb.SelectResponse are serialized into data only for the last packet. - message MPPDataPacket { -@@ -67,6 +75,7 @@ message MPPDataPacket { +@@ -67,6 +81,7 @@ message MPPDataPacket { Error error = 2; repeated bytes chunks = 3; repeated uint64 stream_ids = 4; From 7fd467ee1108050a69a80b34b9d4443ee6afec73 Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Mon, 12 Dec 2022 17:39:21 +0800 Subject: [PATCH 10/93] 4 Signed-off-by: Zhigao Tong --- contrib/tzg.mpp.proto.patch | 15 ++- .../Flash/Coprocessor/CHBlockChunkCodec.cpp | 1 - .../Coprocessor/ChunkDecodeAndSquash.cpp | 3 +- .../Coprocessor/CompressedCHBlockChunkCodec.h | 7 ++ dbms/src/Flash/Coprocessor/DAGContext.h | 7 +- dbms/src/Flash/Coprocessor/tzg-metrics.h | 17 ++-- .../Mpp/BroadcastOrPassThroughWriter.cpp | 50 +++++++++- .../Flash/Mpp/BroadcastOrPassThroughWriter.h | 2 + .../Flash/Mpp/FineGrainedShuffleWriter.cpp | 37 +++++-- dbms/src/Flash/Mpp/FineGrainedShuffleWriter.h | 3 + dbms/src/Flash/Mpp/HashPartitionWriter.cpp | 27 ++++-- dbms/src/Flash/Mpp/HashPartitionWriter.h | 21 +--- dbms/src/Flash/Mpp/LocalBlockMPPTunnelSet.h | 96 ------------------- dbms/src/Flash/Mpp/MPPTask.cpp | 15 ++- dbms/src/Flash/Mpp/MPPTunnelSet.cpp | 21 ++++ dbms/src/Flash/Mpp/MPPTunnelSet.h | 2 + dbms/src/Flash/Mpp/newMPPExchangeWriter.h | 22 +---- .../Transaction/ProxyFFIStatusService.cpp | 5 +- 18 files changed, 175 insertions(+), 176 deletions(-) delete mode 100644 dbms/src/Flash/Mpp/LocalBlockMPPTunnelSet.h diff --git a/contrib/tzg.mpp.proto.patch b/contrib/tzg.mpp.proto.patch index a771b4171c9..361e910e8ff 100644 --- a/contrib/tzg.mpp.proto.patch +++ b/contrib/tzg.mpp.proto.patch @@ -1,8 +1,8 @@ diff --git a/proto/mpp.proto b/proto/mpp.proto -index f8af03b..bea4ae9 100644 +index f8af03b..1582074 100644 --- a/proto/mpp.proto +++ b/proto/mpp.proto -@@ -11,12 +11,25 @@ option (gogoproto.unmarshaler_all) = true; +@@ -11,12 +11,24 @@ option (gogoproto.unmarshaler_all) = true; option java_package = "org.tikv.kvproto"; @@ -24,11 +24,10 @@ index f8af03b..bea4ae9 100644 int64 partition_id = 3; // Only used for hash partition string address = 4; // target address of this task. + int64 mpp_version = 5; // mpp version -+ optional ExchangeSenderMeta exchange_sender_meta = 6; } message IsAliveRequest { -@@ -24,6 +37,7 @@ message IsAliveRequest { +@@ -24,6 +36,7 @@ message IsAliveRequest { message IsAliveResponse { bool available = 1; @@ -36,6 +35,14 @@ index f8af03b..bea4ae9 100644 } // Dipsatch the task request to different tiflash servers. +@@ -36,6 +49,7 @@ message DispatchTaskRequest { + int64 schema_ver = 5; + // Used for partition table scan + repeated coprocessor.TableRegions table_regions = 6; ++ optional ExchangeSenderMeta exchange_sender_meta = 7; + } + + // Get response of DispatchTaskRequest. @@ -67,6 +81,7 @@ message MPPDataPacket { Error error = 2; repeated bytes chunks = 3; diff --git a/dbms/src/Flash/Coprocessor/CHBlockChunkCodec.cpp b/dbms/src/Flash/Coprocessor/CHBlockChunkCodec.cpp index b3f892a294a..8768c67207b 100644 --- a/dbms/src/Flash/Coprocessor/CHBlockChunkCodec.cpp +++ b/dbms/src/Flash/Coprocessor/CHBlockChunkCodec.cpp @@ -23,7 +23,6 @@ #include #include -#include "Flash/Coprocessor/tzg-metrics.h" #include "ext/scope_guard.h" #include "mpp.pb.h" diff --git a/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.cpp b/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.cpp index 9d36c10eca0..9edbf36ab09 100644 --- a/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.cpp +++ b/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.cpp @@ -15,7 +15,6 @@ #include #include -#include "Flash/Coprocessor/tzg-metrics.h" #include "Flash/Mpp/HashPartitionWriter.h" #include "IO/CompressedReadBuffer.h" #include "IO/CompressedStream.h" @@ -42,7 +41,7 @@ std::optional CHBlockChunkDecodeAndSquash::decodeAndSquash(const String & } std::unique_ptr compress_buffer{}; ReadBuffer * istr_ptr = &istr; - if (ToCompressionMethod(compress_method) != CompressionMethod::NONE) + if (ToInternalCompressionMethod(compress_method) != CompressionMethod::NONE) { compress_buffer = std::make_unique>(istr); istr_ptr = compress_buffer.get(); diff --git a/dbms/src/Flash/Coprocessor/CompressedCHBlockChunkCodec.h b/dbms/src/Flash/Coprocessor/CompressedCHBlockChunkCodec.h index 90975bbe923..3d5bcc1addb 100644 --- a/dbms/src/Flash/Coprocessor/CompressedCHBlockChunkCodec.h +++ b/dbms/src/Flash/Coprocessor/CompressedCHBlockChunkCodec.h @@ -18,6 +18,11 @@ #include #include +namespace mpp +{ +enum CompressMethod : int; +} + namespace DB { class CompressedCHBlockChunkCodec final @@ -42,4 +47,6 @@ class CompressedCHBlockChunkCodec final CHBlockChunkCodec chunk_codec; }; +CompressionMethod ToInternalCompressionMethod(mpp::CompressMethod compress_method); + } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/DAGContext.h b/dbms/src/Flash/Coprocessor/DAGContext.h index 004e7decb7b..737e9e9095f 100644 --- a/dbms/src/Flash/Coprocessor/DAGContext.h +++ b/dbms/src/Flash/Coprocessor/DAGContext.h @@ -14,6 +14,7 @@ #pragma once +#include #pragma GCC diagnostic push #pragma GCC diagnostic ignored "-Wunused-parameter" #ifdef __clang__ @@ -144,7 +145,7 @@ class DAGContext } // for mpp - DAGContext(const tipb::DAGRequest & dag_request_, const mpp::TaskMeta & meta_, bool is_root_mpp_task_) + DAGContext(const tipb::DAGRequest & dag_request_, const mpp::TaskMeta & meta_, const mpp::ExchangeSenderMeta & exchange_sender_meta_, bool is_root_mpp_task_) : dag_request(&dag_request_) , dummy_query_string(dag_request->DebugString()) , dummy_ast(makeDummyQuery()) @@ -155,6 +156,7 @@ class DAGContext , flags(dag_request->flags()) , sql_mode(dag_request->sql_mode()) , mpp_task_meta(meta_) + , exchange_sender_meta(exchange_sender_meta_) , mpp_task_id(mpp_task_meta.start_ts(), mpp_task_meta.task_id()) , max_recorded_error_count(getMaxErrorCount(*dag_request)) , warnings(max_recorded_error_count) @@ -169,7 +171,6 @@ class DAGContext // for test explicit DAGContext(UInt64 max_error_count_) : dag_request(nullptr) - , dummy_query_string("") , dummy_ast(makeDummyQuery()) , collect_execution_summaries(false) , is_mpp_task(false) @@ -245,6 +246,7 @@ class DAGContext } UInt64 getWarningCount() { return warning_count; } const mpp::TaskMeta & getMPPTaskMeta() const { return mpp_task_meta; } + const mpp::ExchangeSenderMeta & getExchangeSenderMeta() const { return exchange_sender_meta; } bool isBatchCop() const { return is_batch_cop; } bool isMPPTask() const { return is_mpp_task; } /// root mpp task means mpp task that send data back to TiDB @@ -382,6 +384,7 @@ class DAGContext UInt64 flags; UInt64 sql_mode; mpp::TaskMeta mpp_task_meta; + mpp::ExchangeSenderMeta exchange_sender_meta; const MPPTaskId mpp_task_id = MPPTaskId::unknown_mpp_task_id; /// max_recorded_error_count is the max error/warning need to be recorded in warnings UInt64 max_recorded_error_count; diff --git a/dbms/src/Flash/Coprocessor/tzg-metrics.h b/dbms/src/Flash/Coprocessor/tzg-metrics.h index 6e2bd17239f..4bb8ad11b5d 100644 --- a/dbms/src/Flash/Coprocessor/tzg-metrics.h +++ b/dbms/src/Flash/Coprocessor/tzg-metrics.h @@ -4,22 +4,17 @@ #include #include +#include "mpp.pb.h" + namespace tzg { struct SnappyStatistic { - enum class Method : int - { - LZ4 = 1, - ZSTD = 3, /// Experimental algorithm: https://github.com/Cyan4973/zstd - NONE = 4, /// No compression - }; - mutable std::atomic_uint64_t compressed_size{}; mutable std::atomic_uint64_t uncompressed_size{}; mutable std::atomic_uint64_t package{}; mutable std::atomic_int64_t chunck_stream_cnt{}, max_chunck_stream_cnt{}; - mutable Method method = Method::NONE; + mutable mpp::CompressMethod method{}; mutable std::atomic durations{}, has_write_dur{}; mutable std::atomic_uint64_t encode_bytes{}, has_write_rows{}; @@ -41,11 +36,11 @@ struct SnappyStatistic return uncompressed_size; } - Method getMethod() const + mpp::CompressMethod getMethod() const { return method; } - void setMethod(Method m) + void setMethod(mpp::CompressMethod m) { method = m; } @@ -70,7 +65,7 @@ struct SnappyStatistic ++package; } - void load(uint64_t & compressed_size_, uint64_t & uncompressed_size_, uint64_t & package_, Method & m) const + void load(uint64_t & compressed_size_, uint64_t & uncompressed_size_, uint64_t & package_, mpp::CompressMethod & m) const { compressed_size_ = getCompressedSize(); uncompressed_size_ = getUncompressedSize(); diff --git a/dbms/src/Flash/Mpp/BroadcastOrPassThroughWriter.cpp b/dbms/src/Flash/Mpp/BroadcastOrPassThroughWriter.cpp index bf5b9e75efe..88a39cead6f 100644 --- a/dbms/src/Flash/Mpp/BroadcastOrPassThroughWriter.cpp +++ b/dbms/src/Flash/Mpp/BroadcastOrPassThroughWriter.cpp @@ -14,6 +14,7 @@ #include #include +#include #include #include @@ -29,10 +30,15 @@ BroadcastOrPassThroughWriter::BroadcastOrPassThroughWriter( , batch_send_min_limit(batch_send_min_limit_) , should_send_exec_summary_at_last(should_send_exec_summary_at_last_) , writer(writer_) + , compress_method(dag_context.getExchangeSenderMeta().compress()) { rows_in_blocks = 0; RUNTIME_CHECK(dag_context.encode_type == tipb::EncodeType::TypeCHBlock); chunk_codec_stream = std::make_unique()->newCodecStream(dag_context.result_field_types); + if (auto method = ToInternalCompressionMethod(compress_method); method != CompressionMethod::NONE) + { + compress_chunk_codec_stream = CompressedCHBlockChunkCodec::newCodecStream(dag_context.result_field_types, method); + } } template @@ -82,17 +88,53 @@ void BroadcastOrPassThroughWriter::encodeThenWriteBlocks() return; auto tracked_packet = std::make_shared(); + decltype(tracked_packet) compressed_tracked_packet = {}; + bool need_compress = compress_method != mpp::CompressMethod::NONE; + if (need_compress) + { + auto all_is_local = std::all_of(writer->getTunnels().begin(), writer->getTunnels().end(), [](const auto & tunnel) { + return tunnel->isLocal(); + }); + if (all_is_local) + need_compress = false; + } + if (need_compress) + { + compressed_tracked_packet = std::make_shared(); + } + while (!blocks.empty()) { const auto & block = blocks.back(); - chunk_codec_stream->encode(block, 0, block.rows()); + + if (need_compress) + { + assert(compressed_tracked_packet); + compress_chunk_codec_stream->encode(block, 0, block.rows()); + compressed_tracked_packet->addChunk(compress_chunk_codec_stream->getString()); + compress_chunk_codec_stream->clear(); + } + + { + assert(tracked_packet); + chunk_codec_stream->encode(block, 0, block.rows()); + tracked_packet->addChunk(chunk_codec_stream->getString()); + chunk_codec_stream->clear(); + } + blocks.pop_back(); - tracked_packet->addChunk(chunk_codec_stream->getString()); - chunk_codec_stream->clear(); } assert(blocks.empty()); rows_in_blocks = 0; - writer->broadcastOrPassThroughWrite(tracked_packet); + + if (!need_compress) + { + writer->broadcastOrPassThroughWrite(tracked_packet); + } + else + { + writer->broadcastOrPassThroughWrite(tracked_packet, compressed_tracked_packet); + } } template class BroadcastOrPassThroughWriter; diff --git a/dbms/src/Flash/Mpp/BroadcastOrPassThroughWriter.h b/dbms/src/Flash/Mpp/BroadcastOrPassThroughWriter.h index a7f88cf90e3..3ff348c51a9 100644 --- a/dbms/src/Flash/Mpp/BroadcastOrPassThroughWriter.h +++ b/dbms/src/Flash/Mpp/BroadcastOrPassThroughWriter.h @@ -46,7 +46,9 @@ class BroadcastOrPassThroughWriter : public DAGResponseWriter ExchangeWriterPtr writer; std::vector blocks; size_t rows_in_blocks; + mpp::CompressMethod compress_method{}; std::unique_ptr chunk_codec_stream; + std::unique_ptr compress_chunk_codec_stream; }; } // namespace DB diff --git a/dbms/src/Flash/Mpp/FineGrainedShuffleWriter.cpp b/dbms/src/Flash/Mpp/FineGrainedShuffleWriter.cpp index c04df4c8e42..4f217db124b 100644 --- a/dbms/src/Flash/Mpp/FineGrainedShuffleWriter.cpp +++ b/dbms/src/Flash/Mpp/FineGrainedShuffleWriter.cpp @@ -14,6 +14,7 @@ #include #include +#include #include #include #include @@ -38,12 +39,17 @@ FineGrainedShuffleWriter::FineGrainedShuffleWriter( , fine_grained_shuffle_batch_size(fine_grained_shuffle_batch_size_) , batch_send_row_limit(fine_grained_shuffle_batch_size * fine_grained_shuffle_stream_count) , hash(0) + , compress_method(dag_context.getExchangeSenderMeta().compress()) { rows_in_blocks = 0; partition_num = writer_->getPartitionNum(); RUNTIME_CHECK(partition_num > 0); RUNTIME_CHECK(dag_context.encode_type == tipb::EncodeType::TypeCHBlock); chunk_codec_stream = std::make_unique()->newCodecStream(dag_context.result_field_types); + if (auto method = ToInternalCompressionMethod(compress_method); method != CompressionMethod::NONE) + { + compress_chunk_codec_stream = CompressedCHBlockChunkCodec::newCodecStream(dag_context.result_field_types, method); + } } template @@ -128,6 +134,16 @@ template void FineGrainedShuffleWriter::batchWriteFineGrainedShuffle() { auto tracked_packets = HashBaseWriterHelper::createPackets(partition_num); + for (size_t part_id = 0; part_id < partition_num; ++part_id) + { + auto method = compress_method; + if (writer->getTunnels()[part_id]->isLocal()) + { + method = mpp::CompressMethod::NONE; + } + tracked_packets[part_id]->getPacket().set_compress(method); + } + if (likely(!blocks.empty())) { assert(rows_in_blocks > 0); @@ -153,13 +169,20 @@ void FineGrainedShuffleWriter::batchWriteFineGrainedShuffle() for (size_t col_id = 0; col_id < num_columns; ++col_id) columns.emplace_back(std::move(scattered[col_id][bucket_idx + stream_idx])); auto block = header.cloneWithColumns(std::move(columns)); - - // encode into packet - chunk_codec_stream->encode(block, 0, block.rows()); - tracked_packets[part_id]->addChunk(chunk_codec_stream->getString()); - tracked_packets[part_id]->getPacket().add_stream_ids(stream_idx); - chunk_codec_stream->clear(); - + { + ChunkCodecStream * codec_stream = chunk_codec_stream.get(); + if (tracked_packets[part_id]->getPacket().compress() != mpp::CompressMethod::NONE) + { + assert(compress_chunk_codec_stream); + // no need compress + codec_stream = compress_chunk_codec_stream.get(); + } + // encode into packet + codec_stream->encode(block, 0, block.rows()); + tracked_packets[part_id]->addChunk(codec_stream->getString()); + tracked_packets[part_id]->getPacket().add_stream_ids(stream_idx); + codec_stream->clear(); + } // disassemble the block back to scatter columns columns = block.mutateColumns(); for (size_t col_id = 0; col_id < num_columns; ++col_id) diff --git a/dbms/src/Flash/Mpp/FineGrainedShuffleWriter.h b/dbms/src/Flash/Mpp/FineGrainedShuffleWriter.h index 895b55817fd..f91fe804366 100644 --- a/dbms/src/Flash/Mpp/FineGrainedShuffleWriter.h +++ b/dbms/src/Flash/Mpp/FineGrainedShuffleWriter.h @@ -57,6 +57,7 @@ class FineGrainedShuffleWriter : public DAGResponseWriter size_t rows_in_blocks = 0; uint16_t partition_num; std::unique_ptr chunk_codec_stream; + std::unique_ptr compress_chunk_codec_stream; UInt64 fine_grained_shuffle_stream_count; UInt64 fine_grained_shuffle_batch_size; @@ -67,6 +68,8 @@ class FineGrainedShuffleWriter : public DAGResponseWriter WeakHash32 hash; IColumn::Selector selector; std::vector scattered; // size = num_columns + + mpp::CompressMethod compress_method{}; }; } // namespace DB diff --git a/dbms/src/Flash/Mpp/HashPartitionWriter.cpp b/dbms/src/Flash/Mpp/HashPartitionWriter.cpp index 6effe888e12..cc72cadd4ca 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriter.cpp +++ b/dbms/src/Flash/Mpp/HashPartitionWriter.cpp @@ -24,7 +24,6 @@ #include "Common/Exception.h" #include "Common/Stopwatch.h" #include "Flash/Coprocessor/CompressedCHBlockChunkCodec.h" -#include "Flash/Coprocessor/tzg-metrics.h" #include "IO/CompressedStream.h" #include "common/logger_useful.h" #include "ext/scope_guard.h" @@ -39,26 +38,26 @@ HashPartitionWriter::HashPartitionWriter( TiDB::TiDBCollators collators_, Int64 batch_send_min_limit_, bool should_send_exec_summary_at_last_, - DAGContext & dag_context_, - mpp::CompressMethod compress_method_) + DAGContext & dag_context_) : DAGResponseWriter(/*records_per_chunk=*/-1, dag_context_) , batch_send_min_limit(batch_send_min_limit_) , should_send_exec_summary_at_last(should_send_exec_summary_at_last_) , writer(writer_) , partition_col_ids(std::move(partition_col_ids_)) , collators(std::move(collators_)) - , compress_method(compress_method_) + , compress_method(dag_context.getExchangeSenderMeta().compress()) { rows_in_blocks = 0; partition_num = writer_->getPartitionNum(); RUNTIME_CHECK(partition_num > 0); RUNTIME_CHECK(dag_context.encode_type == tipb::EncodeType::TypeCHBlock); - auto method = ToCompressionMethod(compress_method); - if (method != CompressionMethod::NONE) + + if (auto method = ToInternalCompressionMethod(compress_method); method != CompressionMethod::NONE) { compress_chunk_codec_stream = CompressedCHBlockChunkCodec::newCodecStream(dag_context.result_field_types, method); } chunk_codec_stream = std::make_unique()->newCodecStream(dag_context.result_field_types); + // LOG_TRACE(&Poco::Logger::get("tzg"), "using mpp CompressMethod {}, partition_num {}", mpp::CompressMethod_Name(compress_method), partition_num); // { // size_t local_cnt = 0; @@ -116,6 +115,8 @@ void HashPartitionWriter::write(const Block & block) template void HashPartitionWriter::partitionAndEncodeThenWriteBlocks() { + assert(chunk_codec_stream); + auto tracked_packets = HashBaseWriterHelper::createPackets(partition_num); for (size_t part_id = 0; part_id < partition_num; ++part_id) @@ -192,4 +193,18 @@ void HashPartitionWriter::writePackets(const TrackedMppDataPa template class HashPartitionWriter; +CompressionMethod ToInternalCompressionMethod(mpp::CompressMethod compress_method) +{ + switch (compress_method) + { + case mpp::NONE: + return CompressionMethod::NONE; + case mpp::LZ4: + return CompressionMethod::LZ4; + case mpp::ZSTD: + return CompressionMethod::ZSTD; + default: + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unkown compress method {}", mpp::CompressMethod_Name(compress_method)); + } +} } // namespace DB diff --git a/dbms/src/Flash/Mpp/HashPartitionWriter.h b/dbms/src/Flash/Mpp/HashPartitionWriter.h index dae51bb5560..fe9817a3487 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriter.h +++ b/dbms/src/Flash/Mpp/HashPartitionWriter.h @@ -21,7 +21,6 @@ #include #include -#include "mpp.pb.h" namespace DB { @@ -35,8 +34,7 @@ class HashPartitionWriter : public DAGResponseWriter TiDB::TiDBCollators collators_, Int64 batch_send_min_limit_, bool should_send_exec_summary_at_last, - DAGContext & dag_context_, - mpp::CompressMethod compress_method_); + DAGContext & dag_context_); void write(const Block & block) override; void flush() override; void finishWrite() override; @@ -57,25 +55,10 @@ class HashPartitionWriter : public DAGResponseWriter TiDB::TiDBCollators collators; size_t rows_in_blocks; uint16_t partition_num; - mpp::CompressMethod compress_method; + mpp::CompressMethod compress_method{}; std::unique_ptr chunk_codec_stream; std::unique_ptr compress_chunk_codec_stream; }; -inline CompressionMethod ToCompressionMethod(mpp::CompressMethod compress_method) -{ - switch (compress_method) - { - case mpp::NONE: - return CompressionMethod::NONE; - case mpp::LZ4: - return CompressionMethod::LZ4; - case mpp::ZSTD: - return CompressionMethod::ZSTD; - default: - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unkown compress method {}", mpp::CompressMethod_Name(compress_method)); - } -} - } // namespace DB diff --git a/dbms/src/Flash/Mpp/LocalBlockMPPTunnelSet.h b/dbms/src/Flash/Mpp/LocalBlockMPPTunnelSet.h deleted file mode 100644 index 3bb90d70617..00000000000 --- a/dbms/src/Flash/Mpp/LocalBlockMPPTunnelSet.h +++ /dev/null @@ -1,96 +0,0 @@ -// Copyright 2022 PingCAP, Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -#pragma once - -#include -#include -#include - -#include - -namespace DB -{ -template -class LocalBlockMPPTunnelSetBase : private boost::noncopyable -{ -public: - using TunnelPtr = std::shared_ptr; - explicit LocalBlockMPPTunnelSetBase(const String & req_id) - : log(Logger::get(req_id)) - {} - - /// for both broadcast writing and partition writing, only - /// return meaningful execution summary for the first tunnel, - /// because in TiDB, it does not know enough information - /// about the execution details for the mpp query, it just - /// add up all the execution summaries for the same executor, - /// so if return execution summary for all the tunnels, the - /// information in TiDB will be amplified, which may make - /// user confused. - // this is a root mpp writing. - void write(tipb::SelectResponse & response); - // this is a broadcast or pass through writing. - void broadcastOrPassThroughWrite(const TrackedMppDataPacketPtr & packet); - // this is a partition writing. - void partitionWrite(Block && block, int16_t partition_id) - { - } - // this is a execution summary writing. - void sendExecutionSummary(tipb::SelectResponse & response); - - void close(const String & reason, bool wait_sender_finish); - void finishWrite(); - void registerTunnel(const MPPTaskId & receiver_task_id, const TunnelPtr & tunnel) - { - if (receiver_task_id_to_index_map.find(receiver_task_id) != receiver_task_id_to_index_map.end()) - throw Exception(fmt::format("the tunnel {} has been registered", tunnel->id())); - - receiver_task_id_to_index_map[receiver_task_id] = tunnels.size(); - tunnels.push_back(tunnel); - if (!tunnel->isLocal()) - { - remote_tunnel_cnt++; - } - } - - TunnelPtr getTunnelByReceiverTaskId(const MPPTaskId & id); - - uint16_t getPartitionNum() const { return tunnels.size(); } - - int getRemoteTunnelCnt() - { - return remote_tunnel_cnt; - } - - const std::vector & getTunnels() const { return tunnels; } - -private: - std::vector tunnels; - std::unordered_map receiver_task_id_to_index_map; - const LoggerPtr log; - - int remote_tunnel_cnt = 0; -}; - -class LocalBlockMPPTunnelSet : public LocalBlockMPPTunnelSetBase -{ -public: - using Base = LocalBlockMPPTunnelSetBase; - using Base::Base; -}; - -using LocalBlockMPPTunnelSetPtr = std::shared_ptr; - -} // namespace DB diff --git a/dbms/src/Flash/Mpp/MPPTask.cpp b/dbms/src/Flash/Mpp/MPPTask.cpp index 37ba73cf926..e2edf01a44f 100644 --- a/dbms/src/Flash/Mpp/MPPTask.cpp +++ b/dbms/src/Flash/Mpp/MPPTask.cpp @@ -39,6 +39,8 @@ #include #include +#include "Flash/Coprocessor/tzg-metrics.h" + namespace DB { namespace FailPoints @@ -277,7 +279,18 @@ void MPPTask::prepare(const mpp::DispatchTaskRequest & task_request) } is_root_mpp_task = task_meta.task_id() == -1; } - dag_context = std::make_unique(dag_req, task_request.meta(), is_root_mpp_task); + + if (true) // nolint + { + auto exchange_sender_meta = task_request.exchange_sender_meta(); + exchange_sender_meta.set_compress(tzg::SnappyStatistic::globalInstance().getMethod()); + dag_context = std::make_unique(dag_req, task_request.meta(), exchange_sender_meta, is_root_mpp_task); + } + else + { + dag_context = std::make_unique(dag_req, task_request.meta(), task_request.exchange_sender_meta(), is_root_mpp_task); + } + dag_context->log = log; dag_context->tables_regions_info = std::move(tables_regions_info); dag_context->tidb_host = context->getClientInfo().current_address.toString(); diff --git a/dbms/src/Flash/Mpp/MPPTunnelSet.cpp b/dbms/src/Flash/Mpp/MPPTunnelSet.cpp index 6fb577e5f6f..1ac6c99e655 100644 --- a/dbms/src/Flash/Mpp/MPPTunnelSet.cpp +++ b/dbms/src/Flash/Mpp/MPPTunnelSet.cpp @@ -67,6 +67,27 @@ void MPPTunnelSetBase::broadcastOrPassThroughWrite(const TrackedMppDataP tunnel->write(tunnel->isLocal() && tunnels.size() > 1 ? packet->copy() : packet); } } +template +void MPPTunnelSetBase::broadcastOrPassThroughWrite(const TrackedMppDataPacketPtr & local_packet, const TrackedMppDataPacketPtr & not_local_packet) +{ + checkPacketSize(local_packet->getPacket().ByteSizeLong()); + checkPacketSize(not_local_packet->getPacket().ByteSizeLong()); + + RUNTIME_CHECK(!tunnels.empty()); + for (auto & tunnel : tunnels) + { + if (tunnel->isLocal()) + { + // We should copy the tracked packet for local tunnel. + // Because `switchMemoryTracker` will be called later in `readForLocal`. + tunnel->write(tunnels.size() > 1 ? local_packet->copy() : local_packet); + } + else + { + tunnel->write(not_local_packet); + } + } +} template void MPPTunnelSetBase::partitionWrite(const TrackedMppDataPacketPtr & packet, int16_t partition_id) diff --git a/dbms/src/Flash/Mpp/MPPTunnelSet.h b/dbms/src/Flash/Mpp/MPPTunnelSet.h index 64fd7f396f2..263a01fb901 100644 --- a/dbms/src/Flash/Mpp/MPPTunnelSet.h +++ b/dbms/src/Flash/Mpp/MPPTunnelSet.h @@ -49,6 +49,8 @@ class MPPTunnelSetBase : private boost::noncopyable void write(tipb::SelectResponse & response); // this is a broadcast or pass through writing. void broadcastOrPassThroughWrite(const TrackedMppDataPacketPtr & packet); + // this is a broadcast or pass through writing. `local_packet` will only be sent by local tunnel, `not_local_packet` is on the contrary. + void broadcastOrPassThroughWrite(const TrackedMppDataPacketPtr & local_packet, const TrackedMppDataPacketPtr & not_local_packet); // this is a partition writing. void partitionWrite(const TrackedMppDataPacketPtr & packet, int16_t partition_id); // this is a execution summary writing. diff --git a/dbms/src/Flash/Mpp/newMPPExchangeWriter.h b/dbms/src/Flash/Mpp/newMPPExchangeWriter.h index 0a27c0ad7a9..3315d3785c6 100644 --- a/dbms/src/Flash/Mpp/newMPPExchangeWriter.h +++ b/dbms/src/Flash/Mpp/newMPPExchangeWriter.h @@ -19,8 +19,6 @@ #include #include -#include "Flash/Coprocessor/tzg-metrics.h" - namespace DB { template @@ -37,23 +35,6 @@ std::unique_ptr NewMPPExchangeWriter( UInt64 fine_grained_shuffle_stream_count, UInt64 fine_grained_shuffle_batch_size) { - auto compress_method = dag_context.getMPPTaskMeta().exchange_sender_meta().compress(); - if (true) // nolint - { - auto mm = tzg::SnappyStatistic::globalInstance().getMethod(); - switch (mm) - { - case tzg::SnappyStatistic::Method::LZ4: - compress_method = mpp::CompressMethod::LZ4; - break; - case tzg::SnappyStatistic::Method::ZSTD: - compress_method = mpp::CompressMethod::ZSTD; - break; - default: - compress_method = mpp::CompressMethod::NONE; - break; - } - } RUNTIME_CHECK(!enable_fine_grained_shuffle); RUNTIME_CHECK(dag_context.isMPPTask()); @@ -92,8 +73,7 @@ std::unique_ptr NewMPPExchangeWriter( partition_col_collators, batch_send_min_limit, should_send_exec_summary_at_last, - dag_context, - compress_method); + dag_context); } } else diff --git a/dbms/src/Storages/Transaction/ProxyFFIStatusService.cpp b/dbms/src/Storages/Transaction/ProxyFFIStatusService.cpp index 4323be82adf..c8acd46da7e 100644 --- a/dbms/src/Storages/Transaction/ProxyFFIStatusService.cpp +++ b/dbms/src/Storages/Transaction/ProxyFFIStatusService.cpp @@ -26,6 +26,7 @@ #include "Flash/Coprocessor/tzg-metrics.h" #include "magic_enum.hpp" +#include "mpp.pb.h" namespace DB { @@ -130,7 +131,7 @@ HttpRequestRes HandleHttpRequestCompressStatus( uint64_t compressed_size; uint64_t uncompressed_size; uint64_t package; - tzg::SnappyStatistic::Method method; + mpp::CompressMethod method; tzg::SnappyStatistic::globalInstance().load(compressed_size, uncompressed_size, package, method); // double f_compressed_size_mb = compressed_size * 1.0 / 1024 / 1024; @@ -223,7 +224,7 @@ HttpRequestRes HandleHttpRequestSetCompressMethod( std::string_view) { auto method_str(path.substr(api_name.size())); - auto method = magic_enum::enum_cast(method_str); + auto method = magic_enum::enum_cast(method_str); if (method) { if (tzg::SnappyStatistic::globalInstance().getMethod() != *method) From 2b811fb2ca31a47a8dcb1fda20f879991e7eef27 Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Mon, 12 Dec 2022 17:47:26 +0800 Subject: [PATCH 11/93] 5 Signed-off-by: Zhigao Tong --- dbms/src/Flash/Coprocessor/tzg-metrics.h | 15 +++++++++++---- dbms/src/Flash/Mpp/MPPTask.cpp | 17 ++++++++++++++++- .../Transaction/ProxyFFIStatusService.cpp | 4 ++-- 3 files changed, 29 insertions(+), 7 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/tzg-metrics.h b/dbms/src/Flash/Coprocessor/tzg-metrics.h index 4bb8ad11b5d..553528cc1d9 100644 --- a/dbms/src/Flash/Coprocessor/tzg-metrics.h +++ b/dbms/src/Flash/Coprocessor/tzg-metrics.h @@ -10,11 +10,18 @@ namespace tzg { struct SnappyStatistic { + enum CompressMethod : int + { + NONE = 0, + LZ4 = 1, + ZSTD = 2, + }; + mutable std::atomic_uint64_t compressed_size{}; mutable std::atomic_uint64_t uncompressed_size{}; mutable std::atomic_uint64_t package{}; mutable std::atomic_int64_t chunck_stream_cnt{}, max_chunck_stream_cnt{}; - mutable mpp::CompressMethod method{}; + mutable CompressMethod method{}; mutable std::atomic durations{}, has_write_dur{}; mutable std::atomic_uint64_t encode_bytes{}, has_write_rows{}; @@ -36,11 +43,11 @@ struct SnappyStatistic return uncompressed_size; } - mpp::CompressMethod getMethod() const + CompressMethod getMethod() const { return method; } - void setMethod(mpp::CompressMethod m) + void setMethod(CompressMethod m) { method = m; } @@ -65,7 +72,7 @@ struct SnappyStatistic ++package; } - void load(uint64_t & compressed_size_, uint64_t & uncompressed_size_, uint64_t & package_, mpp::CompressMethod & m) const + void load(uint64_t & compressed_size_, uint64_t & uncompressed_size_, uint64_t & package_, CompressMethod & m) const { compressed_size_ = getCompressedSize(); uncompressed_size_ = getUncompressedSize(); diff --git a/dbms/src/Flash/Mpp/MPPTask.cpp b/dbms/src/Flash/Mpp/MPPTask.cpp index e2edf01a44f..6c8e3773d43 100644 --- a/dbms/src/Flash/Mpp/MPPTask.cpp +++ b/dbms/src/Flash/Mpp/MPPTask.cpp @@ -40,6 +40,7 @@ #include #include "Flash/Coprocessor/tzg-metrics.h" +#include "mpp.pb.h" namespace DB { @@ -283,7 +284,21 @@ void MPPTask::prepare(const mpp::DispatchTaskRequest & task_request) if (true) // nolint { auto exchange_sender_meta = task_request.exchange_sender_meta(); - exchange_sender_meta.set_compress(tzg::SnappyStatistic::globalInstance().getMethod()); + mpp::CompressMethod compress_method{}; + switch (tzg::SnappyStatistic::globalInstance().getMethod()) + { + case tzg::SnappyStatistic::CompressMethod::LZ4: + compress_method = mpp::CompressMethod::LZ4; + break; + case tzg::SnappyStatistic::CompressMethod::ZSTD: + compress_method = mpp::CompressMethod::ZSTD; + break; + default: + compress_method = mpp::CompressMethod::NONE; + break; + } + auto m = reinterpret_cast(compress_method); + exchange_sender_meta.set_compress(m); dag_context = std::make_unique(dag_req, task_request.meta(), exchange_sender_meta, is_root_mpp_task); } else diff --git a/dbms/src/Storages/Transaction/ProxyFFIStatusService.cpp b/dbms/src/Storages/Transaction/ProxyFFIStatusService.cpp index c8acd46da7e..6895398c4a0 100644 --- a/dbms/src/Storages/Transaction/ProxyFFIStatusService.cpp +++ b/dbms/src/Storages/Transaction/ProxyFFIStatusService.cpp @@ -131,7 +131,7 @@ HttpRequestRes HandleHttpRequestCompressStatus( uint64_t compressed_size; uint64_t uncompressed_size; uint64_t package; - mpp::CompressMethod method; + tzg::SnappyStatistic::CompressMethod method; tzg::SnappyStatistic::globalInstance().load(compressed_size, uncompressed_size, package, method); // double f_compressed_size_mb = compressed_size * 1.0 / 1024 / 1024; @@ -224,7 +224,7 @@ HttpRequestRes HandleHttpRequestSetCompressMethod( std::string_view) { auto method_str(path.substr(api_name.size())); - auto method = magic_enum::enum_cast(method_str); + auto method = magic_enum::enum_cast(method_str); if (method) { if (tzg::SnappyStatistic::globalInstance().getMethod() != *method) From e1448ca54abb7d7404e721106fb0829dc452361d Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Mon, 12 Dec 2022 17:47:56 +0800 Subject: [PATCH 12/93] 6 Signed-off-by: Zhigao Tong --- dbms/src/Flash/Coprocessor/tzg-metrics.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/tzg-metrics.h b/dbms/src/Flash/Coprocessor/tzg-metrics.h index 553528cc1d9..f49179d93d6 100644 --- a/dbms/src/Flash/Coprocessor/tzg-metrics.h +++ b/dbms/src/Flash/Coprocessor/tzg-metrics.h @@ -4,8 +4,6 @@ #include #include -#include "mpp.pb.h" - namespace tzg { struct SnappyStatistic From 8ce25ecc53a0ffe33af72bb2426a200a2bd9a00d Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Mon, 12 Dec 2022 17:51:54 +0800 Subject: [PATCH 13/93] 7 Signed-off-by: Zhigao Tong --- dbms/src/Flash/Mpp/MPPTask.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Flash/Mpp/MPPTask.cpp b/dbms/src/Flash/Mpp/MPPTask.cpp index 6c8e3773d43..46e6203af38 100644 --- a/dbms/src/Flash/Mpp/MPPTask.cpp +++ b/dbms/src/Flash/Mpp/MPPTask.cpp @@ -297,8 +297,8 @@ void MPPTask::prepare(const mpp::DispatchTaskRequest & task_request) compress_method = mpp::CompressMethod::NONE; break; } - auto m = reinterpret_cast(compress_method); - exchange_sender_meta.set_compress(m); + exchange_sender_meta.set_compress(compress_method); + LOG_DEBUG(log, "DAGContext use compress method {}", mpp::CompressMethod_Name(compress_method)); dag_context = std::make_unique(dag_req, task_request.meta(), exchange_sender_meta, is_root_mpp_task); } else From f671ba3c1b01cb538d6e2a95f2f36d3fb619c318 Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Wed, 14 Dec 2022 17:03:16 +0800 Subject: [PATCH 14/93] 8 Signed-off-by: Zhigao Tong --- contrib/kvproto | 2 +- contrib/tzg.mpp.proto.patch | 755 ++++++++++++++++++++++++++++++++++- dbms/src/Flash/Mpp/Utils.cpp | 1 + 3 files changed, 755 insertions(+), 3 deletions(-) diff --git a/contrib/kvproto b/contrib/kvproto index 26e28e6a281..6c56ac56fe5 160000 --- a/contrib/kvproto +++ b/contrib/kvproto @@ -1 +1 @@ -Subproject commit 26e28e6a281abb927f91ef992eb8f93b39698ffa +Subproject commit 6c56ac56fe5fbe244cfa377115bca8d931c7fdd0 diff --git a/contrib/tzg.mpp.proto.patch b/contrib/tzg.mpp.proto.patch index 361e910e8ff..b886cf4e20e 100644 --- a/contrib/tzg.mpp.proto.patch +++ b/contrib/tzg.mpp.proto.patch @@ -1,5 +1,756 @@ +diff --git a/pkg/mpp/mpp.pb.go b/pkg/mpp/mpp.pb.go +index 16fd5dd..9eda45d 100644 +--- a/pkg/mpp/mpp.pb.go ++++ b/pkg/mpp/mpp.pb.go +@@ -26,12 +26,89 @@ var _ = math.Inf + // proto package needs to be updated. + const _ = proto.ProtoPackageIsVersion3 // please upgrade the proto package + ++// Data compression method ++type CompressMethod int32 ++ ++const ( ++ CompressMethod_NONE CompressMethod = 0 ++ CompressMethod_LZ4 CompressMethod = 1 ++ CompressMethod_ZSTD CompressMethod = 2 ++) ++ ++var CompressMethod_name = map[int32]string{ ++ 0: "NONE", ++ 1: "LZ4", ++ 2: "ZSTD", ++} ++ ++var CompressMethod_value = map[string]int32{ ++ "NONE": 0, ++ "LZ4": 1, ++ "ZSTD": 2, ++} ++ ++func (x CompressMethod) String() string { ++ return proto.EnumName(CompressMethod_name, int32(x)) ++} ++ ++func (CompressMethod) EnumDescriptor() ([]byte, []int) { ++ return fileDescriptor_819623c7fa76fc55, []int{0} ++} ++ ++type ExchangeSenderMeta struct { ++ Compress CompressMethod `protobuf:"varint,1,opt,name=compress,proto3,enum=mpp.CompressMethod" json:"compress,omitempty"` ++ XXX_NoUnkeyedLiteral struct{} `json:"-"` ++ XXX_unrecognized []byte `json:"-"` ++ XXX_sizecache int32 `json:"-"` ++} ++ ++func (m *ExchangeSenderMeta) Reset() { *m = ExchangeSenderMeta{} } ++func (m *ExchangeSenderMeta) String() string { return proto.CompactTextString(m) } ++func (*ExchangeSenderMeta) ProtoMessage() {} ++func (*ExchangeSenderMeta) Descriptor() ([]byte, []int) { ++ return fileDescriptor_819623c7fa76fc55, []int{0} ++} ++func (m *ExchangeSenderMeta) XXX_Unmarshal(b []byte) error { ++ return m.Unmarshal(b) ++} ++func (m *ExchangeSenderMeta) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { ++ if deterministic { ++ return xxx_messageInfo_ExchangeSenderMeta.Marshal(b, m, deterministic) ++ } else { ++ b = b[:cap(b)] ++ n, err := m.MarshalToSizedBuffer(b) ++ if err != nil { ++ return nil, err ++ } ++ return b[:n], nil ++ } ++} ++func (m *ExchangeSenderMeta) XXX_Merge(src proto.Message) { ++ xxx_messageInfo_ExchangeSenderMeta.Merge(m, src) ++} ++func (m *ExchangeSenderMeta) XXX_Size() int { ++ return m.Size() ++} ++func (m *ExchangeSenderMeta) XXX_DiscardUnknown() { ++ xxx_messageInfo_ExchangeSenderMeta.DiscardUnknown(m) ++} ++ ++var xxx_messageInfo_ExchangeSenderMeta proto.InternalMessageInfo ++ ++func (m *ExchangeSenderMeta) GetCompress() CompressMethod { ++ if m != nil { ++ return m.Compress ++ } ++ return CompressMethod_NONE ++} ++ + // TaskMeta contains meta of a mpp plan, including query's ts and task address. + type TaskMeta struct { + StartTs uint64 `protobuf:"varint,1,opt,name=start_ts,json=startTs,proto3" json:"start_ts,omitempty"` + TaskId int64 `protobuf:"varint,2,opt,name=task_id,json=taskId,proto3" json:"task_id,omitempty"` + PartitionId int64 `protobuf:"varint,3,opt,name=partition_id,json=partitionId,proto3" json:"partition_id,omitempty"` + Address string `protobuf:"bytes,4,opt,name=address,proto3" json:"address,omitempty"` ++ MppVersion int64 `protobuf:"varint,5,opt,name=mpp_version,json=mppVersion,proto3" json:"mpp_version,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +@@ -41,7 +118,7 @@ func (m *TaskMeta) Reset() { *m = TaskMeta{} } + func (m *TaskMeta) String() string { return proto.CompactTextString(m) } + func (*TaskMeta) ProtoMessage() {} + func (*TaskMeta) Descriptor() ([]byte, []int) { +- return fileDescriptor_819623c7fa76fc55, []int{0} ++ return fileDescriptor_819623c7fa76fc55, []int{1} + } + func (m *TaskMeta) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +@@ -98,6 +175,13 @@ func (m *TaskMeta) GetAddress() string { + return "" + } + ++func (m *TaskMeta) GetMppVersion() int64 { ++ if m != nil { ++ return m.MppVersion ++ } ++ return 0 ++} ++ + type IsAliveRequest struct { + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` +@@ -108,7 +192,7 @@ func (m *IsAliveRequest) Reset() { *m = IsAliveRequest{} } + func (m *IsAliveRequest) String() string { return proto.CompactTextString(m) } + func (*IsAliveRequest) ProtoMessage() {} + func (*IsAliveRequest) Descriptor() ([]byte, []int) { +- return fileDescriptor_819623c7fa76fc55, []int{1} ++ return fileDescriptor_819623c7fa76fc55, []int{2} + } + func (m *IsAliveRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +@@ -139,6 +223,7 @@ var xxx_messageInfo_IsAliveRequest proto.InternalMessageInfo + + type IsAliveResponse struct { + Available bool `protobuf:"varint,1,opt,name=available,proto3" json:"available,omitempty"` ++ MppVersion int64 `protobuf:"varint,2,opt,name=mpp_version,json=mppVersion,proto3" json:"mpp_version,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +@@ -148,7 +233,7 @@ func (m *IsAliveResponse) Reset() { *m = IsAliveResponse{} } + func (m *IsAliveResponse) String() string { return proto.CompactTextString(m) } + func (*IsAliveResponse) ProtoMessage() {} + func (*IsAliveResponse) Descriptor() ([]byte, []int) { +- return fileDescriptor_819623c7fa76fc55, []int{2} ++ return fileDescriptor_819623c7fa76fc55, []int{3} + } + func (m *IsAliveResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +@@ -184,6 +269,13 @@ func (m *IsAliveResponse) GetAvailable() bool { + return false + } + ++func (m *IsAliveResponse) GetMppVersion() int64 { ++ if m != nil { ++ return m.MppVersion ++ } ++ return 0 ++} ++ + // Dipsatch the task request to different tiflash servers. + type DispatchTaskRequest struct { + Meta *TaskMeta `protobuf:"bytes,1,opt,name=meta,proto3" json:"meta,omitempty"` +@@ -194,6 +286,7 @@ type DispatchTaskRequest struct { + SchemaVer int64 `protobuf:"varint,5,opt,name=schema_ver,json=schemaVer,proto3" json:"schema_ver,omitempty"` + // Used for partition table scan + TableRegions []*coprocessor.TableRegions `protobuf:"bytes,6,rep,name=table_regions,json=tableRegions,proto3" json:"table_regions,omitempty"` ++ ExchangeSenderMeta *ExchangeSenderMeta `protobuf:"bytes,7,opt,name=exchange_sender_meta,json=exchangeSenderMeta,proto3" json:"exchange_sender_meta,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +@@ -203,7 +296,7 @@ func (m *DispatchTaskRequest) Reset() { *m = DispatchTaskRequest{} } + func (m *DispatchTaskRequest) String() string { return proto.CompactTextString(m) } + func (*DispatchTaskRequest) ProtoMessage() {} + func (*DispatchTaskRequest) Descriptor() ([]byte, []int) { +- return fileDescriptor_819623c7fa76fc55, []int{3} ++ return fileDescriptor_819623c7fa76fc55, []int{4} + } + func (m *DispatchTaskRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +@@ -274,6 +367,13 @@ func (m *DispatchTaskRequest) GetTableRegions() []*coprocessor.TableRegions { + return nil + } + ++func (m *DispatchTaskRequest) GetExchangeSenderMeta() *ExchangeSenderMeta { ++ if m != nil { ++ return m.ExchangeSenderMeta ++ } ++ return nil ++} ++ + // Get response of DispatchTaskRequest. + type DispatchTaskResponse struct { + Error *Error `protobuf:"bytes,1,opt,name=error,proto3" json:"error,omitempty"` +@@ -287,7 +387,7 @@ func (m *DispatchTaskResponse) Reset() { *m = DispatchTaskResponse{} } + func (m *DispatchTaskResponse) String() string { return proto.CompactTextString(m) } + func (*DispatchTaskResponse) ProtoMessage() {} + func (*DispatchTaskResponse) Descriptor() ([]byte, []int) { +- return fileDescriptor_819623c7fa76fc55, []int{4} ++ return fileDescriptor_819623c7fa76fc55, []int{5} + } + func (m *DispatchTaskResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +@@ -343,7 +443,7 @@ func (m *CancelTaskRequest) Reset() { *m = CancelTaskRequest{} } + func (m *CancelTaskRequest) String() string { return proto.CompactTextString(m) } + func (*CancelTaskRequest) ProtoMessage() {} + func (*CancelTaskRequest) Descriptor() ([]byte, []int) { +- return fileDescriptor_819623c7fa76fc55, []int{5} ++ return fileDescriptor_819623c7fa76fc55, []int{6} + } + func (m *CancelTaskRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +@@ -397,7 +497,7 @@ func (m *CancelTaskResponse) Reset() { *m = CancelTaskResponse{} } + func (m *CancelTaskResponse) String() string { return proto.CompactTextString(m) } + func (*CancelTaskResponse) ProtoMessage() {} + func (*CancelTaskResponse) Descriptor() ([]byte, []int) { +- return fileDescriptor_819623c7fa76fc55, []int{6} ++ return fileDescriptor_819623c7fa76fc55, []int{7} + } + func (m *CancelTaskResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +@@ -446,7 +546,7 @@ func (m *EstablishMPPConnectionRequest) Reset() { *m = EstablishMPPConne + func (m *EstablishMPPConnectionRequest) String() string { return proto.CompactTextString(m) } + func (*EstablishMPPConnectionRequest) ProtoMessage() {} + func (*EstablishMPPConnectionRequest) Descriptor() ([]byte, []int) { +- return fileDescriptor_819623c7fa76fc55, []int{7} ++ return fileDescriptor_819623c7fa76fc55, []int{8} + } + func (m *EstablishMPPConnectionRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +@@ -492,20 +592,21 @@ func (m *EstablishMPPConnectionRequest) GetReceiverMeta() *TaskMeta { + // when TiFlash sends data to TiDB, Data packets wrap tipb.SelectResponse, i.e., serialize tipb.SelectResponse into data; + // when TiFlash sends data to TiFlash, data blocks are serialized into chunks, and the execution_summaries in tipb.SelectResponse are serialized into data only for the last packet. + type MPPDataPacket struct { +- Data []byte `protobuf:"bytes,1,opt,name=data,proto3" json:"data,omitempty"` +- Error *Error `protobuf:"bytes,2,opt,name=error,proto3" json:"error,omitempty"` +- Chunks [][]byte `protobuf:"bytes,3,rep,name=chunks,proto3" json:"chunks,omitempty"` +- StreamIds []uint64 `protobuf:"varint,4,rep,packed,name=stream_ids,json=streamIds,proto3" json:"stream_ids,omitempty"` +- XXX_NoUnkeyedLiteral struct{} `json:"-"` +- XXX_unrecognized []byte `json:"-"` +- XXX_sizecache int32 `json:"-"` ++ Data []byte `protobuf:"bytes,1,opt,name=data,proto3" json:"data,omitempty"` ++ Error *Error `protobuf:"bytes,2,opt,name=error,proto3" json:"error,omitempty"` ++ Chunks [][]byte `protobuf:"bytes,3,rep,name=chunks,proto3" json:"chunks,omitempty"` ++ StreamIds []uint64 `protobuf:"varint,4,rep,packed,name=stream_ids,json=streamIds,proto3" json:"stream_ids,omitempty"` ++ Compress CompressMethod `protobuf:"varint,5,opt,name=compress,proto3,enum=mpp.CompressMethod" json:"compress,omitempty"` ++ XXX_NoUnkeyedLiteral struct{} `json:"-"` ++ XXX_unrecognized []byte `json:"-"` ++ XXX_sizecache int32 `json:"-"` + } + + func (m *MPPDataPacket) Reset() { *m = MPPDataPacket{} } + func (m *MPPDataPacket) String() string { return proto.CompactTextString(m) } + func (*MPPDataPacket) ProtoMessage() {} + func (*MPPDataPacket) Descriptor() ([]byte, []int) { +- return fileDescriptor_819623c7fa76fc55, []int{8} ++ return fileDescriptor_819623c7fa76fc55, []int{9} + } + func (m *MPPDataPacket) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +@@ -562,6 +663,13 @@ func (m *MPPDataPacket) GetStreamIds() []uint64 { + return nil + } + ++func (m *MPPDataPacket) GetCompress() CompressMethod { ++ if m != nil { ++ return m.Compress ++ } ++ return CompressMethod_NONE ++} ++ + type Error struct { + Code int32 `protobuf:"varint,1,opt,name=code,proto3" json:"code,omitempty"` + Msg string `protobuf:"bytes,2,opt,name=msg,proto3" json:"msg,omitempty"` +@@ -574,7 +682,7 @@ func (m *Error) Reset() { *m = Error{} } + func (m *Error) String() string { return proto.CompactTextString(m) } + func (*Error) ProtoMessage() {} + func (*Error) Descriptor() ([]byte, []int) { +- return fileDescriptor_819623c7fa76fc55, []int{9} ++ return fileDescriptor_819623c7fa76fc55, []int{10} + } + func (m *Error) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +@@ -618,6 +726,8 @@ func (m *Error) GetMsg() string { + } + + func init() { ++ proto.RegisterEnum("mpp.CompressMethod", CompressMethod_name, CompressMethod_value) ++ proto.RegisterType((*ExchangeSenderMeta)(nil), "mpp.ExchangeSenderMeta") + proto.RegisterType((*TaskMeta)(nil), "mpp.TaskMeta") + proto.RegisterType((*IsAliveRequest)(nil), "mpp.IsAliveRequest") + proto.RegisterType((*IsAliveResponse)(nil), "mpp.IsAliveResponse") +@@ -633,45 +743,85 @@ func init() { + func init() { proto.RegisterFile("mpp.proto", fileDescriptor_819623c7fa76fc55) } + + var fileDescriptor_819623c7fa76fc55 = []byte{ +- // 600 bytes of a gzipped FileDescriptorProto +- 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x54, 0xcf, 0x6e, 0xd3, 0x4e, +- 0x10, 0xfe, 0x39, 0xce, 0x9f, 0x66, 0xe2, 0xf4, 0xd7, 0x2e, 0x15, 0x4d, 0x2b, 0x1a, 0xa5, 0x3e, +- 0xe5, 0x82, 0x2b, 0x5a, 0x89, 0x23, 0x12, 0xb4, 0x3d, 0xe4, 0x50, 0x29, 0x5a, 0x55, 0x88, 0x9b, +- 0xb5, 0xb5, 0x87, 0x64, 0x15, 0x7b, 0xd7, 0xec, 0x6e, 0x2d, 0x21, 0xb8, 0x71, 0xe0, 0x15, 0x78, +- 0x24, 0x8e, 0x3c, 0x02, 0x2a, 0x2f, 0x82, 0x76, 0x6d, 0x87, 0x54, 0xa2, 0x02, 0x4e, 0x99, 0xef, +- 0x9b, 0xd9, 0x99, 0xf9, 0x3e, 0x4d, 0x0c, 0xfd, 0xbc, 0x28, 0xa2, 0x42, 0x49, 0x23, 0x89, 0x9f, +- 0x17, 0xc5, 0xe1, 0xde, 0x42, 0x2e, 0xa4, 0xc3, 0x27, 0x36, 0xaa, 0x52, 0x87, 0xbb, 0x89, 0xe5, +- 0x12, 0xd4, 0x5a, 0xaa, 0x9a, 0x0a, 0x72, 0x34, 0xac, 0xb8, 0xa9, 0x50, 0xf8, 0x01, 0xb6, 0xae, +- 0x99, 0x5e, 0x5d, 0xa1, 0x61, 0xe4, 0x00, 0xb6, 0xb4, 0x61, 0xca, 0xc4, 0x46, 0x8f, 0xbc, 0x89, +- 0x37, 0x6d, 0xd3, 0x9e, 0xc3, 0xd7, 0x9a, 0xec, 0x43, 0xcf, 0x30, 0xbd, 0x8a, 0x79, 0x3a, 0x6a, +- 0x4d, 0xbc, 0xa9, 0x4f, 0xbb, 0x16, 0xce, 0x52, 0x72, 0x0c, 0x41, 0xc1, 0x94, 0xe1, 0x86, 0x4b, +- 0x61, 0xb3, 0xbe, 0xcb, 0x0e, 0xd6, 0xdc, 0x2c, 0x25, 0x23, 0xe8, 0xb1, 0x34, 0x55, 0xa8, 0xf5, +- 0xa8, 0x3d, 0xf1, 0xa6, 0x7d, 0xda, 0xc0, 0x70, 0x07, 0xb6, 0x67, 0xfa, 0x65, 0xc6, 0x4b, 0xa4, +- 0xf8, 0xee, 0x16, 0xb5, 0x09, 0x4f, 0xe0, 0xff, 0x35, 0xa3, 0x0b, 0x29, 0x34, 0x92, 0x27, 0xd0, +- 0x67, 0x25, 0xe3, 0x19, 0xbb, 0xc9, 0xd0, 0xad, 0xb5, 0x45, 0x7f, 0x11, 0xe1, 0xe7, 0x16, 0x3c, +- 0xba, 0xe0, 0xba, 0x60, 0x26, 0x59, 0x5a, 0x21, 0x75, 0x23, 0x72, 0x0c, 0x6d, 0xab, 0xd3, 0x3d, +- 0x18, 0x9c, 0x0e, 0x23, 0xeb, 0x56, 0x23, 0x94, 0xba, 0x94, 0x5d, 0x1d, 0x45, 0x22, 0x53, 0x4c, +- 0xe3, 0x22, 0x63, 0xc2, 0x09, 0x0b, 0xe8, 0xa0, 0xe6, 0xe6, 0x19, 0x13, 0x76, 0x75, 0xc3, 0x73, +- 0x94, 0xb7, 0xa6, 0x16, 0xd6, 0x40, 0xf2, 0x0c, 0x7a, 0x0a, 0x17, 0x5c, 0x0a, 0x2b, 0xca, 0x9f, +- 0x0e, 0x4e, 0xf7, 0xa3, 0x4d, 0xab, 0xa9, 0xcb, 0xcd, 0xc4, 0x5b, 0x49, 0x9b, 0x3a, 0x72, 0x04, +- 0xa0, 0x93, 0x25, 0xe6, 0x2c, 0x2e, 0x51, 0x8d, 0x3a, 0xae, 0x5f, 0xbf, 0x62, 0x5e, 0xa3, 0x22, +- 0x2f, 0x60, 0x68, 0xac, 0xa4, 0xb8, 0xe9, 0xdb, 0x75, 0x7d, 0x0f, 0xee, 0xf5, 0xbd, 0xb6, 0x15, +- 0x55, 0x73, 0x4d, 0x03, 0xb3, 0x81, 0xc2, 0x1c, 0xf6, 0xee, 0x1b, 0x51, 0xfb, 0x37, 0x81, 0x0e, +- 0x2a, 0x25, 0x55, 0x6d, 0x05, 0x38, 0x2b, 0x2e, 0x2d, 0x43, 0xab, 0x04, 0x39, 0x83, 0xa1, 0x42, +- 0xa3, 0xde, 0xaf, 0x27, 0xb7, 0xdc, 0xe4, 0xed, 0xa8, 0xbe, 0x94, 0x6a, 0x02, 0x0d, 0x5c, 0x51, +- 0x33, 0xee, 0x0d, 0xec, 0x9e, 0x33, 0x91, 0x60, 0xf6, 0x8f, 0xae, 0xaf, 0xd7, 0x69, 0x3d, 0xb0, +- 0x4e, 0xf8, 0x1c, 0xc8, 0x66, 0xe7, 0xbf, 0x95, 0x11, 0x7e, 0xf2, 0xe0, 0xe8, 0x52, 0x5b, 0x4f, +- 0xb8, 0x5e, 0x5e, 0xcd, 0xe7, 0xe7, 0x52, 0x08, 0x4c, 0xec, 0x11, 0x36, 0xeb, 0x45, 0x30, 0xd0, +- 0x28, 0x52, 0x54, 0xf1, 0xc3, 0x5b, 0x42, 0x55, 0xe1, 0xfe, 0x10, 0xa7, 0xd6, 0x98, 0x04, 0x79, +- 0xd9, 0xbc, 0x68, 0xfd, 0xee, 0x45, 0xd0, 0xd4, 0x58, 0x14, 0x7e, 0x84, 0xe1, 0xd5, 0x7c, 0x7e, +- 0xc1, 0x0c, 0x9b, 0xb3, 0x64, 0x85, 0x86, 0x10, 0x68, 0xa7, 0xac, 0x9e, 0x16, 0x50, 0x17, 0xff, +- 0xd9, 0x04, 0xf2, 0x18, 0xba, 0xc9, 0xf2, 0x56, 0xac, 0xf4, 0xc8, 0x9f, 0xf8, 0xd3, 0x80, 0xd6, +- 0xc8, 0x1d, 0x91, 0x51, 0xc8, 0xf2, 0x98, 0xa7, 0xd5, 0xe9, 0xb5, 0x69, 0xbf, 0x62, 0x66, 0xa9, +- 0x0e, 0x9f, 0x42, 0xc7, 0xb5, 0xb1, 0x53, 0xed, 0x19, 0xbb, 0xa9, 0x1d, 0xea, 0x62, 0xb2, 0x03, +- 0x7e, 0xae, 0x17, 0x6e, 0x66, 0x9f, 0xda, 0xf0, 0x55, 0xf8, 0xf5, 0x6e, 0xec, 0x7d, 0xbb, 0x1b, +- 0x7b, 0xdf, 0xef, 0xc6, 0xde, 0x97, 0x1f, 0xe3, 0xff, 0x60, 0x47, 0xaa, 0x45, 0x64, 0xf8, 0xaa, +- 0x8c, 0x56, 0xa5, 0xfb, 0x42, 0xdc, 0x74, 0xdd, 0xcf, 0xd9, 0xcf, 0x00, 0x00, 0x00, 0xff, 0xff, +- 0xef, 0x8c, 0x66, 0x5e, 0x71, 0x04, 0x00, 0x00, ++ // 722 bytes of a gzipped FileDescriptorProto ++ 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x54, 0xcd, 0x6e, 0xd3, 0x4a, ++ 0x14, 0xae, 0xe3, 0xfc, 0x9e, 0x38, 0xb9, 0xe9, 0xb4, 0xba, 0x4d, 0xab, 0xdb, 0xdc, 0xd4, 0xab, ++ 0x08, 0xa9, 0xa9, 0x48, 0x11, 0x4b, 0x24, 0x68, 0xb3, 0x88, 0x44, 0x4a, 0x34, 0x8d, 0x2a, 0xd4, ++ 0x8d, 0x35, 0xb5, 0x87, 0xc4, 0x4a, 0xec, 0x19, 0x66, 0xa6, 0x11, 0xac, 0x79, 0x09, 0x16, 0xbc, ++ 0x02, 0xef, 0xc1, 0x12, 0xf1, 0x04, 0xa8, 0xbc, 0x08, 0x9a, 0xb1, 0x9d, 0x26, 0xb4, 0x88, 0xb2, ++ 0xf2, 0x9c, 0xef, 0xfc, 0x7d, 0xe7, 0x9b, 0xe3, 0x81, 0x4a, 0xc4, 0x79, 0x97, 0x0b, 0xa6, 0x18, ++ 0xb2, 0x23, 0xce, 0xf7, 0xb6, 0x27, 0x6c, 0xc2, 0x8c, 0x7d, 0xa4, 0x4f, 0x89, 0x6b, 0x6f, 0xd3, ++ 0xd7, 0x98, 0x4f, 0xa5, 0x64, 0x22, 0x85, 0x9c, 0x88, 0x2a, 0xc2, 0xaf, 0x12, 0xcb, 0xed, 0x03, ++ 0xea, 0xbf, 0xf3, 0xa7, 0x24, 0x9e, 0xd0, 0x73, 0x1a, 0x07, 0x54, 0x0c, 0xa9, 0x22, 0xe8, 0x08, ++ 0xca, 0x3e, 0x8b, 0xb8, 0xa0, 0x52, 0x36, 0xad, 0xb6, 0xd5, 0xa9, 0xf7, 0xb6, 0xba, 0xba, 0xdf, ++ 0x49, 0x0a, 0x0e, 0xa9, 0x9a, 0xb2, 0x00, 0x2f, 0x83, 0xdc, 0x4f, 0x16, 0x94, 0xc7, 0x44, 0xce, ++ 0x4c, 0xf6, 0x2e, 0x94, 0xa5, 0x22, 0x42, 0x79, 0x2a, 0xc9, 0xce, 0xe3, 0x92, 0xb1, 0xc7, 0x12, ++ 0xed, 0x40, 0x49, 0x11, 0x39, 0xf3, 0xc2, 0xa0, 0x99, 0x6b, 0x5b, 0x1d, 0x1b, 0x17, 0xb5, 0x39, ++ 0x08, 0xd0, 0x01, 0x38, 0x9c, 0x08, 0x15, 0xaa, 0x90, 0xc5, 0xda, 0x6b, 0x1b, 0x6f, 0x75, 0x89, ++ 0x0d, 0x02, 0xd4, 0x84, 0x12, 0x09, 0x02, 0xc3, 0x29, 0xdf, 0xb6, 0x3a, 0x15, 0x9c, 0x99, 0xe8, ++ 0x7f, 0xa8, 0x46, 0x9c, 0x7b, 0x0b, 0x2a, 0x64, 0xc8, 0xe2, 0x66, 0xc1, 0xe4, 0x42, 0xc4, 0xf9, ++ 0x45, 0x82, 0xb8, 0x0d, 0xa8, 0x0f, 0xe4, 0xf3, 0x79, 0xb8, 0xa0, 0x98, 0xbe, 0xbd, 0xa6, 0x52, ++ 0xb9, 0x23, 0xf8, 0x67, 0x89, 0x48, 0xce, 0x62, 0x49, 0xd1, 0x7f, 0x50, 0x21, 0x0b, 0x12, 0xce, ++ 0xc9, 0xd5, 0x9c, 0x1a, 0xde, 0x65, 0x7c, 0x0b, 0xfc, 0xda, 0x23, 0x77, 0xa7, 0xc7, 0xb7, 0x1c, ++ 0x6c, 0x9d, 0x86, 0x92, 0x13, 0xe5, 0x4f, 0xb5, 0x14, 0x69, 0x27, 0x74, 0x00, 0x79, 0xad, 0xb8, ++ 0xa9, 0x58, 0xed, 0xd5, 0x8c, 0x8e, 0x99, 0x54, 0xd8, 0xb8, 0xf4, 0xf0, 0x34, 0xf6, 0x59, 0x40, ++ 0x03, 0x8f, 0xcf, 0x49, 0x52, 0xdc, 0xc1, 0xd5, 0x14, 0x1b, 0xcd, 0x49, 0xac, 0x87, 0x57, 0x61, ++ 0x44, 0xd9, 0xb5, 0x4a, 0xa5, 0xc9, 0x4c, 0xf4, 0x18, 0x4a, 0x82, 0x4e, 0x42, 0x16, 0x6b, 0x59, ++ 0xec, 0x4e, 0xb5, 0xb7, 0xd3, 0x5d, 0xbd, 0x74, 0x6c, 0x7c, 0x83, 0xf8, 0x0d, 0xc3, 0x59, 0x1c, ++ 0xda, 0x07, 0x90, 0xfe, 0x94, 0x46, 0x44, 0x8f, 0x93, 0xca, 0x55, 0x49, 0x90, 0x0b, 0x2a, 0xd0, ++ 0x33, 0xa8, 0x29, 0x3d, 0xb3, 0x97, 0xd5, 0x2d, 0x9a, 0xba, 0xbb, 0x6b, 0x75, 0xc7, 0x3a, 0x22, ++ 0x29, 0x2e, 0xb1, 0xa3, 0x56, 0x2c, 0x34, 0x80, 0x6d, 0x9a, 0xee, 0x94, 0x27, 0xcd, 0x52, 0x79, ++ 0x46, 0x81, 0x92, 0x51, 0x60, 0xc7, 0x28, 0x70, 0x77, 0xe9, 0x30, 0xa2, 0x77, 0x30, 0x37, 0x82, ++ 0xed, 0x75, 0x4d, 0xd3, 0xbb, 0x6a, 0x43, 0x81, 0x0a, 0xc1, 0x44, 0xaa, 0x2a, 0x24, 0x35, 0x35, ++ 0x82, 0x13, 0x07, 0x3a, 0x86, 0x9a, 0xa0, 0x4a, 0xbc, 0x5f, 0x0e, 0x91, 0x33, 0x43, 0xd4, 0xbb, ++ 0xe9, 0xfa, 0x27, 0x64, 0xb1, 0x63, 0x82, 0x52, 0xe6, 0xee, 0x6b, 0xd8, 0x3c, 0x21, 0xb1, 0x4f, ++ 0xe7, 0x7f, 0x79, 0x81, 0x4b, 0x3a, 0xb9, 0xdf, 0xd0, 0x71, 0x9f, 0x02, 0x5a, 0xad, 0xfc, 0xd0, ++ 0x31, 0xdc, 0x0f, 0x16, 0xec, 0xf7, 0xa5, 0x96, 0x37, 0x94, 0xd3, 0xe1, 0x68, 0x74, 0xc2, 0xe2, ++ 0x98, 0xfa, 0xfa, 0x8f, 0xc8, 0xe8, 0x75, 0xa1, 0xba, 0x2a, 0xf2, 0xbd, 0x2c, 0x41, 0xde, 0xfe, ++ 0xdb, 0x3d, 0x2d, 0x8c, 0x4f, 0xc3, 0x45, 0x96, 0x91, 0xbb, 0x2f, 0xc3, 0xc9, 0x62, 0xcc, 0x35, ++ 0x7c, 0xb6, 0xa0, 0x36, 0x1c, 0x8d, 0x4e, 0x89, 0x22, 0x23, 0xe2, 0xcf, 0xa8, 0x42, 0x08, 0xf2, ++ 0x01, 0x49, 0xdb, 0x39, 0xd8, 0x9c, 0xff, 0xac, 0x02, 0xfa, 0x17, 0x8a, 0xfe, 0xf4, 0x3a, 0x9e, ++ 0xc9, 0xa6, 0xdd, 0xb6, 0x3b, 0x0e, 0x4e, 0x2d, 0xb3, 0x90, 0x4a, 0x50, 0x12, 0x79, 0x61, 0x90, ++ 0xac, 0x71, 0x1e, 0x57, 0x12, 0x64, 0x10, 0xc8, 0xb5, 0xe7, 0xa8, 0xf0, 0x90, 0xe7, 0xe8, 0x10, ++ 0x0a, 0xa6, 0xaf, 0xa6, 0xa9, 0xff, 0x21, 0x43, 0xb3, 0x80, 0xcd, 0x19, 0x35, 0xc0, 0x8e, 0xe4, ++ 0xc4, 0x90, 0xac, 0x60, 0x7d, 0x7c, 0x74, 0x08, 0xf5, 0xf5, 0x52, 0xa8, 0x0c, 0xf9, 0xb3, 0x57, ++ 0x67, 0xfd, 0xc6, 0x06, 0x2a, 0x81, 0xfd, 0xf2, 0xf2, 0x49, 0xc3, 0xd2, 0xd0, 0xe5, 0xf9, 0xf8, ++ 0xb4, 0x91, 0x7b, 0xe1, 0x7e, 0xb9, 0x69, 0x59, 0x5f, 0x6f, 0x5a, 0xd6, 0xf7, 0x9b, 0x96, 0xf5, ++ 0xf1, 0x47, 0x6b, 0x03, 0x1a, 0x4c, 0x4c, 0xba, 0x2a, 0x9c, 0x2d, 0xba, 0xb3, 0x85, 0x79, 0x57, ++ 0xaf, 0x8a, 0xe6, 0x73, 0xfc, 0x33, 0x00, 0x00, 0xff, 0xff, 0x2d, 0xa1, 0xda, 0xda, 0xa7, 0x05, ++ 0x00, 0x00, ++} ++ ++func (m *ExchangeSenderMeta) Marshal() (dAtA []byte, err error) { ++ size := m.Size() ++ dAtA = make([]byte, size) ++ n, err := m.MarshalToSizedBuffer(dAtA[:size]) ++ if err != nil { ++ return nil, err ++ } ++ return dAtA[:n], nil ++} ++ ++func (m *ExchangeSenderMeta) MarshalTo(dAtA []byte) (int, error) { ++ size := m.Size() ++ return m.MarshalToSizedBuffer(dAtA[:size]) ++} ++ ++func (m *ExchangeSenderMeta) MarshalToSizedBuffer(dAtA []byte) (int, error) { ++ i := len(dAtA) ++ _ = i ++ var l int ++ _ = l ++ if m.XXX_unrecognized != nil { ++ i -= len(m.XXX_unrecognized) ++ copy(dAtA[i:], m.XXX_unrecognized) ++ } ++ if m.Compress != 0 { ++ i = encodeVarintMpp(dAtA, i, uint64(m.Compress)) ++ i-- ++ dAtA[i] = 0x8 ++ } ++ return len(dAtA) - i, nil + } + + func (m *TaskMeta) Marshal() (dAtA []byte, err error) { +@@ -698,6 +848,11 @@ func (m *TaskMeta) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i -= len(m.XXX_unrecognized) + copy(dAtA[i:], m.XXX_unrecognized) + } ++ if m.MppVersion != 0 { ++ i = encodeVarintMpp(dAtA, i, uint64(m.MppVersion)) ++ i-- ++ dAtA[i] = 0x28 ++ } + if len(m.Address) > 0 { + i -= len(m.Address) + copy(dAtA[i:], m.Address) +@@ -774,6 +929,11 @@ func (m *IsAliveResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i -= len(m.XXX_unrecognized) + copy(dAtA[i:], m.XXX_unrecognized) + } ++ if m.MppVersion != 0 { ++ i = encodeVarintMpp(dAtA, i, uint64(m.MppVersion)) ++ i-- ++ dAtA[i] = 0x10 ++ } + if m.Available { + i-- + if m.Available { +@@ -811,6 +971,18 @@ func (m *DispatchTaskRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i -= len(m.XXX_unrecognized) + copy(dAtA[i:], m.XXX_unrecognized) + } ++ if m.ExchangeSenderMeta != nil { ++ { ++ size, err := m.ExchangeSenderMeta.MarshalToSizedBuffer(dAtA[:i]) ++ if err != nil { ++ return 0, err ++ } ++ i -= size ++ i = encodeVarintMpp(dAtA, i, uint64(size)) ++ } ++ i-- ++ dAtA[i] = 0x3a ++ } + if len(m.TableRegions) > 0 { + for iNdEx := len(m.TableRegions) - 1; iNdEx >= 0; iNdEx-- { + { +@@ -1089,21 +1261,26 @@ func (m *MPPDataPacket) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i -= len(m.XXX_unrecognized) + copy(dAtA[i:], m.XXX_unrecognized) + } ++ if m.Compress != 0 { ++ i = encodeVarintMpp(dAtA, i, uint64(m.Compress)) ++ i-- ++ dAtA[i] = 0x28 ++ } + if len(m.StreamIds) > 0 { +- dAtA9 := make([]byte, len(m.StreamIds)*10) +- var j8 int ++ dAtA10 := make([]byte, len(m.StreamIds)*10) ++ var j9 int + for _, num := range m.StreamIds { + for num >= 1<<7 { +- dAtA9[j8] = uint8(uint64(num)&0x7f | 0x80) ++ dAtA10[j9] = uint8(uint64(num)&0x7f | 0x80) + num >>= 7 +- j8++ ++ j9++ + } +- dAtA9[j8] = uint8(num) +- j8++ ++ dAtA10[j9] = uint8(num) ++ j9++ + } +- i -= j8 +- copy(dAtA[i:], dAtA9[:j8]) +- i = encodeVarintMpp(dAtA, i, uint64(j8)) ++ i -= j9 ++ copy(dAtA[i:], dAtA10[:j9]) ++ i = encodeVarintMpp(dAtA, i, uint64(j9)) + i-- + dAtA[i] = 0x22 + } +@@ -1188,6 +1365,21 @@ func encodeVarintMpp(dAtA []byte, offset int, v uint64) int { + dAtA[offset] = uint8(v) + return base + } ++func (m *ExchangeSenderMeta) Size() (n int) { ++ if m == nil { ++ return 0 ++ } ++ var l int ++ _ = l ++ if m.Compress != 0 { ++ n += 1 + sovMpp(uint64(m.Compress)) ++ } ++ if m.XXX_unrecognized != nil { ++ n += len(m.XXX_unrecognized) ++ } ++ return n ++} ++ + func (m *TaskMeta) Size() (n int) { + if m == nil { + return 0 +@@ -1207,6 +1399,9 @@ func (m *TaskMeta) Size() (n int) { + if l > 0 { + n += 1 + l + sovMpp(uint64(l)) + } ++ if m.MppVersion != 0 { ++ n += 1 + sovMpp(uint64(m.MppVersion)) ++ } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } +@@ -1234,6 +1429,9 @@ func (m *IsAliveResponse) Size() (n int) { + if m.Available { + n += 2 + } ++ if m.MppVersion != 0 { ++ n += 1 + sovMpp(uint64(m.MppVersion)) ++ } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } +@@ -1272,6 +1470,10 @@ func (m *DispatchTaskRequest) Size() (n int) { + n += 1 + l + sovMpp(uint64(l)) + } + } ++ if m.ExchangeSenderMeta != nil { ++ l = m.ExchangeSenderMeta.Size() ++ n += 1 + l + sovMpp(uint64(l)) ++ } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } +@@ -1383,6 +1585,9 @@ func (m *MPPDataPacket) Size() (n int) { + } + n += 1 + sovMpp(uint64(l)) + l + } ++ if m.Compress != 0 { ++ n += 1 + sovMpp(uint64(m.Compress)) ++ } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } +@@ -1414,6 +1619,76 @@ func sovMpp(x uint64) (n int) { + func sozMpp(x uint64) (n int) { + return sovMpp(uint64((x << 1) ^ uint64((int64(x) >> 63)))) + } ++func (m *ExchangeSenderMeta) Unmarshal(dAtA []byte) error { ++ l := len(dAtA) ++ iNdEx := 0 ++ for iNdEx < l { ++ preIndex := iNdEx ++ var wire uint64 ++ for shift := uint(0); ; shift += 7 { ++ if shift >= 64 { ++ return ErrIntOverflowMpp ++ } ++ if iNdEx >= l { ++ return io.ErrUnexpectedEOF ++ } ++ b := dAtA[iNdEx] ++ iNdEx++ ++ wire |= uint64(b&0x7F) << shift ++ if b < 0x80 { ++ break ++ } ++ } ++ fieldNum := int32(wire >> 3) ++ wireType := int(wire & 0x7) ++ if wireType == 4 { ++ return fmt.Errorf("proto: ExchangeSenderMeta: wiretype end group for non-group") ++ } ++ if fieldNum <= 0 { ++ return fmt.Errorf("proto: ExchangeSenderMeta: illegal tag %d (wire type %d)", fieldNum, wire) ++ } ++ switch fieldNum { ++ case 1: ++ if wireType != 0 { ++ return fmt.Errorf("proto: wrong wireType = %d for field Compress", wireType) ++ } ++ m.Compress = 0 ++ for shift := uint(0); ; shift += 7 { ++ if shift >= 64 { ++ return ErrIntOverflowMpp ++ } ++ if iNdEx >= l { ++ return io.ErrUnexpectedEOF ++ } ++ b := dAtA[iNdEx] ++ iNdEx++ ++ m.Compress |= CompressMethod(b&0x7F) << shift ++ if b < 0x80 { ++ break ++ } ++ } ++ default: ++ iNdEx = preIndex ++ skippy, err := skipMpp(dAtA[iNdEx:]) ++ if err != nil { ++ return err ++ } ++ if (skippy < 0) || (iNdEx+skippy) < 0 { ++ return ErrInvalidLengthMpp ++ } ++ if (iNdEx + skippy) > l { ++ return io.ErrUnexpectedEOF ++ } ++ m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...) ++ iNdEx += skippy ++ } ++ } ++ ++ if iNdEx > l { ++ return io.ErrUnexpectedEOF ++ } ++ return nil ++} + func (m *TaskMeta) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 +@@ -1532,6 +1807,25 @@ func (m *TaskMeta) Unmarshal(dAtA []byte) error { + } + m.Address = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex ++ case 5: ++ if wireType != 0 { ++ return fmt.Errorf("proto: wrong wireType = %d for field MppVersion", wireType) ++ } ++ m.MppVersion = 0 ++ for shift := uint(0); ; shift += 7 { ++ if shift >= 64 { ++ return ErrIntOverflowMpp ++ } ++ if iNdEx >= l { ++ return io.ErrUnexpectedEOF ++ } ++ b := dAtA[iNdEx] ++ iNdEx++ ++ m.MppVersion |= int64(b&0x7F) << shift ++ if b < 0x80 { ++ break ++ } ++ } + default: + iNdEx = preIndex + skippy, err := skipMpp(dAtA[iNdEx:]) +@@ -1654,6 +1948,25 @@ func (m *IsAliveResponse) Unmarshal(dAtA []byte) error { + } + } + m.Available = bool(v != 0) ++ case 2: ++ if wireType != 0 { ++ return fmt.Errorf("proto: wrong wireType = %d for field MppVersion", wireType) ++ } ++ m.MppVersion = 0 ++ for shift := uint(0); ; shift += 7 { ++ if shift >= 64 { ++ return ErrIntOverflowMpp ++ } ++ if iNdEx >= l { ++ return io.ErrUnexpectedEOF ++ } ++ b := dAtA[iNdEx] ++ iNdEx++ ++ m.MppVersion |= int64(b&0x7F) << shift ++ if b < 0x80 { ++ break ++ } ++ } + default: + iNdEx = preIndex + skippy, err := skipMpp(dAtA[iNdEx:]) +@@ -1881,6 +2194,42 @@ func (m *DispatchTaskRequest) Unmarshal(dAtA []byte) error { + return err + } + iNdEx = postIndex ++ case 7: ++ if wireType != 2 { ++ return fmt.Errorf("proto: wrong wireType = %d for field ExchangeSenderMeta", wireType) ++ } ++ var msglen int ++ for shift := uint(0); ; shift += 7 { ++ if shift >= 64 { ++ return ErrIntOverflowMpp ++ } ++ if iNdEx >= l { ++ return io.ErrUnexpectedEOF ++ } ++ b := dAtA[iNdEx] ++ iNdEx++ ++ msglen |= int(b&0x7F) << shift ++ if b < 0x80 { ++ break ++ } ++ } ++ if msglen < 0 { ++ return ErrInvalidLengthMpp ++ } ++ postIndex := iNdEx + msglen ++ if postIndex < 0 { ++ return ErrInvalidLengthMpp ++ } ++ if postIndex > l { ++ return io.ErrUnexpectedEOF ++ } ++ if m.ExchangeSenderMeta == nil { ++ m.ExchangeSenderMeta = &ExchangeSenderMeta{} ++ } ++ if err := m.ExchangeSenderMeta.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { ++ return err ++ } ++ iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipMpp(dAtA[iNdEx:]) +@@ -2564,6 +2913,25 @@ func (m *MPPDataPacket) Unmarshal(dAtA []byte) error { + } else { + return fmt.Errorf("proto: wrong wireType = %d for field StreamIds", wireType) + } ++ case 5: ++ if wireType != 0 { ++ return fmt.Errorf("proto: wrong wireType = %d for field Compress", wireType) ++ } ++ m.Compress = 0 ++ for shift := uint(0); ; shift += 7 { ++ if shift >= 64 { ++ return ErrIntOverflowMpp ++ } ++ if iNdEx >= l { ++ return io.ErrUnexpectedEOF ++ } ++ b := dAtA[iNdEx] ++ iNdEx++ ++ m.Compress |= CompressMethod(b&0x7F) << shift ++ if b < 0x80 { ++ break ++ } ++ } + default: + iNdEx = preIndex + skippy, err := skipMpp(dAtA[iNdEx:]) diff --git a/proto/mpp.proto b/proto/mpp.proto -index f8af03b..1582074 100644 +index f8af03b..801bee3 100644 --- a/proto/mpp.proto +++ b/proto/mpp.proto @@ -11,12 +11,24 @@ option (gogoproto.unmarshaler_all) = true; @@ -39,7 +790,7 @@ index f8af03b..1582074 100644 int64 schema_ver = 5; // Used for partition table scan repeated coprocessor.TableRegions table_regions = 6; -+ optional ExchangeSenderMeta exchange_sender_meta = 7; ++ ExchangeSenderMeta exchange_sender_meta = 7; } // Get response of DispatchTaskRequest. diff --git a/dbms/src/Flash/Mpp/Utils.cpp b/dbms/src/Flash/Mpp/Utils.cpp index 3ce15aa274c..b6783769491 100644 --- a/dbms/src/Flash/Mpp/Utils.cpp +++ b/dbms/src/Flash/Mpp/Utils.cpp @@ -14,6 +14,7 @@ #include #include +#include #include From b80398427c8b0c0bfbd7b1ca7a16ba4e30049e45 Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Thu, 15 Dec 2022 16:18:21 +0800 Subject: [PATCH 15/93] 9 --- dbms/src/Flash/Mpp/MPPTask.cpp | 43 ++++++++++++----------- dbms/src/Flash/Mpp/Utils.cpp | 2 +- dbms/src/Flash/Mpp/newMPPExchangeWriter.h | 3 ++ 3 files changed, 26 insertions(+), 22 deletions(-) diff --git a/dbms/src/Flash/Mpp/MPPTask.cpp b/dbms/src/Flash/Mpp/MPPTask.cpp index 46e6203af38..8171cbfe7b9 100644 --- a/dbms/src/Flash/Mpp/MPPTask.cpp +++ b/dbms/src/Flash/Mpp/MPPTask.cpp @@ -281,28 +281,29 @@ void MPPTask::prepare(const mpp::DispatchTaskRequest & task_request) is_root_mpp_task = task_meta.task_id() == -1; } - if (true) // nolint - { - auto exchange_sender_meta = task_request.exchange_sender_meta(); - mpp::CompressMethod compress_method{}; - switch (tzg::SnappyStatistic::globalInstance().getMethod()) - { - case tzg::SnappyStatistic::CompressMethod::LZ4: - compress_method = mpp::CompressMethod::LZ4; - break; - case tzg::SnappyStatistic::CompressMethod::ZSTD: - compress_method = mpp::CompressMethod::ZSTD; - break; - default: - compress_method = mpp::CompressMethod::NONE; - break; - } - exchange_sender_meta.set_compress(compress_method); - LOG_DEBUG(log, "DAGContext use compress method {}", mpp::CompressMethod_Name(compress_method)); - dag_context = std::make_unique(dag_req, task_request.meta(), exchange_sender_meta, is_root_mpp_task); - } - else + // if (true) // nolint + // { + // auto exchange_sender_meta = task_request.exchange_sender_meta(); + // mpp::CompressMethod compress_method{}; + // switch (tzg::SnappyStatistic::globalInstance().getMethod()) + // { + // case tzg::SnappyStatistic::CompressMethod::LZ4: + // compress_method = mpp::CompressMethod::LZ4; + // break; + // case tzg::SnappyStatistic::CompressMethod::ZSTD: + // compress_method = mpp::CompressMethod::ZSTD; + // break; + // default: + // compress_method = mpp::CompressMethod::NONE; + // break; + // } + // exchange_sender_meta.set_compress(compress_method); + // LOG_DEBUG(log, "DAGContext use compress method {}", mpp::CompressMethod_Name(compress_method)); + // dag_context = std::make_unique(dag_req, task_request.meta(), exchange_sender_meta, is_root_mpp_task); + // } + // else { + LOG_DEBUG(log, "DAGContext use compress method {}", mpp::CompressMethod_Name(task_request.exchange_sender_meta().compress())); dag_context = std::make_unique(dag_req, task_request.meta(), task_request.exchange_sender_meta(), is_root_mpp_task); } diff --git a/dbms/src/Flash/Mpp/Utils.cpp b/dbms/src/Flash/Mpp/Utils.cpp index b6783769491..b751fc06051 100644 --- a/dbms/src/Flash/Mpp/Utils.cpp +++ b/dbms/src/Flash/Mpp/Utils.cpp @@ -44,7 +44,7 @@ void trimStackTrace(String & message) namespace TiDB { -constexpr int64_t MPP_VERSION = 1; +constexpr int64_t MPP_VERSION = 1 << 10; static const char * MPP_TIFLASH_RELEASE_VERSION = "v6.5.0"; constexpr int64_t MIN_MPP_VERSION = 0; diff --git a/dbms/src/Flash/Mpp/newMPPExchangeWriter.h b/dbms/src/Flash/Mpp/newMPPExchangeWriter.h index 3315d3785c6..6f51998c0d2 100644 --- a/dbms/src/Flash/Mpp/newMPPExchangeWriter.h +++ b/dbms/src/Flash/Mpp/newMPPExchangeWriter.h @@ -41,6 +41,7 @@ std::unique_ptr NewMPPExchangeWriter( should_send_exec_summary_at_last = dag_context.collect_execution_summaries && should_send_exec_summary_at_last; if (dag_context.isRootMPPTask()) { + RUNTIME_CHECK(dag_context.getExchangeSenderMeta().compress() == mpp::CompressMethod::NONE); RUNTIME_CHECK(!enable_fine_grained_shuffle); RUNTIME_CHECK(exchange_type == tipb::ExchangeType::PassThrough); return std::make_unique>( @@ -52,6 +53,8 @@ std::unique_ptr NewMPPExchangeWriter( } else { + RUNTIME_CHECK(dag_context.getExchangeSenderMeta().compress() != mpp::CompressMethod::NONE); + if (exchange_type == tipb::ExchangeType::Hash) { if (enable_fine_grained_shuffle) From bca36f19ca0bd6d0478be1c9681785c71a1317a5 Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Thu, 15 Dec 2022 18:00:52 +0800 Subject: [PATCH 16/93] 10 Signed-off-by: Zhigao Tong --- dbms/src/Flash/FlashService.cpp | 1 + dbms/src/Flash/Mpp/Utils.cpp | 14 ++++++++------ 2 files changed, 9 insertions(+), 6 deletions(-) diff --git a/dbms/src/Flash/FlashService.cpp b/dbms/src/Flash/FlashService.cpp index 259051d57a5..75bffb27615 100644 --- a/dbms/src/Flash/FlashService.cpp +++ b/dbms/src/Flash/FlashService.cpp @@ -245,6 +245,7 @@ grpc::Status FlashService::IsAlive(grpc::ServerContext * grpc_context [[maybe_un auto & tmt_context = context->getTMTContext(); response->set_available(tmt_context.checkRunning()); + response->set_mpp_version(TiDB::GetMppVersion()); return grpc::Status::OK; } diff --git a/dbms/src/Flash/Mpp/Utils.cpp b/dbms/src/Flash/Mpp/Utils.cpp index b751fc06051..2c8eaad4b82 100644 --- a/dbms/src/Flash/Mpp/Utils.cpp +++ b/dbms/src/Flash/Mpp/Utils.cpp @@ -44,21 +44,23 @@ void trimStackTrace(String & message) namespace TiDB { -constexpr int64_t MPP_VERSION = 1 << 10; -static const char * MPP_TIFLASH_RELEASE_VERSION = "v6.5.0"; -constexpr int64_t MIN_MPP_VERSION = 0; +constexpr int64_t MPP_VERSION_V0 = 0; +constexpr int64_t MPP_VERSION_V1 = 1 << 10; +// constexpr int64_t MPP_VERSION_V2 = MPP_VERSION_V1 * 2; +constexpr int64_t MPP_VERSION = MPP_VERSION_V1; +static const char * MPP_TIFLASH_RELEASE_VERSION = "v6.6.0"; bool CheckMppVersion(int64_t mpp_version) { - return mpp_version >= MIN_MPP_VERSION && mpp_version <= MPP_VERSION; + return mpp_version >= MPP_VERSION_V0 && mpp_version <= MPP_VERSION_V1; } std::string GenMppVersionErrorMessage(int64_t mpp_version) { auto err_msg = fmt::format("Invalid mpp version `{}`, expect version: min `{}`, max `{}` release version `{}`", mpp_version, - TiDB::MIN_MPP_VERSION, - TiDB::MPP_VERSION, + TiDB::MPP_VERSION_V0, + TiDB::MPP_VERSION_V1, MPP_TIFLASH_RELEASE_VERSION); return err_msg; } From 42ae446376e231a3f3e553d00ce660cccd62af28 Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Fri, 16 Dec 2022 10:11:14 +0800 Subject: [PATCH 17/93] 11 Signed-off-by: Zhigao Tong --- dbms/src/Flash/Mpp/Utils.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Flash/Mpp/Utils.cpp b/dbms/src/Flash/Mpp/Utils.cpp index 2c8eaad4b82..e971c4f2703 100644 --- a/dbms/src/Flash/Mpp/Utils.cpp +++ b/dbms/src/Flash/Mpp/Utils.cpp @@ -45,8 +45,8 @@ namespace TiDB { constexpr int64_t MPP_VERSION_V0 = 0; -constexpr int64_t MPP_VERSION_V1 = 1 << 10; -// constexpr int64_t MPP_VERSION_V2 = MPP_VERSION_V1 * 2; +constexpr int64_t MPP_VERSION_V1 = 1000; +MAYBE_UNUSED_MEMBER constexpr int64_t MPP_VERSION_V2 = MPP_VERSION_V1 * 2; constexpr int64_t MPP_VERSION = MPP_VERSION_V1; static const char * MPP_TIFLASH_RELEASE_VERSION = "v6.6.0"; From 2b56ec21b7c35d1127fdb5cd83011a649f0f6f9c Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Fri, 16 Dec 2022 10:38:22 +0800 Subject: [PATCH 18/93] 12 --- dbms/src/Flash/Mpp/newMPPExchangeWriter.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Flash/Mpp/newMPPExchangeWriter.h b/dbms/src/Flash/Mpp/newMPPExchangeWriter.h index 6f51998c0d2..9b12d1ab7be 100644 --- a/dbms/src/Flash/Mpp/newMPPExchangeWriter.h +++ b/dbms/src/Flash/Mpp/newMPPExchangeWriter.h @@ -53,10 +53,10 @@ std::unique_ptr NewMPPExchangeWriter( } else { - RUNTIME_CHECK(dag_context.getExchangeSenderMeta().compress() != mpp::CompressMethod::NONE); - if (exchange_type == tipb::ExchangeType::Hash) { + RUNTIME_CHECK(dag_context.getExchangeSenderMeta().compress() != mpp::CompressMethod::NONE); + if (enable_fine_grained_shuffle) { return std::make_unique>( From f24976421f1c2ede677c7f2ca17a2da3b3de9f9a Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Fri, 16 Dec 2022 14:24:43 +0800 Subject: [PATCH 19/93] 13 --- dbms/src/Flash/Mpp/Utils.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/dbms/src/Flash/Mpp/Utils.cpp b/dbms/src/Flash/Mpp/Utils.cpp index e971c4f2703..4032f269351 100644 --- a/dbms/src/Flash/Mpp/Utils.cpp +++ b/dbms/src/Flash/Mpp/Utils.cpp @@ -14,6 +14,7 @@ #include #include +#include #include #include From 8c5cd379663bdfb994ec11cbc63efa2cd9347abc Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Tue, 20 Dec 2022 10:07:39 +0800 Subject: [PATCH 20/93] 14 --- dbms/src/Flash/Mpp/Utils.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Flash/Mpp/Utils.cpp b/dbms/src/Flash/Mpp/Utils.cpp index 4032f269351..6ea535f345d 100644 --- a/dbms/src/Flash/Mpp/Utils.cpp +++ b/dbms/src/Flash/Mpp/Utils.cpp @@ -46,10 +46,10 @@ namespace TiDB { constexpr int64_t MPP_VERSION_V0 = 0; -constexpr int64_t MPP_VERSION_V1 = 1000; +constexpr int64_t MPP_VERSION_V1 = 1; MAYBE_UNUSED_MEMBER constexpr int64_t MPP_VERSION_V2 = MPP_VERSION_V1 * 2; constexpr int64_t MPP_VERSION = MPP_VERSION_V1; -static const char * MPP_TIFLASH_RELEASE_VERSION = "v6.6.0"; +static const char * MPP_TIFLASH_RELEASE_VERSION = "v6.7.0"; bool CheckMppVersion(int64_t mpp_version) { From 70107904f7d563b97aee2be9a7695e60e9eb3d68 Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Wed, 21 Dec 2022 14:32:50 +0800 Subject: [PATCH 21/93] 15 Signed-off-by: Zhigao Tong --- dbms/src/Common/TiFlashMetrics.h | 6 ++ .../Flash/Coprocessor/CHBlockChunkCodec.cpp | 7 +- dbms/src/Flash/Mpp/HashPartitionWriter.cpp | 50 +++++++-- dbms/src/Flash/Mpp/Utils.cpp | 6 +- dbms/src/Flash/Mpp/newMPPExchangeWriter.h | 2 +- dbms/src/IO/CompressedWriteBuffer.cpp | 25 ----- metrics/grafana/tiflash_summary.json | 101 +++++++++++++++++- 7 files changed, 158 insertions(+), 39 deletions(-) diff --git a/dbms/src/Common/TiFlashMetrics.h b/dbms/src/Common/TiFlashMetrics.h index 2e372ca0806..fae81d7a992 100644 --- a/dbms/src/Common/TiFlashMetrics.h +++ b/dbms/src/Common/TiFlashMetrics.h @@ -82,6 +82,12 @@ namespace DB F(type_cancel_mpp_task, {{"type", "cancel_mpp_task"}}, ExpBuckets{0.001, 2, 20}), \ F(type_run_mpp_task, {{"type", "run_mpp_task"}}, ExpBuckets{0.001, 2, 20})) \ M(tiflash_coprocessor_response_bytes, "Total bytes of response body", Counter) \ + M(tiflash_exchange_data_bytes, "Total bytes of exchange operator", Counter, \ + F(type_hash_original_all, {"type", "hash_original_all"}), \ + F(type_hash_none, {"type", "hash_none"}), \ + F(type_hash_none_local, {"type", "hash_none_local"}), \ + F(type_hash_lz4, {"type", "hash_lz4"}), \ + F(type_hash_zstd, {"type", "hash_zstd"})) \ M(tiflash_schema_version, "Current version of tiflash cached schema", Gauge) \ M(tiflash_schema_applying, "Whether the schema is applying or not (holding lock)", Gauge) \ M(tiflash_schema_apply_count, "Total number of each kinds of apply", Counter, F(type_diff, {"type", "diff"}), \ diff --git a/dbms/src/Flash/Coprocessor/CHBlockChunkCodec.cpp b/dbms/src/Flash/Coprocessor/CHBlockChunkCodec.cpp index 8768c67207b..86a344e1a84 100644 --- a/dbms/src/Flash/Coprocessor/CHBlockChunkCodec.cpp +++ b/dbms/src/Flash/Coprocessor/CHBlockChunkCodec.cpp @@ -129,6 +129,11 @@ size_t getExtraInfoSize(const Block & block) return size; } +size_t ApproxBlockBytes(const Block & block) +{ + return block.bytes() + getExtraInfoSize(block); +} + void writeData(const IDataType & type, const ColumnPtr & column, WriteBuffer & ostr, size_t offset, size_t limit) { /** If there are columns-constants - then we materialize them. @@ -165,7 +170,7 @@ void CHBlockChunkCodecStream::encode(const Block & block, size_t start, size_t e if (start != 0 || end != block.rows()) throw TiFlashException("CHBlock encode only support encode whole block", Errors::Coprocessor::Internal); - size_t init_size = block.bytes() + getExtraInfoSize(block); + size_t init_size = ApproxBlockBytes(block); WriteBuffer * ostr_ptr = initOutput(init_size); block.checkNumberOfRows(); diff --git a/dbms/src/Flash/Mpp/HashPartitionWriter.cpp b/dbms/src/Flash/Mpp/HashPartitionWriter.cpp index cc72cadd4ca..fd14675c466 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriter.cpp +++ b/dbms/src/Flash/Mpp/HashPartitionWriter.cpp @@ -13,6 +13,7 @@ // limitations under the License. #include +#include #include #include #include @@ -28,7 +29,6 @@ #include "common/logger_useful.h" #include "ext/scope_guard.h" #include "mpp.pb.h" - namespace DB { template @@ -112,6 +112,8 @@ void HashPartitionWriter::write(const Block & block) partitionAndEncodeThenWriteBlocks(); } +extern size_t ApproxBlockBytes(const Block & block); + template void HashPartitionWriter::partitionAndEncodeThenWriteBlocks() { @@ -129,6 +131,8 @@ void HashPartitionWriter::partitionAndEncodeThenWriteBlocks() tracked_packets[part_id]->getPacket().set_compress(method); } + size_t ori_block_mem_size = 0; + if (!blocks.empty()) { @@ -141,6 +145,8 @@ void HashPartitionWriter::partitionAndEncodeThenWriteBlocks() while (!blocks.empty()) { const auto & block = blocks.back(); + ori_block_mem_size += ApproxBlockBytes(block); + auto dest_tbl_cols = HashBaseWriterHelper::createDestColumns(block, partition_num); HashBaseWriterHelper::scatterColumns(block, partition_num, collators, partition_key_containers, partition_col_ids, dest_tbl_cols); blocks.pop_back(); @@ -159,6 +165,7 @@ void HashPartitionWriter::partitionAndEncodeThenWriteBlocks() codec_stream = compress_chunk_codec_stream.get(); } codec_stream->encode(dest_block, 0, dest_block_rows); + // ori_block_mem_size += ApproxBlockBytes(dest_block); tracked_packets[part_id]->addChunk(codec_stream->getString()); codec_stream->clear(); } @@ -169,24 +176,51 @@ void HashPartitionWriter::partitionAndEncodeThenWriteBlocks() } writePackets(tracked_packets); + + GET_METRIC(tiflash_exchange_data_bytes, type_hash_original_all).Increment(ori_block_mem_size); } template void HashPartitionWriter::writePackets(const TrackedMppDataPacketPtrs & packets) { - // auto * logger = &Poco::Logger::get("tzg"); - for (size_t part_id = 0; part_id < packets.size(); ++part_id) { const auto & packet = packets[part_id]; assert(packet); - if (likely(packet->getPacket().chunks_size() > 0)) + + auto & inner_packet = packet->getPacket(); + if (likely(inner_packet.chunks_size() > 0)) { - // Stopwatch watch{}; writer->partitionWrite(packet, part_id); - // auto cost = watch.elapsedSeconds(); - // if (cost > 1.001) - // LOG_DEBUG(logger, "finish to write partition {}, chunck cnt {}, time cost {:.3f}s", part_id, packet->getPacket().chunks_size(), cost); + + auto sz = inner_packet.ByteSizeLong(); + switch (inner_packet.compress()) + { + case mpp::NONE: + { + if (writer->getTunnels()[part_id]->isLocal()) + { + GET_METRIC(tiflash_exchange_data_bytes, type_hash_none_local).Increment(sz); + } + else + { + GET_METRIC(tiflash_exchange_data_bytes, type_hash_none).Increment(sz); + } + break; + } + case mpp::LZ4: + { + GET_METRIC(tiflash_exchange_data_bytes, type_hash_lz4).Increment(sz); + break; + } + case mpp::ZSTD: + { + GET_METRIC(tiflash_exchange_data_bytes, type_hash_zstd).Increment(sz); + break; + } + default: + break; + } } } } diff --git a/dbms/src/Flash/Mpp/Utils.cpp b/dbms/src/Flash/Mpp/Utils.cpp index 6ea535f345d..c128f3936cc 100644 --- a/dbms/src/Flash/Mpp/Utils.cpp +++ b/dbms/src/Flash/Mpp/Utils.cpp @@ -47,9 +47,9 @@ namespace TiDB constexpr int64_t MPP_VERSION_V0 = 0; constexpr int64_t MPP_VERSION_V1 = 1; -MAYBE_UNUSED_MEMBER constexpr int64_t MPP_VERSION_V2 = MPP_VERSION_V1 * 2; +static const char * MPP_V1_TIFLASH_RELEASE_VERSION = "?"; // TODO: set version after committed +// constexpr int64_t MPP_VERSION_V2 = MPP_VERSION_V1 * 2; constexpr int64_t MPP_VERSION = MPP_VERSION_V1; -static const char * MPP_TIFLASH_RELEASE_VERSION = "v6.7.0"; bool CheckMppVersion(int64_t mpp_version) { @@ -62,7 +62,7 @@ std::string GenMppVersionErrorMessage(int64_t mpp_version) mpp_version, TiDB::MPP_VERSION_V0, TiDB::MPP_VERSION_V1, - MPP_TIFLASH_RELEASE_VERSION); + MPP_V1_TIFLASH_RELEASE_VERSION); return err_msg; } diff --git a/dbms/src/Flash/Mpp/newMPPExchangeWriter.h b/dbms/src/Flash/Mpp/newMPPExchangeWriter.h index 9b12d1ab7be..18818aeced3 100644 --- a/dbms/src/Flash/Mpp/newMPPExchangeWriter.h +++ b/dbms/src/Flash/Mpp/newMPPExchangeWriter.h @@ -55,7 +55,7 @@ std::unique_ptr NewMPPExchangeWriter( { if (exchange_type == tipb::ExchangeType::Hash) { - RUNTIME_CHECK(dag_context.getExchangeSenderMeta().compress() != mpp::CompressMethod::NONE); + // RUNTIME_CHECK(dag_context.getExchangeSenderMeta().compress() != mpp::CompressMethod::NONE); if (enable_fine_grained_shuffle) { diff --git a/dbms/src/IO/CompressedWriteBuffer.cpp b/dbms/src/IO/CompressedWriteBuffer.cpp index 46f000d63e7..92bf73c1dc8 100644 --- a/dbms/src/IO/CompressedWriteBuffer.cpp +++ b/dbms/src/IO/CompressedWriteBuffer.cpp @@ -122,31 +122,6 @@ void CompressedWriteBuffer::nextImpl() compressed_buffer_ptr = &compressed_buffer[0]; break; } - // case CompressionMethod::SNAPPY: - // { - // static constexpr size_t header_size = 1 + sizeof(UInt32) + sizeof(UInt32); - // compressed_size = snappy::MaxCompressedLength(uncompressed_size); - // compressed_buffer.resize(header_size + compressed_size); - - // compressed_buffer[0] = static_cast(CompressionMethodByte::SNAPPY); - - // snappy::RawCompress( - // working_buffer.begin(), - // uncompressed_size, - // &compressed_buffer[header_size], - // &compressed_size); - - // compressed_size = header_size + compressed_size; - - // UInt32 compressed_size_32 = compressed_size; - // UInt32 uncompressed_size_32 = uncompressed_size; - - // unalignedStore(&compressed_buffer[1], compressed_size_32); - // unalignedStore(&compressed_buffer[5], uncompressed_size_32); - - // compressed_buffer_ptr = &compressed_buffer[0]; - // break; - // } default: throw Exception("Unknown compression method", ErrorCodes::UNKNOWN_COMPRESSION_METHOD); } diff --git a/metrics/grafana/tiflash_summary.json b/metrics/grafana/tiflash_summary.json index 7cbbdf47935..86d7c052bff 100644 --- a/metrics/grafana/tiflash_summary.json +++ b/metrics/grafana/tiflash_summary.json @@ -3366,6 +3366,105 @@ "alignLevel": null } }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 0, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 54 + }, + "hiddenSeries": false, + "id": 165, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.11", + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(rate(tiflash_exchange_data_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{type}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Exchange Bytes/Seconds", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "bytes", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, { "aliasColors": {}, "bars": false, @@ -10475,4 +10574,4 @@ "title": "Test-Cluster-TiFlash-Summary", "uid": "SVbh2xUWk", "version": 1 -} +} \ No newline at end of file From c87ee5a79dafebd8076977b77bbe34f34808c23b Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Thu, 22 Dec 2022 14:53:12 +0800 Subject: [PATCH 22/93] 16 Signed-off-by: Zhigao Tong --- contrib/CMakeLists.txt | 2 +- contrib/tzg.mpp.proto.patch | 468 +++++++++++++----- dbms/src/Common/TiFlashBuildInfo.cpp | 4 +- dbms/src/Flash/FlashService.cpp | 68 ++- dbms/src/Flash/Mpp/ExchangeReceiver.cpp | 6 +- .../Flash/Mpp/FineGrainedShuffleWriter.cpp | 4 +- dbms/src/Flash/Mpp/HashPartitionWriter.cpp | 6 +- dbms/src/Flash/Mpp/MPPTask.cpp | 5 - dbms/src/Flash/Mpp/Utils.cpp | 2 +- dbms/src/Flash/Mpp/newMPPExchangeWriter.h | 12 +- 10 files changed, 432 insertions(+), 145 deletions(-) diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 8c84924eaaa..f4346e0b69d 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -112,7 +112,7 @@ if (USE_INTERNAL_GRPC_LIBRARY) endif () execute_process( - COMMAND grep "CompressMethod compress = 5;" "${TiFlash_SOURCE_DIR}/contrib/kvproto/proto/mpp.proto" + COMMAND grep "CompressMethod compress" "${TiFlash_SOURCE_DIR}/contrib/kvproto/proto/mpp.proto" RESULT_VARIABLE HAVE_APPLY_PATCH) # grep - Normally, the exit status is 0 if selected lines are found and 1 otherwise. But the exit status is 2 if an error occurred. diff --git a/contrib/tzg.mpp.proto.patch b/contrib/tzg.mpp.proto.patch index b886cf4e20e..36f2883d5dc 100644 --- a/contrib/tzg.mpp.proto.patch +++ b/contrib/tzg.mpp.proto.patch @@ -1,5 +1,5 @@ diff --git a/pkg/mpp/mpp.pb.go b/pkg/mpp/mpp.pb.go -index 16fd5dd..9eda45d 100644 +index 16fd5dd..63a6d6c 100644 --- a/pkg/mpp/mpp.pb.go +++ b/pkg/mpp/mpp.pb.go @@ -26,12 +26,89 @@ var _ = math.Inf @@ -155,15 +155,23 @@ index 16fd5dd..9eda45d 100644 // Dipsatch the task request to different tiflash servers. type DispatchTaskRequest struct { Meta *TaskMeta `protobuf:"bytes,1,opt,name=meta,proto3" json:"meta,omitempty"` -@@ -194,6 +286,7 @@ type DispatchTaskRequest struct { +@@ -193,17 +285,19 @@ type DispatchTaskRequest struct { + // If this task contains table scan, we still need their region info. SchemaVer int64 `protobuf:"varint,5,opt,name=schema_ver,json=schemaVer,proto3" json:"schema_ver,omitempty"` // Used for partition table scan - TableRegions []*coprocessor.TableRegions `protobuf:"bytes,6,rep,name=table_regions,json=tableRegions,proto3" json:"table_regions,omitempty"` -+ ExchangeSenderMeta *ExchangeSenderMeta `protobuf:"bytes,7,opt,name=exchange_sender_meta,json=exchangeSenderMeta,proto3" json:"exchange_sender_meta,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -@@ -203,7 +296,7 @@ func (m *DispatchTaskRequest) Reset() { *m = DispatchTaskRequest{} } +- TableRegions []*coprocessor.TableRegions `protobuf:"bytes,6,rep,name=table_regions,json=tableRegions,proto3" json:"table_regions,omitempty"` +- XXX_NoUnkeyedLiteral struct{} `json:"-"` +- XXX_unrecognized []byte `json:"-"` +- XXX_sizecache int32 `json:"-"` ++ TableRegions []*coprocessor.TableRegions `protobuf:"bytes,6,rep,name=table_regions,json=tableRegions,proto3" json:"table_regions,omitempty"` ++ // Used for exchange sender ++ ExchangeSenderMeta *ExchangeSenderMeta `protobuf:"bytes,7,opt,name=exchange_sender_meta,json=exchangeSenderMeta,proto3" json:"exchange_sender_meta,omitempty"` ++ XXX_NoUnkeyedLiteral struct{} `json:"-"` ++ XXX_unrecognized []byte `json:"-"` ++ XXX_sizecache int32 `json:"-"` + } + + func (m *DispatchTaskRequest) Reset() { *m = DispatchTaskRequest{} } func (m *DispatchTaskRequest) String() string { return proto.CompactTextString(m) } func (*DispatchTaskRequest) ProtoMessage() {} func (*DispatchTaskRequest) Descriptor() ([]byte, []int) { @@ -172,7 +180,7 @@ index 16fd5dd..9eda45d 100644 } func (m *DispatchTaskRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) -@@ -274,6 +367,13 @@ func (m *DispatchTaskRequest) GetTableRegions() []*coprocessor.TableRegions { +@@ -274,6 +368,13 @@ func (m *DispatchTaskRequest) GetTableRegions() []*coprocessor.TableRegions { return nil } @@ -186,7 +194,7 @@ index 16fd5dd..9eda45d 100644 // Get response of DispatchTaskRequest. type DispatchTaskResponse struct { Error *Error `protobuf:"bytes,1,opt,name=error,proto3" json:"error,omitempty"` -@@ -287,7 +387,7 @@ func (m *DispatchTaskResponse) Reset() { *m = DispatchTaskResponse{} } +@@ -287,7 +388,7 @@ func (m *DispatchTaskResponse) Reset() { *m = DispatchTaskResponse{} } func (m *DispatchTaskResponse) String() string { return proto.CompactTextString(m) } func (*DispatchTaskResponse) ProtoMessage() {} func (*DispatchTaskResponse) Descriptor() ([]byte, []int) { @@ -195,7 +203,7 @@ index 16fd5dd..9eda45d 100644 } func (m *DispatchTaskResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) -@@ -343,7 +443,7 @@ func (m *CancelTaskRequest) Reset() { *m = CancelTaskRequest{} } +@@ -343,7 +444,7 @@ func (m *CancelTaskRequest) Reset() { *m = CancelTaskRequest{} } func (m *CancelTaskRequest) String() string { return proto.CompactTextString(m) } func (*CancelTaskRequest) ProtoMessage() {} func (*CancelTaskRequest) Descriptor() ([]byte, []int) { @@ -204,7 +212,7 @@ index 16fd5dd..9eda45d 100644 } func (m *CancelTaskRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) -@@ -397,7 +497,7 @@ func (m *CancelTaskResponse) Reset() { *m = CancelTaskResponse{} } +@@ -397,7 +498,7 @@ func (m *CancelTaskResponse) Reset() { *m = CancelTaskResponse{} } func (m *CancelTaskResponse) String() string { return proto.CompactTextString(m) } func (*CancelTaskResponse) ProtoMessage() {} func (*CancelTaskResponse) Descriptor() ([]byte, []int) { @@ -213,7 +221,7 @@ index 16fd5dd..9eda45d 100644 } func (m *CancelTaskResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) -@@ -446,7 +546,7 @@ func (m *EstablishMPPConnectionRequest) Reset() { *m = EstablishMPPConne +@@ -446,7 +547,7 @@ func (m *EstablishMPPConnectionRequest) Reset() { *m = EstablishMPPConne func (m *EstablishMPPConnectionRequest) String() string { return proto.CompactTextString(m) } func (*EstablishMPPConnectionRequest) ProtoMessage() {} func (*EstablishMPPConnectionRequest) Descriptor() ([]byte, []int) { @@ -222,7 +230,57 @@ index 16fd5dd..9eda45d 100644 } func (m *EstablishMPPConnectionRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) -@@ -492,20 +592,21 @@ func (m *EstablishMPPConnectionRequest) GetReceiverMeta() *TaskMeta { +@@ -489,23 +590,71 @@ func (m *EstablishMPPConnectionRequest) GetReceiverMeta() *TaskMeta { + return nil + } + ++type CompressMeta struct { ++ Method CompressMethod `protobuf:"varint,1,opt,name=method,proto3,enum=mpp.CompressMethod" json:"method,omitempty"` ++ XXX_NoUnkeyedLiteral struct{} `json:"-"` ++ XXX_unrecognized []byte `json:"-"` ++ XXX_sizecache int32 `json:"-"` ++} ++ ++func (m *CompressMeta) Reset() { *m = CompressMeta{} } ++func (m *CompressMeta) String() string { return proto.CompactTextString(m) } ++func (*CompressMeta) ProtoMessage() {} ++func (*CompressMeta) Descriptor() ([]byte, []int) { ++ return fileDescriptor_819623c7fa76fc55, []int{9} ++} ++func (m *CompressMeta) XXX_Unmarshal(b []byte) error { ++ return m.Unmarshal(b) ++} ++func (m *CompressMeta) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { ++ if deterministic { ++ return xxx_messageInfo_CompressMeta.Marshal(b, m, deterministic) ++ } else { ++ b = b[:cap(b)] ++ n, err := m.MarshalToSizedBuffer(b) ++ if err != nil { ++ return nil, err ++ } ++ return b[:n], nil ++ } ++} ++func (m *CompressMeta) XXX_Merge(src proto.Message) { ++ xxx_messageInfo_CompressMeta.Merge(m, src) ++} ++func (m *CompressMeta) XXX_Size() int { ++ return m.Size() ++} ++func (m *CompressMeta) XXX_DiscardUnknown() { ++ xxx_messageInfo_CompressMeta.DiscardUnknown(m) ++} ++ ++var xxx_messageInfo_CompressMeta proto.InternalMessageInfo ++ ++func (m *CompressMeta) GetMethod() CompressMethod { ++ if m != nil { ++ return m.Method ++ } ++ return CompressMethod_NONE ++} ++ // when TiFlash sends data to TiDB, Data packets wrap tipb.SelectResponse, i.e., serialize tipb.SelectResponse into data; // when TiFlash sends data to TiFlash, data blocks are serialized into chunks, and the execution_summaries in tipb.SelectResponse are serialized into data only for the last packet. type MPPDataPacket struct { @@ -233,14 +291,14 @@ index 16fd5dd..9eda45d 100644 - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -+ Data []byte `protobuf:"bytes,1,opt,name=data,proto3" json:"data,omitempty"` -+ Error *Error `protobuf:"bytes,2,opt,name=error,proto3" json:"error,omitempty"` -+ Chunks [][]byte `protobuf:"bytes,3,rep,name=chunks,proto3" json:"chunks,omitempty"` -+ StreamIds []uint64 `protobuf:"varint,4,rep,packed,name=stream_ids,json=streamIds,proto3" json:"stream_ids,omitempty"` -+ Compress CompressMethod `protobuf:"varint,5,opt,name=compress,proto3,enum=mpp.CompressMethod" json:"compress,omitempty"` -+ XXX_NoUnkeyedLiteral struct{} `json:"-"` -+ XXX_unrecognized []byte `json:"-"` -+ XXX_sizecache int32 `json:"-"` ++ Data []byte `protobuf:"bytes,1,opt,name=data,proto3" json:"data,omitempty"` ++ Error *Error `protobuf:"bytes,2,opt,name=error,proto3" json:"error,omitempty"` ++ Chunks [][]byte `protobuf:"bytes,3,rep,name=chunks,proto3" json:"chunks,omitempty"` ++ StreamIds []uint64 `protobuf:"varint,4,rep,packed,name=stream_ids,json=streamIds,proto3" json:"stream_ids,omitempty"` ++ Compress *CompressMeta `protobuf:"bytes,5,opt,name=compress,proto3" json:"compress,omitempty"` ++ XXX_NoUnkeyedLiteral struct{} `json:"-"` ++ XXX_unrecognized []byte `json:"-"` ++ XXX_sizecache int32 `json:"-"` } func (m *MPPDataPacket) Reset() { *m = MPPDataPacket{} } @@ -248,34 +306,34 @@ index 16fd5dd..9eda45d 100644 func (*MPPDataPacket) ProtoMessage() {} func (*MPPDataPacket) Descriptor() ([]byte, []int) { - return fileDescriptor_819623c7fa76fc55, []int{8} -+ return fileDescriptor_819623c7fa76fc55, []int{9} ++ return fileDescriptor_819623c7fa76fc55, []int{10} } func (m *MPPDataPacket) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) -@@ -562,6 +663,13 @@ func (m *MPPDataPacket) GetStreamIds() []uint64 { +@@ -562,6 +711,13 @@ func (m *MPPDataPacket) GetStreamIds() []uint64 { return nil } -+func (m *MPPDataPacket) GetCompress() CompressMethod { ++func (m *MPPDataPacket) GetCompress() *CompressMeta { + if m != nil { + return m.Compress + } -+ return CompressMethod_NONE ++ return nil +} + type Error struct { Code int32 `protobuf:"varint,1,opt,name=code,proto3" json:"code,omitempty"` Msg string `protobuf:"bytes,2,opt,name=msg,proto3" json:"msg,omitempty"` -@@ -574,7 +682,7 @@ func (m *Error) Reset() { *m = Error{} } +@@ -574,7 +730,7 @@ func (m *Error) Reset() { *m = Error{} } func (m *Error) String() string { return proto.CompactTextString(m) } func (*Error) ProtoMessage() {} func (*Error) Descriptor() ([]byte, []int) { - return fileDescriptor_819623c7fa76fc55, []int{9} -+ return fileDescriptor_819623c7fa76fc55, []int{10} ++ return fileDescriptor_819623c7fa76fc55, []int{11} } func (m *Error) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) -@@ -618,6 +726,8 @@ func (m *Error) GetMsg() string { +@@ -618,6 +774,8 @@ func (m *Error) GetMsg() string { } func init() { @@ -284,7 +342,15 @@ index 16fd5dd..9eda45d 100644 proto.RegisterType((*TaskMeta)(nil), "mpp.TaskMeta") proto.RegisterType((*IsAliveRequest)(nil), "mpp.IsAliveRequest") proto.RegisterType((*IsAliveResponse)(nil), "mpp.IsAliveResponse") -@@ -633,45 +743,85 @@ func init() { +@@ -626,6 +784,7 @@ func init() { + proto.RegisterType((*CancelTaskRequest)(nil), "mpp.CancelTaskRequest") + proto.RegisterType((*CancelTaskResponse)(nil), "mpp.CancelTaskResponse") + proto.RegisterType((*EstablishMPPConnectionRequest)(nil), "mpp.EstablishMPPConnectionRequest") ++ proto.RegisterType((*CompressMeta)(nil), "mpp.CompressMeta") + proto.RegisterType((*MPPDataPacket)(nil), "mpp.MPPDataPacket") + proto.RegisterType((*Error)(nil), "mpp.Error") + } +@@ -633,45 +792,86 @@ func init() { func init() { proto.RegisterFile("mpp.proto", fileDescriptor_819623c7fa76fc55) } var fileDescriptor_819623c7fa76fc55 = []byte{ @@ -327,53 +393,54 @@ index 16fd5dd..9eda45d 100644 - 0x7b, 0xdf, 0xef, 0xc6, 0xde, 0x97, 0x1f, 0xe3, 0xff, 0x60, 0x47, 0xaa, 0x45, 0x64, 0xf8, 0xaa, - 0x8c, 0x56, 0xa5, 0xfb, 0x42, 0xdc, 0x74, 0xdd, 0xcf, 0xd9, 0xcf, 0x00, 0x00, 0x00, 0xff, 0xff, - 0xef, 0x8c, 0x66, 0x5e, 0x71, 0x04, 0x00, 0x00, -+ // 722 bytes of a gzipped FileDescriptorProto -+ 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x54, 0xcd, 0x6e, 0xd3, 0x4a, -+ 0x14, 0xae, 0xe3, 0xfc, 0x9e, 0x38, 0xb9, 0xe9, 0xb4, 0xba, 0x4d, 0xab, 0xdb, 0xdc, 0xd4, 0xab, -+ 0x08, 0xa9, 0xa9, 0x48, 0x11, 0x4b, 0x24, 0x68, 0xb3, 0x88, 0x44, 0x4a, 0x34, 0x8d, 0x2a, 0xd4, -+ 0x8d, 0x35, 0xb5, 0x87, 0xc4, 0x4a, 0xec, 0x19, 0x66, 0xa6, 0x11, 0xac, 0x79, 0x09, 0x16, 0xbc, -+ 0x02, 0xef, 0xc1, 0x12, 0xf1, 0x04, 0xa8, 0xbc, 0x08, 0x9a, 0xb1, 0x9d, 0x26, 0xb4, 0x88, 0xb2, -+ 0xf2, 0x9c, 0xef, 0xfc, 0x7d, 0xe7, 0x9b, 0xe3, 0x81, 0x4a, 0xc4, 0x79, 0x97, 0x0b, 0xa6, 0x18, -+ 0xb2, 0x23, 0xce, 0xf7, 0xb6, 0x27, 0x6c, 0xc2, 0x8c, 0x7d, 0xa4, 0x4f, 0x89, 0x6b, 0x6f, 0xd3, -+ 0xd7, 0x98, 0x4f, 0xa5, 0x64, 0x22, 0x85, 0x9c, 0x88, 0x2a, 0xc2, 0xaf, 0x12, 0xcb, 0xed, 0x03, -+ 0xea, 0xbf, 0xf3, 0xa7, 0x24, 0x9e, 0xd0, 0x73, 0x1a, 0x07, 0x54, 0x0c, 0xa9, 0x22, 0xe8, 0x08, -+ 0xca, 0x3e, 0x8b, 0xb8, 0xa0, 0x52, 0x36, 0xad, 0xb6, 0xd5, 0xa9, 0xf7, 0xb6, 0xba, 0xba, 0xdf, -+ 0x49, 0x0a, 0x0e, 0xa9, 0x9a, 0xb2, 0x00, 0x2f, 0x83, 0xdc, 0x4f, 0x16, 0x94, 0xc7, 0x44, 0xce, -+ 0x4c, 0xf6, 0x2e, 0x94, 0xa5, 0x22, 0x42, 0x79, 0x2a, 0xc9, 0xce, 0xe3, 0x92, 0xb1, 0xc7, 0x12, -+ 0xed, 0x40, 0x49, 0x11, 0x39, 0xf3, 0xc2, 0xa0, 0x99, 0x6b, 0x5b, 0x1d, 0x1b, 0x17, 0xb5, 0x39, -+ 0x08, 0xd0, 0x01, 0x38, 0x9c, 0x08, 0x15, 0xaa, 0x90, 0xc5, 0xda, 0x6b, 0x1b, 0x6f, 0x75, 0x89, -+ 0x0d, 0x02, 0xd4, 0x84, 0x12, 0x09, 0x02, 0xc3, 0x29, 0xdf, 0xb6, 0x3a, 0x15, 0x9c, 0x99, 0xe8, -+ 0x7f, 0xa8, 0x46, 0x9c, 0x7b, 0x0b, 0x2a, 0x64, 0xc8, 0xe2, 0x66, 0xc1, 0xe4, 0x42, 0xc4, 0xf9, -+ 0x45, 0x82, 0xb8, 0x0d, 0xa8, 0x0f, 0xe4, 0xf3, 0x79, 0xb8, 0xa0, 0x98, 0xbe, 0xbd, 0xa6, 0x52, -+ 0xb9, 0x23, 0xf8, 0x67, 0x89, 0x48, 0xce, 0x62, 0x49, 0xd1, 0x7f, 0x50, 0x21, 0x0b, 0x12, 0xce, -+ 0xc9, 0xd5, 0x9c, 0x1a, 0xde, 0x65, 0x7c, 0x0b, 0xfc, 0xda, 0x23, 0x77, 0xa7, 0xc7, 0xb7, 0x1c, -+ 0x6c, 0x9d, 0x86, 0x92, 0x13, 0xe5, 0x4f, 0xb5, 0x14, 0x69, 0x27, 0x74, 0x00, 0x79, 0xad, 0xb8, -+ 0xa9, 0x58, 0xed, 0xd5, 0x8c, 0x8e, 0x99, 0x54, 0xd8, 0xb8, 0xf4, 0xf0, 0x34, 0xf6, 0x59, 0x40, -+ 0x03, 0x8f, 0xcf, 0x49, 0x52, 0xdc, 0xc1, 0xd5, 0x14, 0x1b, 0xcd, 0x49, 0xac, 0x87, 0x57, 0x61, -+ 0x44, 0xd9, 0xb5, 0x4a, 0xa5, 0xc9, 0x4c, 0xf4, 0x18, 0x4a, 0x82, 0x4e, 0x42, 0x16, 0x6b, 0x59, -+ 0xec, 0x4e, 0xb5, 0xb7, 0xd3, 0x5d, 0xbd, 0x74, 0x6c, 0x7c, 0x83, 0xf8, 0x0d, 0xc3, 0x59, 0x1c, -+ 0xda, 0x07, 0x90, 0xfe, 0x94, 0x46, 0x44, 0x8f, 0x93, 0xca, 0x55, 0x49, 0x90, 0x0b, 0x2a, 0xd0, -+ 0x33, 0xa8, 0x29, 0x3d, 0xb3, 0x97, 0xd5, 0x2d, 0x9a, 0xba, 0xbb, 0x6b, 0x75, 0xc7, 0x3a, 0x22, -+ 0x29, 0x2e, 0xb1, 0xa3, 0x56, 0x2c, 0x34, 0x80, 0x6d, 0x9a, 0xee, 0x94, 0x27, 0xcd, 0x52, 0x79, -+ 0x46, 0x81, 0x92, 0x51, 0x60, 0xc7, 0x28, 0x70, 0x77, 0xe9, 0x30, 0xa2, 0x77, 0x30, 0x37, 0x82, -+ 0xed, 0x75, 0x4d, 0xd3, 0xbb, 0x6a, 0x43, 0x81, 0x0a, 0xc1, 0x44, 0xaa, 0x2a, 0x24, 0x35, 0x35, -+ 0x82, 0x13, 0x07, 0x3a, 0x86, 0x9a, 0xa0, 0x4a, 0xbc, 0x5f, 0x0e, 0x91, 0x33, 0x43, 0xd4, 0xbb, -+ 0xe9, 0xfa, 0x27, 0x64, 0xb1, 0x63, 0x82, 0x52, 0xe6, 0xee, 0x6b, 0xd8, 0x3c, 0x21, 0xb1, 0x4f, -+ 0xe7, 0x7f, 0x79, 0x81, 0x4b, 0x3a, 0xb9, 0xdf, 0xd0, 0x71, 0x9f, 0x02, 0x5a, 0xad, 0xfc, 0xd0, -+ 0x31, 0xdc, 0x0f, 0x16, 0xec, 0xf7, 0xa5, 0x96, 0x37, 0x94, 0xd3, 0xe1, 0x68, 0x74, 0xc2, 0xe2, -+ 0x98, 0xfa, 0xfa, 0x8f, 0xc8, 0xe8, 0x75, 0xa1, 0xba, 0x2a, 0xf2, 0xbd, 0x2c, 0x41, 0xde, 0xfe, -+ 0xdb, 0x3d, 0x2d, 0x8c, 0x4f, 0xc3, 0x45, 0x96, 0x91, 0xbb, 0x2f, 0xc3, 0xc9, 0x62, 0xcc, 0x35, -+ 0x7c, 0xb6, 0xa0, 0x36, 0x1c, 0x8d, 0x4e, 0x89, 0x22, 0x23, 0xe2, 0xcf, 0xa8, 0x42, 0x08, 0xf2, -+ 0x01, 0x49, 0xdb, 0x39, 0xd8, 0x9c, 0xff, 0xac, 0x02, 0xfa, 0x17, 0x8a, 0xfe, 0xf4, 0x3a, 0x9e, -+ 0xc9, 0xa6, 0xdd, 0xb6, 0x3b, 0x0e, 0x4e, 0x2d, 0xb3, 0x90, 0x4a, 0x50, 0x12, 0x79, 0x61, 0x90, -+ 0xac, 0x71, 0x1e, 0x57, 0x12, 0x64, 0x10, 0xc8, 0xb5, 0xe7, 0xa8, 0xf0, 0x90, 0xe7, 0xe8, 0x10, -+ 0x0a, 0xa6, 0xaf, 0xa6, 0xa9, 0xff, 0x21, 0x43, 0xb3, 0x80, 0xcd, 0x19, 0x35, 0xc0, 0x8e, 0xe4, -+ 0xc4, 0x90, 0xac, 0x60, 0x7d, 0x7c, 0x74, 0x08, 0xf5, 0xf5, 0x52, 0xa8, 0x0c, 0xf9, 0xb3, 0x57, -+ 0x67, 0xfd, 0xc6, 0x06, 0x2a, 0x81, 0xfd, 0xf2, 0xf2, 0x49, 0xc3, 0xd2, 0xd0, 0xe5, 0xf9, 0xf8, -+ 0xb4, 0x91, 0x7b, 0xe1, 0x7e, 0xb9, 0x69, 0x59, 0x5f, 0x6f, 0x5a, 0xd6, 0xf7, 0x9b, 0x96, 0xf5, -+ 0xf1, 0x47, 0x6b, 0x03, 0x1a, 0x4c, 0x4c, 0xba, 0x2a, 0x9c, 0x2d, 0xba, 0xb3, 0x85, 0x79, 0x57, -+ 0xaf, 0x8a, 0xe6, 0x73, 0xfc, 0x33, 0x00, 0x00, 0xff, 0xff, 0x2d, 0xa1, 0xda, 0xda, 0xa7, 0x05, -+ 0x00, 0x00, ++ // 750 bytes of a gzipped FileDescriptorProto ++ 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x54, 0xcd, 0x6e, 0xdb, 0x38, ++ 0x10, 0x8e, 0x2c, 0xff, 0x8e, 0x65, 0xaf, 0xc3, 0x04, 0x1b, 0x27, 0xd8, 0x78, 0x1d, 0x9d, 0x8c, ++ 0x5d, 0xc4, 0x41, 0x9d, 0xa2, 0x97, 0x02, 0x05, 0xda, 0xc4, 0x07, 0x03, 0x75, 0x6a, 0x30, 0x46, ++ 0x50, 0xe4, 0x22, 0x30, 0x12, 0x6b, 0x0b, 0xb6, 0x44, 0x96, 0x64, 0x8c, 0xf6, 0xdc, 0x97, 0xe8, ++ 0xa1, 0x6f, 0xd0, 0x17, 0xe9, 0xb1, 0xe8, 0x13, 0x14, 0xe9, 0x8b, 0x14, 0xa4, 0x24, 0xc7, 0xf9, ++ 0xe9, 0xdf, 0x49, 0x9a, 0x6f, 0x86, 0xc3, 0x6f, 0x3e, 0xce, 0x0c, 0x54, 0x22, 0xce, 0xbb, 0x5c, ++ 0x30, 0xc5, 0x90, 0x1d, 0x71, 0xbe, 0xb3, 0x39, 0x61, 0x13, 0x66, 0xec, 0x03, 0xfd, 0x97, 0xb8, ++ 0x76, 0xd6, 0x7d, 0x8d, 0xf9, 0x54, 0x4a, 0x26, 0x52, 0xc8, 0x89, 0xa8, 0x22, 0xfc, 0x22, 0xb1, ++ 0xdc, 0x3e, 0xa0, 0xfe, 0x1b, 0x7f, 0x4a, 0xe2, 0x09, 0x3d, 0xa5, 0x71, 0x40, 0xc5, 0x90, 0x2a, ++ 0x82, 0x0e, 0xa0, 0xec, 0xb3, 0x88, 0x0b, 0x2a, 0x65, 0xd3, 0x6a, 0x5b, 0x9d, 0x7a, 0x6f, 0xa3, ++ 0xab, 0xef, 0x3b, 0x4a, 0xc1, 0x21, 0x55, 0x53, 0x16, 0xe0, 0x65, 0x90, 0xfb, 0xc1, 0x82, 0xf2, ++ 0x98, 0xc8, 0x99, 0x39, 0xbd, 0x0d, 0x65, 0xa9, 0x88, 0x50, 0x9e, 0x4a, 0x4e, 0xe7, 0x71, 0xc9, ++ 0xd8, 0x63, 0x89, 0xb6, 0xa0, 0xa4, 0x88, 0x9c, 0x79, 0x61, 0xd0, 0xcc, 0xb5, 0xad, 0x8e, 0x8d, ++ 0x8b, 0xda, 0x1c, 0x04, 0x68, 0x0f, 0x1c, 0x4e, 0x84, 0x0a, 0x55, 0xc8, 0x62, 0xed, 0xb5, 0x8d, ++ 0xb7, 0xba, 0xc4, 0x06, 0x01, 0x6a, 0x42, 0x89, 0x04, 0x81, 0xe1, 0x94, 0x6f, 0x5b, 0x9d, 0x0a, ++ 0xce, 0x4c, 0xf4, 0x2f, 0x54, 0x23, 0xce, 0xbd, 0x05, 0x15, 0x32, 0x64, 0x71, 0xb3, 0x60, 0xce, ++ 0x42, 0xc4, 0xf9, 0x59, 0x82, 0xb8, 0x0d, 0xa8, 0x0f, 0xe4, 0xd3, 0x79, 0xb8, 0xa0, 0x98, 0xbe, ++ 0xbe, 0xa4, 0x52, 0xb9, 0x23, 0xf8, 0x6b, 0x89, 0x48, 0xce, 0x62, 0x49, 0xd1, 0x3f, 0x50, 0x21, ++ 0x0b, 0x12, 0xce, 0xc9, 0xc5, 0x9c, 0x1a, 0xde, 0x65, 0x7c, 0x0d, 0xdc, 0xbe, 0x23, 0x77, 0xe7, ++ 0x8e, 0x2f, 0x39, 0xd8, 0x38, 0x0e, 0x25, 0x27, 0xca, 0x9f, 0x6a, 0x29, 0xd2, 0x9b, 0xd0, 0x1e, ++ 0xe4, 0xb5, 0xe2, 0x26, 0x63, 0xb5, 0x57, 0x33, 0x3a, 0x66, 0x52, 0x61, 0xe3, 0xd2, 0xc5, 0xd3, ++ 0xd8, 0x67, 0x01, 0x0d, 0x3c, 0x3e, 0x27, 0x49, 0x72, 0x07, 0x57, 0x53, 0x6c, 0x34, 0x27, 0xb1, ++ 0x2e, 0x5e, 0x85, 0x11, 0x65, 0x97, 0x2a, 0x95, 0x26, 0x33, 0xd1, 0x03, 0x28, 0x09, 0x3a, 0x09, ++ 0x59, 0xac, 0x65, 0xb1, 0x3b, 0xd5, 0xde, 0x56, 0x77, 0xf5, 0xd1, 0xb1, 0xf1, 0x0d, 0xe2, 0x57, ++ 0x0c, 0x67, 0x71, 0x68, 0x17, 0x40, 0xfa, 0x53, 0x1a, 0x11, 0x5d, 0x4e, 0x2a, 0x57, 0x25, 0x41, ++ 0xce, 0xa8, 0x40, 0x4f, 0xa0, 0xa6, 0x74, 0xcd, 0x5e, 0x96, 0xb7, 0x68, 0xf2, 0x6e, 0xdf, 0xc8, ++ 0x3b, 0xd6, 0x11, 0x49, 0x72, 0x89, 0x1d, 0xb5, 0x62, 0xa1, 0x01, 0x6c, 0xd2, 0xb4, 0xa7, 0x3c, ++ 0x69, 0x9a, 0xca, 0x33, 0x0a, 0x94, 0x8c, 0x02, 0x5b, 0x46, 0x81, 0xbb, 0x4d, 0x87, 0x11, 0xbd, ++ 0x83, 0xb9, 0x11, 0x6c, 0xde, 0xd4, 0x34, 0x7d, 0xab, 0x36, 0x14, 0xa8, 0x10, 0x4c, 0xa4, 0xaa, ++ 0x42, 0x92, 0x53, 0x23, 0x38, 0x71, 0xa0, 0x43, 0xa8, 0x09, 0xaa, 0xc4, 0xdb, 0x65, 0x11, 0x39, ++ 0x53, 0x44, 0xbd, 0x9b, 0xb6, 0x7f, 0x42, 0x16, 0x3b, 0x26, 0x28, 0x65, 0xee, 0xbe, 0x84, 0xf5, ++ 0x23, 0x12, 0xfb, 0x74, 0xfe, 0x87, 0x0f, 0xb8, 0xa4, 0x93, 0xfb, 0x01, 0x1d, 0xf7, 0x11, 0xa0, ++ 0xd5, 0xcc, 0xbf, 0x5b, 0x86, 0xfb, 0xce, 0x82, 0xdd, 0xbe, 0xd4, 0xf2, 0x86, 0x72, 0x3a, 0x1c, ++ 0x8d, 0x8e, 0x58, 0x1c, 0x53, 0x5f, 0x4f, 0x44, 0x46, 0xaf, 0x0b, 0xd5, 0x55, 0x91, 0xef, 0x65, ++ 0x09, 0xf2, 0x7a, 0xb6, 0x7b, 0x5a, 0x18, 0x9f, 0x86, 0x8b, 0xec, 0x44, 0xee, 0xbe, 0x13, 0x4e, ++ 0x16, 0x63, 0x9e, 0xe1, 0x31, 0x38, 0x2b, 0xa3, 0x4f, 0xd0, 0xff, 0x50, 0x8c, 0xcc, 0x0a, 0xf8, ++ 0xd9, 0x76, 0x48, 0x43, 0xdc, 0x8f, 0x16, 0xd4, 0x86, 0xa3, 0xd1, 0x31, 0x51, 0x64, 0x44, 0xfc, ++ 0x19, 0x55, 0x08, 0x41, 0x3e, 0x20, 0x29, 0x57, 0x07, 0x9b, 0xff, 0x5f, 0x4b, 0x88, 0xfe, 0x86, ++ 0xa2, 0x3f, 0xbd, 0x8c, 0x67, 0xb2, 0x69, 0xb7, 0xed, 0x8e, 0x83, 0x53, 0xcb, 0x74, 0xb3, 0x12, ++ 0x94, 0x44, 0x5e, 0x18, 0x24, 0x33, 0x90, 0xc7, 0x95, 0x04, 0x19, 0x04, 0x12, 0xed, 0xaf, 0xec, ++ 0xb2, 0x82, 0xc9, 0xbd, 0x7e, 0x9b, 0x2d, 0x59, 0xd9, 0x64, 0xfb, 0x50, 0x30, 0xb7, 0x6a, 0x92, ++ 0x7a, 0xfc, 0x0c, 0xc9, 0x02, 0x36, 0xff, 0xa8, 0x01, 0x76, 0x24, 0x27, 0x86, 0x62, 0x05, 0xeb, ++ 0xdf, 0xff, 0xf6, 0xa1, 0x7e, 0xb3, 0x6c, 0x54, 0x86, 0xfc, 0xc9, 0x8b, 0x93, 0x7e, 0x63, 0x0d, ++ 0x95, 0xc0, 0x7e, 0x7e, 0xfe, 0xb0, 0x61, 0x69, 0xe8, 0xfc, 0x74, 0x7c, 0xdc, 0xc8, 0x3d, 0x73, ++ 0x3f, 0x5d, 0xb5, 0xac, 0xcf, 0x57, 0x2d, 0xeb, 0xeb, 0x55, 0xcb, 0x7a, 0xff, 0xad, 0xb5, 0x06, ++ 0x0d, 0x26, 0x26, 0x5d, 0x15, 0xce, 0x16, 0xdd, 0xd9, 0xc2, 0xac, 0xe4, 0x8b, 0xa2, 0xf9, 0x1c, ++ 0x7e, 0x0f, 0x00, 0x00, 0xff, 0xff, 0x91, 0xaa, 0x0c, 0xe0, 0xe2, 0x05, 0x00, 0x00, +} + +func (m *ExchangeSenderMeta) Marshal() (dAtA []byte, err error) { @@ -409,7 +476,7 @@ index 16fd5dd..9eda45d 100644 } func (m *TaskMeta) Marshal() (dAtA []byte, err error) { -@@ -698,6 +848,11 @@ func (m *TaskMeta) MarshalToSizedBuffer(dAtA []byte) (int, error) { +@@ -698,6 +898,11 @@ func (m *TaskMeta) MarshalToSizedBuffer(dAtA []byte) (int, error) { i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } @@ -421,7 +488,7 @@ index 16fd5dd..9eda45d 100644 if len(m.Address) > 0 { i -= len(m.Address) copy(dAtA[i:], m.Address) -@@ -774,6 +929,11 @@ func (m *IsAliveResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { +@@ -774,6 +979,11 @@ func (m *IsAliveResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } @@ -433,7 +500,7 @@ index 16fd5dd..9eda45d 100644 if m.Available { i-- if m.Available { -@@ -811,6 +971,18 @@ func (m *DispatchTaskRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { +@@ -811,6 +1021,18 @@ func (m *DispatchTaskRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } @@ -452,43 +519,89 @@ index 16fd5dd..9eda45d 100644 if len(m.TableRegions) > 0 { for iNdEx := len(m.TableRegions) - 1; iNdEx >= 0; iNdEx-- { { -@@ -1089,21 +1261,26 @@ func (m *MPPDataPacket) MarshalToSizedBuffer(dAtA []byte) (int, error) { +@@ -1065,6 +1287,38 @@ func (m *EstablishMPPConnectionRequest) MarshalToSizedBuffer(dAtA []byte) (int, + return len(dAtA) - i, nil + } + ++func (m *CompressMeta) Marshal() (dAtA []byte, err error) { ++ size := m.Size() ++ dAtA = make([]byte, size) ++ n, err := m.MarshalToSizedBuffer(dAtA[:size]) ++ if err != nil { ++ return nil, err ++ } ++ return dAtA[:n], nil ++} ++ ++func (m *CompressMeta) MarshalTo(dAtA []byte) (int, error) { ++ size := m.Size() ++ return m.MarshalToSizedBuffer(dAtA[:size]) ++} ++ ++func (m *CompressMeta) MarshalToSizedBuffer(dAtA []byte) (int, error) { ++ i := len(dAtA) ++ _ = i ++ var l int ++ _ = l ++ if m.XXX_unrecognized != nil { ++ i -= len(m.XXX_unrecognized) ++ copy(dAtA[i:], m.XXX_unrecognized) ++ } ++ if m.Method != 0 { ++ i = encodeVarintMpp(dAtA, i, uint64(m.Method)) ++ i-- ++ dAtA[i] = 0x8 ++ } ++ return len(dAtA) - i, nil ++} ++ + func (m *MPPDataPacket) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) +@@ -1089,21 +1343,33 @@ func (m *MPPDataPacket) MarshalToSizedBuffer(dAtA []byte) (int, error) { i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } -+ if m.Compress != 0 { -+ i = encodeVarintMpp(dAtA, i, uint64(m.Compress)) ++ if m.Compress != nil { ++ { ++ size, err := m.Compress.MarshalToSizedBuffer(dAtA[:i]) ++ if err != nil { ++ return 0, err ++ } ++ i -= size ++ i = encodeVarintMpp(dAtA, i, uint64(size)) ++ } + i-- -+ dAtA[i] = 0x28 ++ dAtA[i] = 0x2a + } if len(m.StreamIds) > 0 { - dAtA9 := make([]byte, len(m.StreamIds)*10) - var j8 int -+ dAtA10 := make([]byte, len(m.StreamIds)*10) -+ var j9 int ++ dAtA11 := make([]byte, len(m.StreamIds)*10) ++ var j10 int for _, num := range m.StreamIds { for num >= 1<<7 { - dAtA9[j8] = uint8(uint64(num)&0x7f | 0x80) -+ dAtA10[j9] = uint8(uint64(num)&0x7f | 0x80) ++ dAtA11[j10] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j8++ -+ j9++ ++ j10++ } - dAtA9[j8] = uint8(num) - j8++ -+ dAtA10[j9] = uint8(num) -+ j9++ ++ dAtA11[j10] = uint8(num) ++ j10++ } - i -= j8 - copy(dAtA[i:], dAtA9[:j8]) - i = encodeVarintMpp(dAtA, i, uint64(j8)) -+ i -= j9 -+ copy(dAtA[i:], dAtA10[:j9]) -+ i = encodeVarintMpp(dAtA, i, uint64(j9)) ++ i -= j10 ++ copy(dAtA[i:], dAtA11[:j10]) ++ i = encodeVarintMpp(dAtA, i, uint64(j10)) i-- dAtA[i] = 0x22 } -@@ -1188,6 +1365,21 @@ func encodeVarintMpp(dAtA []byte, offset int, v uint64) int { +@@ -1188,6 +1454,21 @@ func encodeVarintMpp(dAtA []byte, offset int, v uint64) int { dAtA[offset] = uint8(v) return base } @@ -510,7 +623,7 @@ index 16fd5dd..9eda45d 100644 func (m *TaskMeta) Size() (n int) { if m == nil { return 0 -@@ -1207,6 +1399,9 @@ func (m *TaskMeta) Size() (n int) { +@@ -1207,6 +1488,9 @@ func (m *TaskMeta) Size() (n int) { if l > 0 { n += 1 + l + sovMpp(uint64(l)) } @@ -520,7 +633,7 @@ index 16fd5dd..9eda45d 100644 if m.XXX_unrecognized != nil { n += len(m.XXX_unrecognized) } -@@ -1234,6 +1429,9 @@ func (m *IsAliveResponse) Size() (n int) { +@@ -1234,6 +1518,9 @@ func (m *IsAliveResponse) Size() (n int) { if m.Available { n += 2 } @@ -530,7 +643,7 @@ index 16fd5dd..9eda45d 100644 if m.XXX_unrecognized != nil { n += len(m.XXX_unrecognized) } -@@ -1272,6 +1470,10 @@ func (m *DispatchTaskRequest) Size() (n int) { +@@ -1272,6 +1559,10 @@ func (m *DispatchTaskRequest) Size() (n int) { n += 1 + l + sovMpp(uint64(l)) } } @@ -541,17 +654,40 @@ index 16fd5dd..9eda45d 100644 if m.XXX_unrecognized != nil { n += len(m.XXX_unrecognized) } -@@ -1383,6 +1585,9 @@ func (m *MPPDataPacket) Size() (n int) { +@@ -1356,6 +1647,21 @@ func (m *EstablishMPPConnectionRequest) Size() (n int) { + return n + } + ++func (m *CompressMeta) Size() (n int) { ++ if m == nil { ++ return 0 ++ } ++ var l int ++ _ = l ++ if m.Method != 0 { ++ n += 1 + sovMpp(uint64(m.Method)) ++ } ++ if m.XXX_unrecognized != nil { ++ n += len(m.XXX_unrecognized) ++ } ++ return n ++} ++ + func (m *MPPDataPacket) Size() (n int) { + if m == nil { + return 0 +@@ -1383,6 +1689,10 @@ func (m *MPPDataPacket) Size() (n int) { } n += 1 + sovMpp(uint64(l)) + l } -+ if m.Compress != 0 { -+ n += 1 + sovMpp(uint64(m.Compress)) ++ if m.Compress != nil { ++ l = m.Compress.Size() ++ n += 1 + l + sovMpp(uint64(l)) + } if m.XXX_unrecognized != nil { n += len(m.XXX_unrecognized) } -@@ -1414,6 +1619,76 @@ func sovMpp(x uint64) (n int) { +@@ -1414,6 +1724,76 @@ func sovMpp(x uint64) (n int) { func sozMpp(x uint64) (n int) { return sovMpp(uint64((x << 1) ^ uint64((int64(x) >> 63)))) } @@ -628,7 +764,7 @@ index 16fd5dd..9eda45d 100644 func (m *TaskMeta) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 -@@ -1532,6 +1807,25 @@ func (m *TaskMeta) Unmarshal(dAtA []byte) error { +@@ -1532,6 +1912,25 @@ func (m *TaskMeta) Unmarshal(dAtA []byte) error { } m.Address = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex @@ -654,7 +790,7 @@ index 16fd5dd..9eda45d 100644 default: iNdEx = preIndex skippy, err := skipMpp(dAtA[iNdEx:]) -@@ -1654,6 +1948,25 @@ func (m *IsAliveResponse) Unmarshal(dAtA []byte) error { +@@ -1654,6 +2053,25 @@ func (m *IsAliveResponse) Unmarshal(dAtA []byte) error { } } m.Available = bool(v != 0) @@ -680,7 +816,7 @@ index 16fd5dd..9eda45d 100644 default: iNdEx = preIndex skippy, err := skipMpp(dAtA[iNdEx:]) -@@ -1881,6 +2194,42 @@ func (m *DispatchTaskRequest) Unmarshal(dAtA []byte) error { +@@ -1881,6 +2299,42 @@ func (m *DispatchTaskRequest) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex @@ -723,15 +859,92 @@ index 16fd5dd..9eda45d 100644 default: iNdEx = preIndex skippy, err := skipMpp(dAtA[iNdEx:]) -@@ -2564,6 +2913,25 @@ func (m *MPPDataPacket) Unmarshal(dAtA []byte) error { +@@ -2357,6 +2811,76 @@ func (m *EstablishMPPConnectionRequest) Unmarshal(dAtA []byte) error { + } + return nil + } ++func (m *CompressMeta) Unmarshal(dAtA []byte) error { ++ l := len(dAtA) ++ iNdEx := 0 ++ for iNdEx < l { ++ preIndex := iNdEx ++ var wire uint64 ++ for shift := uint(0); ; shift += 7 { ++ if shift >= 64 { ++ return ErrIntOverflowMpp ++ } ++ if iNdEx >= l { ++ return io.ErrUnexpectedEOF ++ } ++ b := dAtA[iNdEx] ++ iNdEx++ ++ wire |= uint64(b&0x7F) << shift ++ if b < 0x80 { ++ break ++ } ++ } ++ fieldNum := int32(wire >> 3) ++ wireType := int(wire & 0x7) ++ if wireType == 4 { ++ return fmt.Errorf("proto: CompressMeta: wiretype end group for non-group") ++ } ++ if fieldNum <= 0 { ++ return fmt.Errorf("proto: CompressMeta: illegal tag %d (wire type %d)", fieldNum, wire) ++ } ++ switch fieldNum { ++ case 1: ++ if wireType != 0 { ++ return fmt.Errorf("proto: wrong wireType = %d for field Method", wireType) ++ } ++ m.Method = 0 ++ for shift := uint(0); ; shift += 7 { ++ if shift >= 64 { ++ return ErrIntOverflowMpp ++ } ++ if iNdEx >= l { ++ return io.ErrUnexpectedEOF ++ } ++ b := dAtA[iNdEx] ++ iNdEx++ ++ m.Method |= CompressMethod(b&0x7F) << shift ++ if b < 0x80 { ++ break ++ } ++ } ++ default: ++ iNdEx = preIndex ++ skippy, err := skipMpp(dAtA[iNdEx:]) ++ if err != nil { ++ return err ++ } ++ if (skippy < 0) || (iNdEx+skippy) < 0 { ++ return ErrInvalidLengthMpp ++ } ++ if (iNdEx + skippy) > l { ++ return io.ErrUnexpectedEOF ++ } ++ m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...) ++ iNdEx += skippy ++ } ++ } ++ ++ if iNdEx > l { ++ return io.ErrUnexpectedEOF ++ } ++ return nil ++} + func (m *MPPDataPacket) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 +@@ -2564,6 +3088,42 @@ func (m *MPPDataPacket) Unmarshal(dAtA []byte) error { } else { return fmt.Errorf("proto: wrong wireType = %d for field StreamIds", wireType) } + case 5: -+ if wireType != 0 { ++ if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Compress", wireType) + } -+ m.Compress = 0 ++ var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMpp @@ -741,16 +954,33 @@ index 16fd5dd..9eda45d 100644 + } + b := dAtA[iNdEx] + iNdEx++ -+ m.Compress |= CompressMethod(b&0x7F) << shift ++ msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } ++ if msglen < 0 { ++ return ErrInvalidLengthMpp ++ } ++ postIndex := iNdEx + msglen ++ if postIndex < 0 { ++ return ErrInvalidLengthMpp ++ } ++ if postIndex > l { ++ return io.ErrUnexpectedEOF ++ } ++ if m.Compress == nil { ++ m.Compress = &CompressMeta{} ++ } ++ if err := m.Compress.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { ++ return err ++ } ++ iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipMpp(dAtA[iNdEx:]) diff --git a/proto/mpp.proto b/proto/mpp.proto -index f8af03b..801bee3 100644 +index f8af03b..8dd93bf 100644 --- a/proto/mpp.proto +++ b/proto/mpp.proto @@ -11,12 +11,24 @@ option (gogoproto.unmarshaler_all) = true; @@ -764,7 +994,7 @@ index f8af03b..801bee3 100644 + ZSTD = 2; +} + -+message ExchangeSenderMeta { ++message ExchangeSenderMeta { + CompressMethod compress = 1; +} + @@ -786,19 +1016,31 @@ index f8af03b..801bee3 100644 } // Dipsatch the task request to different tiflash servers. -@@ -36,6 +49,7 @@ message DispatchTaskRequest { +@@ -36,6 +49,8 @@ message DispatchTaskRequest { int64 schema_ver = 5; // Used for partition table scan repeated coprocessor.TableRegions table_regions = 6; ++ // Used for exchange sender + ExchangeSenderMeta exchange_sender_meta = 7; } // Get response of DispatchTaskRequest. -@@ -67,6 +81,7 @@ message MPPDataPacket { +@@ -60,6 +75,10 @@ message EstablishMPPConnectionRequest { + TaskMeta receiver_meta = 2; // node closer to the tidb mpp gather. + } + ++message CompressMeta { ++ CompressMethod method = 1; ++} ++ + // when TiFlash sends data to TiDB, Data packets wrap tipb.SelectResponse, i.e., serialize tipb.SelectResponse into data; + // when TiFlash sends data to TiFlash, data blocks are serialized into chunks, and the execution_summaries in tipb.SelectResponse are serialized into data only for the last packet. + message MPPDataPacket { +@@ -67,6 +86,7 @@ message MPPDataPacket { Error error = 2; repeated bytes chunks = 3; repeated uint64 stream_ids = 4; -+ CompressMethod compress = 5; ++ CompressMeta compress = 5; } message Error { diff --git a/dbms/src/Common/TiFlashBuildInfo.cpp b/dbms/src/Common/TiFlashBuildInfo.cpp index 8e34e62655a..187ce8d9751 100644 --- a/dbms/src/Common/TiFlashBuildInfo.cpp +++ b/dbms/src/Common/TiFlashBuildInfo.cpp @@ -14,6 +14,7 @@ #include #include +#include #include #include #include @@ -145,6 +146,7 @@ void outputDetail(std::ostream & os) << "Git Branch: " << getGitBranch() << std::endl << "UTC Build Time: " << getUTCBuildTime() << std::endl << "Enable Features: " << getEnabledFeatures() << std::endl - << "Profile: " << getProfile() << std::endl; + << "Profile: " << getProfile() << std::endl + << "Mpp Version: " << TiDB::GetMppVersion() << std::endl; } } // namespace TiFlashBuildInfo diff --git a/dbms/src/Flash/FlashService.cpp b/dbms/src/Flash/FlashService.cpp index 75bffb27615..1a879a65d3a 100644 --- a/dbms/src/Flash/FlashService.cpp +++ b/dbms/src/Flash/FlashService.cpp @@ -203,6 +203,14 @@ grpc::Status FlashService::DispatchMPPTask( auto check_result = checkGrpcContext(grpc_context); if (!check_result.ok()) return check_result; + + if (auto mpp_version = request->meta().mpp_version(); !TiDB::CheckMppVersion(mpp_version)) + { + auto && err_msg = fmt::format("Failed to handling mpp dispatch request, reason=`{}`", TiDB::GenMppVersionErrorMessage(mpp_version)); + LOG_WARNING(log, err_msg); + return grpc::Status(grpc::StatusCode::INTERNAL, std::move(err_msg)); + } + GET_METRIC(tiflash_coprocessor_request_count, type_dispatch_mpp_task).Increment(); GET_METRIC(tiflash_coprocessor_handling_request_count, type_dispatch_mpp_task).Increment(); GET_METRIC(tiflash_thread_count, type_active_threads_of_dispatch_mpp).Increment(); @@ -249,6 +257,33 @@ grpc::Status FlashService::IsAlive(grpc::ServerContext * grpc_context [[maybe_un return grpc::Status::OK; } +static grpc::Status CheckMppVersionForEstablishMPPConnection(const mpp::EstablishMPPConnectionRequest * request) +{ + const auto & sender_mpp_version = request->sender_meta().mpp_version(); + const auto & receiver_mpp_version = request->receiver_meta().mpp_version(); + + std::string && err_reason{}; + + if (!TiDB::CheckMppVersion(sender_mpp_version)) + { + err_reason += fmt::format("sender failed: {};", TiDB::GenMppVersionErrorMessage(sender_mpp_version)); + } + if (!TiDB::CheckMppVersion(receiver_mpp_version)) + { + err_reason += fmt::format("receiver failed: {};", TiDB::GenMppVersionErrorMessage(receiver_mpp_version)); + } + + if (!err_reason.empty()) + { + auto && err_msg = fmt::format("Failed to establish MPP connection, sender-meta=`{}`, receiver-meta=`{}`, reason=`{}`", + request->sender_meta().DebugString(), + request->receiver_meta().DebugString(), + err_reason); + return grpc::Status(grpc::StatusCode::INTERNAL, std::move(err_msg)); + } + return grpc::Status::OK; +} + grpc::Status AsyncFlashService::establishMPPConnectionAsync(grpc::ServerContext * grpc_context, const mpp::EstablishMPPConnectionRequest * request, EstablishCallData * call_data) @@ -262,6 +297,12 @@ grpc::Status AsyncFlashService::establishMPPConnectionAsync(grpc::ServerContext if (!check_result.ok()) return check_result; + if (auto res = CheckMppVersionForEstablishMPPConnection(request); !res.ok()) + { + LOG_WARNING(log, res.error_message()); + return res; + } + GET_METRIC(tiflash_coprocessor_request_count, type_mpp_establish_conn).Increment(); GET_METRIC(tiflash_coprocessor_handling_request_count, type_mpp_establish_conn).Increment(); @@ -272,18 +313,6 @@ grpc::Status AsyncFlashService::establishMPPConnectionAsync(grpc::ServerContext grpc::Status FlashService::EstablishMPPConnection(grpc::ServerContext * grpc_context, const mpp::EstablishMPPConnectionRequest * request, grpc::ServerWriter * sync_writer) { - { - const auto & sender_mpp_version = request->sender_meta().mpp_version(); - const auto & receiver_mpp_version = request->receiver_meta().mpp_version(); - if (!TiDB::CheckMppVersion(sender_mpp_version) || !TiDB::CheckMppVersion(receiver_mpp_version)) - { - auto && err_msg = fmt::format("Failed to establish MPP connection, sender: {}, receiver: {}", - TiDB::GenMppVersionErrorMessage(sender_mpp_version), - TiDB::GenMppVersionErrorMessage(receiver_mpp_version)); - return grpc::Status(grpc::StatusCode::INTERNAL, std::move(err_msg)); - } - } - CPUAffinityManager::getInstance().bindSelfGrpcThread(); // Establish a pipe for data transferring. The pipes have registered by the task in advance. // We need to find it out and bind the grpc stream with it. @@ -292,6 +321,13 @@ grpc::Status FlashService::EstablishMPPConnection(grpc::ServerContext * grpc_con auto check_result = checkGrpcContext(grpc_context); if (!check_result.ok()) return check_result; + + if (auto res = CheckMppVersionForEstablishMPPConnection(request); !res.ok()) + { + LOG_WARNING(log, res.error_message()); + return res; + } + GET_METRIC(tiflash_coprocessor_request_count, type_mpp_establish_conn).Increment(); GET_METRIC(tiflash_coprocessor_handling_request_count, type_mpp_establish_conn).Increment(); GET_METRIC(tiflash_thread_count, type_active_threads_of_establish_mpp).Increment(); @@ -345,6 +381,14 @@ grpc::Status FlashService::CancelMPPTask( auto check_result = checkGrpcContext(grpc_context); if (!check_result.ok()) return check_result; + + if (auto mpp_version = request->meta().mpp_version(); !TiDB::CheckMppVersion(mpp_version)) + { + auto && err_msg = fmt::format("Failed to cancel mpp task, reason=`{}`", TiDB::GenMppVersionErrorMessage(mpp_version)); + LOG_WARNING(log, err_msg); + return grpc::Status(grpc::StatusCode::INTERNAL, std::move(err_msg)); + } + GET_METRIC(tiflash_coprocessor_request_count, type_cancel_mpp_task).Increment(); GET_METRIC(tiflash_coprocessor_handling_request_count, type_cancel_mpp_task).Increment(); Stopwatch watch; diff --git a/dbms/src/Flash/Mpp/ExchangeReceiver.cpp b/dbms/src/Flash/Mpp/ExchangeReceiver.cpp index 9ddba8b622d..ac54083a6ec 100644 --- a/dbms/src/Flash/Mpp/ExchangeReceiver.cpp +++ b/dbms/src/Flash/Mpp/ExchangeReceiver.cpp @@ -58,7 +58,7 @@ bool pushPacket(size_t source_index, bool push_succeed = true; const mpp::Error * error_ptr = nullptr; - auto & packet = tracked_packet->packet; + auto & packet = tracked_packet->getPacket(); if (packet.has_error()) error_ptr = &packet.error(); const String * resp_ptr = nullptr; @@ -688,8 +688,8 @@ DecodeDetail ExchangeReceiverBase::decodeChunks( if (recv_msg->chunks.empty()) return detail; - auto & packet = recv_msg->packet->packet; - auto compress_method = recv_msg->packet->getPacket().compress(); + auto & packet = recv_msg->packet->getPacket(); + auto compress_method = packet.compress().method(); // Record total packet size even if fine grained shuffle is enabled. detail.packet_bytes = packet.ByteSizeLong(); diff --git a/dbms/src/Flash/Mpp/FineGrainedShuffleWriter.cpp b/dbms/src/Flash/Mpp/FineGrainedShuffleWriter.cpp index 4f217db124b..ff0757d76f5 100644 --- a/dbms/src/Flash/Mpp/FineGrainedShuffleWriter.cpp +++ b/dbms/src/Flash/Mpp/FineGrainedShuffleWriter.cpp @@ -141,7 +141,7 @@ void FineGrainedShuffleWriter::batchWriteFineGrainedShuffle() { method = mpp::CompressMethod::NONE; } - tracked_packets[part_id]->getPacket().set_compress(method); + tracked_packets[part_id]->getPacket().mutable_compress()->set_method(method); } if (likely(!blocks.empty())) @@ -171,7 +171,7 @@ void FineGrainedShuffleWriter::batchWriteFineGrainedShuffle() auto block = header.cloneWithColumns(std::move(columns)); { ChunkCodecStream * codec_stream = chunk_codec_stream.get(); - if (tracked_packets[part_id]->getPacket().compress() != mpp::CompressMethod::NONE) + if (tracked_packets[part_id]->getPacket().compress().method() != mpp::CompressMethod::NONE) { assert(compress_chunk_codec_stream); // no need compress diff --git a/dbms/src/Flash/Mpp/HashPartitionWriter.cpp b/dbms/src/Flash/Mpp/HashPartitionWriter.cpp index fd14675c466..33155d9cd31 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriter.cpp +++ b/dbms/src/Flash/Mpp/HashPartitionWriter.cpp @@ -128,7 +128,7 @@ void HashPartitionWriter::partitionAndEncodeThenWriteBlocks() { method = mpp::CompressMethod::NONE; } - tracked_packets[part_id]->getPacket().set_compress(method); + tracked_packets[part_id]->getPacket().mutable_compress()->set_method(method); } size_t ori_block_mem_size = 0; @@ -158,7 +158,7 @@ void HashPartitionWriter::partitionAndEncodeThenWriteBlocks() if (dest_block_rows > 0) { auto * codec_stream = chunk_codec_stream.get(); - if (tracked_packets[part_id]->getPacket().compress() != mpp::CompressMethod::NONE) + if (tracked_packets[part_id]->getPacket().compress().method() != mpp::CompressMethod::NONE) { assert(compress_chunk_codec_stream); // no need compress @@ -194,7 +194,7 @@ void HashPartitionWriter::writePackets(const TrackedMppDataPa writer->partitionWrite(packet, part_id); auto sz = inner_packet.ByteSizeLong(); - switch (inner_packet.compress()) + switch (inner_packet.compress().method()) { case mpp::NONE: { diff --git a/dbms/src/Flash/Mpp/MPPTask.cpp b/dbms/src/Flash/Mpp/MPPTask.cpp index 8171cbfe7b9..81d8b1c21b6 100644 --- a/dbms/src/Flash/Mpp/MPPTask.cpp +++ b/dbms/src/Flash/Mpp/MPPTask.cpp @@ -227,11 +227,6 @@ void MPPTask::unregisterTask() void MPPTask::prepare(const mpp::DispatchTaskRequest & task_request) { - if (auto mpp_version = task_request.meta().mpp_version(); !TiDB::CheckMppVersion(mpp_version)) - { - throw TiFlashException(TiDB::GenMppVersionErrorMessage(mpp_version), Errors::MPP::Internal); - } - dag_req = getDAGRequestFromStringWithRetry(task_request.encoded_plan()); TMTContext & tmt_context = context->getTMTContext(); /// MPP task will only use key ranges in mpp::DispatchTaskRequest::regions/mpp::DispatchTaskRequest::table_regions. diff --git a/dbms/src/Flash/Mpp/Utils.cpp b/dbms/src/Flash/Mpp/Utils.cpp index c128f3936cc..b0375949cbb 100644 --- a/dbms/src/Flash/Mpp/Utils.cpp +++ b/dbms/src/Flash/Mpp/Utils.cpp @@ -58,7 +58,7 @@ bool CheckMppVersion(int64_t mpp_version) std::string GenMppVersionErrorMessage(int64_t mpp_version) { - auto err_msg = fmt::format("Invalid mpp version `{}`, expect version: min `{}`, max `{}` release version `{}`", + auto err_msg = fmt::format("invalid mpp version `{}`, expect version: min `{}`, max `{}` release version `{}`", mpp_version, TiDB::MPP_VERSION_V0, TiDB::MPP_VERSION_V1, diff --git a/dbms/src/Flash/Mpp/newMPPExchangeWriter.h b/dbms/src/Flash/Mpp/newMPPExchangeWriter.h index 18818aeced3..5ea25c86ec3 100644 --- a/dbms/src/Flash/Mpp/newMPPExchangeWriter.h +++ b/dbms/src/Flash/Mpp/newMPPExchangeWriter.h @@ -35,13 +35,13 @@ std::unique_ptr NewMPPExchangeWriter( UInt64 fine_grained_shuffle_stream_count, UInt64 fine_grained_shuffle_batch_size) { - RUNTIME_CHECK(!enable_fine_grained_shuffle); - RUNTIME_CHECK(dag_context.isMPPTask()); should_send_exec_summary_at_last = dag_context.collect_execution_summaries && should_send_exec_summary_at_last; if (dag_context.isRootMPPTask()) { + // No need to use use data compression RUNTIME_CHECK(dag_context.getExchangeSenderMeta().compress() == mpp::CompressMethod::NONE); + RUNTIME_CHECK(!enable_fine_grained_shuffle); RUNTIME_CHECK(exchange_type == tipb::ExchangeType::PassThrough); return std::make_unique>( @@ -55,10 +55,11 @@ std::unique_ptr NewMPPExchangeWriter( { if (exchange_type == tipb::ExchangeType::Hash) { - // RUNTIME_CHECK(dag_context.getExchangeSenderMeta().compress() != mpp::CompressMethod::NONE); - if (enable_fine_grained_shuffle) { + // TODO: support data compression if necessary + RUNTIME_CHECK(dag_context.getExchangeSenderMeta().compress() == mpp::CompressMethod::NONE); + return std::make_unique>( writer, partition_col_ids, @@ -81,6 +82,9 @@ std::unique_ptr NewMPPExchangeWriter( } else { + // TODO: support data compression if necessary + RUNTIME_CHECK(dag_context.getExchangeSenderMeta().compress() == mpp::CompressMethod::NONE); + RUNTIME_CHECK(!enable_fine_grained_shuffle); return std::make_unique>( writer, From 16dd20d80bb61465fdb0ab7c0ab37ca42de54208 Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Thu, 22 Dec 2022 17:11:56 +0800 Subject: [PATCH 23/93] 17 Signed-off-by: Zhigao Tong --- dbms/src/Common/TiFlashBuildInfo.cpp | 2 +- dbms/src/Flash/Mpp/MppVersion.h | 1 + dbms/src/Flash/Mpp/Utils.cpp | 20 ++++++++++++++++++-- 3 files changed, 20 insertions(+), 3 deletions(-) diff --git a/dbms/src/Common/TiFlashBuildInfo.cpp b/dbms/src/Common/TiFlashBuildInfo.cpp index 187ce8d9751..cc558c2a43a 100644 --- a/dbms/src/Common/TiFlashBuildInfo.cpp +++ b/dbms/src/Common/TiFlashBuildInfo.cpp @@ -147,6 +147,6 @@ void outputDetail(std::ostream & os) << "UTC Build Time: " << getUTCBuildTime() << std::endl << "Enable Features: " << getEnabledFeatures() << std::endl << "Profile: " << getProfile() << std::endl - << "Mpp Version: " << TiDB::GetMppVersion() << std::endl; + << "Mpp Version: " << fmt::format("{} (release version {})", TiDB::GetMppVersion(), TiDB::GetMppVersionReleaseInfo(TiDB::GetMppVersion())) << std::endl; } } // namespace TiFlashBuildInfo diff --git a/dbms/src/Flash/Mpp/MppVersion.h b/dbms/src/Flash/Mpp/MppVersion.h index e60cd88bd2f..f0346361e36 100644 --- a/dbms/src/Flash/Mpp/MppVersion.h +++ b/dbms/src/Flash/Mpp/MppVersion.h @@ -7,4 +7,5 @@ namespace TiDB bool CheckMppVersion(int64_t mpp_version); std::string GenMppVersionErrorMessage(int64_t mpp_version); int64_t GetMppVersion(); +std::string GetMppVersionReleaseInfo(int64_t mpp_version); } // namespace TiDB \ No newline at end of file diff --git a/dbms/src/Flash/Mpp/Utils.cpp b/dbms/src/Flash/Mpp/Utils.cpp index b0375949cbb..1db7519d41f 100644 --- a/dbms/src/Flash/Mpp/Utils.cpp +++ b/dbms/src/Flash/Mpp/Utils.cpp @@ -17,6 +17,7 @@ #include #include +#include #include namespace DB @@ -47,10 +48,16 @@ namespace TiDB constexpr int64_t MPP_VERSION_V0 = 0; constexpr int64_t MPP_VERSION_V1 = 1; -static const char * MPP_V1_TIFLASH_RELEASE_VERSION = "?"; // TODO: set version after committed +constexpr int64_t MPP_VERSION_MAX = 2; + // constexpr int64_t MPP_VERSION_V2 = MPP_VERSION_V1 * 2; constexpr int64_t MPP_VERSION = MPP_VERSION_V1; +static_assert(MPP_VERSION < MPP_VERSION_MAX && MPP_VERSION >= MPP_VERSION_V0); + +// TODO: set version after committed +constexpr std::array MPP_TIFLASH_RELEASE_VERSION = {"", "?"}; + bool CheckMppVersion(int64_t mpp_version) { return mpp_version >= MPP_VERSION_V0 && mpp_version <= MPP_VERSION_V1; @@ -62,7 +69,7 @@ std::string GenMppVersionErrorMessage(int64_t mpp_version) mpp_version, TiDB::MPP_VERSION_V0, TiDB::MPP_VERSION_V1, - MPP_V1_TIFLASH_RELEASE_VERSION); + MPP_TIFLASH_RELEASE_VERSION[TiDB::MPP_VERSION_V1]); return err_msg; } @@ -71,4 +78,13 @@ int64_t GetMppVersion() return MPP_VERSION; } +std::string GetMppVersionReleaseInfo(int64_t mpp_version) +{ + if (mpp_version >= MPP_VERSION_V0 && mpp_version < MPP_VERSION_MAX) + { + return MPP_TIFLASH_RELEASE_VERSION[mpp_version]; + } + return "unknown"; +} + } // namespace TiDB \ No newline at end of file From e2f17be180793771c0018fd6267c5a76d23d86c7 Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Fri, 23 Dec 2022 17:09:05 +0800 Subject: [PATCH 24/93] 18 --- contrib/kvproto | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/kvproto b/contrib/kvproto index 6c56ac56fe5..7a9ffcbb833 160000 --- a/contrib/kvproto +++ b/contrib/kvproto @@ -1 +1 @@ -Subproject commit 6c56ac56fe5fbe244cfa377115bca8d931c7fdd0 +Subproject commit 7a9ffcbb83360d770f922710f47f931a9d85d1b4 From 686105daf7c20bfba25f7256f3a1ed49e812bcd6 Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Tue, 27 Dec 2022 14:16:05 +0800 Subject: [PATCH 25/93] 19 --- contrib/kvproto | 2 +- contrib/tzg.mpp.proto.patch | 381 +++++++++++++++++++----------------- 2 files changed, 198 insertions(+), 185 deletions(-) diff --git a/contrib/kvproto b/contrib/kvproto index 7a9ffcbb833..0a9b14f1fc2 160000 --- a/contrib/kvproto +++ b/contrib/kvproto @@ -1 +1 @@ -Subproject commit 7a9ffcbb83360d770f922710f47f931a9d85d1b4 +Subproject commit 0a9b14f1fc265bb5ad5a9ccc91f278c9f3716fda diff --git a/contrib/tzg.mpp.proto.patch b/contrib/tzg.mpp.proto.patch index 36f2883d5dc..d81c94bdabe 100644 --- a/contrib/tzg.mpp.proto.patch +++ b/contrib/tzg.mpp.proto.patch @@ -1,8 +1,8 @@ diff --git a/pkg/mpp/mpp.pb.go b/pkg/mpp/mpp.pb.go -index 16fd5dd..63a6d6c 100644 +index fcd4691..b6a15bf 100644 --- a/pkg/mpp/mpp.pb.go +++ b/pkg/mpp/mpp.pb.go -@@ -26,12 +26,89 @@ var _ = math.Inf +@@ -26,6 +26,82 @@ var _ = math.Inf // proto package needs to be updated. const _ = proto.ProtoPackageIsVersion3 // please upgrade the proto package @@ -85,14 +85,15 @@ index 16fd5dd..63a6d6c 100644 // TaskMeta contains meta of a mpp plan, including query's ts and task address. type TaskMeta struct { StartTs uint64 `protobuf:"varint,1,opt,name=start_ts,json=startTs,proto3" json:"start_ts,omitempty"` - TaskId int64 `protobuf:"varint,2,opt,name=task_id,json=taskId,proto3" json:"task_id,omitempty"` - PartitionId int64 `protobuf:"varint,3,opt,name=partition_id,json=partitionId,proto3" json:"partition_id,omitempty"` - Address string `protobuf:"bytes,4,opt,name=address,proto3" json:"address,omitempty"` -+ MppVersion int64 `protobuf:"varint,5,opt,name=mpp_version,json=mppVersion,proto3" json:"mpp_version,omitempty"` +@@ -36,6 +112,7 @@ type TaskMeta struct { + QueryTs uint64 `protobuf:"varint,6,opt,name=query_ts,json=queryTs,proto3" json:"query_ts,omitempty"` + LocalQueryId uint64 `protobuf:"varint,7,opt,name=local_query_id,json=localQueryId,proto3" json:"local_query_id,omitempty"` + ServerId uint64 `protobuf:"varint,8,opt,name=server_id,json=serverId,proto3" json:"server_id,omitempty"` ++ MppVersion int64 `protobuf:"varint,9,opt,name=mpp_version,json=mppVersion,proto3" json:"mpp_version,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` -@@ -41,7 +118,7 @@ func (m *TaskMeta) Reset() { *m = TaskMeta{} } +@@ -45,7 +122,7 @@ func (m *TaskMeta) Reset() { *m = TaskMeta{} } func (m *TaskMeta) String() string { return proto.CompactTextString(m) } func (*TaskMeta) ProtoMessage() {} func (*TaskMeta) Descriptor() ([]byte, []int) { @@ -101,8 +102,8 @@ index 16fd5dd..63a6d6c 100644 } func (m *TaskMeta) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) -@@ -98,6 +175,13 @@ func (m *TaskMeta) GetAddress() string { - return "" +@@ -130,6 +207,13 @@ func (m *TaskMeta) GetServerId() uint64 { + return 0 } +func (m *TaskMeta) GetMppVersion() int64 { @@ -115,7 +116,7 @@ index 16fd5dd..63a6d6c 100644 type IsAliveRequest struct { XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` -@@ -108,7 +192,7 @@ func (m *IsAliveRequest) Reset() { *m = IsAliveRequest{} } +@@ -140,7 +224,7 @@ func (m *IsAliveRequest) Reset() { *m = IsAliveRequest{} } func (m *IsAliveRequest) String() string { return proto.CompactTextString(m) } func (*IsAliveRequest) ProtoMessage() {} func (*IsAliveRequest) Descriptor() ([]byte, []int) { @@ -124,7 +125,7 @@ index 16fd5dd..63a6d6c 100644 } func (m *IsAliveRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) -@@ -139,6 +223,7 @@ var xxx_messageInfo_IsAliveRequest proto.InternalMessageInfo +@@ -171,6 +255,7 @@ var xxx_messageInfo_IsAliveRequest proto.InternalMessageInfo type IsAliveResponse struct { Available bool `protobuf:"varint,1,opt,name=available,proto3" json:"available,omitempty"` @@ -132,7 +133,7 @@ index 16fd5dd..63a6d6c 100644 XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` -@@ -148,7 +233,7 @@ func (m *IsAliveResponse) Reset() { *m = IsAliveResponse{} } +@@ -180,7 +265,7 @@ func (m *IsAliveResponse) Reset() { *m = IsAliveResponse{} } func (m *IsAliveResponse) String() string { return proto.CompactTextString(m) } func (*IsAliveResponse) ProtoMessage() {} func (*IsAliveResponse) Descriptor() ([]byte, []int) { @@ -141,7 +142,7 @@ index 16fd5dd..63a6d6c 100644 } func (m *IsAliveResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) -@@ -184,6 +269,13 @@ func (m *IsAliveResponse) GetAvailable() bool { +@@ -216,6 +301,13 @@ func (m *IsAliveResponse) GetAvailable() bool { return false } @@ -155,7 +156,7 @@ index 16fd5dd..63a6d6c 100644 // Dipsatch the task request to different tiflash servers. type DispatchTaskRequest struct { Meta *TaskMeta `protobuf:"bytes,1,opt,name=meta,proto3" json:"meta,omitempty"` -@@ -193,17 +285,19 @@ type DispatchTaskRequest struct { +@@ -225,17 +317,19 @@ type DispatchTaskRequest struct { // If this task contains table scan, we still need their region info. SchemaVer int64 `protobuf:"varint,5,opt,name=schema_ver,json=schemaVer,proto3" json:"schema_ver,omitempty"` // Used for partition table scan @@ -180,7 +181,7 @@ index 16fd5dd..63a6d6c 100644 } func (m *DispatchTaskRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) -@@ -274,6 +368,13 @@ func (m *DispatchTaskRequest) GetTableRegions() []*coprocessor.TableRegions { +@@ -306,6 +400,13 @@ func (m *DispatchTaskRequest) GetTableRegions() []*coprocessor.TableRegions { return nil } @@ -194,7 +195,7 @@ index 16fd5dd..63a6d6c 100644 // Get response of DispatchTaskRequest. type DispatchTaskResponse struct { Error *Error `protobuf:"bytes,1,opt,name=error,proto3" json:"error,omitempty"` -@@ -287,7 +388,7 @@ func (m *DispatchTaskResponse) Reset() { *m = DispatchTaskResponse{} } +@@ -319,7 +420,7 @@ func (m *DispatchTaskResponse) Reset() { *m = DispatchTaskResponse{} } func (m *DispatchTaskResponse) String() string { return proto.CompactTextString(m) } func (*DispatchTaskResponse) ProtoMessage() {} func (*DispatchTaskResponse) Descriptor() ([]byte, []int) { @@ -203,7 +204,7 @@ index 16fd5dd..63a6d6c 100644 } func (m *DispatchTaskResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) -@@ -343,7 +444,7 @@ func (m *CancelTaskRequest) Reset() { *m = CancelTaskRequest{} } +@@ -375,7 +476,7 @@ func (m *CancelTaskRequest) Reset() { *m = CancelTaskRequest{} } func (m *CancelTaskRequest) String() string { return proto.CompactTextString(m) } func (*CancelTaskRequest) ProtoMessage() {} func (*CancelTaskRequest) Descriptor() ([]byte, []int) { @@ -212,7 +213,7 @@ index 16fd5dd..63a6d6c 100644 } func (m *CancelTaskRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) -@@ -397,7 +498,7 @@ func (m *CancelTaskResponse) Reset() { *m = CancelTaskResponse{} } +@@ -429,7 +530,7 @@ func (m *CancelTaskResponse) Reset() { *m = CancelTaskResponse{} } func (m *CancelTaskResponse) String() string { return proto.CompactTextString(m) } func (*CancelTaskResponse) ProtoMessage() {} func (*CancelTaskResponse) Descriptor() ([]byte, []int) { @@ -221,7 +222,7 @@ index 16fd5dd..63a6d6c 100644 } func (m *CancelTaskResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) -@@ -446,7 +547,7 @@ func (m *EstablishMPPConnectionRequest) Reset() { *m = EstablishMPPConne +@@ -478,7 +579,7 @@ func (m *EstablishMPPConnectionRequest) Reset() { *m = EstablishMPPConne func (m *EstablishMPPConnectionRequest) String() string { return proto.CompactTextString(m) } func (*EstablishMPPConnectionRequest) ProtoMessage() {} func (*EstablishMPPConnectionRequest) Descriptor() ([]byte, []int) { @@ -230,29 +231,29 @@ index 16fd5dd..63a6d6c 100644 } func (m *EstablishMPPConnectionRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) -@@ -489,23 +590,71 @@ func (m *EstablishMPPConnectionRequest) GetReceiverMeta() *TaskMeta { +@@ -521,23 +622,72 @@ func (m *EstablishMPPConnectionRequest) GetReceiverMeta() *TaskMeta { return nil } -+type CompressMeta struct { ++type DataPacketCompressMeta struct { + Method CompressMethod `protobuf:"varint,1,opt,name=method,proto3,enum=mpp.CompressMethod" json:"method,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + -+func (m *CompressMeta) Reset() { *m = CompressMeta{} } -+func (m *CompressMeta) String() string { return proto.CompactTextString(m) } -+func (*CompressMeta) ProtoMessage() {} -+func (*CompressMeta) Descriptor() ([]byte, []int) { ++func (m *DataPacketCompressMeta) Reset() { *m = DataPacketCompressMeta{} } ++func (m *DataPacketCompressMeta) String() string { return proto.CompactTextString(m) } ++func (*DataPacketCompressMeta) ProtoMessage() {} ++func (*DataPacketCompressMeta) Descriptor() ([]byte, []int) { + return fileDescriptor_819623c7fa76fc55, []int{9} +} -+func (m *CompressMeta) XXX_Unmarshal(b []byte) error { ++func (m *DataPacketCompressMeta) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} -+func (m *CompressMeta) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { ++func (m *DataPacketCompressMeta) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { -+ return xxx_messageInfo_CompressMeta.Marshal(b, m, deterministic) ++ return xxx_messageInfo_DataPacketCompressMeta.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) @@ -262,19 +263,19 @@ index 16fd5dd..63a6d6c 100644 + return b[:n], nil + } +} -+func (m *CompressMeta) XXX_Merge(src proto.Message) { -+ xxx_messageInfo_CompressMeta.Merge(m, src) ++func (m *DataPacketCompressMeta) XXX_Merge(src proto.Message) { ++ xxx_messageInfo_DataPacketCompressMeta.Merge(m, src) +} -+func (m *CompressMeta) XXX_Size() int { ++func (m *DataPacketCompressMeta) XXX_Size() int { + return m.Size() +} -+func (m *CompressMeta) XXX_DiscardUnknown() { -+ xxx_messageInfo_CompressMeta.DiscardUnknown(m) ++func (m *DataPacketCompressMeta) XXX_DiscardUnknown() { ++ xxx_messageInfo_DataPacketCompressMeta.DiscardUnknown(m) +} + -+var xxx_messageInfo_CompressMeta proto.InternalMessageInfo ++var xxx_messageInfo_DataPacketCompressMeta proto.InternalMessageInfo + -+func (m *CompressMeta) GetMethod() CompressMethod { ++func (m *DataPacketCompressMeta) GetMethod() CompressMethod { + if m != nil { + return m.Method + } @@ -291,14 +292,15 @@ index 16fd5dd..63a6d6c 100644 - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -+ Data []byte `protobuf:"bytes,1,opt,name=data,proto3" json:"data,omitempty"` -+ Error *Error `protobuf:"bytes,2,opt,name=error,proto3" json:"error,omitempty"` -+ Chunks [][]byte `protobuf:"bytes,3,rep,name=chunks,proto3" json:"chunks,omitempty"` -+ StreamIds []uint64 `protobuf:"varint,4,rep,packed,name=stream_ids,json=streamIds,proto3" json:"stream_ids,omitempty"` -+ Compress *CompressMeta `protobuf:"bytes,5,opt,name=compress,proto3" json:"compress,omitempty"` -+ XXX_NoUnkeyedLiteral struct{} `json:"-"` -+ XXX_unrecognized []byte `json:"-"` -+ XXX_sizecache int32 `json:"-"` ++ Data []byte `protobuf:"bytes,1,opt,name=data,proto3" json:"data,omitempty"` ++ Error *Error `protobuf:"bytes,2,opt,name=error,proto3" json:"error,omitempty"` ++ Chunks [][]byte `protobuf:"bytes,3,rep,name=chunks,proto3" json:"chunks,omitempty"` ++ StreamIds []uint64 `protobuf:"varint,4,rep,packed,name=stream_ids,json=streamIds,proto3" json:"stream_ids,omitempty"` ++ // data compression ++ Compress *DataPacketCompressMeta `protobuf:"bytes,5,opt,name=compress,proto3" json:"compress,omitempty"` ++ XXX_NoUnkeyedLiteral struct{} `json:"-"` ++ XXX_unrecognized []byte `json:"-"` ++ XXX_sizecache int32 `json:"-"` } func (m *MPPDataPacket) Reset() { *m = MPPDataPacket{} } @@ -310,11 +312,11 @@ index 16fd5dd..63a6d6c 100644 } func (m *MPPDataPacket) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) -@@ -562,6 +711,13 @@ func (m *MPPDataPacket) GetStreamIds() []uint64 { +@@ -594,6 +744,13 @@ func (m *MPPDataPacket) GetStreamIds() []uint64 { return nil } -+func (m *MPPDataPacket) GetCompress() *CompressMeta { ++func (m *MPPDataPacket) GetCompress() *DataPacketCompressMeta { + if m != nil { + return m.Compress + } @@ -324,7 +326,7 @@ index 16fd5dd..63a6d6c 100644 type Error struct { Code int32 `protobuf:"varint,1,opt,name=code,proto3" json:"code,omitempty"` Msg string `protobuf:"bytes,2,opt,name=msg,proto3" json:"msg,omitempty"` -@@ -574,7 +730,7 @@ func (m *Error) Reset() { *m = Error{} } +@@ -606,7 +763,7 @@ func (m *Error) Reset() { *m = Error{} } func (m *Error) String() string { return proto.CompactTextString(m) } func (*Error) ProtoMessage() {} func (*Error) Descriptor() ([]byte, []int) { @@ -333,7 +335,7 @@ index 16fd5dd..63a6d6c 100644 } func (m *Error) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) -@@ -618,6 +774,8 @@ func (m *Error) GetMsg() string { +@@ -650,6 +807,8 @@ func (m *Error) GetMsg() string { } func init() { @@ -342,105 +344,114 @@ index 16fd5dd..63a6d6c 100644 proto.RegisterType((*TaskMeta)(nil), "mpp.TaskMeta") proto.RegisterType((*IsAliveRequest)(nil), "mpp.IsAliveRequest") proto.RegisterType((*IsAliveResponse)(nil), "mpp.IsAliveResponse") -@@ -626,6 +784,7 @@ func init() { +@@ -658,6 +817,7 @@ func init() { proto.RegisterType((*CancelTaskRequest)(nil), "mpp.CancelTaskRequest") proto.RegisterType((*CancelTaskResponse)(nil), "mpp.CancelTaskResponse") proto.RegisterType((*EstablishMPPConnectionRequest)(nil), "mpp.EstablishMPPConnectionRequest") -+ proto.RegisterType((*CompressMeta)(nil), "mpp.CompressMeta") ++ proto.RegisterType((*DataPacketCompressMeta)(nil), "mpp.DataPacketCompressMeta") proto.RegisterType((*MPPDataPacket)(nil), "mpp.MPPDataPacket") proto.RegisterType((*Error)(nil), "mpp.Error") } -@@ -633,45 +792,86 @@ func init() { +@@ -665,49 +825,91 @@ func init() { func init() { proto.RegisterFile("mpp.proto", fileDescriptor_819623c7fa76fc55) } var fileDescriptor_819623c7fa76fc55 = []byte{ -- // 600 bytes of a gzipped FileDescriptorProto -- 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x54, 0xcf, 0x6e, 0xd3, 0x4e, -- 0x10, 0xfe, 0x39, 0xce, 0x9f, 0x66, 0xe2, 0xf4, 0xd7, 0x2e, 0x15, 0x4d, 0x2b, 0x1a, 0xa5, 0x3e, -- 0xe5, 0x82, 0x2b, 0x5a, 0x89, 0x23, 0x12, 0xb4, 0x3d, 0xe4, 0x50, 0x29, 0x5a, 0x55, 0x88, 0x9b, -- 0xb5, 0xb5, 0x87, 0x64, 0x15, 0x7b, 0xd7, 0xec, 0x6e, 0x2d, 0x21, 0xb8, 0x71, 0xe0, 0x15, 0x78, -- 0x24, 0x8e, 0x3c, 0x02, 0x2a, 0x2f, 0x82, 0x76, 0x6d, 0x87, 0x54, 0xa2, 0x02, 0x4e, 0x99, 0xef, -- 0x9b, 0xd9, 0x99, 0xf9, 0x3e, 0x4d, 0x0c, 0xfd, 0xbc, 0x28, 0xa2, 0x42, 0x49, 0x23, 0x89, 0x9f, -- 0x17, 0xc5, 0xe1, 0xde, 0x42, 0x2e, 0xa4, 0xc3, 0x27, 0x36, 0xaa, 0x52, 0x87, 0xbb, 0x89, 0xe5, -- 0x12, 0xd4, 0x5a, 0xaa, 0x9a, 0x0a, 0x72, 0x34, 0xac, 0xb8, 0xa9, 0x50, 0xf8, 0x01, 0xb6, 0xae, -- 0x99, 0x5e, 0x5d, 0xa1, 0x61, 0xe4, 0x00, 0xb6, 0xb4, 0x61, 0xca, 0xc4, 0x46, 0x8f, 0xbc, 0x89, -- 0x37, 0x6d, 0xd3, 0x9e, 0xc3, 0xd7, 0x9a, 0xec, 0x43, 0xcf, 0x30, 0xbd, 0x8a, 0x79, 0x3a, 0x6a, -- 0x4d, 0xbc, 0xa9, 0x4f, 0xbb, 0x16, 0xce, 0x52, 0x72, 0x0c, 0x41, 0xc1, 0x94, 0xe1, 0x86, 0x4b, -- 0x61, 0xb3, 0xbe, 0xcb, 0x0e, 0xd6, 0xdc, 0x2c, 0x25, 0x23, 0xe8, 0xb1, 0x34, 0x55, 0xa8, 0xf5, -- 0xa8, 0x3d, 0xf1, 0xa6, 0x7d, 0xda, 0xc0, 0x70, 0x07, 0xb6, 0x67, 0xfa, 0x65, 0xc6, 0x4b, 0xa4, -- 0xf8, 0xee, 0x16, 0xb5, 0x09, 0x4f, 0xe0, 0xff, 0x35, 0xa3, 0x0b, 0x29, 0x34, 0x92, 0x27, 0xd0, -- 0x67, 0x25, 0xe3, 0x19, 0xbb, 0xc9, 0xd0, 0xad, 0xb5, 0x45, 0x7f, 0x11, 0xe1, 0xe7, 0x16, 0x3c, -- 0xba, 0xe0, 0xba, 0x60, 0x26, 0x59, 0x5a, 0x21, 0x75, 0x23, 0x72, 0x0c, 0x6d, 0xab, 0xd3, 0x3d, -- 0x18, 0x9c, 0x0e, 0x23, 0xeb, 0x56, 0x23, 0x94, 0xba, 0x94, 0x5d, 0x1d, 0x45, 0x22, 0x53, 0x4c, -- 0xe3, 0x22, 0x63, 0xc2, 0x09, 0x0b, 0xe8, 0xa0, 0xe6, 0xe6, 0x19, 0x13, 0x76, 0x75, 0xc3, 0x73, -- 0x94, 0xb7, 0xa6, 0x16, 0xd6, 0x40, 0xf2, 0x0c, 0x7a, 0x0a, 0x17, 0x5c, 0x0a, 0x2b, 0xca, 0x9f, -- 0x0e, 0x4e, 0xf7, 0xa3, 0x4d, 0xab, 0xa9, 0xcb, 0xcd, 0xc4, 0x5b, 0x49, 0x9b, 0x3a, 0x72, 0x04, -- 0xa0, 0x93, 0x25, 0xe6, 0x2c, 0x2e, 0x51, 0x8d, 0x3a, 0xae, 0x5f, 0xbf, 0x62, 0x5e, 0xa3, 0x22, -- 0x2f, 0x60, 0x68, 0xac, 0xa4, 0xb8, 0xe9, 0xdb, 0x75, 0x7d, 0x0f, 0xee, 0xf5, 0xbd, 0xb6, 0x15, -- 0x55, 0x73, 0x4d, 0x03, 0xb3, 0x81, 0xc2, 0x1c, 0xf6, 0xee, 0x1b, 0x51, 0xfb, 0x37, 0x81, 0x0e, -- 0x2a, 0x25, 0x55, 0x6d, 0x05, 0x38, 0x2b, 0x2e, 0x2d, 0x43, 0xab, 0x04, 0x39, 0x83, 0xa1, 0x42, -- 0xa3, 0xde, 0xaf, 0x27, 0xb7, 0xdc, 0xe4, 0xed, 0xa8, 0xbe, 0x94, 0x6a, 0x02, 0x0d, 0x5c, 0x51, -- 0x33, 0xee, 0x0d, 0xec, 0x9e, 0x33, 0x91, 0x60, 0xf6, 0x8f, 0xae, 0xaf, 0xd7, 0x69, 0x3d, 0xb0, -- 0x4e, 0xf8, 0x1c, 0xc8, 0x66, 0xe7, 0xbf, 0x95, 0x11, 0x7e, 0xf2, 0xe0, 0xe8, 0x52, 0x5b, 0x4f, -- 0xb8, 0x5e, 0x5e, 0xcd, 0xe7, 0xe7, 0x52, 0x08, 0x4c, 0xec, 0x11, 0x36, 0xeb, 0x45, 0x30, 0xd0, -- 0x28, 0x52, 0x54, 0xf1, 0xc3, 0x5b, 0x42, 0x55, 0xe1, 0xfe, 0x10, 0xa7, 0xd6, 0x98, 0x04, 0x79, -- 0xd9, 0xbc, 0x68, 0xfd, 0xee, 0x45, 0xd0, 0xd4, 0x58, 0x14, 0x7e, 0x84, 0xe1, 0xd5, 0x7c, 0x7e, -- 0xc1, 0x0c, 0x9b, 0xb3, 0x64, 0x85, 0x86, 0x10, 0x68, 0xa7, 0xac, 0x9e, 0x16, 0x50, 0x17, 0xff, -- 0xd9, 0x04, 0xf2, 0x18, 0xba, 0xc9, 0xf2, 0x56, 0xac, 0xf4, 0xc8, 0x9f, 0xf8, 0xd3, 0x80, 0xd6, -- 0xc8, 0x1d, 0x91, 0x51, 0xc8, 0xf2, 0x98, 0xa7, 0xd5, 0xe9, 0xb5, 0x69, 0xbf, 0x62, 0x66, 0xa9, -- 0x0e, 0x9f, 0x42, 0xc7, 0xb5, 0xb1, 0x53, 0xed, 0x19, 0xbb, 0xa9, 0x1d, 0xea, 0x62, 0xb2, 0x03, -- 0x7e, 0xae, 0x17, 0x6e, 0x66, 0x9f, 0xda, 0xf0, 0x55, 0xf8, 0xf5, 0x6e, 0xec, 0x7d, 0xbb, 0x1b, -- 0x7b, 0xdf, 0xef, 0xc6, 0xde, 0x97, 0x1f, 0xe3, 0xff, 0x60, 0x47, 0xaa, 0x45, 0x64, 0xf8, 0xaa, -- 0x8c, 0x56, 0xa5, 0xfb, 0x42, 0xdc, 0x74, 0xdd, 0xcf, 0xd9, 0xcf, 0x00, 0x00, 0x00, 0xff, 0xff, -- 0xef, 0x8c, 0x66, 0x5e, 0x71, 0x04, 0x00, 0x00, -+ // 750 bytes of a gzipped FileDescriptorProto -+ 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x54, 0xcd, 0x6e, 0xdb, 0x38, -+ 0x10, 0x8e, 0x2c, 0xff, 0x8e, 0x65, 0xaf, 0xc3, 0x04, 0x1b, 0x27, 0xd8, 0x78, 0x1d, 0x9d, 0x8c, -+ 0x5d, 0xc4, 0x41, 0x9d, 0xa2, 0x97, 0x02, 0x05, 0xda, 0xc4, 0x07, 0x03, 0x75, 0x6a, 0x30, 0x46, -+ 0x50, 0xe4, 0x22, 0x30, 0x12, 0x6b, 0x0b, 0xb6, 0x44, 0x96, 0x64, 0x8c, 0xf6, 0xdc, 0x97, 0xe8, -+ 0xa1, 0x6f, 0xd0, 0x17, 0xe9, 0xb1, 0xe8, 0x13, 0x14, 0xe9, 0x8b, 0x14, 0xa4, 0x24, 0xc7, 0xf9, -+ 0xe9, 0xdf, 0x49, 0x9a, 0x6f, 0x86, 0xc3, 0x6f, 0x3e, 0xce, 0x0c, 0x54, 0x22, 0xce, 0xbb, 0x5c, -+ 0x30, 0xc5, 0x90, 0x1d, 0x71, 0xbe, 0xb3, 0x39, 0x61, 0x13, 0x66, 0xec, 0x03, 0xfd, 0x97, 0xb8, -+ 0x76, 0xd6, 0x7d, 0x8d, 0xf9, 0x54, 0x4a, 0x26, 0x52, 0xc8, 0x89, 0xa8, 0x22, 0xfc, 0x22, 0xb1, -+ 0xdc, 0x3e, 0xa0, 0xfe, 0x1b, 0x7f, 0x4a, 0xe2, 0x09, 0x3d, 0xa5, 0x71, 0x40, 0xc5, 0x90, 0x2a, -+ 0x82, 0x0e, 0xa0, 0xec, 0xb3, 0x88, 0x0b, 0x2a, 0x65, 0xd3, 0x6a, 0x5b, 0x9d, 0x7a, 0x6f, 0xa3, -+ 0xab, 0xef, 0x3b, 0x4a, 0xc1, 0x21, 0x55, 0x53, 0x16, 0xe0, 0x65, 0x90, 0xfb, 0xc1, 0x82, 0xf2, -+ 0x98, 0xc8, 0x99, 0x39, 0xbd, 0x0d, 0x65, 0xa9, 0x88, 0x50, 0x9e, 0x4a, 0x4e, 0xe7, 0x71, 0xc9, -+ 0xd8, 0x63, 0x89, 0xb6, 0xa0, 0xa4, 0x88, 0x9c, 0x79, 0x61, 0xd0, 0xcc, 0xb5, 0xad, 0x8e, 0x8d, -+ 0x8b, 0xda, 0x1c, 0x04, 0x68, 0x0f, 0x1c, 0x4e, 0x84, 0x0a, 0x55, 0xc8, 0x62, 0xed, 0xb5, 0x8d, -+ 0xb7, 0xba, 0xc4, 0x06, 0x01, 0x6a, 0x42, 0x89, 0x04, 0x81, 0xe1, 0x94, 0x6f, 0x5b, 0x9d, 0x0a, -+ 0xce, 0x4c, 0xf4, 0x2f, 0x54, 0x23, 0xce, 0xbd, 0x05, 0x15, 0x32, 0x64, 0x71, 0xb3, 0x60, 0xce, -+ 0x42, 0xc4, 0xf9, 0x59, 0x82, 0xb8, 0x0d, 0xa8, 0x0f, 0xe4, 0xd3, 0x79, 0xb8, 0xa0, 0x98, 0xbe, -+ 0xbe, 0xa4, 0x52, 0xb9, 0x23, 0xf8, 0x6b, 0x89, 0x48, 0xce, 0x62, 0x49, 0xd1, 0x3f, 0x50, 0x21, -+ 0x0b, 0x12, 0xce, 0xc9, 0xc5, 0x9c, 0x1a, 0xde, 0x65, 0x7c, 0x0d, 0xdc, 0xbe, 0x23, 0x77, 0xe7, -+ 0x8e, 0x2f, 0x39, 0xd8, 0x38, 0x0e, 0x25, 0x27, 0xca, 0x9f, 0x6a, 0x29, 0xd2, 0x9b, 0xd0, 0x1e, -+ 0xe4, 0xb5, 0xe2, 0x26, 0x63, 0xb5, 0x57, 0x33, 0x3a, 0x66, 0x52, 0x61, 0xe3, 0xd2, 0xc5, 0xd3, -+ 0xd8, 0x67, 0x01, 0x0d, 0x3c, 0x3e, 0x27, 0x49, 0x72, 0x07, 0x57, 0x53, 0x6c, 0x34, 0x27, 0xb1, -+ 0x2e, 0x5e, 0x85, 0x11, 0x65, 0x97, 0x2a, 0x95, 0x26, 0x33, 0xd1, 0x03, 0x28, 0x09, 0x3a, 0x09, -+ 0x59, 0xac, 0x65, 0xb1, 0x3b, 0xd5, 0xde, 0x56, 0x77, 0xf5, 0xd1, 0xb1, 0xf1, 0x0d, 0xe2, 0x57, -+ 0x0c, 0x67, 0x71, 0x68, 0x17, 0x40, 0xfa, 0x53, 0x1a, 0x11, 0x5d, 0x4e, 0x2a, 0x57, 0x25, 0x41, -+ 0xce, 0xa8, 0x40, 0x4f, 0xa0, 0xa6, 0x74, 0xcd, 0x5e, 0x96, 0xb7, 0x68, 0xf2, 0x6e, 0xdf, 0xc8, -+ 0x3b, 0xd6, 0x11, 0x49, 0x72, 0x89, 0x1d, 0xb5, 0x62, 0xa1, 0x01, 0x6c, 0xd2, 0xb4, 0xa7, 0x3c, -+ 0x69, 0x9a, 0xca, 0x33, 0x0a, 0x94, 0x8c, 0x02, 0x5b, 0x46, 0x81, 0xbb, 0x4d, 0x87, 0x11, 0xbd, -+ 0x83, 0xb9, 0x11, 0x6c, 0xde, 0xd4, 0x34, 0x7d, 0xab, 0x36, 0x14, 0xa8, 0x10, 0x4c, 0xa4, 0xaa, -+ 0x42, 0x92, 0x53, 0x23, 0x38, 0x71, 0xa0, 0x43, 0xa8, 0x09, 0xaa, 0xc4, 0xdb, 0x65, 0x11, 0x39, -+ 0x53, 0x44, 0xbd, 0x9b, 0xb6, 0x7f, 0x42, 0x16, 0x3b, 0x26, 0x28, 0x65, 0xee, 0xbe, 0x84, 0xf5, -+ 0x23, 0x12, 0xfb, 0x74, 0xfe, 0x87, 0x0f, 0xb8, 0xa4, 0x93, 0xfb, 0x01, 0x1d, 0xf7, 0x11, 0xa0, -+ 0xd5, 0xcc, 0xbf, 0x5b, 0x86, 0xfb, 0xce, 0x82, 0xdd, 0xbe, 0xd4, 0xf2, 0x86, 0x72, 0x3a, 0x1c, -+ 0x8d, 0x8e, 0x58, 0x1c, 0x53, 0x5f, 0x4f, 0x44, 0x46, 0xaf, 0x0b, 0xd5, 0x55, 0x91, 0xef, 0x65, -+ 0x09, 0xf2, 0x7a, 0xb6, 0x7b, 0x5a, 0x18, 0x9f, 0x86, 0x8b, 0xec, 0x44, 0xee, 0xbe, 0x13, 0x4e, -+ 0x16, 0x63, 0x9e, 0xe1, 0x31, 0x38, 0x2b, 0xa3, 0x4f, 0xd0, 0xff, 0x50, 0x8c, 0xcc, 0x0a, 0xf8, -+ 0xd9, 0x76, 0x48, 0x43, 0xdc, 0x8f, 0x16, 0xd4, 0x86, 0xa3, 0xd1, 0x31, 0x51, 0x64, 0x44, 0xfc, -+ 0x19, 0x55, 0x08, 0x41, 0x3e, 0x20, 0x29, 0x57, 0x07, 0x9b, 0xff, 0x5f, 0x4b, 0x88, 0xfe, 0x86, -+ 0xa2, 0x3f, 0xbd, 0x8c, 0x67, 0xb2, 0x69, 0xb7, 0xed, 0x8e, 0x83, 0x53, 0xcb, 0x74, 0xb3, 0x12, -+ 0x94, 0x44, 0x5e, 0x18, 0x24, 0x33, 0x90, 0xc7, 0x95, 0x04, 0x19, 0x04, 0x12, 0xed, 0xaf, 0xec, -+ 0xb2, 0x82, 0xc9, 0xbd, 0x7e, 0x9b, 0x2d, 0x59, 0xd9, 0x64, 0xfb, 0x50, 0x30, 0xb7, 0x6a, 0x92, -+ 0x7a, 0xfc, 0x0c, 0xc9, 0x02, 0x36, 0xff, 0xa8, 0x01, 0x76, 0x24, 0x27, 0x86, 0x62, 0x05, 0xeb, -+ 0xdf, 0xff, 0xf6, 0xa1, 0x7e, 0xb3, 0x6c, 0x54, 0x86, 0xfc, 0xc9, 0x8b, 0x93, 0x7e, 0x63, 0x0d, -+ 0x95, 0xc0, 0x7e, 0x7e, 0xfe, 0xb0, 0x61, 0x69, 0xe8, 0xfc, 0x74, 0x7c, 0xdc, 0xc8, 0x3d, 0x73, -+ 0x3f, 0x5d, 0xb5, 0xac, 0xcf, 0x57, 0x2d, 0xeb, 0xeb, 0x55, 0xcb, 0x7a, 0xff, 0xad, 0xb5, 0x06, -+ 0x0d, 0x26, 0x26, 0x5d, 0x15, 0xce, 0x16, 0xdd, 0xd9, 0xc2, 0xac, 0xe4, 0x8b, 0xa2, 0xf9, 0x1c, -+ 0x7e, 0x0f, 0x00, 0x00, 0xff, 0xff, 0x91, 0xaa, 0x0c, 0xe0, 0xe2, 0x05, 0x00, 0x00, +- // 664 bytes of a gzipped FileDescriptorProto +- 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x54, 0xcd, 0x6e, 0xd3, 0x4c, +- 0x14, 0xfd, 0x1c, 0xe7, 0xcf, 0x37, 0x4e, 0xbf, 0x76, 0xbe, 0xea, 0x6b, 0x5a, 0x68, 0x94, 0x5a, +- 0x2c, 0xb2, 0xc1, 0x15, 0xad, 0xc4, 0x12, 0x09, 0xda, 0x2e, 0xbc, 0xa8, 0x14, 0x46, 0x11, 0x62, +- 0x67, 0x4d, 0x3d, 0x43, 0x62, 0xc5, 0xf6, 0xb8, 0x33, 0x13, 0x4b, 0x95, 0xd8, 0xb1, 0xe0, 0x15, +- 0x78, 0x24, 0x96, 0x3c, 0x02, 0x2a, 0xef, 0xc0, 0x1a, 0xcd, 0x8c, 0x1d, 0x5a, 0x89, 0x0a, 0x58, +- 0x65, 0xce, 0xb9, 0x77, 0xce, 0xb9, 0x3e, 0xbe, 0x31, 0x78, 0x79, 0x59, 0x86, 0xa5, 0xe0, 0x8a, +- 0x23, 0x37, 0x2f, 0xcb, 0x83, 0xdd, 0x05, 0x5f, 0x70, 0x83, 0x8f, 0xf5, 0xc9, 0x96, 0x0e, 0x76, +- 0x12, 0xcd, 0x25, 0x4c, 0x4a, 0x2e, 0x6a, 0xca, 0xcf, 0x99, 0x22, 0xe5, 0x95, 0x45, 0xc1, 0x77, +- 0x07, 0xfa, 0x73, 0x22, 0x57, 0x97, 0x4c, 0x11, 0xb4, 0x0f, 0x7d, 0xa9, 0x88, 0x50, 0xb1, 0x92, +- 0x23, 0x67, 0xe2, 0x4c, 0xdb, 0xb8, 0x67, 0xf0, 0x5c, 0xa2, 0x3d, 0xe8, 0x29, 0x22, 0x57, 0x71, +- 0x4a, 0x47, 0xad, 0x89, 0x33, 0x75, 0x71, 0x57, 0xc3, 0x88, 0xa2, 0x23, 0xf0, 0x4b, 0x22, 0x54, +- 0xaa, 0x52, 0x5e, 0xe8, 0xaa, 0x6b, 0xaa, 0x83, 0x0d, 0x17, 0x51, 0x34, 0x82, 0x1e, 0xa1, 0x54, +- 0x30, 0x29, 0x47, 0xed, 0x89, 0x33, 0xf5, 0x70, 0x03, 0xd1, 0x23, 0xf0, 0x16, 0x44, 0x2d, 0x99, +- 0xd0, 0x37, 0x3b, 0xc6, 0xb1, 0x6f, 0x89, 0x88, 0xea, 0x69, 0xae, 0xd7, 0x4c, 0xdc, 0xe8, 0x69, +- 0xba, 0x76, 0x1a, 0x83, 0xe7, 0x12, 0x3d, 0x81, 0xad, 0x8c, 0x27, 0x24, 0x8b, 0x6d, 0x43, 0x4a, +- 0x47, 0x3d, 0xd3, 0xe0, 0x1b, 0xf6, 0xb5, 0x26, 0x23, 0xaa, 0xd5, 0x25, 0x13, 0x95, 0x55, 0xef, +- 0x5b, 0x75, 0x4b, 0x44, 0x34, 0xd8, 0x86, 0xad, 0x48, 0xbe, 0xcc, 0xd2, 0x8a, 0x61, 0x76, 0xbd, +- 0x66, 0x52, 0x05, 0xc7, 0xf0, 0xef, 0x86, 0x91, 0x25, 0x2f, 0x24, 0x43, 0x8f, 0xc1, 0x23, 0x15, +- 0x49, 0x33, 0x72, 0x95, 0x31, 0x93, 0x48, 0x1f, 0xff, 0x24, 0x82, 0x8f, 0x2d, 0xf8, 0xef, 0x3c, +- 0x95, 0x25, 0x51, 0xc9, 0x52, 0x67, 0x58, 0x0b, 0xa1, 0x23, 0x68, 0xeb, 0x8c, 0xcd, 0x85, 0xc1, +- 0xc9, 0x30, 0xd4, 0x6f, 0xaa, 0xc9, 0x18, 0x9b, 0x92, 0x4e, 0x8d, 0x15, 0x09, 0xa7, 0x8c, 0xc6, +- 0x65, 0x46, 0x0a, 0x93, 0xa9, 0x8f, 0x07, 0x35, 0x37, 0xcb, 0x48, 0xa1, 0x53, 0x53, 0x69, 0xce, +- 0xf8, 0x5a, 0xd5, 0x99, 0x36, 0x10, 0x3d, 0x83, 0x9e, 0x60, 0x8b, 0x94, 0x17, 0x3a, 0x4f, 0x77, +- 0x3a, 0x38, 0xd9, 0x0b, 0xef, 0xbe, 0x66, 0x6c, 0x6a, 0x51, 0xf1, 0x8e, 0xe3, 0xa6, 0x0f, 0x1d, +- 0x02, 0xc8, 0x64, 0xc9, 0x72, 0x12, 0x57, 0x4c, 0x98, 0xa4, 0x5d, 0xec, 0x59, 0xe6, 0x0d, 0x13, +- 0xe8, 0x05, 0x0c, 0x95, 0x7e, 0xa4, 0xb8, 0xd1, 0xed, 0x1a, 0xdd, 0xfd, 0x7b, 0xba, 0x73, 0xdd, +- 0x61, 0xc5, 0x25, 0xf6, 0xd5, 0x1d, 0x14, 0xe4, 0xb0, 0x7b, 0x3f, 0x88, 0x3a, 0xbf, 0x09, 0x74, +- 0x98, 0x10, 0x5c, 0xd4, 0x51, 0x80, 0x89, 0xe2, 0x42, 0x33, 0xd8, 0x16, 0xd0, 0x29, 0x0c, 0x05, +- 0x53, 0xe2, 0x66, 0xe3, 0xdc, 0x32, 0xce, 0x5b, 0x61, 0xbd, 0xa5, 0xd6, 0x01, 0xfb, 0xa6, 0xa9, +- 0xb1, 0x7b, 0x0b, 0x3b, 0x67, 0xa4, 0x48, 0x58, 0xf6, 0x97, 0xa9, 0x6f, 0xc6, 0x69, 0x3d, 0x30, +- 0x4e, 0xf0, 0x1c, 0xd0, 0x5d, 0xe5, 0x3f, 0x7d, 0x8c, 0xe0, 0x83, 0x03, 0x87, 0x17, 0x52, 0x67, +- 0x92, 0xca, 0xe5, 0xe5, 0x6c, 0x76, 0xc6, 0x8b, 0x82, 0x25, 0x7a, 0xff, 0x9b, 0xf1, 0x42, 0x18, +- 0x48, 0x56, 0x50, 0x26, 0xe2, 0x87, 0xa7, 0x04, 0xdb, 0x61, 0xfe, 0x8b, 0x27, 0x3a, 0x98, 0x84, +- 0xa5, 0x55, 0x73, 0xa3, 0xf5, 0xab, 0x1b, 0x7e, 0xd3, 0xa3, 0x51, 0xf0, 0x1e, 0x86, 0x97, 0xb3, +- 0xd9, 0x39, 0x51, 0x64, 0x46, 0x92, 0x15, 0x53, 0x08, 0x41, 0x9b, 0x92, 0xda, 0xcd, 0xc7, 0xe6, +- 0xfc, 0xfb, 0x10, 0xd0, 0xff, 0xd0, 0x4d, 0x96, 0xeb, 0x62, 0x25, 0x47, 0xee, 0xc4, 0x9d, 0xfa, +- 0xb8, 0x46, 0x66, 0x89, 0x94, 0x60, 0x24, 0x8f, 0x53, 0x6a, 0x57, 0xaf, 0x8d, 0x3d, 0xcb, 0x44, +- 0x54, 0x06, 0x4f, 0xa1, 0x63, 0x64, 0xb4, 0xab, 0x5e, 0x63, 0xe3, 0xda, 0xc1, 0xe6, 0x8c, 0xb6, +- 0xc1, 0xcd, 0xe5, 0xc2, 0x78, 0x7a, 0x58, 0x1f, 0x5f, 0x05, 0x9f, 0x6f, 0xc7, 0xce, 0x97, 0xdb, +- 0xb1, 0xf3, 0xf5, 0x76, 0xec, 0x7c, 0xfa, 0x36, 0xfe, 0x07, 0xb6, 0xb9, 0x58, 0x84, 0x2a, 0x5d, +- 0x55, 0xe1, 0xaa, 0x32, 0x5f, 0xa7, 0xab, 0xae, 0xf9, 0x39, 0xfd, 0x11, 0x00, 0x00, 0xff, 0xff, +- 0xcd, 0x41, 0xdb, 0xe2, 0xed, 0x04, 0x00, 0x00, ++ // 820 bytes of a gzipped FileDescriptorProto ++ 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x55, 0xcd, 0x8e, 0x1b, 0x45, ++ 0x10, 0xce, 0x78, 0xfc, 0x5b, 0x9e, 0x35, 0x4e, 0x67, 0x95, 0x75, 0x12, 0xb2, 0x38, 0x23, 0x0e, ++ 0x16, 0x28, 0x8e, 0xd8, 0x20, 0xb8, 0x21, 0xc1, 0xae, 0x0f, 0x23, 0xb1, 0xc1, 0x74, 0xac, 0x08, ++ 0xe5, 0x32, 0xea, 0x9d, 0x29, 0xec, 0x91, 0x3d, 0xd3, 0x93, 0xee, 0xb6, 0x45, 0xce, 0x3c, 0x04, ++ 0xbc, 0x08, 0xef, 0xc0, 0x11, 0xf1, 0x04, 0x68, 0x79, 0x11, 0xd4, 0xd5, 0x63, 0xaf, 0x97, 0x5d, ++ 0x7e, 0x72, 0xf2, 0xd4, 0xd7, 0xd5, 0x5f, 0x7f, 0xf5, 0x55, 0x77, 0x19, 0x3a, 0x79, 0x59, 0x8e, ++ 0x4b, 0x25, 0x8d, 0x64, 0x7e, 0x5e, 0x96, 0x0f, 0x0f, 0xe7, 0x72, 0x2e, 0x29, 0x7e, 0x66, 0xbf, ++ 0xdc, 0xd2, 0xc3, 0xbb, 0x89, 0xc5, 0x12, 0xd4, 0x5a, 0xaa, 0x0a, 0x0a, 0x72, 0x34, 0xa2, 0xbc, ++ 0x70, 0x51, 0x38, 0x01, 0x36, 0xf9, 0x21, 0x59, 0x88, 0x62, 0x8e, 0x2f, 0xb1, 0x48, 0x51, 0x9d, ++ 0xa3, 0x11, 0xec, 0x19, 0xb4, 0x13, 0x99, 0x97, 0x0a, 0xb5, 0x1e, 0x78, 0x43, 0x6f, 0xd4, 0x3b, ++ 0xb9, 0x37, 0xb6, 0xe7, 0x9d, 0x56, 0xe0, 0x39, 0x9a, 0x85, 0x4c, 0xf9, 0x2e, 0x29, 0xfc, 0xa9, ++ 0x06, 0xed, 0x99, 0xd0, 0x4b, 0xda, 0xfd, 0x00, 0xda, 0xda, 0x08, 0x65, 0x62, 0xe3, 0x76, 0xd7, ++ 0x79, 0x8b, 0xe2, 0x99, 0x66, 0x47, 0xd0, 0x32, 0x42, 0x2f, 0xe3, 0x2c, 0x1d, 0xd4, 0x86, 0xde, ++ 0xc8, 0xe7, 0x4d, 0x1b, 0x46, 0x29, 0x7b, 0x02, 0x41, 0x29, 0x94, 0xc9, 0x4c, 0x26, 0x0b, 0xbb, ++ 0xea, 0xd3, 0x6a, 0x77, 0x87, 0x45, 0x29, 0x1b, 0x40, 0x4b, 0xa4, 0x29, 0x69, 0xaa, 0x0f, 0xbd, ++ 0x51, 0x87, 0x6f, 0x43, 0xf6, 0x08, 0x3a, 0x73, 0x61, 0x16, 0xa8, 0xec, 0xce, 0x06, 0x9d, 0xd8, ++ 0x76, 0x40, 0x94, 0x5a, 0x35, 0x6f, 0xd6, 0xa8, 0xde, 0x5a, 0x35, 0x4d, 0xa7, 0x86, 0xe2, 0x99, ++ 0x66, 0x1f, 0x42, 0x6f, 0x25, 0x13, 0xb1, 0x8a, 0x5d, 0x42, 0x96, 0x0e, 0x5a, 0x94, 0x10, 0x10, ++ 0xfa, 0xad, 0x05, 0xa3, 0xd4, 0xb2, 0x6b, 0x54, 0x1b, 0xc7, 0xde, 0x76, 0xec, 0x0e, 0x88, 0x52, ++ 0xf6, 0x01, 0x74, 0xf3, 0xb2, 0x8c, 0x37, 0xa8, 0x74, 0x26, 0x8b, 0x41, 0x87, 0x64, 0x43, 0x5e, ++ 0x96, 0xaf, 0x1c, 0x12, 0xf6, 0xa1, 0x17, 0xe9, 0x2f, 0x57, 0xd9, 0x06, 0x39, 0xbe, 0x59, 0xa3, ++ 0x36, 0xe1, 0x14, 0xde, 0xdb, 0x21, 0xba, 0x94, 0x85, 0x46, 0xf6, 0x3e, 0x74, 0xc4, 0x46, 0x64, ++ 0x2b, 0x71, 0xb1, 0x42, 0xb2, 0xac, 0xcd, 0xaf, 0x80, 0xbf, 0x9f, 0x51, 0xbb, 0x71, 0xc6, 0xef, ++ 0x35, 0xb8, 0x77, 0x96, 0xe9, 0x52, 0x98, 0x64, 0x61, 0xbb, 0x50, 0x9d, 0xc4, 0x9e, 0x40, 0xdd, ++ 0x36, 0x9b, 0x18, 0xbb, 0x27, 0x07, 0xd4, 0xc2, 0x6d, 0x97, 0x38, 0x2d, 0x59, 0xdf, 0xb1, 0x48, ++ 0x64, 0x8a, 0x69, 0x5c, 0xae, 0x84, 0x23, 0x0f, 0x78, 0xb7, 0xc2, 0xa6, 0x2b, 0x51, 0x58, 0xdf, ++ 0x4d, 0x96, 0xa3, 0x5c, 0x9b, 0xaa, 0x2b, 0xdb, 0x90, 0x7d, 0x02, 0x2d, 0x85, 0xf3, 0x4c, 0x16, ++ 0xb6, 0x23, 0xfe, 0xa8, 0x7b, 0x72, 0x34, 0xde, 0xbf, 0x6f, 0x9c, 0xd6, 0xa2, 0xe2, 0x7b, 0xc9, ++ 0xb7, 0x79, 0xec, 0x31, 0x80, 0x4e, 0x16, 0x98, 0x0b, 0x5b, 0x0e, 0xf5, 0xca, 0xe7, 0x1d, 0x87, ++ 0xbc, 0x42, 0xc5, 0xbe, 0x80, 0x03, 0x63, 0x6b, 0x8e, 0xb7, 0xbc, 0x4d, 0xe2, 0x7d, 0x70, 0x8d, ++ 0x77, 0x66, 0x33, 0x1c, 0xb9, 0xe6, 0x81, 0xd9, 0x8b, 0x58, 0x04, 0x87, 0x58, 0x5d, 0xe7, 0x58, ++ 0xd3, 0x7d, 0x8e, 0xc9, 0x81, 0x16, 0x39, 0x70, 0x44, 0x0e, 0xdc, 0xbc, 0xef, 0x9c, 0xe1, 0x0d, ++ 0x2c, 0xcc, 0xe1, 0xf0, 0xba, 0xa7, 0x55, 0xaf, 0x86, 0xd0, 0x40, 0xa5, 0xa4, 0xaa, 0x5c, 0x05, ++ 0xc7, 0x69, 0x11, 0xee, 0x16, 0xd8, 0x73, 0x38, 0x50, 0x68, 0xd4, 0xdb, 0x5d, 0x11, 0x35, 0x2a, ++ 0xa2, 0x37, 0xae, 0x5e, 0x9e, 0x13, 0xcb, 0x03, 0x4a, 0xaa, 0x94, 0x87, 0xdf, 0xc1, 0xdd, 0x53, ++ 0x51, 0x24, 0xb8, 0x7a, 0xc7, 0x06, 0xee, 0xe4, 0xd4, 0xfe, 0x41, 0x4e, 0xf8, 0x19, 0xb0, 0x7d, ++ 0xe6, 0xff, 0x5b, 0x46, 0xf8, 0xa3, 0x07, 0x8f, 0x27, 0xda, 0xda, 0x9b, 0xe9, 0xc5, 0xf9, 0x74, ++ 0x7a, 0x2a, 0x8b, 0x02, 0x13, 0xfb, 0x18, 0xb7, 0xf2, 0xc6, 0xd0, 0xdd, 0x37, 0xf9, 0x56, 0x95, ++ 0xa0, 0xaf, 0xc6, 0xca, 0x89, 0x35, 0x26, 0xc1, 0x6c, 0xb3, 0xdd, 0x51, 0xbb, 0x6d, 0x47, 0xb0, ++ 0xcd, 0xa1, 0x36, 0x4c, 0xe0, 0xfe, 0x99, 0x30, 0x62, 0x2a, 0x92, 0x25, 0x9a, 0xbd, 0xf9, 0x23, ++ 0xd8, 0xc7, 0xd0, 0xcc, 0x69, 0x0e, 0xfd, 0xdb, 0x88, 0xaa, 0x52, 0xc2, 0x5f, 0x3c, 0x38, 0x38, ++ 0x9f, 0x4e, 0xaf, 0xa8, 0x18, 0x83, 0x7a, 0x2a, 0x2a, 0xd5, 0x01, 0xa7, 0xef, 0xff, 0x36, 0x93, ++ 0xdd, 0x87, 0x66, 0xb2, 0x58, 0x17, 0x4b, 0x3d, 0xf0, 0x87, 0xfe, 0x28, 0xe0, 0x55, 0x44, 0xf7, ++ 0xda, 0x28, 0x14, 0x79, 0x9c, 0xa5, 0xee, 0x35, 0xd4, 0x79, 0xc7, 0x21, 0x51, 0xaa, 0xd9, 0xe7, ++ 0x7b, 0x03, 0xb5, 0x41, 0xdc, 0x8f, 0x88, 0xfb, 0xf6, 0xd2, 0xf6, 0x06, 0xeb, 0x53, 0x68, 0xd0, ++ 0xf9, 0x56, 0xae, 0x7d, 0x92, 0x24, 0xb7, 0xc1, 0xe9, 0x9b, 0xf5, 0xc1, 0xcf, 0xf5, 0x9c, 0xc4, ++ 0x76, 0xb8, 0xfd, 0xfc, 0xe8, 0x29, 0xf4, 0xae, 0x1b, 0xc0, 0xda, 0x50, 0x7f, 0xf1, 0xcd, 0x8b, ++ 0x49, 0xff, 0x0e, 0x6b, 0x81, 0xff, 0xf5, 0xeb, 0x4f, 0xfb, 0x9e, 0x85, 0x5e, 0xbf, 0x9c, 0x9d, ++ 0xf5, 0x6b, 0x5f, 0x85, 0xbf, 0x5e, 0x1e, 0x7b, 0xbf, 0x5d, 0x1e, 0x7b, 0x7f, 0x5c, 0x1e, 0x7b, ++ 0x3f, 0xff, 0x79, 0x7c, 0x07, 0xfa, 0x52, 0xcd, 0xc7, 0x26, 0x5b, 0x6e, 0xc6, 0xcb, 0x0d, 0xfd, ++ 0x43, 0x5c, 0x34, 0xe9, 0xe7, 0xf9, 0x5f, 0x01, 0x00, 0x00, 0xff, 0xff, 0x50, 0xee, 0x3d, 0x86, ++ 0x71, 0x06, 0x00, 0x00, +} + +func (m *ExchangeSenderMeta) Marshal() (dAtA []byte, err error) { @@ -476,19 +487,19 @@ index 16fd5dd..63a6d6c 100644 } func (m *TaskMeta) Marshal() (dAtA []byte, err error) { -@@ -698,6 +898,11 @@ func (m *TaskMeta) MarshalToSizedBuffer(dAtA []byte) (int, error) { +@@ -734,6 +936,11 @@ func (m *TaskMeta) MarshalToSizedBuffer(dAtA []byte) (int, error) { i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } + if m.MppVersion != 0 { + i = encodeVarintMpp(dAtA, i, uint64(m.MppVersion)) + i-- -+ dAtA[i] = 0x28 ++ dAtA[i] = 0x48 + } - if len(m.Address) > 0 { - i -= len(m.Address) - copy(dAtA[i:], m.Address) -@@ -774,6 +979,11 @@ func (m *IsAliveResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { + if m.ServerId != 0 { + i = encodeVarintMpp(dAtA, i, uint64(m.ServerId)) + i-- +@@ -830,6 +1037,11 @@ func (m *IsAliveResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } @@ -500,7 +511,7 @@ index 16fd5dd..63a6d6c 100644 if m.Available { i-- if m.Available { -@@ -811,6 +1021,18 @@ func (m *DispatchTaskRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { +@@ -867,6 +1079,18 @@ func (m *DispatchTaskRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } @@ -519,11 +530,11 @@ index 16fd5dd..63a6d6c 100644 if len(m.TableRegions) > 0 { for iNdEx := len(m.TableRegions) - 1; iNdEx >= 0; iNdEx-- { { -@@ -1065,6 +1287,38 @@ func (m *EstablishMPPConnectionRequest) MarshalToSizedBuffer(dAtA []byte) (int, +@@ -1121,6 +1345,38 @@ func (m *EstablishMPPConnectionRequest) MarshalToSizedBuffer(dAtA []byte) (int, return len(dAtA) - i, nil } -+func (m *CompressMeta) Marshal() (dAtA []byte, err error) { ++func (m *DataPacketCompressMeta) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) @@ -533,12 +544,12 @@ index 16fd5dd..63a6d6c 100644 + return dAtA[:n], nil +} + -+func (m *CompressMeta) MarshalTo(dAtA []byte) (int, error) { ++func (m *DataPacketCompressMeta) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + -+func (m *CompressMeta) MarshalToSizedBuffer(dAtA []byte) (int, error) { ++func (m *DataPacketCompressMeta) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int @@ -558,7 +569,7 @@ index 16fd5dd..63a6d6c 100644 func (m *MPPDataPacket) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) -@@ -1089,21 +1343,33 @@ func (m *MPPDataPacket) MarshalToSizedBuffer(dAtA []byte) (int, error) { +@@ -1145,21 +1401,33 @@ func (m *MPPDataPacket) MarshalToSizedBuffer(dAtA []byte) (int, error) { i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } @@ -601,7 +612,7 @@ index 16fd5dd..63a6d6c 100644 i-- dAtA[i] = 0x22 } -@@ -1188,6 +1454,21 @@ func encodeVarintMpp(dAtA []byte, offset int, v uint64) int { +@@ -1244,6 +1512,21 @@ func encodeVarintMpp(dAtA []byte, offset int, v uint64) int { dAtA[offset] = uint8(v) return base } @@ -623,9 +634,9 @@ index 16fd5dd..63a6d6c 100644 func (m *TaskMeta) Size() (n int) { if m == nil { return 0 -@@ -1207,6 +1488,9 @@ func (m *TaskMeta) Size() (n int) { - if l > 0 { - n += 1 + l + sovMpp(uint64(l)) +@@ -1275,6 +1558,9 @@ func (m *TaskMeta) Size() (n int) { + if m.ServerId != 0 { + n += 1 + sovMpp(uint64(m.ServerId)) } + if m.MppVersion != 0 { + n += 1 + sovMpp(uint64(m.MppVersion)) @@ -633,7 +644,7 @@ index 16fd5dd..63a6d6c 100644 if m.XXX_unrecognized != nil { n += len(m.XXX_unrecognized) } -@@ -1234,6 +1518,9 @@ func (m *IsAliveResponse) Size() (n int) { +@@ -1302,6 +1588,9 @@ func (m *IsAliveResponse) Size() (n int) { if m.Available { n += 2 } @@ -643,7 +654,7 @@ index 16fd5dd..63a6d6c 100644 if m.XXX_unrecognized != nil { n += len(m.XXX_unrecognized) } -@@ -1272,6 +1559,10 @@ func (m *DispatchTaskRequest) Size() (n int) { +@@ -1340,6 +1629,10 @@ func (m *DispatchTaskRequest) Size() (n int) { n += 1 + l + sovMpp(uint64(l)) } } @@ -654,11 +665,11 @@ index 16fd5dd..63a6d6c 100644 if m.XXX_unrecognized != nil { n += len(m.XXX_unrecognized) } -@@ -1356,6 +1647,21 @@ func (m *EstablishMPPConnectionRequest) Size() (n int) { +@@ -1424,6 +1717,21 @@ func (m *EstablishMPPConnectionRequest) Size() (n int) { return n } -+func (m *CompressMeta) Size() (n int) { ++func (m *DataPacketCompressMeta) Size() (n int) { + if m == nil { + return 0 + } @@ -676,7 +687,7 @@ index 16fd5dd..63a6d6c 100644 func (m *MPPDataPacket) Size() (n int) { if m == nil { return 0 -@@ -1383,6 +1689,10 @@ func (m *MPPDataPacket) Size() (n int) { +@@ -1451,6 +1759,10 @@ func (m *MPPDataPacket) Size() (n int) { } n += 1 + sovMpp(uint64(l)) + l } @@ -687,7 +698,7 @@ index 16fd5dd..63a6d6c 100644 if m.XXX_unrecognized != nil { n += len(m.XXX_unrecognized) } -@@ -1414,6 +1724,76 @@ func sovMpp(x uint64) (n int) { +@@ -1482,6 +1794,76 @@ func sovMpp(x uint64) (n int) { func sozMpp(x uint64) (n int) { return sovMpp(uint64((x << 1) ^ uint64((int64(x) >> 63)))) } @@ -764,11 +775,11 @@ index 16fd5dd..63a6d6c 100644 func (m *TaskMeta) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 -@@ -1532,6 +1912,25 @@ func (m *TaskMeta) Unmarshal(dAtA []byte) error { +@@ -1676,6 +2058,25 @@ func (m *TaskMeta) Unmarshal(dAtA []byte) error { + break + } } - m.Address = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex -+ case 5: ++ case 9: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field MppVersion", wireType) + } @@ -790,7 +801,7 @@ index 16fd5dd..63a6d6c 100644 default: iNdEx = preIndex skippy, err := skipMpp(dAtA[iNdEx:]) -@@ -1654,6 +2053,25 @@ func (m *IsAliveResponse) Unmarshal(dAtA []byte) error { +@@ -1798,6 +2199,25 @@ func (m *IsAliveResponse) Unmarshal(dAtA []byte) error { } } m.Available = bool(v != 0) @@ -816,7 +827,7 @@ index 16fd5dd..63a6d6c 100644 default: iNdEx = preIndex skippy, err := skipMpp(dAtA[iNdEx:]) -@@ -1881,6 +2299,42 @@ func (m *DispatchTaskRequest) Unmarshal(dAtA []byte) error { +@@ -2025,6 +2445,42 @@ func (m *DispatchTaskRequest) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex @@ -859,11 +870,11 @@ index 16fd5dd..63a6d6c 100644 default: iNdEx = preIndex skippy, err := skipMpp(dAtA[iNdEx:]) -@@ -2357,6 +2811,76 @@ func (m *EstablishMPPConnectionRequest) Unmarshal(dAtA []byte) error { +@@ -2501,6 +2957,76 @@ func (m *EstablishMPPConnectionRequest) Unmarshal(dAtA []byte) error { } return nil } -+func (m *CompressMeta) Unmarshal(dAtA []byte) error { ++func (m *DataPacketCompressMeta) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { @@ -886,10 +897,10 @@ index 16fd5dd..63a6d6c 100644 + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { -+ return fmt.Errorf("proto: CompressMeta: wiretype end group for non-group") ++ return fmt.Errorf("proto: DataPacketCompressMeta: wiretype end group for non-group") + } + if fieldNum <= 0 { -+ return fmt.Errorf("proto: CompressMeta: illegal tag %d (wire type %d)", fieldNum, wire) ++ return fmt.Errorf("proto: DataPacketCompressMeta: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: @@ -936,7 +947,7 @@ index 16fd5dd..63a6d6c 100644 func (m *MPPDataPacket) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 -@@ -2564,6 +3088,42 @@ func (m *MPPDataPacket) Unmarshal(dAtA []byte) error { +@@ -2708,6 +3234,42 @@ func (m *MPPDataPacket) Unmarshal(dAtA []byte) error { } else { return fmt.Errorf("proto: wrong wireType = %d for field StreamIds", wireType) } @@ -970,7 +981,7 @@ index 16fd5dd..63a6d6c 100644 + return io.ErrUnexpectedEOF + } + if m.Compress == nil { -+ m.Compress = &CompressMeta{} ++ m.Compress = &DataPacketCompressMeta{} + } + if err := m.Compress.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err @@ -980,10 +991,10 @@ index 16fd5dd..63a6d6c 100644 iNdEx = preIndex skippy, err := skipMpp(dAtA[iNdEx:]) diff --git a/proto/mpp.proto b/proto/mpp.proto -index f8af03b..8dd93bf 100644 +index 6739455..377b875 100644 --- a/proto/mpp.proto +++ b/proto/mpp.proto -@@ -11,12 +11,24 @@ option (gogoproto.unmarshaler_all) = true; +@@ -11,6 +11,17 @@ option (gogoproto.unmarshaler_all) = true; option java_package = "org.tikv.kvproto"; @@ -1001,14 +1012,15 @@ index f8af03b..8dd93bf 100644 // TaskMeta contains meta of a mpp plan, including query's ts and task address. message TaskMeta { uint64 start_ts = 1; // start ts of a query - int64 task_id = 2; // if task id is -1 , it indicates a tidb task. - int64 partition_id = 3; // Only used for hash partition - string address = 4; // target address of this task. -+ int64 mpp_version = 5; // mpp version +@@ -21,6 +32,7 @@ message TaskMeta { + uint64 query_ts = 6; // timestamp when start to execute query, used for TiFlash miniTSO schedule. + uint64 local_query_id = 7; // unique local query_id if tidb don't restart. So we can use query_ts + local_query_id + server_id to represent a global unique query. + uint64 server_id = 8; // TiDB server id ++ int64 mpp_version = 9; // mpp version } message IsAliveRequest { -@@ -24,6 +36,7 @@ message IsAliveRequest { +@@ -28,6 +40,7 @@ message IsAliveRequest { message IsAliveResponse { bool available = 1; @@ -1016,31 +1028,32 @@ index f8af03b..8dd93bf 100644 } // Dipsatch the task request to different tiflash servers. -@@ -36,6 +49,8 @@ message DispatchTaskRequest { +@@ -40,6 +53,8 @@ message DispatchTaskRequest { int64 schema_ver = 5; // Used for partition table scan repeated coprocessor.TableRegions table_regions = 6; + // Used for exchange sender -+ ExchangeSenderMeta exchange_sender_meta = 7; ++ ExchangeSenderMeta exchange_sender_meta = 7; } // Get response of DispatchTaskRequest. -@@ -60,6 +75,10 @@ message EstablishMPPConnectionRequest { +@@ -64,6 +79,10 @@ message EstablishMPPConnectionRequest { TaskMeta receiver_meta = 2; // node closer to the tidb mpp gather. } -+message CompressMeta { ++message DataPacketCompressMeta { + CompressMethod method = 1; +} + // when TiFlash sends data to TiDB, Data packets wrap tipb.SelectResponse, i.e., serialize tipb.SelectResponse into data; // when TiFlash sends data to TiFlash, data blocks are serialized into chunks, and the execution_summaries in tipb.SelectResponse are serialized into data only for the last packet. message MPPDataPacket { -@@ -67,6 +86,7 @@ message MPPDataPacket { +@@ -71,6 +90,8 @@ message MPPDataPacket { Error error = 2; repeated bytes chunks = 3; repeated uint64 stream_ids = 4; -+ CompressMeta compress = 5; ++ // data compression ++ DataPacketCompressMeta compress = 5; } message Error { From 94ddcf24ba411beabf0220976bddded2cbc62dc8 Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Tue, 27 Dec 2022 16:23:43 +0800 Subject: [PATCH 26/93] 20 --- .../Flash/Coprocessor/CHBlockChunkCodec.cpp | 75 +++++++------------ .../src/Flash/Coprocessor/CHBlockChunkCodec.h | 2 - .../Coprocessor/ChunkDecodeAndSquash.cpp | 7 +- .../Coprocessor/CompressedCHBlockChunkCodec.h | 20 ++--- .../Mpp/BroadcastOrPassThroughWriter.cpp | 50 +------------ .../Flash/Mpp/BroadcastOrPassThroughWriter.h | 2 - .../Flash/Mpp/FineGrainedShuffleWriter.cpp | 37 ++------- dbms/src/Flash/Mpp/FineGrainedShuffleWriter.h | 3 - dbms/src/Flash/Mpp/HashPartitionWriter.cpp | 16 ---- 9 files changed, 43 insertions(+), 169 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/CHBlockChunkCodec.cpp b/dbms/src/Flash/Coprocessor/CHBlockChunkCodec.cpp index 86a344e1a84..6de7d465104 100644 --- a/dbms/src/Flash/Coprocessor/CHBlockChunkCodec.cpp +++ b/dbms/src/Flash/Coprocessor/CHBlockChunkCodec.cpp @@ -48,7 +48,7 @@ class CHBlockChunkCodecStream : public ChunkCodecStream } return output->releaseStr(); } - virtual WriteBuffer * initOutput(size_t init_size) + virtual WriteBuffer * initOutputBuffer(size_t init_size) { assert(output == nullptr); output = std::make_unique(init_size); @@ -68,15 +68,15 @@ class CompressCHBlockChunkCodecStream final : public CHBlockChunkCodecStream using Base = CHBlockChunkCodecStream; public: - explicit CompressCHBlockChunkCodecStream(const std::vector & field_types, CompressionMethod compress_method_ = CompressionMethod::LZ4) + explicit CompressCHBlockChunkCodecStream(const std::vector & field_types, CompressionMethod compress_method_) : Base(field_types) , compress_method(compress_method_) { } - WriteBuffer * initOutput(size_t init_size) override + WriteBuffer * initOutputBuffer(size_t init_size) override { assert(compress_write_buffer == nullptr); - compress_write_buffer = std::make_unique>(*Base::initOutput(init_size), CompressionSettings(compress_method), init_size); + compress_write_buffer = std::make_unique(*Base::initOutputBuffer(init_size), CompressionSettings(compress_method), init_size); return compress_write_buffer.get(); } void clear() override @@ -94,7 +94,7 @@ class CompressCHBlockChunkCodecStream final : public CHBlockChunkCodecStream return Base::getString(); } CompressionMethod compress_method; - std::unique_ptr> compress_write_buffer{}; + std::unique_ptr compress_write_buffer{}; ~CompressCHBlockChunkCodecStream() override = default; }; @@ -134,6 +134,21 @@ size_t ApproxBlockBytes(const Block & block) return block.bytes() + getExtraInfoSize(block); } +CompressionMethod ToInternalCompressionMethod(mpp::CompressMethod compress_method) +{ + switch (compress_method) + { + case mpp::NONE: + return CompressionMethod::NONE; + case mpp::LZ4: + return CompressionMethod::LZ4; + case mpp::ZSTD: + return CompressionMethod::ZSTD; + default: + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unkown compress method {}", mpp::CompressMethod_Name(compress_method)); + } +} + void writeData(const IDataType & type, const ColumnPtr & column, WriteBuffer & ostr, size_t offset, size_t limit) { /** If there are columns-constants - then we materialize them. @@ -171,7 +186,7 @@ void CHBlockChunkCodecStream::encode(const Block & block, size_t start, size_t e throw TiFlashException("CHBlock encode only support encode whole block", Errors::Coprocessor::Internal); size_t init_size = ApproxBlockBytes(block); - WriteBuffer * ostr_ptr = initOutput(init_size); + WriteBuffer * ostr_ptr = initOutputBuffer(init_size); block.checkNumberOfRows(); size_t columns = block.columns(); @@ -201,9 +216,7 @@ Block CHBlockChunkCodec::decodeImpl(ReadBuffer & istr, size_t reserve_size) { Block res; - ReadBuffer * istr_ptr = &istr; - - if (istr_ptr->eof()) + if (istr.eof()) { return res; } @@ -211,12 +224,12 @@ Block CHBlockChunkCodec::decodeImpl(ReadBuffer & istr, size_t reserve_size) /// Dimensions size_t columns = 0; size_t rows = 0; - readBlockMeta(*istr_ptr, columns, rows); + readBlockMeta(istr, columns, rows); for (size_t i = 0; i < columns; ++i) { ColumnWithTypeAndName column; - readColumnMeta(i, *istr_ptr, column); + readColumnMeta(i, istr, column); /// Data MutableColumnPtr read_column = column.type->createColumn(); @@ -226,7 +239,7 @@ Block CHBlockChunkCodec::decodeImpl(ReadBuffer & istr, size_t reserve_size) read_column->reserve(rows); if (rows) /// If no rows, nothing to read. - readData(*column.type, *read_column, *istr_ptr, rows); + readData(*column.type, *read_column, istr, rows); column.column = std::move(read_column); res.insert(std::move(column)); @@ -285,42 +298,4 @@ std::unique_ptr CompressedCHBlockChunkCodec::newCodecStream(co { return std::make_unique(field_types, compress_method); } - -CompressedCHBlockChunkCodec::CompressedCHBlockChunkCodec( - const Block & header_) - : chunk_codec(header_) -{ -} -CompressedCHBlockChunkCodec::CompressedCHBlockChunkCodec(const DAGSchema & schema) - : chunk_codec(schema) -{ -} -Block CompressedCHBlockChunkCodec::decode(const String & str, const DAGSchema & schema) -{ - ReadBufferFromString read_buffer(str); - CompressedReadBuffer compress_read_buffer(read_buffer); - return CHBlockChunkCodec(schema).decodeImpl(compress_read_buffer); -} -Block CompressedCHBlockChunkCodec::decode(const String & str, const Block & header) -{ - ReadBufferFromString read_buffer(str); - CompressedReadBuffer compress_read_buffer(read_buffer); - return CHBlockChunkCodec(header).decodeImpl(compress_read_buffer); -} -Block CompressedCHBlockChunkCodec::decodeImpl(CompressedReadBuffer & istr, size_t reserve_size) -{ - return chunk_codec.decodeImpl(istr, reserve_size); -} -void CompressedCHBlockChunkCodec::readColumnMeta(size_t i, CompressedReadBuffer & istr, ColumnWithTypeAndName & column) -{ - return chunk_codec.readColumnMeta(i, istr, column); -} -void CompressedCHBlockChunkCodec::readBlockMeta(CompressedReadBuffer & istr, size_t & columns, size_t & rows) const -{ - return chunk_codec.readBlockMeta(istr, columns, rows); -} -void CompressedCHBlockChunkCodec::readData(const IDataType & type, IColumn & column, CompressedReadBuffer & istr, size_t rows) -{ - return CHBlockChunkCodec::readData(type, column, istr, rows); -} } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/CHBlockChunkCodec.h b/dbms/src/Flash/Coprocessor/CHBlockChunkCodec.h index 9c91834d264..73f4a890054 100644 --- a/dbms/src/Flash/Coprocessor/CHBlockChunkCodec.h +++ b/dbms/src/Flash/Coprocessor/CHBlockChunkCodec.h @@ -20,7 +20,6 @@ namespace DB { class CHBlockChunkDecodeAndSquash; -class CompressedCHBlockChunkCodec; class CHBlockChunkCodec final : public ChunkCodec { @@ -35,7 +34,6 @@ class CHBlockChunkCodec final : public ChunkCodec private: friend class CHBlockChunkDecodeAndSquash; - friend class CompressedCHBlockChunkCodec; void readColumnMeta(size_t i, ReadBuffer & istr, ColumnWithTypeAndName & column); void readBlockMeta(ReadBuffer & istr, size_t & columns, size_t & rows) const; static void readData(const IDataType & type, IColumn & column, ReadBuffer & istr, size_t rows); diff --git a/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.cpp b/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.cpp index 9edbf36ab09..ea3c1da6d8b 100644 --- a/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.cpp +++ b/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.cpp @@ -13,12 +13,9 @@ // limitations under the License. #include +#include #include -#include "Flash/Mpp/HashPartitionWriter.h" -#include "IO/CompressedReadBuffer.h" -#include "IO/CompressedStream.h" - namespace DB { CHBlockChunkDecodeAndSquash::CHBlockChunkDecodeAndSquash( @@ -43,7 +40,7 @@ std::optional CHBlockChunkDecodeAndSquash::decodeAndSquash(const String & ReadBuffer * istr_ptr = &istr; if (ToInternalCompressionMethod(compress_method) != CompressionMethod::NONE) { - compress_buffer = std::make_unique>(istr); + compress_buffer = std::make_unique(istr); istr_ptr = compress_buffer.get(); } diff --git a/dbms/src/Flash/Coprocessor/CompressedCHBlockChunkCodec.h b/dbms/src/Flash/Coprocessor/CompressedCHBlockChunkCodec.h index 3d5bcc1addb..bd8db5191ec 100644 --- a/dbms/src/Flash/Coprocessor/CompressedCHBlockChunkCodec.h +++ b/dbms/src/Flash/Coprocessor/CompressedCHBlockChunkCodec.h @@ -17,6 +17,7 @@ #include #include #include +#include namespace mpp { @@ -25,28 +26,17 @@ enum CompressMethod : int; namespace DB { -class CompressedCHBlockChunkCodec final +class CompressedCHBlockChunkCodec { public: using CompressedReadBuffer = CompressedReadBuffer; - CompressedCHBlockChunkCodec() = default; - explicit CompressedCHBlockChunkCodec(const Block & header_); - explicit CompressedCHBlockChunkCodec(const DAGSchema & schema); + using CompressedWriteBuffer = CompressedWriteBuffer; - static Block decode(const String &, const DAGSchema & schema); - static Block decode(const String &, const Block & header); static std::unique_ptr newCodecStream(const std::vector & field_types, CompressionMethod compress_method); - -private: - void readColumnMeta(size_t i, CompressedReadBuffer & istr, ColumnWithTypeAndName & column); - void readBlockMeta(CompressedReadBuffer & istr, size_t & columns, size_t & rows) const; - static void readData(const IDataType & type, IColumn & column, CompressedReadBuffer & istr, size_t rows); - /// 'reserve_size' used for Squash usage, and takes effect when 'reserve_size' > 0 - Block decodeImpl(CompressedReadBuffer & istr, size_t reserve_size = 0); - - CHBlockChunkCodec chunk_codec; }; CompressionMethod ToInternalCompressionMethod(mpp::CompressMethod compress_method); +size_t ApproxBlockBytes(const Block & block); + } // namespace DB diff --git a/dbms/src/Flash/Mpp/BroadcastOrPassThroughWriter.cpp b/dbms/src/Flash/Mpp/BroadcastOrPassThroughWriter.cpp index 88a39cead6f..bf5b9e75efe 100644 --- a/dbms/src/Flash/Mpp/BroadcastOrPassThroughWriter.cpp +++ b/dbms/src/Flash/Mpp/BroadcastOrPassThroughWriter.cpp @@ -14,7 +14,6 @@ #include #include -#include #include #include @@ -30,15 +29,10 @@ BroadcastOrPassThroughWriter::BroadcastOrPassThroughWriter( , batch_send_min_limit(batch_send_min_limit_) , should_send_exec_summary_at_last(should_send_exec_summary_at_last_) , writer(writer_) - , compress_method(dag_context.getExchangeSenderMeta().compress()) { rows_in_blocks = 0; RUNTIME_CHECK(dag_context.encode_type == tipb::EncodeType::TypeCHBlock); chunk_codec_stream = std::make_unique()->newCodecStream(dag_context.result_field_types); - if (auto method = ToInternalCompressionMethod(compress_method); method != CompressionMethod::NONE) - { - compress_chunk_codec_stream = CompressedCHBlockChunkCodec::newCodecStream(dag_context.result_field_types, method); - } } template @@ -88,53 +82,17 @@ void BroadcastOrPassThroughWriter::encodeThenWriteBlocks() return; auto tracked_packet = std::make_shared(); - decltype(tracked_packet) compressed_tracked_packet = {}; - bool need_compress = compress_method != mpp::CompressMethod::NONE; - if (need_compress) - { - auto all_is_local = std::all_of(writer->getTunnels().begin(), writer->getTunnels().end(), [](const auto & tunnel) { - return tunnel->isLocal(); - }); - if (all_is_local) - need_compress = false; - } - if (need_compress) - { - compressed_tracked_packet = std::make_shared(); - } - while (!blocks.empty()) { const auto & block = blocks.back(); - - if (need_compress) - { - assert(compressed_tracked_packet); - compress_chunk_codec_stream->encode(block, 0, block.rows()); - compressed_tracked_packet->addChunk(compress_chunk_codec_stream->getString()); - compress_chunk_codec_stream->clear(); - } - - { - assert(tracked_packet); - chunk_codec_stream->encode(block, 0, block.rows()); - tracked_packet->addChunk(chunk_codec_stream->getString()); - chunk_codec_stream->clear(); - } - + chunk_codec_stream->encode(block, 0, block.rows()); blocks.pop_back(); + tracked_packet->addChunk(chunk_codec_stream->getString()); + chunk_codec_stream->clear(); } assert(blocks.empty()); rows_in_blocks = 0; - - if (!need_compress) - { - writer->broadcastOrPassThroughWrite(tracked_packet); - } - else - { - writer->broadcastOrPassThroughWrite(tracked_packet, compressed_tracked_packet); - } + writer->broadcastOrPassThroughWrite(tracked_packet); } template class BroadcastOrPassThroughWriter; diff --git a/dbms/src/Flash/Mpp/BroadcastOrPassThroughWriter.h b/dbms/src/Flash/Mpp/BroadcastOrPassThroughWriter.h index 3ff348c51a9..a7f88cf90e3 100644 --- a/dbms/src/Flash/Mpp/BroadcastOrPassThroughWriter.h +++ b/dbms/src/Flash/Mpp/BroadcastOrPassThroughWriter.h @@ -46,9 +46,7 @@ class BroadcastOrPassThroughWriter : public DAGResponseWriter ExchangeWriterPtr writer; std::vector blocks; size_t rows_in_blocks; - mpp::CompressMethod compress_method{}; std::unique_ptr chunk_codec_stream; - std::unique_ptr compress_chunk_codec_stream; }; } // namespace DB diff --git a/dbms/src/Flash/Mpp/FineGrainedShuffleWriter.cpp b/dbms/src/Flash/Mpp/FineGrainedShuffleWriter.cpp index ff0757d76f5..c04df4c8e42 100644 --- a/dbms/src/Flash/Mpp/FineGrainedShuffleWriter.cpp +++ b/dbms/src/Flash/Mpp/FineGrainedShuffleWriter.cpp @@ -14,7 +14,6 @@ #include #include -#include #include #include #include @@ -39,17 +38,12 @@ FineGrainedShuffleWriter::FineGrainedShuffleWriter( , fine_grained_shuffle_batch_size(fine_grained_shuffle_batch_size_) , batch_send_row_limit(fine_grained_shuffle_batch_size * fine_grained_shuffle_stream_count) , hash(0) - , compress_method(dag_context.getExchangeSenderMeta().compress()) { rows_in_blocks = 0; partition_num = writer_->getPartitionNum(); RUNTIME_CHECK(partition_num > 0); RUNTIME_CHECK(dag_context.encode_type == tipb::EncodeType::TypeCHBlock); chunk_codec_stream = std::make_unique()->newCodecStream(dag_context.result_field_types); - if (auto method = ToInternalCompressionMethod(compress_method); method != CompressionMethod::NONE) - { - compress_chunk_codec_stream = CompressedCHBlockChunkCodec::newCodecStream(dag_context.result_field_types, method); - } } template @@ -134,16 +128,6 @@ template void FineGrainedShuffleWriter::batchWriteFineGrainedShuffle() { auto tracked_packets = HashBaseWriterHelper::createPackets(partition_num); - for (size_t part_id = 0; part_id < partition_num; ++part_id) - { - auto method = compress_method; - if (writer->getTunnels()[part_id]->isLocal()) - { - method = mpp::CompressMethod::NONE; - } - tracked_packets[part_id]->getPacket().mutable_compress()->set_method(method); - } - if (likely(!blocks.empty())) { assert(rows_in_blocks > 0); @@ -169,20 +153,13 @@ void FineGrainedShuffleWriter::batchWriteFineGrainedShuffle() for (size_t col_id = 0; col_id < num_columns; ++col_id) columns.emplace_back(std::move(scattered[col_id][bucket_idx + stream_idx])); auto block = header.cloneWithColumns(std::move(columns)); - { - ChunkCodecStream * codec_stream = chunk_codec_stream.get(); - if (tracked_packets[part_id]->getPacket().compress().method() != mpp::CompressMethod::NONE) - { - assert(compress_chunk_codec_stream); - // no need compress - codec_stream = compress_chunk_codec_stream.get(); - } - // encode into packet - codec_stream->encode(block, 0, block.rows()); - tracked_packets[part_id]->addChunk(codec_stream->getString()); - tracked_packets[part_id]->getPacket().add_stream_ids(stream_idx); - codec_stream->clear(); - } + + // encode into packet + chunk_codec_stream->encode(block, 0, block.rows()); + tracked_packets[part_id]->addChunk(chunk_codec_stream->getString()); + tracked_packets[part_id]->getPacket().add_stream_ids(stream_idx); + chunk_codec_stream->clear(); + // disassemble the block back to scatter columns columns = block.mutateColumns(); for (size_t col_id = 0; col_id < num_columns; ++col_id) diff --git a/dbms/src/Flash/Mpp/FineGrainedShuffleWriter.h b/dbms/src/Flash/Mpp/FineGrainedShuffleWriter.h index f91fe804366..895b55817fd 100644 --- a/dbms/src/Flash/Mpp/FineGrainedShuffleWriter.h +++ b/dbms/src/Flash/Mpp/FineGrainedShuffleWriter.h @@ -57,7 +57,6 @@ class FineGrainedShuffleWriter : public DAGResponseWriter size_t rows_in_blocks = 0; uint16_t partition_num; std::unique_ptr chunk_codec_stream; - std::unique_ptr compress_chunk_codec_stream; UInt64 fine_grained_shuffle_stream_count; UInt64 fine_grained_shuffle_batch_size; @@ -68,8 +67,6 @@ class FineGrainedShuffleWriter : public DAGResponseWriter WeakHash32 hash; IColumn::Selector selector; std::vector scattered; // size = num_columns - - mpp::CompressMethod compress_method{}; }; } // namespace DB diff --git a/dbms/src/Flash/Mpp/HashPartitionWriter.cpp b/dbms/src/Flash/Mpp/HashPartitionWriter.cpp index 33155d9cd31..547f1bdc59c 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriter.cpp +++ b/dbms/src/Flash/Mpp/HashPartitionWriter.cpp @@ -112,8 +112,6 @@ void HashPartitionWriter::write(const Block & block) partitionAndEncodeThenWriteBlocks(); } -extern size_t ApproxBlockBytes(const Block & block); - template void HashPartitionWriter::partitionAndEncodeThenWriteBlocks() { @@ -227,18 +225,4 @@ void HashPartitionWriter::writePackets(const TrackedMppDataPa template class HashPartitionWriter; -CompressionMethod ToInternalCompressionMethod(mpp::CompressMethod compress_method) -{ - switch (compress_method) - { - case mpp::NONE: - return CompressionMethod::NONE; - case mpp::LZ4: - return CompressionMethod::LZ4; - case mpp::ZSTD: - return CompressionMethod::ZSTD; - default: - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unkown compress method {}", mpp::CompressMethod_Name(compress_method)); - } -} } // namespace DB From b43aeb0e64019b0590ee176fd9a2ed6bcdfc33ff Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Tue, 27 Dec 2022 18:01:44 +0800 Subject: [PATCH 27/93] 21 --- dbms/src/Flash/Mpp/HashPartitionWriter.cpp | 13 ------------- 1 file changed, 13 deletions(-) diff --git a/dbms/src/Flash/Mpp/HashPartitionWriter.cpp b/dbms/src/Flash/Mpp/HashPartitionWriter.cpp index 547f1bdc59c..9609d63d9fe 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriter.cpp +++ b/dbms/src/Flash/Mpp/HashPartitionWriter.cpp @@ -57,19 +57,6 @@ HashPartitionWriter::HashPartitionWriter( compress_chunk_codec_stream = CompressedCHBlockChunkCodec::newCodecStream(dag_context.result_field_types, method); } chunk_codec_stream = std::make_unique()->newCodecStream(dag_context.result_field_types); - - // LOG_TRACE(&Poco::Logger::get("tzg"), "using mpp CompressMethod {}, partition_num {}", mpp::CompressMethod_Name(compress_method), partition_num); - // { - // size_t local_cnt = 0; - // for (size_t part_id = 0; part_id < partition_num; ++part_id) - // { - // if (writer->getTunnels()[part_id]->isLocal()) - // { - // ++local_cnt; - // } - // } - // LOG_TRACE(&Poco::Logger::get("tzg"), "local_cnt is {}", local_cnt); - // } } template From d6578dfe15a84b4ce73e176329ce1e4507bc6794 Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Thu, 29 Dec 2022 09:44:35 +0800 Subject: [PATCH 28/93] 22 --- dbms/src/Flash/Mpp/MPPTask.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Flash/Mpp/MPPTask.cpp b/dbms/src/Flash/Mpp/MPPTask.cpp index 81d8b1c21b6..c6904b7e517 100644 --- a/dbms/src/Flash/Mpp/MPPTask.cpp +++ b/dbms/src/Flash/Mpp/MPPTask.cpp @@ -129,7 +129,7 @@ void MPPTask::registerTunnels(const mpp::DispatchTaskRequest & task_request) throw TiFlashException("Failed to decode task meta info in ExchangeSender", Errors::Coprocessor::BadRequest); bool is_local = context->getSettingsRef().enable_local_tunnel && sender_meta.address() == receiver_meta.address(); - receiver_meta.mpp_version(); + bool is_async = !is_local && context->getSettingsRef().enable_async_server; MPPTunnelPtr tunnel = std::make_shared(receiver_meta, sender_meta, timeout, context->getSettingsRef().max_threads, is_local, is_async, log->identifier()); From 3b15dab139ae8402c99e89db25177538495b3c2e Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Fri, 30 Dec 2022 17:01:13 +0800 Subject: [PATCH 29/93] 23 --- contrib/tzg.mpp.proto.patch | 272 +++++++++++----- .../Flash/Coprocessor/CHBlockChunkCodec.cpp | 125 +++----- .../Coprocessor/CHBlockChunkCodecStream.h | 52 +++ .../Coprocessor/ChunkDecodeAndSquash.cpp | 52 ++- .../Flash/Coprocessor/ChunkDecodeAndSquash.h | 5 +- .../CompressCHBlockChunkCodecStream.cpp | 121 +++++++ .../CompressCHBlockChunkCodecStream.h | 112 +++++++ .../Coprocessor/CompressedCHBlockChunkCodec.h | 5 - dbms/src/Flash/FlashService.cpp | 1 + dbms/src/Flash/Mpp/ExchangeReceiver.cpp | 23 +- dbms/src/Flash/Mpp/HashPartitionWriter.cpp | 37 ++- dbms/src/Flash/Mpp/HashPartitionWriter.h | 4 +- dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp | 303 ++++++++++++++++++ dbms/src/Flash/Mpp/HashPartitionWriterV1.h | 66 ++++ dbms/src/Flash/Mpp/Utils.cpp | 36 ++- dbms/src/Flash/Mpp/newMPPExchangeWriter.h | 62 +--- .../Planner/plans/PhysicalExchangeSender.cpp | 88 +++++ 17 files changed, 1091 insertions(+), 273 deletions(-) create mode 100644 dbms/src/Flash/Coprocessor/CHBlockChunkCodecStream.h create mode 100644 dbms/src/Flash/Coprocessor/CompressCHBlockChunkCodecStream.cpp create mode 100644 dbms/src/Flash/Coprocessor/CompressCHBlockChunkCodecStream.h create mode 100644 dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp create mode 100644 dbms/src/Flash/Mpp/HashPartitionWriterV1.h diff --git a/contrib/tzg.mpp.proto.patch b/contrib/tzg.mpp.proto.patch index d81c94bdabe..ec4959bfbc0 100644 --- a/contrib/tzg.mpp.proto.patch +++ b/contrib/tzg.mpp.proto.patch @@ -1,5 +1,5 @@ diff --git a/pkg/mpp/mpp.pb.go b/pkg/mpp/mpp.pb.go -index fcd4691..b6a15bf 100644 +index fcd4691..783c2eb 100644 --- a/pkg/mpp/mpp.pb.go +++ b/pkg/mpp/mpp.pb.go @@ -26,6 +26,82 @@ var _ = math.Inf @@ -292,12 +292,12 @@ index fcd4691..b6a15bf 100644 - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -+ Data []byte `protobuf:"bytes,1,opt,name=data,proto3" json:"data,omitempty"` -+ Error *Error `protobuf:"bytes,2,opt,name=error,proto3" json:"error,omitempty"` -+ Chunks [][]byte `protobuf:"bytes,3,rep,name=chunks,proto3" json:"chunks,omitempty"` -+ StreamIds []uint64 `protobuf:"varint,4,rep,packed,name=stream_ids,json=streamIds,proto3" json:"stream_ids,omitempty"` -+ // data compression -+ Compress *DataPacketCompressMeta `protobuf:"bytes,5,opt,name=compress,proto3" json:"compress,omitempty"` ++ Data []byte `protobuf:"bytes,1,opt,name=data,proto3" json:"data,omitempty"` ++ Error *Error `protobuf:"bytes,2,opt,name=error,proto3" json:"error,omitempty"` ++ Chunks [][]byte `protobuf:"bytes,3,rep,name=chunks,proto3" json:"chunks,omitempty"` ++ StreamIds []uint64 `protobuf:"varint,4,rep,packed,name=stream_ids,json=streamIds,proto3" json:"stream_ids,omitempty"` ++ MppVersion int64 `protobuf:"varint,5,opt,name=mpp_version,json=mppVersion,proto3" json:"mpp_version,omitempty"` ++ Compress *DataPacketCompressMeta `protobuf:"bytes,6,opt,name=compress,proto3" json:"compress,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` @@ -312,10 +312,17 @@ index fcd4691..b6a15bf 100644 } func (m *MPPDataPacket) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) -@@ -594,6 +744,13 @@ func (m *MPPDataPacket) GetStreamIds() []uint64 { +@@ -594,9 +744,24 @@ func (m *MPPDataPacket) GetStreamIds() []uint64 { return nil } ++func (m *MPPDataPacket) GetMppVersion() int64 { ++ if m != nil { ++ return m.MppVersion ++ } ++ return 0 ++} ++ +func (m *MPPDataPacket) GetCompress() *DataPacketCompressMeta { + if m != nil { + return m.Compress @@ -326,7 +333,11 @@ index fcd4691..b6a15bf 100644 type Error struct { Code int32 `protobuf:"varint,1,opt,name=code,proto3" json:"code,omitempty"` Msg string `protobuf:"bytes,2,opt,name=msg,proto3" json:"msg,omitempty"` -@@ -606,7 +763,7 @@ func (m *Error) Reset() { *m = Error{} } ++ MppVersion int64 `protobuf:"varint,3,opt,name=mpp_version,json=mppVersion,proto3" json:"mpp_version,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +@@ -606,7 +771,7 @@ func (m *Error) Reset() { *m = Error{} } func (m *Error) String() string { return proto.CompactTextString(m) } func (*Error) ProtoMessage() {} func (*Error) Descriptor() ([]byte, []int) { @@ -335,16 +346,24 @@ index fcd4691..b6a15bf 100644 } func (m *Error) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) -@@ -650,6 +807,8 @@ func (m *Error) GetMsg() string { +@@ -649,7 +814,16 @@ func (m *Error) GetMsg() string { + return "" } ++func (m *Error) GetMppVersion() int64 { ++ if m != nil { ++ return m.MppVersion ++ } ++ return 0 ++} ++ func init() { + proto.RegisterEnum("mpp.CompressMethod", CompressMethod_name, CompressMethod_value) + proto.RegisterType((*ExchangeSenderMeta)(nil), "mpp.ExchangeSenderMeta") proto.RegisterType((*TaskMeta)(nil), "mpp.TaskMeta") proto.RegisterType((*IsAliveRequest)(nil), "mpp.IsAliveRequest") proto.RegisterType((*IsAliveResponse)(nil), "mpp.IsAliveResponse") -@@ -658,6 +817,7 @@ func init() { +@@ -658,6 +832,7 @@ func init() { proto.RegisterType((*CancelTaskRequest)(nil), "mpp.CancelTaskRequest") proto.RegisterType((*CancelTaskResponse)(nil), "mpp.CancelTaskResponse") proto.RegisterType((*EstablishMPPConnectionRequest)(nil), "mpp.EstablishMPPConnectionRequest") @@ -352,7 +371,7 @@ index fcd4691..b6a15bf 100644 proto.RegisterType((*MPPDataPacket)(nil), "mpp.MPPDataPacket") proto.RegisterType((*Error)(nil), "mpp.Error") } -@@ -665,49 +825,91 @@ func init() { +@@ -665,49 +840,91 @@ func init() { func init() { proto.RegisterFile("mpp.proto", fileDescriptor_819623c7fa76fc55) } var fileDescriptor_819623c7fa76fc55 = []byte{ @@ -399,59 +418,59 @@ index fcd4691..b6a15bf 100644 - 0xb1, 0xf3, 0xf5, 0x76, 0xec, 0x7c, 0xfa, 0x36, 0xfe, 0x07, 0xb6, 0xb9, 0x58, 0x84, 0x2a, 0x5d, - 0x55, 0xe1, 0xaa, 0x32, 0x5f, 0xa7, 0xab, 0xae, 0xf9, 0x39, 0xfd, 0x11, 0x00, 0x00, 0xff, 0xff, - 0xcd, 0x41, 0xdb, 0xe2, 0xed, 0x04, 0x00, 0x00, -+ // 820 bytes of a gzipped FileDescriptorProto -+ 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x55, 0xcd, 0x8e, 0x1b, 0x45, -+ 0x10, 0xce, 0x78, 0xfc, 0x5b, 0x9e, 0x35, 0x4e, 0x67, 0x95, 0x75, 0x12, 0xb2, 0x38, 0x23, 0x0e, -+ 0x16, 0x28, 0x8e, 0xd8, 0x20, 0xb8, 0x21, 0xc1, 0xae, 0x0f, 0x23, 0xb1, 0xc1, 0x74, 0xac, 0x08, -+ 0xe5, 0x32, 0xea, 0x9d, 0x29, 0xec, 0x91, 0x3d, 0xd3, 0x93, 0xee, 0xb6, 0x45, 0xce, 0x3c, 0x04, -+ 0xbc, 0x08, 0xef, 0xc0, 0x11, 0xf1, 0x04, 0x68, 0x79, 0x11, 0xd4, 0xd5, 0x63, 0xaf, 0x97, 0x5d, -+ 0x7e, 0x72, 0xf2, 0xd4, 0xd7, 0xd5, 0x5f, 0x7f, 0xf5, 0x55, 0x77, 0x19, 0x3a, 0x79, 0x59, 0x8e, -+ 0x4b, 0x25, 0x8d, 0x64, 0x7e, 0x5e, 0x96, 0x0f, 0x0f, 0xe7, 0x72, 0x2e, 0x29, 0x7e, 0x66, 0xbf, -+ 0xdc, 0xd2, 0xc3, 0xbb, 0x89, 0xc5, 0x12, 0xd4, 0x5a, 0xaa, 0x0a, 0x0a, 0x72, 0x34, 0xa2, 0xbc, -+ 0x70, 0x51, 0x38, 0x01, 0x36, 0xf9, 0x21, 0x59, 0x88, 0x62, 0x8e, 0x2f, 0xb1, 0x48, 0x51, 0x9d, -+ 0xa3, 0x11, 0xec, 0x19, 0xb4, 0x13, 0x99, 0x97, 0x0a, 0xb5, 0x1e, 0x78, 0x43, 0x6f, 0xd4, 0x3b, -+ 0xb9, 0x37, 0xb6, 0xe7, 0x9d, 0x56, 0xe0, 0x39, 0x9a, 0x85, 0x4c, 0xf9, 0x2e, 0x29, 0xfc, 0xa9, -+ 0x06, 0xed, 0x99, 0xd0, 0x4b, 0xda, 0xfd, 0x00, 0xda, 0xda, 0x08, 0x65, 0x62, 0xe3, 0x76, 0xd7, -+ 0x79, 0x8b, 0xe2, 0x99, 0x66, 0x47, 0xd0, 0x32, 0x42, 0x2f, 0xe3, 0x2c, 0x1d, 0xd4, 0x86, 0xde, -+ 0xc8, 0xe7, 0x4d, 0x1b, 0x46, 0x29, 0x7b, 0x02, 0x41, 0x29, 0x94, 0xc9, 0x4c, 0x26, 0x0b, 0xbb, -+ 0xea, 0xd3, 0x6a, 0x77, 0x87, 0x45, 0x29, 0x1b, 0x40, 0x4b, 0xa4, 0x29, 0x69, 0xaa, 0x0f, 0xbd, -+ 0x51, 0x87, 0x6f, 0x43, 0xf6, 0x08, 0x3a, 0x73, 0x61, 0x16, 0xa8, 0xec, 0xce, 0x06, 0x9d, 0xd8, -+ 0x76, 0x40, 0x94, 0x5a, 0x35, 0x6f, 0xd6, 0xa8, 0xde, 0x5a, 0x35, 0x4d, 0xa7, 0x86, 0xe2, 0x99, -+ 0x66, 0x1f, 0x42, 0x6f, 0x25, 0x13, 0xb1, 0x8a, 0x5d, 0x42, 0x96, 0x0e, 0x5a, 0x94, 0x10, 0x10, -+ 0xfa, 0xad, 0x05, 0xa3, 0xd4, 0xb2, 0x6b, 0x54, 0x1b, 0xc7, 0xde, 0x76, 0xec, 0x0e, 0x88, 0x52, -+ 0xf6, 0x01, 0x74, 0xf3, 0xb2, 0x8c, 0x37, 0xa8, 0x74, 0x26, 0x8b, 0x41, 0x87, 0x64, 0x43, 0x5e, -+ 0x96, 0xaf, 0x1c, 0x12, 0xf6, 0xa1, 0x17, 0xe9, 0x2f, 0x57, 0xd9, 0x06, 0x39, 0xbe, 0x59, 0xa3, -+ 0x36, 0xe1, 0x14, 0xde, 0xdb, 0x21, 0xba, 0x94, 0x85, 0x46, 0xf6, 0x3e, 0x74, 0xc4, 0x46, 0x64, -+ 0x2b, 0x71, 0xb1, 0x42, 0xb2, 0xac, 0xcd, 0xaf, 0x80, 0xbf, 0x9f, 0x51, 0xbb, 0x71, 0xc6, 0xef, -+ 0x35, 0xb8, 0x77, 0x96, 0xe9, 0x52, 0x98, 0x64, 0x61, 0xbb, 0x50, 0x9d, 0xc4, 0x9e, 0x40, 0xdd, -+ 0x36, 0x9b, 0x18, 0xbb, 0x27, 0x07, 0xd4, 0xc2, 0x6d, 0x97, 0x38, 0x2d, 0x59, 0xdf, 0xb1, 0x48, -+ 0x64, 0x8a, 0x69, 0x5c, 0xae, 0x84, 0x23, 0x0f, 0x78, 0xb7, 0xc2, 0xa6, 0x2b, 0x51, 0x58, 0xdf, -+ 0x4d, 0x96, 0xa3, 0x5c, 0x9b, 0xaa, 0x2b, 0xdb, 0x90, 0x7d, 0x02, 0x2d, 0x85, 0xf3, 0x4c, 0x16, -+ 0xb6, 0x23, 0xfe, 0xa8, 0x7b, 0x72, 0x34, 0xde, 0xbf, 0x6f, 0x9c, 0xd6, 0xa2, 0xe2, 0x7b, 0xc9, -+ 0xb7, 0x79, 0xec, 0x31, 0x80, 0x4e, 0x16, 0x98, 0x0b, 0x5b, 0x0e, 0xf5, 0xca, 0xe7, 0x1d, 0x87, -+ 0xbc, 0x42, 0xc5, 0xbe, 0x80, 0x03, 0x63, 0x6b, 0x8e, 0xb7, 0xbc, 0x4d, 0xe2, 0x7d, 0x70, 0x8d, -+ 0x77, 0x66, 0x33, 0x1c, 0xb9, 0xe6, 0x81, 0xd9, 0x8b, 0x58, 0x04, 0x87, 0x58, 0x5d, 0xe7, 0x58, -+ 0xd3, 0x7d, 0x8e, 0xc9, 0x81, 0x16, 0x39, 0x70, 0x44, 0x0e, 0xdc, 0xbc, 0xef, 0x9c, 0xe1, 0x0d, -+ 0x2c, 0xcc, 0xe1, 0xf0, 0xba, 0xa7, 0x55, 0xaf, 0x86, 0xd0, 0x40, 0xa5, 0xa4, 0xaa, 0x5c, 0x05, -+ 0xc7, 0x69, 0x11, 0xee, 0x16, 0xd8, 0x73, 0x38, 0x50, 0x68, 0xd4, 0xdb, 0x5d, 0x11, 0x35, 0x2a, -+ 0xa2, 0x37, 0xae, 0x5e, 0x9e, 0x13, 0xcb, 0x03, 0x4a, 0xaa, 0x94, 0x87, 0xdf, 0xc1, 0xdd, 0x53, -+ 0x51, 0x24, 0xb8, 0x7a, 0xc7, 0x06, 0xee, 0xe4, 0xd4, 0xfe, 0x41, 0x4e, 0xf8, 0x19, 0xb0, 0x7d, -+ 0xe6, 0xff, 0x5b, 0x46, 0xf8, 0xa3, 0x07, 0x8f, 0x27, 0xda, 0xda, 0x9b, 0xe9, 0xc5, 0xf9, 0x74, -+ 0x7a, 0x2a, 0x8b, 0x02, 0x13, 0xfb, 0x18, 0xb7, 0xf2, 0xc6, 0xd0, 0xdd, 0x37, 0xf9, 0x56, 0x95, -+ 0xa0, 0xaf, 0xc6, 0xca, 0x89, 0x35, 0x26, 0xc1, 0x6c, 0xb3, 0xdd, 0x51, 0xbb, 0x6d, 0x47, 0xb0, -+ 0xcd, 0xa1, 0x36, 0x4c, 0xe0, 0xfe, 0x99, 0x30, 0x62, 0x2a, 0x92, 0x25, 0x9a, 0xbd, 0xf9, 0x23, -+ 0xd8, 0xc7, 0xd0, 0xcc, 0x69, 0x0e, 0xfd, 0xdb, 0x88, 0xaa, 0x52, 0xc2, 0x5f, 0x3c, 0x38, 0x38, -+ 0x9f, 0x4e, 0xaf, 0xa8, 0x18, 0x83, 0x7a, 0x2a, 0x2a, 0xd5, 0x01, 0xa7, 0xef, 0xff, 0x36, 0x93, -+ 0xdd, 0x87, 0x66, 0xb2, 0x58, 0x17, 0x4b, 0x3d, 0xf0, 0x87, 0xfe, 0x28, 0xe0, 0x55, 0x44, 0xf7, -+ 0xda, 0x28, 0x14, 0x79, 0x9c, 0xa5, 0xee, 0x35, 0xd4, 0x79, 0xc7, 0x21, 0x51, 0xaa, 0xd9, 0xe7, -+ 0x7b, 0x03, 0xb5, 0x41, 0xdc, 0x8f, 0x88, 0xfb, 0xf6, 0xd2, 0xf6, 0x06, 0xeb, 0x53, 0x68, 0xd0, -+ 0xf9, 0x56, 0xae, 0x7d, 0x92, 0x24, 0xb7, 0xc1, 0xe9, 0x9b, 0xf5, 0xc1, 0xcf, 0xf5, 0x9c, 0xc4, -+ 0x76, 0xb8, 0xfd, 0xfc, 0xe8, 0x29, 0xf4, 0xae, 0x1b, 0xc0, 0xda, 0x50, 0x7f, 0xf1, 0xcd, 0x8b, -+ 0x49, 0xff, 0x0e, 0x6b, 0x81, 0xff, 0xf5, 0xeb, 0x4f, 0xfb, 0x9e, 0x85, 0x5e, 0xbf, 0x9c, 0x9d, -+ 0xf5, 0x6b, 0x5f, 0x85, 0xbf, 0x5e, 0x1e, 0x7b, 0xbf, 0x5d, 0x1e, 0x7b, 0x7f, 0x5c, 0x1e, 0x7b, -+ 0x3f, 0xff, 0x79, 0x7c, 0x07, 0xfa, 0x52, 0xcd, 0xc7, 0x26, 0x5b, 0x6e, 0xc6, 0xcb, 0x0d, 0xfd, -+ 0x43, 0x5c, 0x34, 0xe9, 0xe7, 0xf9, 0x5f, 0x01, 0x00, 0x00, 0xff, 0xff, 0x50, 0xee, 0x3d, 0x86, -+ 0x71, 0x06, 0x00, 0x00, ++ // 831 bytes of a gzipped FileDescriptorProto ++ 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x55, 0xcf, 0x8f, 0x1b, 0x35, ++ 0x14, 0xee, 0x64, 0xf2, 0xf3, 0x65, 0x36, 0xa4, 0xee, 0xaa, 0x9b, 0xb6, 0x74, 0x49, 0x47, 0x1c, ++ 0x22, 0x10, 0xa9, 0xd8, 0x22, 0xb8, 0x21, 0xc1, 0x6e, 0x0e, 0x23, 0xb1, 0x4b, 0x70, 0xa3, 0x0a, ++ 0xf5, 0x32, 0xf2, 0x8e, 0x4d, 0x32, 0x4a, 0x66, 0x3c, 0xb5, 0x9d, 0x88, 0x9e, 0xf9, 0x23, 0xe0, ++ 0x4f, 0xe2, 0x88, 0x38, 0x70, 0x46, 0xcb, 0x3f, 0x82, 0xfc, 0x3c, 0x93, 0xcd, 0x66, 0x97, 0x1f, ++ 0x3d, 0x65, 0xde, 0xe7, 0xe7, 0xef, 0x7d, 0xef, 0x7b, 0xb6, 0x03, 0x9d, 0xac, 0x28, 0xc6, 0x85, ++ 0x92, 0x46, 0x12, 0x3f, 0x2b, 0x8a, 0xc7, 0x87, 0x73, 0x39, 0x97, 0x18, 0x3f, 0xb7, 0x5f, 0x6e, ++ 0xe9, 0xf1, 0xfd, 0xc4, 0x62, 0x89, 0xd0, 0x5a, 0xaa, 0x12, 0x0a, 0x32, 0x61, 0x58, 0x71, 0xe9, ++ 0xa2, 0x70, 0x02, 0x64, 0xf2, 0x63, 0xb2, 0x60, 0xf9, 0x5c, 0xbc, 0x14, 0x39, 0x17, 0xea, 0x5c, ++ 0x18, 0x46, 0x9e, 0x43, 0x3b, 0x91, 0x59, 0xa1, 0x84, 0xd6, 0x03, 0x6f, 0xe8, 0x8d, 0x7a, 0x27, ++ 0x0f, 0xc6, 0xb6, 0xde, 0x69, 0x09, 0x9e, 0x0b, 0xb3, 0x90, 0x9c, 0x6e, 0x93, 0xc2, 0x9f, 0x6b, ++ 0xd0, 0x9e, 0x31, 0xbd, 0xc4, 0xdd, 0x8f, 0xa0, 0xad, 0x0d, 0x53, 0x26, 0x36, 0x6e, 0x77, 0x9d, ++ 0xb6, 0x30, 0x9e, 0x69, 0x72, 0x04, 0x2d, 0xc3, 0xf4, 0x32, 0x4e, 0xf9, 0xa0, 0x36, 0xf4, 0x46, ++ 0x3e, 0x6d, 0xda, 0x30, 0xe2, 0xe4, 0x19, 0x04, 0x05, 0x53, 0x26, 0x35, 0xa9, 0xcc, 0xed, 0xaa, ++ 0x8f, 0xab, 0xdd, 0x2d, 0x16, 0x71, 0x32, 0x80, 0x16, 0xe3, 0x1c, 0x35, 0xd5, 0x87, 0xde, 0xa8, ++ 0x43, 0xab, 0x90, 0x3c, 0x81, 0xce, 0x9c, 0x99, 0x85, 0x50, 0x76, 0x67, 0x03, 0x2b, 0xb6, 0x1d, ++ 0x10, 0x71, 0xab, 0xe6, 0xcd, 0x5a, 0xa8, 0xb7, 0x56, 0x4d, 0xd3, 0xa9, 0xc1, 0x78, 0xa6, 0xc9, ++ 0x87, 0xd0, 0x5b, 0xc9, 0x84, 0xad, 0x62, 0x97, 0x90, 0xf2, 0x41, 0x0b, 0x13, 0x02, 0x44, 0xbf, ++ 0xb3, 0x60, 0xc4, 0x2d, 0xbb, 0x16, 0x6a, 0xe3, 0xd8, 0xdb, 0x8e, 0xdd, 0x01, 0x11, 0x27, 0x1f, ++ 0x40, 0x37, 0x2b, 0x8a, 0x78, 0x23, 0x94, 0x4e, 0x65, 0x3e, 0xe8, 0xa0, 0x6c, 0xc8, 0x8a, 0xe2, ++ 0x95, 0x43, 0xc2, 0x3e, 0xf4, 0x22, 0xfd, 0xd5, 0x2a, 0xdd, 0x08, 0x2a, 0xde, 0xac, 0x85, 0x36, ++ 0xe1, 0x14, 0xde, 0xdb, 0x22, 0xba, 0x90, 0xb9, 0x16, 0xe4, 0x7d, 0xe8, 0xb0, 0x0d, 0x4b, 0x57, ++ 0xec, 0x72, 0x25, 0xd0, 0xb2, 0x36, 0xbd, 0x06, 0xf6, 0x6b, 0xd4, 0x6e, 0xd5, 0xf8, 0xbd, 0x06, ++ 0x0f, 0xce, 0x52, 0x5d, 0x30, 0x93, 0x2c, 0xec, 0x14, 0xca, 0x4a, 0xe4, 0x19, 0xd4, 0xed, 0xb0, ++ 0x91, 0xb1, 0x7b, 0x72, 0x80, 0x23, 0xac, 0xa6, 0x44, 0x71, 0xc9, 0xfa, 0x2e, 0xf2, 0x44, 0x72, ++ 0xc1, 0xe3, 0x62, 0xc5, 0x1c, 0x79, 0x40, 0xbb, 0x25, 0x36, 0x5d, 0xb1, 0xdc, 0xfa, 0x6e, 0xd2, ++ 0x4c, 0xc8, 0xb5, 0x29, 0xa7, 0x52, 0x85, 0xe4, 0x53, 0x68, 0x29, 0x31, 0x4f, 0x65, 0x6e, 0x27, ++ 0xe2, 0x8f, 0xba, 0x27, 0x47, 0xe3, 0xdd, 0xf3, 0x46, 0x71, 0x2d, 0xca, 0x7f, 0x90, 0xb4, 0xca, ++ 0x23, 0x4f, 0x01, 0x74, 0xb2, 0x10, 0x19, 0xb3, 0xed, 0xe0, 0xac, 0x7c, 0xda, 0x71, 0xc8, 0x2b, ++ 0xa1, 0xc8, 0x97, 0x70, 0x60, 0x6c, 0xcf, 0x71, 0xc5, 0xdb, 0x44, 0xde, 0x47, 0x37, 0x78, 0x67, ++ 0x36, 0xc3, 0x91, 0x6b, 0x1a, 0x98, 0x9d, 0x88, 0x44, 0x70, 0x28, 0xca, 0xe3, 0x1c, 0x6b, 0x3c, ++ 0xcf, 0x31, 0x3a, 0xd0, 0x42, 0x07, 0x8e, 0xd0, 0x81, 0xdb, 0xe7, 0x9d, 0x12, 0x71, 0x0b, 0x0b, ++ 0x33, 0x38, 0xbc, 0xe9, 0x69, 0x39, 0xab, 0x21, 0x34, 0x84, 0x52, 0x52, 0x95, 0xae, 0x82, 0xe3, ++ 0xb4, 0x08, 0x75, 0x0b, 0xe4, 0x05, 0x1c, 0x28, 0x61, 0xd4, 0xdb, 0x6d, 0x13, 0x35, 0x6c, 0xa2, ++ 0x37, 0x2e, 0x6f, 0x9e, 0x13, 0x4b, 0x03, 0x4c, 0x2a, 0x95, 0x87, 0xdf, 0xc3, 0xfd, 0x53, 0x96, ++ 0x27, 0x62, 0xf5, 0x8e, 0x03, 0xdc, 0xca, 0xa9, 0xfd, 0x83, 0x9c, 0xf0, 0x73, 0x20, 0xbb, 0xcc, ++ 0xff, 0xb7, 0x8d, 0xf0, 0x27, 0x0f, 0x9e, 0x4e, 0xb4, 0xb5, 0x37, 0xd5, 0x8b, 0xf3, 0xe9, 0xf4, ++ 0x54, 0xe6, 0xb9, 0x48, 0xec, 0x65, 0xac, 0xe4, 0x8d, 0xa1, 0xbb, 0x6b, 0xf2, 0x9d, 0x2a, 0x41, ++ 0x5f, 0x3f, 0x2b, 0x27, 0xd6, 0x98, 0x44, 0xa4, 0x9b, 0x6a, 0x47, 0xed, 0xae, 0x1d, 0x41, 0x95, ++ 0x83, 0x63, 0x98, 0xc0, 0xc3, 0x33, 0x66, 0xd8, 0x94, 0x25, 0x4b, 0x61, 0x76, 0xde, 0x1f, 0x46, ++ 0x3e, 0x86, 0x66, 0x86, 0xef, 0xd0, 0xbf, 0x3d, 0x51, 0x65, 0x4a, 0xf8, 0x87, 0x07, 0x07, 0xe7, ++ 0xd3, 0xe9, 0x35, 0x15, 0x21, 0x50, 0xe7, 0xac, 0x54, 0x1d, 0x50, 0xfc, 0xfe, 0x6f, 0x33, 0xc9, ++ 0x43, 0x68, 0x26, 0x8b, 0x75, 0xbe, 0xd4, 0x03, 0x7f, 0xe8, 0x8f, 0x02, 0x5a, 0x46, 0x78, 0xae, ++ 0x8d, 0x12, 0x2c, 0x8b, 0x53, 0xee, 0x6e, 0x43, 0x9d, 0x76, 0x1c, 0x12, 0x71, 0xbd, 0x7f, 0x85, ++ 0x1b, 0xfb, 0x57, 0x98, 0x7c, 0xb1, 0xf3, 0xe2, 0x36, 0xb1, 0xf8, 0x13, 0x2c, 0x7e, 0x77, 0xef, ++ 0x3b, 0x2f, 0xef, 0x05, 0x34, 0x50, 0xa0, 0xed, 0xc7, 0xde, 0x59, 0xec, 0xa7, 0x41, 0xf1, 0x9b, ++ 0xf4, 0xc1, 0xcf, 0xf4, 0x1c, 0xbb, 0xe9, 0x50, 0xfb, 0xb9, 0x2f, 0xc4, 0xdf, 0x17, 0xf2, 0xd1, ++ 0x27, 0xd0, 0xbb, 0x69, 0x21, 0x69, 0x43, 0xfd, 0xe2, 0xdb, 0x8b, 0x49, 0xff, 0x1e, 0x69, 0x81, ++ 0xff, 0xcd, 0xeb, 0xcf, 0xfa, 0x9e, 0x85, 0x5e, 0xbf, 0x9c, 0x9d, 0xf5, 0x6b, 0x5f, 0x87, 0xbf, ++ 0x5e, 0x1d, 0x7b, 0xbf, 0x5d, 0x1d, 0x7b, 0x7f, 0x5e, 0x1d, 0x7b, 0xbf, 0xfc, 0x75, 0x7c, 0x0f, ++ 0xfa, 0x52, 0xcd, 0xc7, 0x26, 0x5d, 0x6e, 0xc6, 0xcb, 0x0d, 0xfe, 0xc7, 0x5c, 0x36, 0xf1, 0xe7, ++ 0xc5, 0xdf, 0x01, 0x00, 0x00, 0xff, 0xff, 0x68, 0x3e, 0x5c, 0x7f, 0xb3, 0x06, 0x00, 0x00, +} + +func (m *ExchangeSenderMeta) Marshal() (dAtA []byte, err error) { @@ -487,7 +506,7 @@ index fcd4691..b6a15bf 100644 } func (m *TaskMeta) Marshal() (dAtA []byte, err error) { -@@ -734,6 +936,11 @@ func (m *TaskMeta) MarshalToSizedBuffer(dAtA []byte) (int, error) { +@@ -734,6 +951,11 @@ func (m *TaskMeta) MarshalToSizedBuffer(dAtA []byte) (int, error) { i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } @@ -499,7 +518,7 @@ index fcd4691..b6a15bf 100644 if m.ServerId != 0 { i = encodeVarintMpp(dAtA, i, uint64(m.ServerId)) i-- -@@ -830,6 +1037,11 @@ func (m *IsAliveResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { +@@ -830,6 +1052,11 @@ func (m *IsAliveResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } @@ -511,7 +530,7 @@ index fcd4691..b6a15bf 100644 if m.Available { i-- if m.Available { -@@ -867,6 +1079,18 @@ func (m *DispatchTaskRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { +@@ -867,6 +1094,18 @@ func (m *DispatchTaskRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } @@ -530,7 +549,7 @@ index fcd4691..b6a15bf 100644 if len(m.TableRegions) > 0 { for iNdEx := len(m.TableRegions) - 1; iNdEx >= 0; iNdEx-- { { -@@ -1121,6 +1345,38 @@ func (m *EstablishMPPConnectionRequest) MarshalToSizedBuffer(dAtA []byte) (int, +@@ -1121,6 +1360,38 @@ func (m *EstablishMPPConnectionRequest) MarshalToSizedBuffer(dAtA []byte) (int, return len(dAtA) - i, nil } @@ -569,7 +588,7 @@ index fcd4691..b6a15bf 100644 func (m *MPPDataPacket) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) -@@ -1145,21 +1401,33 @@ func (m *MPPDataPacket) MarshalToSizedBuffer(dAtA []byte) (int, error) { +@@ -1145,21 +1416,38 @@ func (m *MPPDataPacket) MarshalToSizedBuffer(dAtA []byte) (int, error) { i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } @@ -583,7 +602,12 @@ index fcd4691..b6a15bf 100644 + i = encodeVarintMpp(dAtA, i, uint64(size)) + } + i-- -+ dAtA[i] = 0x2a ++ dAtA[i] = 0x32 ++ } ++ if m.MppVersion != 0 { ++ i = encodeVarintMpp(dAtA, i, uint64(m.MppVersion)) ++ i-- ++ dAtA[i] = 0x28 + } if len(m.StreamIds) > 0 { - dAtA9 := make([]byte, len(m.StreamIds)*10) @@ -612,7 +636,19 @@ index fcd4691..b6a15bf 100644 i-- dAtA[i] = 0x22 } -@@ -1244,6 +1512,21 @@ func encodeVarintMpp(dAtA []byte, offset int, v uint64) int { +@@ -1218,6 +1506,11 @@ func (m *Error) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i -= len(m.XXX_unrecognized) + copy(dAtA[i:], m.XXX_unrecognized) + } ++ if m.MppVersion != 0 { ++ i = encodeVarintMpp(dAtA, i, uint64(m.MppVersion)) ++ i-- ++ dAtA[i] = 0x18 ++ } + if len(m.Msg) > 0 { + i -= len(m.Msg) + copy(dAtA[i:], m.Msg) +@@ -1244,6 +1537,21 @@ func encodeVarintMpp(dAtA []byte, offset int, v uint64) int { dAtA[offset] = uint8(v) return base } @@ -634,7 +670,7 @@ index fcd4691..b6a15bf 100644 func (m *TaskMeta) Size() (n int) { if m == nil { return 0 -@@ -1275,6 +1558,9 @@ func (m *TaskMeta) Size() (n int) { +@@ -1275,6 +1583,9 @@ func (m *TaskMeta) Size() (n int) { if m.ServerId != 0 { n += 1 + sovMpp(uint64(m.ServerId)) } @@ -644,7 +680,7 @@ index fcd4691..b6a15bf 100644 if m.XXX_unrecognized != nil { n += len(m.XXX_unrecognized) } -@@ -1302,6 +1588,9 @@ func (m *IsAliveResponse) Size() (n int) { +@@ -1302,6 +1613,9 @@ func (m *IsAliveResponse) Size() (n int) { if m.Available { n += 2 } @@ -654,7 +690,7 @@ index fcd4691..b6a15bf 100644 if m.XXX_unrecognized != nil { n += len(m.XXX_unrecognized) } -@@ -1340,6 +1629,10 @@ func (m *DispatchTaskRequest) Size() (n int) { +@@ -1340,6 +1654,10 @@ func (m *DispatchTaskRequest) Size() (n int) { n += 1 + l + sovMpp(uint64(l)) } } @@ -665,7 +701,7 @@ index fcd4691..b6a15bf 100644 if m.XXX_unrecognized != nil { n += len(m.XXX_unrecognized) } -@@ -1424,6 +1717,21 @@ func (m *EstablishMPPConnectionRequest) Size() (n int) { +@@ -1424,6 +1742,21 @@ func (m *EstablishMPPConnectionRequest) Size() (n int) { return n } @@ -687,10 +723,13 @@ index fcd4691..b6a15bf 100644 func (m *MPPDataPacket) Size() (n int) { if m == nil { return 0 -@@ -1451,6 +1759,10 @@ func (m *MPPDataPacket) Size() (n int) { +@@ -1451,6 +1784,13 @@ func (m *MPPDataPacket) Size() (n int) { } n += 1 + sovMpp(uint64(l)) + l } ++ if m.MppVersion != 0 { ++ n += 1 + sovMpp(uint64(m.MppVersion)) ++ } + if m.Compress != nil { + l = m.Compress.Size() + n += 1 + l + sovMpp(uint64(l)) @@ -698,7 +737,17 @@ index fcd4691..b6a15bf 100644 if m.XXX_unrecognized != nil { n += len(m.XXX_unrecognized) } -@@ -1482,6 +1794,76 @@ func sovMpp(x uint64) (n int) { +@@ -1470,6 +1810,9 @@ func (m *Error) Size() (n int) { + if l > 0 { + n += 1 + l + sovMpp(uint64(l)) + } ++ if m.MppVersion != 0 { ++ n += 1 + sovMpp(uint64(m.MppVersion)) ++ } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } +@@ -1482,6 +1825,76 @@ func sovMpp(x uint64) (n int) { func sozMpp(x uint64) (n int) { return sovMpp(uint64((x << 1) ^ uint64((int64(x) >> 63)))) } @@ -775,7 +824,7 @@ index fcd4691..b6a15bf 100644 func (m *TaskMeta) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 -@@ -1676,6 +2058,25 @@ func (m *TaskMeta) Unmarshal(dAtA []byte) error { +@@ -1676,6 +2089,25 @@ func (m *TaskMeta) Unmarshal(dAtA []byte) error { break } } @@ -801,7 +850,7 @@ index fcd4691..b6a15bf 100644 default: iNdEx = preIndex skippy, err := skipMpp(dAtA[iNdEx:]) -@@ -1798,6 +2199,25 @@ func (m *IsAliveResponse) Unmarshal(dAtA []byte) error { +@@ -1798,6 +2230,25 @@ func (m *IsAliveResponse) Unmarshal(dAtA []byte) error { } } m.Available = bool(v != 0) @@ -827,7 +876,7 @@ index fcd4691..b6a15bf 100644 default: iNdEx = preIndex skippy, err := skipMpp(dAtA[iNdEx:]) -@@ -2025,6 +2445,42 @@ func (m *DispatchTaskRequest) Unmarshal(dAtA []byte) error { +@@ -2025,6 +2476,42 @@ func (m *DispatchTaskRequest) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex @@ -870,7 +919,7 @@ index fcd4691..b6a15bf 100644 default: iNdEx = preIndex skippy, err := skipMpp(dAtA[iNdEx:]) -@@ -2501,6 +2957,76 @@ func (m *EstablishMPPConnectionRequest) Unmarshal(dAtA []byte) error { +@@ -2501,6 +2988,76 @@ func (m *EstablishMPPConnectionRequest) Unmarshal(dAtA []byte) error { } return nil } @@ -947,11 +996,30 @@ index fcd4691..b6a15bf 100644 func (m *MPPDataPacket) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 -@@ -2708,6 +3234,42 @@ func (m *MPPDataPacket) Unmarshal(dAtA []byte) error { +@@ -2708,6 +3265,61 @@ func (m *MPPDataPacket) Unmarshal(dAtA []byte) error { } else { return fmt.Errorf("proto: wrong wireType = %d for field StreamIds", wireType) } + case 5: ++ if wireType != 0 { ++ return fmt.Errorf("proto: wrong wireType = %d for field MppVersion", wireType) ++ } ++ m.MppVersion = 0 ++ for shift := uint(0); ; shift += 7 { ++ if shift >= 64 { ++ return ErrIntOverflowMpp ++ } ++ if iNdEx >= l { ++ return io.ErrUnexpectedEOF ++ } ++ b := dAtA[iNdEx] ++ iNdEx++ ++ m.MppVersion |= int64(b&0x7F) << shift ++ if b < 0x80 { ++ break ++ } ++ } ++ case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Compress", wireType) + } @@ -990,8 +1058,34 @@ index fcd4691..b6a15bf 100644 default: iNdEx = preIndex skippy, err := skipMpp(dAtA[iNdEx:]) +@@ -2810,6 +3422,25 @@ func (m *Error) Unmarshal(dAtA []byte) error { + } + m.Msg = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex ++ case 3: ++ if wireType != 0 { ++ return fmt.Errorf("proto: wrong wireType = %d for field MppVersion", wireType) ++ } ++ m.MppVersion = 0 ++ for shift := uint(0); ; shift += 7 { ++ if shift >= 64 { ++ return ErrIntOverflowMpp ++ } ++ if iNdEx >= l { ++ return io.ErrUnexpectedEOF ++ } ++ b := dAtA[iNdEx] ++ iNdEx++ ++ m.MppVersion |= int64(b&0x7F) << shift ++ if b < 0x80 { ++ break ++ } ++ } + default: + iNdEx = preIndex + skippy, err := skipMpp(dAtA[iNdEx:]) diff --git a/proto/mpp.proto b/proto/mpp.proto -index 6739455..377b875 100644 +index 6739455..b61e03c 100644 --- a/proto/mpp.proto +++ b/proto/mpp.proto @@ -11,6 +11,17 @@ option (gogoproto.unmarshaler_all) = true; @@ -1048,12 +1142,16 @@ index 6739455..377b875 100644 // when TiFlash sends data to TiDB, Data packets wrap tipb.SelectResponse, i.e., serialize tipb.SelectResponse into data; // when TiFlash sends data to TiFlash, data blocks are serialized into chunks, and the execution_summaries in tipb.SelectResponse are serialized into data only for the last packet. message MPPDataPacket { -@@ -71,6 +90,8 @@ message MPPDataPacket { +@@ -71,9 +90,12 @@ message MPPDataPacket { Error error = 2; repeated bytes chunks = 3; repeated uint64 stream_ids = 4; -+ // data compression -+ DataPacketCompressMeta compress = 5; ++ int64 mpp_version = 5; ++ DataPacketCompressMeta compress = 6; // data compression } message Error { + int32 code = 1; + string msg = 2; ++ int64 mpp_version = 3; + } diff --git a/dbms/src/Flash/Coprocessor/CHBlockChunkCodec.cpp b/dbms/src/Flash/Coprocessor/CHBlockChunkCodec.cpp index 6de7d465104..7de63acb975 100644 --- a/dbms/src/Flash/Coprocessor/CHBlockChunkCodec.cpp +++ b/dbms/src/Flash/Coprocessor/CHBlockChunkCodec.cpp @@ -17,6 +17,7 @@ #include #include #include +#include #include #include #include @@ -28,75 +29,36 @@ namespace DB { -class CHBlockChunkCodecStream : public ChunkCodecStream + +CHBlockChunkCodecStream::CHBlockChunkCodecStream(const std::vector & field_types) + : ChunkCodecStream(field_types) { -public: - explicit CHBlockChunkCodecStream(const std::vector & field_types) - : ChunkCodecStream(field_types) + for (const auto & field_type : field_types) { - for (const auto & field_type : field_types) - { - expected_types.emplace_back(getDataTypeByFieldTypeForComputingLayer(field_type)); - } + expected_types.emplace_back(getDataTypeByFieldTypeForComputingLayer(field_type)); } +} - String getString() override - { - if (output == nullptr) - { - throw Exception("The output should not be null in getString()"); - } - return output->releaseStr(); - } - virtual WriteBuffer * initOutputBuffer(size_t init_size) +String CHBlockChunkCodecStream::getString() +{ + if (output == nullptr) { - assert(output == nullptr); - output = std::make_unique(init_size); - return output.get(); + throw Exception("The output should not be null in getString()"); } + return output->releaseStr(); +} - void clear() override { output = nullptr; } - void encode(const Block & block, size_t start, size_t end) override; - std::unique_ptr output; - DataTypes expected_types; - - ~CHBlockChunkCodecStream() override = default; -}; - -class CompressCHBlockChunkCodecStream final : public CHBlockChunkCodecStream +WriteBuffer * CHBlockChunkCodecStream::initOutputBuffer(size_t init_size) { - using Base = CHBlockChunkCodecStream; + assert(output == nullptr); + output = std::make_unique(init_size); + return output.get(); +} -public: - explicit CompressCHBlockChunkCodecStream(const std::vector & field_types, CompressionMethod compress_method_) - : Base(field_types) - , compress_method(compress_method_) - { - } - WriteBuffer * initOutputBuffer(size_t init_size) override - { - assert(compress_write_buffer == nullptr); - compress_write_buffer = std::make_unique(*Base::initOutputBuffer(init_size), CompressionSettings(compress_method), init_size); - return compress_write_buffer.get(); - } - void clear() override - { - compress_write_buffer = nullptr; - Base::clear(); - } - String getString() override - { - if (compress_write_buffer == nullptr) - { - throw Exception("The output should not be null in getString()"); - } - compress_write_buffer->next(); - return Base::getString(); - } - CompressionMethod compress_method; - std::unique_ptr compress_write_buffer{}; - ~CompressCHBlockChunkCodecStream() override = default; -}; +void CHBlockChunkCodecStream::clear() +{ + output = nullptr; +} CHBlockChunkCodec::CHBlockChunkCodec( const Block & header_) @@ -175,20 +137,8 @@ void CHBlockChunkCodec::readData(const IDataType & type, IColumn & column, ReadB type.deserializeBinaryBulkWithMultipleStreams(column, input_stream_getter, rows, 0, false, {}); } -void CHBlockChunkCodecStream::encode(const Block & block, size_t start, size_t end) +void EncodeCHBlockChunk(WriteBuffer * ostr_ptr, const Block & block) { - /// only check block schema in CHBlock codec because for both - /// Default codec and Arrow codec, it implicitly convert the - /// input to the target output types. - assertBlockSchema(expected_types, block, "CHBlockChunkCodecStream"); - // Encode data in chunk by chblock encode - if (start != 0 || end != block.rows()) - throw TiFlashException("CHBlock encode only support encode whole block", Errors::Coprocessor::Internal); - - size_t init_size = ApproxBlockBytes(block); - WriteBuffer * ostr_ptr = initOutputBuffer(init_size); - - block.checkNumberOfRows(); size_t columns = block.columns(); size_t rows = block.rows(); @@ -207,11 +157,34 @@ void CHBlockChunkCodecStream::encode(const Block & block, size_t start, size_t e } } -std::unique_ptr CHBlockChunkCodec::newCodecStream(const std::vector & field_types) +void CHBlockChunkCodecStream::encode(const Block & block, size_t start, size_t end) +{ + /// only check block schema in CHBlock codec because for both + /// Default codec and Arrow codec, it implicitly convert the + /// input to the target output types. + assertBlockSchema(expected_types, block, "CHBlockChunkCodecStream"); + // Encode data in chunk by chblock encode + if (start != 0 || end != block.rows()) + throw TiFlashException("CHBlock encode only support encode whole block", Errors::Coprocessor::Internal); + + block.checkNumberOfRows(); + + size_t init_size = ApproxBlockBytes(block); + WriteBuffer * ostr_ptr = initOutputBuffer(init_size); + + return EncodeCHBlockChunk(ostr_ptr, block); +} + +std::unique_ptr NewCHBlockChunkCodecStream(const std::vector & field_types) { return std::make_unique(field_types); } +std::unique_ptr CHBlockChunkCodec::newCodecStream(const std::vector & field_types) +{ + return NewCHBlockChunkCodecStream(field_types); +} + Block CHBlockChunkCodec::decodeImpl(ReadBuffer & istr, size_t reserve_size) { Block res; @@ -294,8 +267,4 @@ Block CHBlockChunkCodec::decode(const String & str, const Block & header) return CHBlockChunkCodec(header).decodeImpl(read_buffer); } -std::unique_ptr CompressedCHBlockChunkCodec::newCodecStream(const std::vector & field_types, CompressionMethod compress_method) -{ - return std::make_unique(field_types, compress_method); -} } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/CHBlockChunkCodecStream.h b/dbms/src/Flash/Coprocessor/CHBlockChunkCodecStream.h new file mode 100644 index 00000000000..cacf94778dc --- /dev/null +++ b/dbms/src/Flash/Coprocessor/CHBlockChunkCodecStream.h @@ -0,0 +1,52 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +class CHBlockChunkCodecStream : public ChunkCodecStream +{ +public: + explicit CHBlockChunkCodecStream(const std::vector & field_types); + String getString() override; + void clear() override; + void encode(const Block & block, size_t start, size_t end) override; + ~CHBlockChunkCodecStream() override = default; + +private: + WriteBuffer * initOutputBuffer(size_t init_size); + std::unique_ptr output; + DataTypes expected_types; +}; + +size_t getExtraInfoSize(const Block & block); +size_t ApproxBlockBytes(const Block & block); +CompressionMethod ToInternalCompressionMethod(mpp::CompressMethod compress_method); +std::unique_ptr NewCHBlockChunkCodecStream(const std::vector & field_types); +void EncodeCHBlockChunk(WriteBuffer * ostr_ptr, const Block & block); + +} // namespace DB diff --git a/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.cpp b/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.cpp index ea3c1da6d8b..acb6c145359 100644 --- a/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.cpp +++ b/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.cpp @@ -16,6 +16,13 @@ #include #include +#include +#include + +#include "Flash/Coprocessor/CHBlockChunkCodecStream.h" +#include "Flash/Coprocessor/CompressCHBlockChunkCodecStream.h" +#include "Flash/Mpp/TrackedMppDataPacket.h" + namespace DB { CHBlockChunkDecodeAndSquash::CHBlockChunkDecodeAndSquash( @@ -26,7 +33,44 @@ CHBlockChunkDecodeAndSquash::CHBlockChunkDecodeAndSquash( { } -std::optional CHBlockChunkDecodeAndSquash::decodeAndSquash(const String & str, mpp::CompressMethod compress_method) +std::optional CHBlockChunkDecodeAndSquash::decodeAndSquashWithCompress(std::string_view sv) +{ + std::optional res; + ReadBufferFromString istr(sv); + auto && compress_buffer = CompressedCHBlockChunkCodec::CompressedReadBuffer(istr); + + if (compress_buffer.eof()) + { + if (accumulated_block) + res.swap(accumulated_block); + return res; + } + + if (!accumulated_block) + { + size_t rows{}; + Block block = DecodeHeader(compress_buffer, codec.header, rows); + DecodeColumns(compress_buffer, block, codec.header.columns(), rows, static_cast(rows_limit * 1.5)); + if (block) + accumulated_block.emplace(std::move(block)); + } + else + { + size_t rows{}; + DecodeHeader(compress_buffer, codec.header, rows); + DecodeColumns(compress_buffer, *accumulated_block, codec.header.columns(), rows, 0); + } + + if (accumulated_block && accumulated_block->rows() >= rows_limit) + { + /// Return accumulated data and reset accumulated_block + res.swap(accumulated_block); + return res; + } + return res; +} + +std::optional CHBlockChunkDecodeAndSquash::decodeAndSquash(const String & str) { std::optional res; ReadBufferFromString istr(str); @@ -36,13 +80,7 @@ std::optional CHBlockChunkDecodeAndSquash::decodeAndSquash(const String & res.swap(accumulated_block); return res; } - std::unique_ptr compress_buffer{}; ReadBuffer * istr_ptr = &istr; - if (ToInternalCompressionMethod(compress_method) != CompressionMethod::NONE) - { - compress_buffer = std::make_unique(istr); - istr_ptr = compress_buffer.get(); - } if (!accumulated_block) { diff --git a/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.h b/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.h index 0b020d48cd4..3fb888d61e9 100644 --- a/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.h +++ b/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.h @@ -18,6 +18,8 @@ #include #include +#include + #include "mpp.pb.h" namespace DB @@ -28,7 +30,8 @@ class CHBlockChunkDecodeAndSquash public: CHBlockChunkDecodeAndSquash(const Block & header, size_t rows_limit_); ~CHBlockChunkDecodeAndSquash() = default; - std::optional decodeAndSquash(const String &, mpp::CompressMethod); + std::optional decodeAndSquash(const String &); + std::optional decodeAndSquashWithCompress(std::string_view); std::optional flush(); private: diff --git a/dbms/src/Flash/Coprocessor/CompressCHBlockChunkCodecStream.cpp b/dbms/src/Flash/Coprocessor/CompressCHBlockChunkCodecStream.cpp new file mode 100644 index 00000000000..126241d2e34 --- /dev/null +++ b/dbms/src/Flash/Coprocessor/CompressCHBlockChunkCodecStream.cpp @@ -0,0 +1,121 @@ + + +#include "CompressCHBlockChunkCodecStream.h" + +#include + +#include +#include + +namespace DB +{ + +void CompressCHBlockChunkCodecStream::encodeHeader(const Block & header, size_t rows) +{ + WriteBuffer * ostr_ptr = getWriter(); + + size_t columns = header.columns(); + writeVarUInt(columns, *ostr_ptr); + writeVarUInt(rows, *ostr_ptr); + + for (size_t i = 0; i < columns; i++) + { + const ColumnWithTypeAndName & column = header.safeGetByPosition(i); + writeStringBinary(column.name, *ostr_ptr); + writeStringBinary(column.type->getName(), *ostr_ptr); + } +} + +extern void writeData(const IDataType & type, const ColumnPtr & column, WriteBuffer & ostr, size_t offset, size_t limit); + +void CompressCHBlockChunkCodecStream::encodeColumn(const ColumnPtr & column, const ColumnWithTypeAndName & type_name) +{ + WriteBuffer * ostr_ptr = getWriter(); + writeVarUInt(column->size(), *ostr_ptr); + writeData(*type_name.type, column, *ostr_ptr, 0, 0); +} +std::unique_ptr NewCompressCHBlockChunkCodecStream(CompressionMethod compress_method) +{ + return std::make_unique(compress_method); +} + +Block DecodeHeader(ReadBuffer & istr, const Block & header, size_t & rows) +{ + Block res; + + if (istr.eof()) + { + return res; + } + + size_t columns = 0; + { + readVarUInt(columns, istr); + readVarUInt(rows, istr); + } + + for (size_t i = 0; i < columns; ++i) + { + ColumnWithTypeAndName column; + { + readBinary(column.name, istr); + if (header) + column.name = header.getByPosition(i).name; + String type_name; + readBinary(type_name, istr); + if (header) + { + CodecUtils::checkDataTypeName(i, header.getByPosition(i).type->getName(), type_name); + column.type = header.getByPosition(i).type; + } + else + { + const auto & data_type_factory = DataTypeFactory::instance(); + column.type = data_type_factory.get(type_name); + } + } + res.insert(std::move(column)); + } + + return res; +} + +void DecodeColumns(ReadBuffer & istr, Block & res, size_t columns, size_t rows, size_t reserve_size) +{ + if (!rows) + return; + + auto && mutable_columns = res.mutateColumns(); + + for (size_t i = 0; i < columns; ++i) + { + /// Data + auto && read_column = mutable_columns[i]; + if (reserve_size > 0) + read_column->reserve(std::max(rows, reserve_size) + read_column->size()); + else + read_column->reserve(rows + read_column->size()); + + size_t read_rows = 0; + for (size_t sz = 0; read_rows < rows; read_rows += sz) + { + readVarUInt(sz, istr); + if (!sz) + continue; + res.getByPosition(i).type->deserializeBinaryBulkWithMultipleStreams( + *read_column, + [&](const IDataType::SubstreamPath &) { + return &istr; + }, + sz, + 0, + {}, + {}); + } + assert(read_rows == rows); + } + + res.setColumns(std::move(mutable_columns)); +} + +} // namespace DB \ No newline at end of file diff --git a/dbms/src/Flash/Coprocessor/CompressCHBlockChunkCodecStream.h b/dbms/src/Flash/Coprocessor/CompressCHBlockChunkCodecStream.h new file mode 100644 index 00000000000..8c96bc10372 --- /dev/null +++ b/dbms/src/Flash/Coprocessor/CompressCHBlockChunkCodecStream.h @@ -0,0 +1,112 @@ +#pragma once + +#include +#include +#include +#include +#include +#include + +#include "Columns/IColumn.h" +#include "CompressedCHBlockChunkCodec.h" + +namespace DB +{ + +struct WriteBufferFromOwnStringList + : public WriteBuffer + , public boost::noncopyable +{ + WriteBufferFromOwnStringList() + : WriteBuffer(nullptr, 0) + { + } + + void nextImpl() override + { + buffs.emplace_back(std::string(DBMS_DEFAULT_BUFFER_SIZE, 0)); + WriteBuffer::set(buffs.back().data(), buffs.back().size()); + }; + + std::string getString() + { + next(); + + std::string res; + size_t sz = std::accumulate(buffs.begin(), buffs.end(), 0, [](const auto r, const auto & s) { + return r + s.size(); + }); + res.resize(sz); + char * start = res.data(); + std::for_each(buffs.begin(), buffs.end(), [&](auto & s) { + std::memcpy(start, s.data(), s.size()); + start += s.size(); + s.clear(); + }); + + clear(); + return res; + } + + void clear() + { + buffs.clear(); + WriteBuffer::set(nullptr, 0); + } + + std::vector buffs; +}; + +struct CompressCHBlockChunkCodecStream +{ + explicit CompressCHBlockChunkCodecStream(CompressionMethod compress_method_) + : compress_method(compress_method_) + { + output_buffer = std::make_unique(); + compress_write_buffer = std::make_unique(*output_buffer, CompressionSettings(compress_method)); + } + void clear() + { + compress_write_buffer->next(); + output_buffer->clear(); + } + + WriteBuffer * getWriterWithoutCompress() const + { + return output_buffer.get(); + } + + WriteBuffer * getWriter() + { + return compress_write_buffer.get(); + } + + std::string getString() + { + if (compress_write_buffer == nullptr) + { + throw Exception("The output should not be null in getString()"); + } + compress_write_buffer->next(); + output_buffer->next(); + return output_buffer->getString(); + } + ~CompressCHBlockChunkCodecStream() = default; + + // void disableCompress() { enable_compress = false; } + // void enableCompress() { enable_compress = true; } + + void encodeHeader(const Block & header, size_t rows); + void encodeColumn(const ColumnPtr & column, const ColumnWithTypeAndName & type_name); + + // bool enable_compress{true}; + CompressionMethod compress_method; + std::unique_ptr output_buffer{}; + std::unique_ptr compress_write_buffer{}; +}; + +void DecodeColumns(ReadBuffer & istr, Block & res, size_t columns, size_t rows, size_t reserve_size = 0); +Block DecodeHeader(ReadBuffer & istr, const Block & header, size_t & rows); + +std::unique_ptr NewCompressCHBlockChunkCodecStream(CompressionMethod compress_method); +} // namespace DB \ No newline at end of file diff --git a/dbms/src/Flash/Coprocessor/CompressedCHBlockChunkCodec.h b/dbms/src/Flash/Coprocessor/CompressedCHBlockChunkCodec.h index bd8db5191ec..831ef536954 100644 --- a/dbms/src/Flash/Coprocessor/CompressedCHBlockChunkCodec.h +++ b/dbms/src/Flash/Coprocessor/CompressedCHBlockChunkCodec.h @@ -31,12 +31,7 @@ class CompressedCHBlockChunkCodec public: using CompressedReadBuffer = CompressedReadBuffer; using CompressedWriteBuffer = CompressedWriteBuffer; - - static std::unique_ptr newCodecStream(const std::vector & field_types, CompressionMethod compress_method); }; -CompressionMethod ToInternalCompressionMethod(mpp::CompressMethod compress_method); - -size_t ApproxBlockBytes(const Block & block); } // namespace DB diff --git a/dbms/src/Flash/FlashService.cpp b/dbms/src/Flash/FlashService.cpp index 1a879a65d3a..e6094f28b33 100644 --- a/dbms/src/Flash/FlashService.cpp +++ b/dbms/src/Flash/FlashService.cpp @@ -436,6 +436,7 @@ ::grpc::Status FlashService::cancelMPPTaskForTest(const ::mpp::CancelTaskRequest if (!status.ok()) { auto err = std::make_unique(); + err->set_mpp_version(TiDB::GetMppVersion()); err->set_msg("error status"); response->set_allocated_error(err.release()); return status; diff --git a/dbms/src/Flash/Mpp/ExchangeReceiver.cpp b/dbms/src/Flash/Mpp/ExchangeReceiver.cpp index ac54083a6ec..eaacd8b76ff 100644 --- a/dbms/src/Flash/Mpp/ExchangeReceiver.cpp +++ b/dbms/src/Flash/Mpp/ExchangeReceiver.cpp @@ -23,8 +23,11 @@ #include #include +#include #include +#include "mpp.pb.h" + namespace DB { namespace FailPoints @@ -689,13 +692,29 @@ DecodeDetail ExchangeReceiverBase::decodeChunks( if (recv_msg->chunks.empty()) return detail; auto & packet = recv_msg->packet->getPacket(); - auto compress_method = packet.compress().method(); // Record total packet size even if fine grained shuffle is enabled. detail.packet_bytes = packet.ByteSizeLong(); + + if (packet.mpp_version() && packet.compress().method() != mpp::CompressMethod::NONE) + { + for (auto && chunk : packet.chunks()) + { + auto && result = decoder_ptr->decodeAndSquashWithCompress(chunk); + if (!result) + continue; + detail.rows += result->rows(); + if likely (result->rows() > 0) + { + block_queue.push(std::move(*result)); + } + } + return detail; + } + for (const String * chunk : recv_msg->chunks) { - auto result = decoder_ptr->decodeAndSquash(*chunk, compress_method); + auto result = decoder_ptr->decodeAndSquash(*chunk); if (!result) continue; detail.rows += result->rows(); diff --git a/dbms/src/Flash/Mpp/HashPartitionWriter.cpp b/dbms/src/Flash/Mpp/HashPartitionWriter.cpp index 9609d63d9fe..1f0befe0736 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriter.cpp +++ b/dbms/src/Flash/Mpp/HashPartitionWriter.cpp @@ -24,7 +24,9 @@ #include "Common/Exception.h" #include "Common/Stopwatch.h" +#include "Flash/Coprocessor/CHBlockChunkCodecStream.h" #include "Flash/Coprocessor/CompressedCHBlockChunkCodec.h" +#include "Flash/Mpp/MppVersion.h" #include "IO/CompressedStream.h" #include "common/logger_useful.h" #include "ext/scope_guard.h" @@ -45,17 +47,17 @@ HashPartitionWriter::HashPartitionWriter( , writer(writer_) , partition_col_ids(std::move(partition_col_ids_)) , collators(std::move(collators_)) - , compress_method(dag_context.getExchangeSenderMeta().compress()) + // , compress_method(dag_context.getExchangeSenderMeta().compress()) { rows_in_blocks = 0; partition_num = writer_->getPartitionNum(); RUNTIME_CHECK(partition_num > 0); RUNTIME_CHECK(dag_context.encode_type == tipb::EncodeType::TypeCHBlock); - if (auto method = ToInternalCompressionMethod(compress_method); method != CompressionMethod::NONE) - { - compress_chunk_codec_stream = CompressedCHBlockChunkCodec::newCodecStream(dag_context.result_field_types, method); - } + // if (auto method = ToInternalCompressionMethod(compress_method); method != CompressionMethod::NONE) + // { + // compress_chunk_codec_stream = CompressedCHBlockChunkCodec::newCodecStream(dag_context.result_field_types, method); + // } chunk_codec_stream = std::make_unique()->newCodecStream(dag_context.result_field_types); } @@ -108,12 +110,13 @@ void HashPartitionWriter::partitionAndEncodeThenWriteBlocks() for (size_t part_id = 0; part_id < partition_num; ++part_id) { - auto method = compress_method; - if (writer->getTunnels()[part_id]->isLocal()) - { - method = mpp::CompressMethod::NONE; - } - tracked_packets[part_id]->getPacket().mutable_compress()->set_method(method); + // auto method = compress_method; + // if (writer->getTunnels()[part_id]->isLocal()) + // { + // method = mpp::CompressMethod::NONE; + // } + // tracked_packets[part_id]->getPacket().mutable_compress()->set_method(method); + tracked_packets[part_id]->getPacket().set_mpp_version(TiDB::GetMppVersion()); } size_t ori_block_mem_size = 0; @@ -143,12 +146,12 @@ void HashPartitionWriter::partitionAndEncodeThenWriteBlocks() if (dest_block_rows > 0) { auto * codec_stream = chunk_codec_stream.get(); - if (tracked_packets[part_id]->getPacket().compress().method() != mpp::CompressMethod::NONE) - { - assert(compress_chunk_codec_stream); - // no need compress - codec_stream = compress_chunk_codec_stream.get(); - } + // if (tracked_packets[part_id]->getPacket().compress().method() != mpp::CompressMethod::NONE) + // { + // assert(compress_chunk_codec_stream); + // // no need compress + // codec_stream = compress_chunk_codec_stream.get(); + // } codec_stream->encode(dest_block, 0, dest_block_rows); // ori_block_mem_size += ApproxBlockBytes(dest_block); tracked_packets[part_id]->addChunk(codec_stream->getString()); diff --git a/dbms/src/Flash/Mpp/HashPartitionWriter.h b/dbms/src/Flash/Mpp/HashPartitionWriter.h index fe9817a3487..3e01661adf7 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriter.h +++ b/dbms/src/Flash/Mpp/HashPartitionWriter.h @@ -56,9 +56,9 @@ class HashPartitionWriter : public DAGResponseWriter size_t rows_in_blocks; uint16_t partition_num; - mpp::CompressMethod compress_method{}; + // mpp::CompressMethod compress_method{}; std::unique_ptr chunk_codec_stream; - std::unique_ptr compress_chunk_codec_stream; + // std::unique_ptr compress_chunk_codec_stream; }; } // namespace DB diff --git a/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp b/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp new file mode 100644 index 00000000000..582aac9d08f --- /dev/null +++ b/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp @@ -0,0 +1,303 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include +#include +#include +#include +#include +#include +#include + +#include +#include + +#include "Common/Exception.h" +#include "Common/Stopwatch.h" +#include "Flash/Coprocessor/CHBlockChunkCodecStream.h" +#include "Flash/Coprocessor/CompressCHBlockChunkCodecStream.h" +#include "Flash/Coprocessor/CompressedCHBlockChunkCodec.h" +#include "Flash/Mpp/MppVersion.h" +#include "IO/CompressedStream.h" +#include "common/logger_useful.h" +#include "ext/scope_guard.h" + +namespace DB +{ +template +HashPartitionWriterV1::HashPartitionWriterV1( + ExchangeWriterPtr writer_, + std::vector partition_col_ids_, + TiDB::TiDBCollators collators_, + Int64 batch_send_min_limit_, + bool should_send_exec_summary_at_last_, + DAGContext & dag_context_) + : DAGResponseWriter(/*records_per_chunk=*/-1, dag_context_) + , batch_send_min_limit(batch_send_min_limit_) + , should_send_exec_summary_at_last(should_send_exec_summary_at_last_) + , writer(writer_) + , partition_col_ids(std::move(partition_col_ids_)) + , collators(std::move(collators_)) + , compress_method(dag_context.getExchangeSenderMeta().compress()) +{ + assert(compress_method != mpp::CompressMethod::NONE); + assert(dag_context.getMPPTaskMeta().mpp_version() > 0); + + rows_in_blocks = 0; + partition_num = writer_->getPartitionNum(); + RUNTIME_CHECK(partition_num > 0); + RUNTIME_CHECK(dag_context.encode_type == tipb::EncodeType::TypeCHBlock); + for (const auto & field_type : dag_context.result_field_types) + { + expected_types.emplace_back(getDataTypeByFieldTypeForComputingLayer(field_type)); + } + compress_chunk_codec_stream = NewCompressCHBlockChunkCodecStream(ToInternalCompressionMethod(compress_method)); +} + +template +void HashPartitionWriterV1::finishWrite() +{ + assert(0 == rows_in_blocks); + if (should_send_exec_summary_at_last) + sendExecutionSummary(); +} + +template +void HashPartitionWriterV1::sendExecutionSummary() +{ + tipb::SelectResponse response; + summary_collector.addExecuteSummaries(response); + writer->sendExecutionSummary(response); +} + +template +void HashPartitionWriterV1::flush() +{ + if (rows_in_blocks > 0) + partitionAndEncodeThenWriteBlocks(); +} + +template +void HashPartitionWriterV1::write(const Block & block) +{ + RUNTIME_CHECK_MSG( + block.columns() == dag_context.result_field_types.size(), + "Output column size mismatch with field type size"); + size_t rows = block.rows(); + rows_in_blocks += rows; + if (rows > 0) + { + blocks.push_back(block); + } + + if (static_cast(rows_in_blocks) > batch_send_min_limit) + partitionAndEncodeThenWriteBlocks(); +} + +template +void HashPartitionWriterV1::partitionAndEncodeThenWriteBlocks() +{ + assert(compress_chunk_codec_stream); + + auto tracked_packets = HashBaseWriterHelper::createPackets(partition_num); + + for (size_t part_id = 0; part_id < partition_num; ++part_id) + { + tracked_packets[part_id]->getPacket().set_mpp_version(TiDB::GetMppVersion()); + + auto method = compress_method; + if (writer->getTunnels()[part_id]->isLocal()) + { + method = mpp::CompressMethod::NONE; + } + tracked_packets[part_id]->getPacket().mutable_compress()->set_method(method); + } + + size_t ori_block_mem_size = 0; + + if (!blocks.empty()) + { + assert(rows_in_blocks > 0); + + HashBaseWriterHelper::materializeBlocks(blocks); + Block dest_block_header = blocks[0].cloneEmpty(); + assertBlockSchema(expected_types, dest_block_header, "HashPartitionWriterV1"); + + std::vector partition_key_containers(collators.size()); + std::vector> dest_columns(partition_num); + size_t total_rows = 0; + + while (!blocks.empty()) + { + const auto & block = blocks.back(); + block.checkNumberOfRows(); + assertBlockSchema(expected_types, block, "HashPartitionWriterV1"); + + ori_block_mem_size += ApproxBlockBytes(block); + total_rows += block.rows(); + + auto dest_tbl_cols = HashBaseWriterHelper::createDestColumns(block, partition_num); + HashBaseWriterHelper::scatterColumns(block, partition_num, collators, partition_key_containers, partition_col_ids, dest_tbl_cols); + blocks.pop_back(); + + for (size_t part_id = 0; part_id < partition_num; ++part_id) + { + auto & columns = dest_tbl_cols[part_id]; + dest_columns[part_id].emplace_back(std::move(columns)); + } + } + { + size_t rows = 0; + for (size_t part_id = 0; part_id < partition_num; ++part_id) + { + for (auto && columns : dest_columns[part_id]) + { + rows += columns[0]->size(); + } + } + RUNTIME_CHECK(rows == total_rows, rows, total_rows); + } + + LOG_DEBUG(&Poco::Logger::get("tzg"), "send total_rows is {}", total_rows); + + for (size_t part_id = 0; part_id < partition_num; ++part_id) + { + if (tracked_packets[part_id]->getPacket().compress().method() == mpp::NONE) + { + auto * ostr_ptr = compress_chunk_codec_stream->getWriterWithoutCompress(); + RUNTIME_CHECK(ostr_ptr != nullptr); + + LOG_DEBUG(&Poco::Logger::get("tzg"), "compress().method local"); + for (auto && columns : dest_columns[part_id]) + { + dest_block_header.setColumns(std::move(columns)); + EncodeCHBlockChunk(ostr_ptr, dest_block_header); + tracked_packets[part_id]->getPacket().add_chunks(compress_chunk_codec_stream->getString()); + compress_chunk_codec_stream->clear(); + + { + const auto & chunks = tracked_packets[part_id]->getPacket().chunks(); + const auto & dd = chunks[chunks.size() - 1]; + + auto res = CHBlockChunkCodec::decode(dd, dest_block_header); + RUNTIME_CHECK(res.rows() == dest_block_header.rows(), res.rows(), dest_block_header.rows()); + RUNTIME_CHECK(res.columns() == dest_block_header.columns(), res.columns(), dest_block_header.columns()); + res.checkNumberOfRows(); + for (size_t i = 0; i < res.columns(); ++i) + { + RUNTIME_CHECK(dest_block_header.getByPosition(i) == res.getByPosition(i)); + } + } + } + + LOG_DEBUG(&Poco::Logger::get("tzg"), "compress().method local done"); + } + else + { + LOG_DEBUG(&Poco::Logger::get("tzg"), "compress().method compress"); + + compress_chunk_codec_stream->encodeHeader(dest_block_header, total_rows); + for (size_t col_index = 0; col_index < dest_block_header.columns(); ++col_index) + { + auto && col_type_name = dest_block_header.getByPosition(col_index); + for (auto && columns : dest_columns[part_id]) + { + compress_chunk_codec_stream->encodeColumn(std::move(columns[col_index]), col_type_name); + } + } + + LOG_DEBUG(&Poco::Logger::get("tzg"), "compress().method compress done"); + + tracked_packets[part_id]->getPacket().add_chunks(compress_chunk_codec_stream->getString()); + compress_chunk_codec_stream->clear(); + } + + if (tracked_packets[part_id]->getPacket().compress().method() != mpp::NONE) + { + // decode + const auto & chunks = tracked_packets[part_id]->getPacket().chunks(); + const auto & dd = chunks[chunks.size() - 1]; + + ReadBufferFromString istr(dd); + auto && compress_buffer = CompressedCHBlockChunkCodec::CompressedReadBuffer(istr); + + size_t rows{}; + Block res = DecodeHeader(compress_buffer, {}, rows); + DecodeColumns(compress_buffer, res, res.columns(), rows); + RUNTIME_CHECK(res.rows() == total_rows, res.rows(), total_rows); + RUNTIME_CHECK(res.columns() == dest_block_header.columns(), res.columns(), dest_block_header.columns()); + res.checkNumberOfRows(); + for (size_t i = 0; i < res.columns(); ++i) + { + RUNTIME_CHECK(dest_block_header.getByPosition(i) == res.getByPosition(i)); + } + } + } + assert(blocks.empty()); + rows_in_blocks = 0; + } + + writePackets(tracked_packets); + + GET_METRIC(tiflash_exchange_data_bytes, type_hash_original_all).Increment(ori_block_mem_size); +} + +template +void HashPartitionWriterV1::writePackets(const TrackedMppDataPacketPtrs & packets) +{ + for (size_t part_id = 0; part_id < packets.size(); ++part_id) + { + const auto & packet = packets[part_id]; + assert(packet); + + auto & inner_packet = packet->getPacket(); + if (likely(inner_packet.chunks_size() > 0)) + { + writer->partitionWrite(packet, part_id); + + auto sz = inner_packet.ByteSizeLong(); + switch (inner_packet.compress().method()) + { + case mpp::NONE: + { + if (writer->getTunnels()[part_id]->isLocal()) + { + GET_METRIC(tiflash_exchange_data_bytes, type_hash_none_local).Increment(sz); + } + else + { + GET_METRIC(tiflash_exchange_data_bytes, type_hash_none).Increment(sz); + } + break; + } + case mpp::LZ4: + { + GET_METRIC(tiflash_exchange_data_bytes, type_hash_lz4).Increment(sz); + break; + } + case mpp::ZSTD: + { + GET_METRIC(tiflash_exchange_data_bytes, type_hash_zstd).Increment(sz); + break; + } + default: + break; + } + } + } +} + +template class HashPartitionWriterV1; + +} // namespace DB diff --git a/dbms/src/Flash/Mpp/HashPartitionWriterV1.h b/dbms/src/Flash/Mpp/HashPartitionWriterV1.h new file mode 100644 index 00000000000..0d7dc804711 --- /dev/null +++ b/dbms/src/Flash/Mpp/HashPartitionWriterV1.h @@ -0,0 +1,66 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#pragma once + +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +struct CompressCHBlockChunkCodecStream; + +template +class HashPartitionWriterV1 : public DAGResponseWriter +{ +public: + HashPartitionWriterV1( + ExchangeWriterPtr writer_, + std::vector partition_col_ids_, + TiDB::TiDBCollators collators_, + Int64 batch_send_min_limit_, + bool should_send_exec_summary_at_last, + DAGContext & dag_context_); + void write(const Block & block) override; + void flush() override; + void finishWrite() override; + +private: + void partitionAndEncodeThenWriteBlocks(); + + void writePackets(const TrackedMppDataPacketPtrs & packets); + + void sendExecutionSummary(); + +private: + Int64 batch_send_min_limit; + bool should_send_exec_summary_at_last; + ExchangeWriterPtr writer; + std::vector blocks; + std::vector partition_col_ids; + TiDB::TiDBCollators collators; + size_t rows_in_blocks; + uint16_t partition_num; + DataTypes expected_types; + mpp::CompressMethod compress_method{}; + std::unique_ptr compress_chunk_codec_stream; +}; + +} // namespace DB diff --git a/dbms/src/Flash/Mpp/Utils.cpp b/dbms/src/Flash/Mpp/Utils.cpp index 1db7519d41f..a2723b812bb 100644 --- a/dbms/src/Flash/Mpp/Utils.cpp +++ b/dbms/src/Flash/Mpp/Utils.cpp @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include #include #include #include @@ -26,6 +27,7 @@ mpp::MPPDataPacket getPacketWithError(String reason) { mpp::MPPDataPacket data; auto err = std::make_unique(); + err->set_mpp_version(TiDB::GetMppVersion()); err->set_msg(std::move(reason)); data.set_allocated_error(err.release()); return data; @@ -46,41 +48,49 @@ void trimStackTrace(String & message) namespace TiDB { -constexpr int64_t MPP_VERSION_V0 = 0; -constexpr int64_t MPP_VERSION_V1 = 1; -constexpr int64_t MPP_VERSION_MAX = 2; +enum class MppVersion : int64_t +{ + V0 = 0, + V1, + MAX, +}; -// constexpr int64_t MPP_VERSION_V2 = MPP_VERSION_V1 * 2; -constexpr int64_t MPP_VERSION = MPP_VERSION_V1; +static constexpr int64_t toInt64(MppVersion v) +{ + return static_cast(v); +} -static_assert(MPP_VERSION < MPP_VERSION_MAX && MPP_VERSION >= MPP_VERSION_V0); +// Latest mpp-version supported by TiFlash +constexpr MppVersion MPP_VERSION = MppVersion(toInt64(MppVersion::MAX) - 1); // TODO: set version after committed -constexpr std::array MPP_TIFLASH_RELEASE_VERSION = {"", "?"}; +constexpr std::array MPP_TIFLASH_RELEASE_VERSION = {"", "?"}; +// Check mpp-version is illegal bool CheckMppVersion(int64_t mpp_version) { - return mpp_version >= MPP_VERSION_V0 && mpp_version <= MPP_VERSION_V1; + return mpp_version >= toInt64(MppVersion::V0) && mpp_version < toInt64(MppVersion::MAX); } std::string GenMppVersionErrorMessage(int64_t mpp_version) { auto err_msg = fmt::format("invalid mpp version `{}`, expect version: min `{}`, max `{}` release version `{}`", mpp_version, - TiDB::MPP_VERSION_V0, - TiDB::MPP_VERSION_V1, - MPP_TIFLASH_RELEASE_VERSION[TiDB::MPP_VERSION_V1]); + toInt64(MppVersion::V0), + toInt64(MPP_VERSION), + MPP_TIFLASH_RELEASE_VERSION[toInt64(MPP_VERSION)]); return err_msg; } +// Get latest mpp-version supported by TiFlash int64_t GetMppVersion() { - return MPP_VERSION; + return toInt64(MPP_VERSION); } std::string GetMppVersionReleaseInfo(int64_t mpp_version) { - if (mpp_version >= MPP_VERSION_V0 && mpp_version < MPP_VERSION_MAX) + if (CheckMppVersion(mpp_version)) { return MPP_TIFLASH_RELEASE_VERSION[mpp_version]; } diff --git a/dbms/src/Flash/Mpp/newMPPExchangeWriter.h b/dbms/src/Flash/Mpp/newMPPExchangeWriter.h index 5ea25c86ec3..548a256a09e 100644 --- a/dbms/src/Flash/Mpp/newMPPExchangeWriter.h +++ b/dbms/src/Flash/Mpp/newMPPExchangeWriter.h @@ -17,7 +17,6 @@ #include #include #include -#include namespace DB { @@ -33,65 +32,6 @@ std::unique_ptr NewMPPExchangeWriter( DAGContext & dag_context, bool enable_fine_grained_shuffle, UInt64 fine_grained_shuffle_stream_count, - UInt64 fine_grained_shuffle_batch_size) -{ - RUNTIME_CHECK(dag_context.isMPPTask()); - should_send_exec_summary_at_last = dag_context.collect_execution_summaries && should_send_exec_summary_at_last; - if (dag_context.isRootMPPTask()) - { - // No need to use use data compression - RUNTIME_CHECK(dag_context.getExchangeSenderMeta().compress() == mpp::CompressMethod::NONE); - - RUNTIME_CHECK(!enable_fine_grained_shuffle); - RUNTIME_CHECK(exchange_type == tipb::ExchangeType::PassThrough); - return std::make_unique>( - writer, - records_per_chunk, - batch_send_min_limit, - should_send_exec_summary_at_last, - dag_context); - } - else - { - if (exchange_type == tipb::ExchangeType::Hash) - { - if (enable_fine_grained_shuffle) - { - // TODO: support data compression if necessary - RUNTIME_CHECK(dag_context.getExchangeSenderMeta().compress() == mpp::CompressMethod::NONE); - - return std::make_unique>( - writer, - partition_col_ids, - partition_col_collators, - should_send_exec_summary_at_last, - dag_context, - fine_grained_shuffle_stream_count, - fine_grained_shuffle_batch_size); - } - else - { - return std::make_unique>( - writer, - partition_col_ids, - partition_col_collators, - batch_send_min_limit, - should_send_exec_summary_at_last, - dag_context); - } - } - else - { - // TODO: support data compression if necessary - RUNTIME_CHECK(dag_context.getExchangeSenderMeta().compress() == mpp::CompressMethod::NONE); + UInt64 fine_grained_shuffle_batch_size); - RUNTIME_CHECK(!enable_fine_grained_shuffle); - return std::make_unique>( - writer, - batch_send_min_limit, - should_send_exec_summary_at_last, - dag_context); - } - } -} } // namespace DB diff --git a/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp b/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp index c123113aef8..7a927a7ca7d 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp +++ b/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp @@ -18,10 +18,14 @@ #include #include #include +#include #include #include #include +#include "Flash/Coprocessor/CompressCHBlockChunkCodecStream.h" +#include "Flash/Mpp/HashPartitionWriterV1.h" + namespace DB { PhysicalPlanNodePtr PhysicalExchangeSender::build( @@ -95,4 +99,88 @@ const Block & PhysicalExchangeSender::getSampleBlock() const { return child->getSampleBlock(); } + +template <> +std::unique_ptr NewMPPExchangeWriter( + const MPPTunnelSetPtr & writer, + const std::vector & partition_col_ids, + const TiDB::TiDBCollators & partition_col_collators, + const tipb::ExchangeType & exchange_type, + Int64 records_per_chunk, + Int64 batch_send_min_limit, + bool should_send_exec_summary_at_last, + DAGContext & dag_context, + bool enable_fine_grained_shuffle, + UInt64 fine_grained_shuffle_stream_count, + UInt64 fine_grained_shuffle_batch_size) +{ + RUNTIME_CHECK(dag_context.isMPPTask()); + should_send_exec_summary_at_last = dag_context.collect_execution_summaries && should_send_exec_summary_at_last; + if (dag_context.isRootMPPTask()) + { + // No need to use use data compression + RUNTIME_CHECK(dag_context.getExchangeSenderMeta().compress() == mpp::CompressMethod::NONE); + + RUNTIME_CHECK(!enable_fine_grained_shuffle); + RUNTIME_CHECK(exchange_type == tipb::ExchangeType::PassThrough); + return std::make_unique>( + writer, + records_per_chunk, + batch_send_min_limit, + should_send_exec_summary_at_last, + dag_context); + } + else + { + if (exchange_type == tipb::ExchangeType::Hash) + { + if (enable_fine_grained_shuffle) + { + // TODO: support data compression if necessary + RUNTIME_CHECK(dag_context.getExchangeSenderMeta().compress() == mpp::CompressMethod::NONE); + + return std::make_unique>( + writer, + partition_col_ids, + partition_col_collators, + should_send_exec_summary_at_last, + dag_context, + fine_grained_shuffle_stream_count, + fine_grained_shuffle_batch_size); + } + else + { + auto && compress_method = dag_context.getExchangeSenderMeta().compress(); + if (compress_method == mpp::CompressMethod::NONE || !dag_context.getMPPTaskMeta().mpp_version()) + return std::make_unique>( + writer, + partition_col_ids, + partition_col_collators, + batch_send_min_limit, + should_send_exec_summary_at_last, + dag_context); + return std::make_unique>( + writer, + partition_col_ids, + partition_col_collators, + batch_send_min_limit, + should_send_exec_summary_at_last, + dag_context); + } + } + else + { + // TODO: support data compression if necessary + RUNTIME_CHECK(dag_context.getExchangeSenderMeta().compress() == mpp::CompressMethod::NONE); + + RUNTIME_CHECK(!enable_fine_grained_shuffle); + return std::make_unique>( + writer, + batch_send_min_limit, + should_send_exec_summary_at_last, + dag_context); + } + } +} + } // namespace DB From 148235bad12a20c5ccf019820535ad7918e98e88 Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Fri, 30 Dec 2022 20:41:38 +0800 Subject: [PATCH 30/93] 24 Signed-off-by: Zhigao Tong --- dbms/CMakeLists.txt | 2 +- .../CompressCHBlockChunkCodecStream.h | 10 ++- dbms/src/Flash/Coprocessor/DAGContext.h | 6 ++ dbms/src/Flash/Mpp/HashPartitionWriter.cpp | 2 +- dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp | 20 ++--- dbms/src/Flash/Mpp/MPPTunnelSet.cpp | 7 ++ dbms/src/Flash/Mpp/MPPTunnelSet.h | 3 + .../Mpp/tests/gtest_mpp_exchange_writer.cpp | 79 ++++++++++++++++++- 8 files changed, 109 insertions(+), 20 deletions(-) diff --git a/dbms/CMakeLists.txt b/dbms/CMakeLists.txt index 025dda30e48..b8613754165 100644 --- a/dbms/CMakeLists.txt +++ b/dbms/CMakeLists.txt @@ -309,7 +309,7 @@ if (ENABLE_TESTS) macro(grep_gtest_sources BASE_DIR DST_VAR) # Cold match files that are not in tests/ directories - file(GLOB_RECURSE "${DST_VAR}" RELATIVE "${BASE_DIR}" "gtest*.cpp") + file(GLOB_RECURSE "${DST_VAR}" RELATIVE "${BASE_DIR}" "gtest_mpp_exchange_writer.cpp") endmacro() macro(grep_bench_sources BASE_DIR DST_VAR) diff --git a/dbms/src/Flash/Coprocessor/CompressCHBlockChunkCodecStream.h b/dbms/src/Flash/Coprocessor/CompressCHBlockChunkCodecStream.h index 8c96bc10372..164469b2a6d 100644 --- a/dbms/src/Flash/Coprocessor/CompressCHBlockChunkCodecStream.h +++ b/dbms/src/Flash/Coprocessor/CompressCHBlockChunkCodecStream.h @@ -13,7 +13,7 @@ namespace DB { -struct WriteBufferFromOwnStringList +struct WriteBufferFromOwnStringList final : public WriteBuffer , public boost::noncopyable { @@ -48,6 +48,12 @@ struct WriteBufferFromOwnStringList return res; } + void init() + { + clear(); + nextImpl(); + } + void clear() { buffs.clear(); @@ -71,7 +77,7 @@ struct CompressCHBlockChunkCodecStream output_buffer->clear(); } - WriteBuffer * getWriterWithoutCompress() const + WriteBufferFromOwnStringList * getWriterWithoutCompress() const { return output_buffer.get(); } diff --git a/dbms/src/Flash/Coprocessor/DAGContext.h b/dbms/src/Flash/Coprocessor/DAGContext.h index 737e9e9095f..19a1888181a 100644 --- a/dbms/src/Flash/Coprocessor/DAGContext.h +++ b/dbms/src/Flash/Coprocessor/DAGContext.h @@ -37,6 +37,11 @@ namespace DB { +namespace tests +{ +class TestMPPExchangeWriter; +} + class Context; class MPPTunnelSet; class ExchangeReceiver; @@ -366,6 +371,7 @@ class DAGContext private: void initExecutorIdToJoinIdMap(); void initOutputInfo(); + friend class tests::TestMPPExchangeWriter; private: std::shared_ptr process_list_entry; diff --git a/dbms/src/Flash/Mpp/HashPartitionWriter.cpp b/dbms/src/Flash/Mpp/HashPartitionWriter.cpp index 1f0befe0736..9fe0fe9e2b3 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriter.cpp +++ b/dbms/src/Flash/Mpp/HashPartitionWriter.cpp @@ -186,7 +186,7 @@ void HashPartitionWriter::writePackets(const TrackedMppDataPa { case mpp::NONE: { - if (writer->getTunnels()[part_id]->isLocal()) + if (writer->isLocal(part_id)) { GET_METRIC(tiflash_exchange_data_bytes, type_hash_none_local).Increment(sz); } diff --git a/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp b/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp index 582aac9d08f..0c6c7482d6a 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp +++ b/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp @@ -117,7 +117,7 @@ void HashPartitionWriterV1::partitionAndEncodeThenWriteBlocks tracked_packets[part_id]->getPacket().set_mpp_version(TiDB::GetMppVersion()); auto method = compress_method; - if (writer->getTunnels()[part_id]->isLocal()) + if (writer->isLocal(part_id)) { method = mpp::CompressMethod::NONE; } @@ -169,22 +169,18 @@ void HashPartitionWriterV1::partitionAndEncodeThenWriteBlocks RUNTIME_CHECK(rows == total_rows, rows, total_rows); } - LOG_DEBUG(&Poco::Logger::get("tzg"), "send total_rows is {}", total_rows); - for (size_t part_id = 0; part_id < partition_num; ++part_id) { if (tracked_packets[part_id]->getPacket().compress().method() == mpp::NONE) { auto * ostr_ptr = compress_chunk_codec_stream->getWriterWithoutCompress(); - RUNTIME_CHECK(ostr_ptr != nullptr); - - LOG_DEBUG(&Poco::Logger::get("tzg"), "compress().method local"); for (auto && columns : dest_columns[part_id]) { dest_block_header.setColumns(std::move(columns)); + ostr_ptr->init(); EncodeCHBlockChunk(ostr_ptr, dest_block_header); - tracked_packets[part_id]->getPacket().add_chunks(compress_chunk_codec_stream->getString()); - compress_chunk_codec_stream->clear(); + tracked_packets[part_id]->getPacket().add_chunks(ostr_ptr->getString()); + ostr_ptr->clear(); { const auto & chunks = tracked_packets[part_id]->getPacket().chunks(); @@ -200,13 +196,9 @@ void HashPartitionWriterV1::partitionAndEncodeThenWriteBlocks } } } - - LOG_DEBUG(&Poco::Logger::get("tzg"), "compress().method local done"); } else { - LOG_DEBUG(&Poco::Logger::get("tzg"), "compress().method compress"); - compress_chunk_codec_stream->encodeHeader(dest_block_header, total_rows); for (size_t col_index = 0; col_index < dest_block_header.columns(); ++col_index) { @@ -217,8 +209,6 @@ void HashPartitionWriterV1::partitionAndEncodeThenWriteBlocks } } - LOG_DEBUG(&Poco::Logger::get("tzg"), "compress().method compress done"); - tracked_packets[part_id]->getPacket().add_chunks(compress_chunk_codec_stream->getString()); compress_chunk_codec_stream->clear(); } @@ -271,7 +261,7 @@ void HashPartitionWriterV1::writePackets(const TrackedMppData { case mpp::NONE: { - if (writer->getTunnels()[part_id]->isLocal()) + if (writer->isLocal(part_id)) { GET_METRIC(tiflash_exchange_data_bytes, type_hash_none_local).Increment(sz); } diff --git a/dbms/src/Flash/Mpp/MPPTunnelSet.cpp b/dbms/src/Flash/Mpp/MPPTunnelSet.cpp index 1ac6c99e655..9be2d67a9b1 100644 --- a/dbms/src/Flash/Mpp/MPPTunnelSet.cpp +++ b/dbms/src/Flash/Mpp/MPPTunnelSet.cpp @@ -137,6 +137,13 @@ typename MPPTunnelSetBase::TunnelPtr MPPTunnelSetBase::getTunnel return tunnels[it->second]; } +template +bool MPPTunnelSetBase::isLocal(size_t index) const +{ + assert(getPartitionNum() > index); + return getTunnels()[index]->isLocal(); +} + /// Explicit template instantiations - to avoid code bloat in headers. template class MPPTunnelSetBase; diff --git a/dbms/src/Flash/Mpp/MPPTunnelSet.h b/dbms/src/Flash/Mpp/MPPTunnelSet.h index 263a01fb901..c5bd314c528 100644 --- a/dbms/src/Flash/Mpp/MPPTunnelSet.h +++ b/dbms/src/Flash/Mpp/MPPTunnelSet.h @@ -71,6 +71,9 @@ class MPPTunnelSetBase : private boost::noncopyable const std::vector & getTunnels() const { return tunnels; } + bool isLocal(size_t index) const; + + private: std::vector tunnels; std::unordered_map receiver_task_id_to_index_map; diff --git a/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp b/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp index ce93fa07406..11653d8883f 100644 --- a/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp +++ b/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp @@ -24,6 +24,10 @@ #include #include #include +#include + +#include "Flash/Mpp/MppVersion.h" +#include "ext/scope_guard.h" namespace DB { @@ -39,6 +43,7 @@ class TestMPPExchangeWriter : public testing::Test dag_context_ptr->is_mpp_task = true; dag_context_ptr->is_root_mpp_task = false; dag_context_ptr->result_field_types = makeFields(); + dag_context_ptr->mpp_task_meta.set_mpp_version(TiDB::GetMppVersion()); context.setDAGContext(dag_context_ptr.get()); } @@ -103,6 +108,11 @@ class TestMPPExchangeWriter : public testing::Test return block; } + void setMppExchangeDataCompress(mpp::CompressMethod m) const + { + dag_context_ptr->exchange_sender_meta.set_compress(m); + } + Context context; std::vector part_col_ids; TiDB::TiDBCollators part_col_collators; @@ -122,7 +132,13 @@ struct MockExchangeWriter void broadcastOrPassThroughWrite(const TrackedMppDataPacketPtr & packet) { checker(packet, 0); } void partitionWrite(const TrackedMppDataPacketPtr & packet, uint16_t part_id) { checker(packet, part_id); } - void write(tipb::SelectResponse &) { FAIL() << "cannot reach here, only consider CH Block format"; } + static void write(tipb::SelectResponse &) { FAIL() << "cannot reach here, only consider CH Block format"; } + bool isLocal(size_t index) const + { + assert(getPartitionNum() > index); + return true; + } + void sendExecutionSummary(tipb::SelectResponse & response) { auto tracked_packet = std::make_shared(); @@ -456,5 +472,66 @@ try } CATCH +TEST_F(TestMPPExchangeWriter, testHashPartitionWriterV1) +try +{ + setMppExchangeDataCompress(mpp::CompressMethod::LZ4); + SCOPE_EXIT({ + setMppExchangeDataCompress(mpp::CompressMethod::NONE); + }); + + const size_t block_rows = 64; + const size_t block_num = 64; + const size_t batch_send_min_limit = 108; + const uint16_t part_num = 4; + + // 1. Build Blocks. + std::vector blocks; + for (size_t i = 0; i < block_num; ++i) + { + blocks.emplace_back(prepareUniformBlock(block_rows)); + blocks.emplace_back(prepareUniformBlock(0)); + } + Block header = blocks.back(); + + // 2. Build MockExchangeWriter. + std::unordered_map write_report; + auto checker = [&write_report](const TrackedMppDataPacketPtr & packet, uint16_t part_id) { + write_report[part_id].emplace_back(packet); + }; + auto mock_writer = std::make_shared(checker, part_num); + + // 3. Start to write. + auto dag_writer = std::make_shared>>( + mock_writer, + part_col_ids, + part_col_collators, + batch_send_min_limit, + /*should_send_exec_summary_at_last=*/false, + *dag_context_ptr); + for (const auto & block : blocks) + dag_writer->write(block); + dag_writer->flush(); + dag_writer->finishWrite(); + + // 4. Start to check write_report. + size_t per_part_rows = block_rows * block_num / part_num; + ASSERT_EQ(write_report.size(), part_num); + for (const auto & ele : write_report) + { + size_t decoded_block_rows = 0; + for (const auto & packet : ele.second) + { + for (int i = 0; i < packet->getPacket().chunks_size(); ++i) + { + auto decoded_block = CHBlockChunkCodec::decode(packet->getPacket().chunks(i), header); + decoded_block_rows += decoded_block.rows(); + } + } + ASSERT_EQ(decoded_block_rows, per_part_rows); + } +} +CATCH + } // namespace tests } // namespace DB From d1d7e82fda95b016d1a9531ac07911035853ba7a Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Mon, 2 Jan 2023 17:57:21 +0800 Subject: [PATCH 31/93] 25 Signed-off-by: Zhigao Tong --- contrib/tzg.mpp.proto.patch | 4 +- .../CompressCHBlockChunkCodecStream.h | 57 ++++++++++--------- dbms/src/Flash/Mpp/ExchangeReceiver.cpp | 4 +- dbms/src/Flash/Mpp/HashBaseWriterHelper.cpp | 5 +- dbms/src/Flash/Mpp/HashBaseWriterHelper.h | 2 +- dbms/src/Flash/Mpp/HashPartitionWriter.cpp | 3 +- dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp | 24 ++++---- dbms/src/Flash/Mpp/HashPartitionWriterV1.h | 3 +- .../Mpp/tests/gtest_mpp_exchange_writer.cpp | 21 +++---- .../Planner/plans/PhysicalExchangeSender.cpp | 3 +- 10 files changed, 65 insertions(+), 61 deletions(-) diff --git a/contrib/tzg.mpp.proto.patch b/contrib/tzg.mpp.proto.patch index ec4959bfbc0..239232a9903 100644 --- a/contrib/tzg.mpp.proto.patch +++ b/contrib/tzg.mpp.proto.patch @@ -1085,7 +1085,7 @@ index fcd4691..783c2eb 100644 iNdEx = preIndex skippy, err := skipMpp(dAtA[iNdEx:]) diff --git a/proto/mpp.proto b/proto/mpp.proto -index 6739455..b61e03c 100644 +index 6739455..d693f8d 100644 --- a/proto/mpp.proto +++ b/proto/mpp.proto @@ -11,6 +11,17 @@ option (gogoproto.unmarshaler_all) = true; @@ -1146,7 +1146,7 @@ index 6739455..b61e03c 100644 Error error = 2; repeated bytes chunks = 3; repeated uint64 stream_ids = 4; -+ int64 mpp_version = 5; ++ int64 version = 5; // version of data format + DataPacketCompressMeta compress = 6; // data compression } diff --git a/dbms/src/Flash/Coprocessor/CompressCHBlockChunkCodecStream.h b/dbms/src/Flash/Coprocessor/CompressCHBlockChunkCodecStream.h index 164469b2a6d..3ea89943141 100644 --- a/dbms/src/Flash/Coprocessor/CompressCHBlockChunkCodecStream.h +++ b/dbms/src/Flash/Coprocessor/CompressCHBlockChunkCodecStream.h @@ -20,6 +20,7 @@ struct WriteBufferFromOwnStringList final WriteBufferFromOwnStringList() : WriteBuffer(nullptr, 0) { + reset(); } void nextImpl() override @@ -33,31 +34,32 @@ struct WriteBufferFromOwnStringList final next(); std::string res; - size_t sz = std::accumulate(buffs.begin(), buffs.end(), 0, [](const auto r, const auto & s) { - return r + s.size(); - }); - res.resize(sz); - char * start = res.data(); - std::for_each(buffs.begin(), buffs.end(), [&](auto & s) { - std::memcpy(start, s.data(), s.size()); - start += s.size(); - s.clear(); - }); - - clear(); + res.resize(bytes); + for (size_t sz = 0; sz < bytes;) + { + for (auto && s : buffs) + { + if (sz + s.size() < bytes) + { + std::memcpy(res.data() + sz, s.data(), s.size()); + sz += s.size(); + } + else + { + std::memcpy(res.data() + sz, s.data(), bytes - sz); + sz = bytes; + break; + } + } + } return res; } - void init() - { - clear(); - nextImpl(); - } - - void clear() + void reset() { buffs.clear(); - WriteBuffer::set(nullptr, 0); + nextImpl(); + bytes = 0; } std::vector buffs; @@ -69,12 +71,16 @@ struct CompressCHBlockChunkCodecStream : compress_method(compress_method_) { output_buffer = std::make_unique(); - compress_write_buffer = std::make_unique(*output_buffer, CompressionSettings(compress_method)); + compress_write_buffer = std::make_unique( + *output_buffer, + CompressionSettings(compress_method), + DBMS_DEFAULT_BUFFER_SIZE); } - void clear() + + void reset() const { compress_write_buffer->next(); - output_buffer->clear(); + output_buffer->reset(); } WriteBufferFromOwnStringList * getWriterWithoutCompress() const @@ -82,19 +88,18 @@ struct CompressCHBlockChunkCodecStream return output_buffer.get(); } - WriteBuffer * getWriter() + CompressedCHBlockChunkCodec::CompressedWriteBuffer * getWriter() const { return compress_write_buffer.get(); } - std::string getString() + std::string getString() const { if (compress_write_buffer == nullptr) { throw Exception("The output should not be null in getString()"); } compress_write_buffer->next(); - output_buffer->next(); return output_buffer->getString(); } ~CompressCHBlockChunkCodecStream() = default; diff --git a/dbms/src/Flash/Mpp/ExchangeReceiver.cpp b/dbms/src/Flash/Mpp/ExchangeReceiver.cpp index eaacd8b76ff..88e0e55748c 100644 --- a/dbms/src/Flash/Mpp/ExchangeReceiver.cpp +++ b/dbms/src/Flash/Mpp/ExchangeReceiver.cpp @@ -696,7 +696,7 @@ DecodeDetail ExchangeReceiverBase::decodeChunks( // Record total packet size even if fine grained shuffle is enabled. detail.packet_bytes = packet.ByteSizeLong(); - if (packet.mpp_version() && packet.compress().method() != mpp::CompressMethod::NONE) + if (packet.version() && packet.compress().method() != mpp::CompressMethod::NONE) { for (auto && chunk : packet.chunks()) { @@ -711,7 +711,7 @@ DecodeDetail ExchangeReceiverBase::decodeChunks( } return detail; } - + for (const String * chunk : recv_msg->chunks) { auto result = decoder_ptr->decodeAndSquash(*chunk); diff --git a/dbms/src/Flash/Mpp/HashBaseWriterHelper.cpp b/dbms/src/Flash/Mpp/HashBaseWriterHelper.cpp index 3e17b20f2a1..ebdbf56e2b3 100644 --- a/dbms/src/Flash/Mpp/HashBaseWriterHelper.cpp +++ b/dbms/src/Flash/Mpp/HashBaseWriterHelper.cpp @@ -92,12 +92,15 @@ void scatterColumns(const Block & input_block, } } -DB::TrackedMppDataPacketPtrs createPackets(size_t partition_num) +DB::TrackedMppDataPacketPtrs createPackets(size_t partition_num, uint64_t version) { DB::TrackedMppDataPacketPtrs tracked_packets; tracked_packets.reserve(partition_num); for (size_t i = 0; i < partition_num; ++i) + { tracked_packets.emplace_back(std::make_shared()); + tracked_packets.back()->getPacket().set_version(version); + } return tracked_packets; } diff --git a/dbms/src/Flash/Mpp/HashBaseWriterHelper.h b/dbms/src/Flash/Mpp/HashBaseWriterHelper.h index 0b323d559ca..63ecad09d52 100644 --- a/dbms/src/Flash/Mpp/HashBaseWriterHelper.h +++ b/dbms/src/Flash/Mpp/HashBaseWriterHelper.h @@ -33,7 +33,7 @@ void computeHash(const Block & block, WeakHash32 & hash, IColumn::Selector & selector); -DB::TrackedMppDataPacketPtrs createPackets(size_t partition_num); +DB::TrackedMppDataPacketPtrs createPackets(size_t partition_num, uint64_t version = 0); void scatterColumns(const Block & input_block, uint32_t bucket_num, diff --git a/dbms/src/Flash/Mpp/HashPartitionWriter.cpp b/dbms/src/Flash/Mpp/HashPartitionWriter.cpp index 9fe0fe9e2b3..874352699c6 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriter.cpp +++ b/dbms/src/Flash/Mpp/HashPartitionWriter.cpp @@ -47,7 +47,6 @@ HashPartitionWriter::HashPartitionWriter( , writer(writer_) , partition_col_ids(std::move(partition_col_ids_)) , collators(std::move(collators_)) - // , compress_method(dag_context.getExchangeSenderMeta().compress()) { rows_in_blocks = 0; partition_num = writer_->getPartitionNum(); @@ -116,7 +115,7 @@ void HashPartitionWriter::partitionAndEncodeThenWriteBlocks() // method = mpp::CompressMethod::NONE; // } // tracked_packets[part_id]->getPacket().mutable_compress()->set_method(method); - tracked_packets[part_id]->getPacket().set_mpp_version(TiDB::GetMppVersion()); + // tracked_packets[part_id]->getPacket().set_mpp_version(TiDB::GetMppVersion()); } size_t ori_block_mem_size = 0; diff --git a/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp b/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp index 0c6c7482d6a..7c81024cdac 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp +++ b/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp @@ -22,6 +22,7 @@ #include #include +#include #include "Common/Exception.h" #include "Common/Stopwatch.h" @@ -42,14 +43,15 @@ HashPartitionWriterV1::HashPartitionWriterV1( TiDB::TiDBCollators collators_, Int64 batch_send_min_limit_, bool should_send_exec_summary_at_last_, - DAGContext & dag_context_) + DAGContext & dag_context_, + mpp::CompressMethod compress_method_) : DAGResponseWriter(/*records_per_chunk=*/-1, dag_context_) , batch_send_min_limit(batch_send_min_limit_) , should_send_exec_summary_at_last(should_send_exec_summary_at_last_) , writer(writer_) , partition_col_ids(std::move(partition_col_ids_)) , collators(std::move(collators_)) - , compress_method(dag_context.getExchangeSenderMeta().compress()) + , compress_method(compress_method_) { assert(compress_method != mpp::CompressMethod::NONE); assert(dag_context.getMPPTaskMeta().mpp_version() > 0); @@ -110,12 +112,10 @@ void HashPartitionWriterV1::partitionAndEncodeThenWriteBlocks { assert(compress_chunk_codec_stream); - auto tracked_packets = HashBaseWriterHelper::createPackets(partition_num); + auto tracked_packets = HashBaseWriterHelper::createPackets(partition_num, 1); for (size_t part_id = 0; part_id < partition_num; ++part_id) { - tracked_packets[part_id]->getPacket().set_mpp_version(TiDB::GetMppVersion()); - auto method = compress_method; if (writer->isLocal(part_id)) { @@ -171,16 +171,16 @@ void HashPartitionWriterV1::partitionAndEncodeThenWriteBlocks for (size_t part_id = 0; part_id < partition_num; ++part_id) { + size_t part_rows{}; if (tracked_packets[part_id]->getPacket().compress().method() == mpp::NONE) { auto * ostr_ptr = compress_chunk_codec_stream->getWriterWithoutCompress(); for (auto && columns : dest_columns[part_id]) { dest_block_header.setColumns(std::move(columns)); - ostr_ptr->init(); EncodeCHBlockChunk(ostr_ptr, dest_block_header); tracked_packets[part_id]->getPacket().add_chunks(ostr_ptr->getString()); - ostr_ptr->clear(); + ostr_ptr->reset(); { const auto & chunks = tracked_packets[part_id]->getPacket().chunks(); @@ -199,7 +199,9 @@ void HashPartitionWriterV1::partitionAndEncodeThenWriteBlocks } else { - compress_chunk_codec_stream->encodeHeader(dest_block_header, total_rows); + size_t rows = std::accumulate(dest_columns[part_id].begin(), dest_columns[part_id].end(), 0, [](const auto & r, const auto & columns) { return r + columns[0]->size(); }); + part_rows = rows; + compress_chunk_codec_stream->encodeHeader(dest_block_header, rows); for (size_t col_index = 0; col_index < dest_block_header.columns(); ++col_index) { auto && col_type_name = dest_block_header.getByPosition(col_index); @@ -210,7 +212,7 @@ void HashPartitionWriterV1::partitionAndEncodeThenWriteBlocks } tracked_packets[part_id]->getPacket().add_chunks(compress_chunk_codec_stream->getString()); - compress_chunk_codec_stream->clear(); + compress_chunk_codec_stream->reset(); } if (tracked_packets[part_id]->getPacket().compress().method() != mpp::NONE) @@ -223,9 +225,9 @@ void HashPartitionWriterV1::partitionAndEncodeThenWriteBlocks auto && compress_buffer = CompressedCHBlockChunkCodec::CompressedReadBuffer(istr); size_t rows{}; - Block res = DecodeHeader(compress_buffer, {}, rows); + Block res = DecodeHeader(compress_buffer, dest_block_header, rows); DecodeColumns(compress_buffer, res, res.columns(), rows); - RUNTIME_CHECK(res.rows() == total_rows, res.rows(), total_rows); + RUNTIME_CHECK(res.rows() == part_rows, res.rows(), part_rows); RUNTIME_CHECK(res.columns() == dest_block_header.columns(), res.columns(), dest_block_header.columns()); res.checkNumberOfRows(); for (size_t i = 0; i < res.columns(); ++i) diff --git a/dbms/src/Flash/Mpp/HashPartitionWriterV1.h b/dbms/src/Flash/Mpp/HashPartitionWriterV1.h index 0d7dc804711..6184e4ae5c8 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriterV1.h +++ b/dbms/src/Flash/Mpp/HashPartitionWriterV1.h @@ -37,7 +37,8 @@ class HashPartitionWriterV1 : public DAGResponseWriter TiDB::TiDBCollators collators_, Int64 batch_send_min_limit_, bool should_send_exec_summary_at_last, - DAGContext & dag_context_); + DAGContext & dag_context_, + mpp::CompressMethod compress_method_); void write(const Block & block) override; void flush() override; void finishWrite() override; diff --git a/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp b/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp index 11653d8883f..2e388e88f9d 100644 --- a/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp +++ b/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp @@ -28,6 +28,7 @@ #include "Flash/Mpp/MppVersion.h" #include "ext/scope_guard.h" +#include "mpp.pb.h" namespace DB { @@ -108,11 +109,6 @@ class TestMPPExchangeWriter : public testing::Test return block; } - void setMppExchangeDataCompress(mpp::CompressMethod m) const - { - dag_context_ptr->exchange_sender_meta.set_compress(m); - } - Context context; std::vector part_col_ids; TiDB::TiDBCollators part_col_collators; @@ -136,7 +132,8 @@ struct MockExchangeWriter bool isLocal(size_t index) const { assert(getPartitionNum() > index); - return true; + // make only part 0 use local tunnel + return index == 0; } void sendExecutionSummary(tipb::SelectResponse & response) @@ -475,15 +472,10 @@ CATCH TEST_F(TestMPPExchangeWriter, testHashPartitionWriterV1) try { - setMppExchangeDataCompress(mpp::CompressMethod::LZ4); - SCOPE_EXIT({ - setMppExchangeDataCompress(mpp::CompressMethod::NONE); - }); - const size_t block_rows = 64; - const size_t block_num = 64; + const size_t block_num = 1; const size_t batch_send_min_limit = 108; - const uint16_t part_num = 4; + const uint16_t part_num = 2; // 1. Build Blocks. std::vector blocks; @@ -508,7 +500,8 @@ try part_col_collators, batch_send_min_limit, /*should_send_exec_summary_at_last=*/false, - *dag_context_ptr); + *dag_context_ptr, + mpp::CompressMethod::LZ4); for (const auto & block : blocks) dag_writer->write(block); dag_writer->flush(); diff --git a/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp b/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp index 7a927a7ca7d..a22e99636a2 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp +++ b/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp @@ -165,7 +165,8 @@ std::unique_ptr NewMPPExchangeWriter( partition_col_collators, batch_send_min_limit, should_send_exec_summary_at_last, - dag_context); + dag_context, + compress_method); } } else From 9ce108bf4346769ae3daba4f7ee8d03bd939c98e Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Mon, 2 Jan 2023 18:29:23 +0800 Subject: [PATCH 32/93] 26 Signed-off-by: Zhigao Tong --- dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp | 62 +++++++++---------- .../Mpp/tests/gtest_mpp_exchange_writer.cpp | 37 ++++++++--- 2 files changed, 57 insertions(+), 42 deletions(-) diff --git a/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp b/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp index 7c81024cdac..db52b707f16 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp +++ b/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp @@ -120,6 +120,7 @@ void HashPartitionWriterV1::partitionAndEncodeThenWriteBlocks if (writer->isLocal(part_id)) { method = mpp::CompressMethod::NONE; + tracked_packets[part_id]->getPacket().set_version(0); } tracked_packets[part_id]->getPacket().mutable_compress()->set_method(method); } @@ -136,7 +137,7 @@ void HashPartitionWriterV1::partitionAndEncodeThenWriteBlocks std::vector partition_key_containers(collators.size()); std::vector> dest_columns(partition_num); - size_t total_rows = 0; + size_t total_rows = 0, encoded_rows = 0; while (!blocks.empty()) { @@ -157,27 +158,17 @@ void HashPartitionWriterV1::partitionAndEncodeThenWriteBlocks dest_columns[part_id].emplace_back(std::move(columns)); } } - { - size_t rows = 0; - for (size_t part_id = 0; part_id < partition_num; ++part_id) - { - for (auto && columns : dest_columns[part_id]) - { - rows += columns[0]->size(); - } - } - RUNTIME_CHECK(rows == total_rows, rows, total_rows); - } for (size_t part_id = 0; part_id < partition_num; ++part_id) { - size_t part_rows{}; if (tracked_packets[part_id]->getPacket().compress().method() == mpp::NONE) { auto * ostr_ptr = compress_chunk_codec_stream->getWriterWithoutCompress(); for (auto && columns : dest_columns[part_id]) { dest_block_header.setColumns(std::move(columns)); + encoded_rows += dest_block_header.rows(); + EncodeCHBlockChunk(ostr_ptr, dest_block_header); tracked_packets[part_id]->getPacket().add_chunks(ostr_ptr->getString()); ostr_ptr->reset(); @@ -199,9 +190,10 @@ void HashPartitionWriterV1::partitionAndEncodeThenWriteBlocks } else { - size_t rows = std::accumulate(dest_columns[part_id].begin(), dest_columns[part_id].end(), 0, [](const auto & r, const auto & columns) { return r + columns[0]->size(); }); - part_rows = rows; - compress_chunk_codec_stream->encodeHeader(dest_block_header, rows); + size_t part_rows = std::accumulate(dest_columns[part_id].begin(), dest_columns[part_id].end(), 0, [](const auto & r, const auto & columns) { return r + columns[0]->size(); }); + encoded_rows += part_rows; + + compress_chunk_codec_stream->encodeHeader(dest_block_header, part_rows); for (size_t col_index = 0; col_index < dest_block_header.columns(); ++col_index) { auto && col_type_name = dest_block_header.getByPosition(col_index); @@ -213,29 +205,31 @@ void HashPartitionWriterV1::partitionAndEncodeThenWriteBlocks tracked_packets[part_id]->getPacket().add_chunks(compress_chunk_codec_stream->getString()); compress_chunk_codec_stream->reset(); - } - if (tracked_packets[part_id]->getPacket().compress().method() != mpp::NONE) - { - // decode - const auto & chunks = tracked_packets[part_id]->getPacket().chunks(); - const auto & dd = chunks[chunks.size() - 1]; - - ReadBufferFromString istr(dd); - auto && compress_buffer = CompressedCHBlockChunkCodec::CompressedReadBuffer(istr); - - size_t rows{}; - Block res = DecodeHeader(compress_buffer, dest_block_header, rows); - DecodeColumns(compress_buffer, res, res.columns(), rows); - RUNTIME_CHECK(res.rows() == part_rows, res.rows(), part_rows); - RUNTIME_CHECK(res.columns() == dest_block_header.columns(), res.columns(), dest_block_header.columns()); - res.checkNumberOfRows(); - for (size_t i = 0; i < res.columns(); ++i) { - RUNTIME_CHECK(dest_block_header.getByPosition(i) == res.getByPosition(i)); + // decode + const auto & chunks = tracked_packets[part_id]->getPacket().chunks(); + const auto & dd = chunks[chunks.size() - 1]; + + ReadBufferFromString istr(dd); + auto && compress_buffer = CompressedCHBlockChunkCodec::CompressedReadBuffer(istr); + + size_t rows{}; + Block res = DecodeHeader(compress_buffer, dest_block_header, rows); + DecodeColumns(compress_buffer, res, res.columns(), rows); + RUNTIME_CHECK(res.rows() == part_rows, res.rows(), part_rows); + RUNTIME_CHECK(res.columns() == dest_block_header.columns(), res.columns(), dest_block_header.columns()); + res.checkNumberOfRows(); + for (size_t i = 0; i < res.columns(); ++i) + { + RUNTIME_CHECK(dest_block_header.getByPosition(i) == res.getByPosition(i)); + } } } } + + RUNTIME_CHECK(encoded_rows == total_rows, encoded_rows, total_rows); + assert(blocks.empty()); rows_in_blocks = 0; } diff --git a/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp b/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp index 2e388e88f9d..1840b586dc4 100644 --- a/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp +++ b/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp @@ -14,7 +14,7 @@ #include #include -#include +#include #include #include #include @@ -25,6 +25,7 @@ #include #include #include +#include #include "Flash/Mpp/MppVersion.h" #include "ext/scope_guard.h" @@ -473,9 +474,9 @@ TEST_F(TestMPPExchangeWriter, testHashPartitionWriterV1) try { const size_t block_rows = 64; - const size_t block_num = 1; - const size_t batch_send_min_limit = 108; - const uint16_t part_num = 2; + const size_t block_num = 64; + const size_t batch_send_min_limit = 100; + const uint16_t part_num = 4; // 1. Build Blocks. std::vector blocks; @@ -510,15 +511,35 @@ try // 4. Start to check write_report. size_t per_part_rows = block_rows * block_num / part_num; ASSERT_EQ(write_report.size(), part_num); + + CHBlockChunkDecodeAndSquash decoder(header, std::numeric_limits::max()); + for (const auto & ele : write_report) { size_t decoded_block_rows = 0; - for (const auto & packet : ele.second) + for (const auto & tracked_packet : ele.second) { - for (int i = 0; i < packet->getPacket().chunks_size(); ++i) + auto & packet = tracked_packet->getPacket(); + + if (packet.version() && packet.compress().method() != mpp::CompressMethod::NONE) { - auto decoded_block = CHBlockChunkCodec::decode(packet->getPacket().chunks(i), header); - decoded_block_rows += decoded_block.rows(); + for (auto && chunk : packet.chunks()) + { + auto && result = decoder.decodeAndSquashWithCompress(chunk); + if (!result) + continue; + decoded_block_rows += result->rows(); + } + } + else + { + for (auto && chunk : packet.chunks()) + { + auto result = decoder.decodeAndSquash(chunk); + if (!result) + continue; + decoded_block_rows += result->rows(); + } } } ASSERT_EQ(decoded_block_rows, per_part_rows); From 409d0200b6277500ab41f6ed1fe05c3209f0a7b4 Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Tue, 3 Jan 2023 13:50:09 +0800 Subject: [PATCH 33/93] 27 Signed-off-by: Zhigao Tong --- .../Coprocessor/ChunkDecodeAndSquash.cpp | 21 ++-- .../Flash/Coprocessor/ChunkDecodeAndSquash.h | 5 +- .../CompressCHBlockChunkCodecStream.cpp | 20 ++-- .../CompressCHBlockChunkCodecStream.h | 15 +-- dbms/src/Flash/Mpp/ExchangeReceiver.cpp | 4 +- dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp | 100 +++++++++--------- dbms/src/Flash/Mpp/HashPartitionWriterV1.h | 2 +- dbms/src/Flash/Mpp/MppVersion.h | 7 ++ dbms/src/Flash/Mpp/Utils.cpp | 27 ++--- .../Mpp/tests/gtest_mpp_exchange_writer.cpp | 33 +++--- 10 files changed, 118 insertions(+), 116 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.cpp b/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.cpp index acb6c145359..b5c2691349b 100644 --- a/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.cpp +++ b/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.cpp @@ -33,13 +33,20 @@ CHBlockChunkDecodeAndSquash::CHBlockChunkDecodeAndSquash( { } -std::optional CHBlockChunkDecodeAndSquash::decodeAndSquashWithCompress(std::string_view sv) +std::optional CHBlockChunkDecodeAndSquash::decodeAndSquash(std::string_view sv, bool compress) { - std::optional res; ReadBufferFromString istr(sv); + if (!compress) + return decodeAndSquashWithCompressImpl(istr); auto && compress_buffer = CompressedCHBlockChunkCodec::CompressedReadBuffer(istr); + return decodeAndSquashWithCompressImpl(compress_buffer); +} + +std::optional CHBlockChunkDecodeAndSquash::decodeAndSquashWithCompressImpl(ReadBuffer & istr) +{ + std::optional res; - if (compress_buffer.eof()) + if (istr.eof()) { if (accumulated_block) res.swap(accumulated_block); @@ -49,16 +56,16 @@ std::optional CHBlockChunkDecodeAndSquash::decodeAndSquashWithCompress(st if (!accumulated_block) { size_t rows{}; - Block block = DecodeHeader(compress_buffer, codec.header, rows); - DecodeColumns(compress_buffer, block, codec.header.columns(), rows, static_cast(rows_limit * 1.5)); + Block block = DecodeHeader(istr, codec.header, rows); + DecodeColumns(istr, block, codec.header.columns(), rows, static_cast(rows_limit * 1.5)); if (block) accumulated_block.emplace(std::move(block)); } else { size_t rows{}; - DecodeHeader(compress_buffer, codec.header, rows); - DecodeColumns(compress_buffer, *accumulated_block, codec.header.columns(), rows, 0); + DecodeHeader(istr, codec.header, rows); + DecodeColumns(istr, *accumulated_block, codec.header.columns(), rows, 0); } if (accumulated_block && accumulated_block->rows() >= rows_limit) diff --git a/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.h b/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.h index 3fb888d61e9..340479361ba 100644 --- a/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.h +++ b/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.h @@ -31,9 +31,12 @@ class CHBlockChunkDecodeAndSquash CHBlockChunkDecodeAndSquash(const Block & header, size_t rows_limit_); ~CHBlockChunkDecodeAndSquash() = default; std::optional decodeAndSquash(const String &); - std::optional decodeAndSquashWithCompress(std::string_view); + std::optional decodeAndSquash(std::string_view, bool compress); std::optional flush(); +private: + std::optional decodeAndSquashWithCompressImpl(ReadBuffer & istr); + private: CHBlockChunkCodec codec; std::optional accumulated_block; diff --git a/dbms/src/Flash/Coprocessor/CompressCHBlockChunkCodecStream.cpp b/dbms/src/Flash/Coprocessor/CompressCHBlockChunkCodecStream.cpp index 126241d2e34..caaeb5b583f 100644 --- a/dbms/src/Flash/Coprocessor/CompressCHBlockChunkCodecStream.cpp +++ b/dbms/src/Flash/Coprocessor/CompressCHBlockChunkCodecStream.cpp @@ -10,30 +10,28 @@ namespace DB { -void CompressCHBlockChunkCodecStream::encodeHeader(const Block & header, size_t rows) +void EncodeHeader(WriteBuffer & ostr, const Block & header, size_t rows) { - WriteBuffer * ostr_ptr = getWriter(); - size_t columns = header.columns(); - writeVarUInt(columns, *ostr_ptr); - writeVarUInt(rows, *ostr_ptr); + writeVarUInt(columns, ostr); + writeVarUInt(rows, ostr); for (size_t i = 0; i < columns; i++) { const ColumnWithTypeAndName & column = header.safeGetByPosition(i); - writeStringBinary(column.name, *ostr_ptr); - writeStringBinary(column.type->getName(), *ostr_ptr); + writeStringBinary(column.name, ostr); + writeStringBinary(column.type->getName(), ostr); } } extern void writeData(const IDataType & type, const ColumnPtr & column, WriteBuffer & ostr, size_t offset, size_t limit); -void CompressCHBlockChunkCodecStream::encodeColumn(const ColumnPtr & column, const ColumnWithTypeAndName & type_name) +void EncodeColumn(WriteBuffer & ostr, const ColumnPtr & column, const ColumnWithTypeAndName & type_name) { - WriteBuffer * ostr_ptr = getWriter(); - writeVarUInt(column->size(), *ostr_ptr); - writeData(*type_name.type, column, *ostr_ptr, 0, 0); + writeVarUInt(column->size(), ostr); + writeData(*type_name.type, column, ostr, 0, 0); } + std::unique_ptr NewCompressCHBlockChunkCodecStream(CompressionMethod compress_method) { return std::make_unique(compress_method); diff --git a/dbms/src/Flash/Coprocessor/CompressCHBlockChunkCodecStream.h b/dbms/src/Flash/Coprocessor/CompressCHBlockChunkCodecStream.h index 3ea89943141..2fa179f47eb 100644 --- a/dbms/src/Flash/Coprocessor/CompressCHBlockChunkCodecStream.h +++ b/dbms/src/Flash/Coprocessor/CompressCHBlockChunkCodecStream.h @@ -5,6 +5,7 @@ #include #include #include +#include #include #include "Columns/IColumn.h" @@ -33,6 +34,12 @@ struct WriteBufferFromOwnStringList final { next(); + if (!buffs.empty() && buffs[0].size() >= bytes) + { + buffs[0].resize(bytes); + return std::move(buffs[0]); + } + std::string res; res.resize(bytes); for (size_t sz = 0; sz < bytes;) @@ -104,18 +111,14 @@ struct CompressCHBlockChunkCodecStream } ~CompressCHBlockChunkCodecStream() = default; - // void disableCompress() { enable_compress = false; } - // void enableCompress() { enable_compress = true; } - - void encodeHeader(const Block & header, size_t rows); - void encodeColumn(const ColumnPtr & column, const ColumnWithTypeAndName & type_name); - // bool enable_compress{true}; CompressionMethod compress_method; std::unique_ptr output_buffer{}; std::unique_ptr compress_write_buffer{}; }; +void EncodeHeader(WriteBuffer & ostr, const Block & header, size_t rows); +void EncodeColumn(WriteBuffer & ostr, const ColumnPtr & column, const ColumnWithTypeAndName & type_name); void DecodeColumns(ReadBuffer & istr, Block & res, size_t columns, size_t rows, size_t reserve_size = 0); Block DecodeHeader(ReadBuffer & istr, const Block & header, size_t & rows); diff --git a/dbms/src/Flash/Mpp/ExchangeReceiver.cpp b/dbms/src/Flash/Mpp/ExchangeReceiver.cpp index 88e0e55748c..bcbfe433694 100644 --- a/dbms/src/Flash/Mpp/ExchangeReceiver.cpp +++ b/dbms/src/Flash/Mpp/ExchangeReceiver.cpp @@ -696,11 +696,11 @@ DecodeDetail ExchangeReceiverBase::decodeChunks( // Record total packet size even if fine grained shuffle is enabled. detail.packet_bytes = packet.ByteSizeLong(); - if (packet.version() && packet.compress().method() != mpp::CompressMethod::NONE) + if (packet.version()) { for (auto && chunk : packet.chunks()) { - auto && result = decoder_ptr->decodeAndSquashWithCompress(chunk); + auto && result = decoder_ptr->decodeAndSquash(chunk, packet.compress().method() != mpp::CompressMethod::NONE); if (!result) continue; detail.rows += result->rows(); diff --git a/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp b/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp index db52b707f16..bf90cb39663 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp +++ b/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp @@ -46,7 +46,8 @@ HashPartitionWriterV1::HashPartitionWriterV1( DAGContext & dag_context_, mpp::CompressMethod compress_method_) : DAGResponseWriter(/*records_per_chunk=*/-1, dag_context_) - , batch_send_min_limit(batch_send_min_limit_) + , partition_num(writer_->getPartitionNum()) + , batch_send_min_limit(batch_send_min_limit_ * partition_num) , should_send_exec_summary_at_last(should_send_exec_summary_at_last_) , writer(writer_) , partition_col_ids(std::move(partition_col_ids_)) @@ -57,7 +58,6 @@ HashPartitionWriterV1::HashPartitionWriterV1( assert(dag_context.getMPPTaskMeta().mpp_version() > 0); rows_in_blocks = 0; - partition_num = writer_->getPartitionNum(); RUNTIME_CHECK(partition_num > 0); RUNTIME_CHECK(dag_context.encode_type == tipb::EncodeType::TypeCHBlock); for (const auto & field_type : dag_context.result_field_types) @@ -120,7 +120,6 @@ void HashPartitionWriterV1::partitionAndEncodeThenWriteBlocks if (writer->isLocal(part_id)) { method = mpp::CompressMethod::NONE; - tracked_packets[part_id]->getPacket().set_version(0); } tracked_packets[part_id]->getPacket().mutable_compress()->set_method(method); } @@ -161,70 +160,75 @@ void HashPartitionWriterV1::partitionAndEncodeThenWriteBlocks for (size_t part_id = 0; part_id < partition_num; ++part_id) { + WriteBuffer * ostr_ptr{}; if (tracked_packets[part_id]->getPacket().compress().method() == mpp::NONE) { - auto * ostr_ptr = compress_chunk_codec_stream->getWriterWithoutCompress(); - for (auto && columns : dest_columns[part_id]) - { - dest_block_header.setColumns(std::move(columns)); - encoded_rows += dest_block_header.rows(); - - EncodeCHBlockChunk(ostr_ptr, dest_block_header); - tracked_packets[part_id]->getPacket().add_chunks(ostr_ptr->getString()); - ostr_ptr->reset(); - - { - const auto & chunks = tracked_packets[part_id]->getPacket().chunks(); - const auto & dd = chunks[chunks.size() - 1]; - - auto res = CHBlockChunkCodec::decode(dd, dest_block_header); - RUNTIME_CHECK(res.rows() == dest_block_header.rows(), res.rows(), dest_block_header.rows()); - RUNTIME_CHECK(res.columns() == dest_block_header.columns(), res.columns(), dest_block_header.columns()); - res.checkNumberOfRows(); - for (size_t i = 0; i < res.columns(); ++i) - { - RUNTIME_CHECK(dest_block_header.getByPosition(i) == res.getByPosition(i)); - } - } - } + ostr_ptr = compress_chunk_codec_stream->getWriterWithoutCompress(); + // for (auto && columns : dest_columns[part_id]) + // { + // dest_block_header.setColumns(std::move(columns)); + // encoded_rows += dest_block_header.rows(); + + // EncodeCHBlockChunk(ostr_ptr, dest_block_header); + // tracked_packets[part_id]->getPacket().add_chunks(ostr_ptr->getString()); + // ostr_ptr->reset(); + + // // { + // // const auto & chunks = tracked_packets[part_id]->getPacket().chunks(); + // // const auto & dd = chunks[chunks.size() - 1]; + + // // auto res = CHBlockChunkCodec::decode(dd, dest_block_header); + // // RUNTIME_CHECK(res.rows() == dest_block_header.rows(), res.rows(), dest_block_header.rows()); + // // RUNTIME_CHECK(res.columns() == dest_block_header.columns(), res.columns(), dest_block_header.columns()); + // // res.checkNumberOfRows(); + // // for (size_t i = 0; i < res.columns(); ++i) + // // { + // // RUNTIME_CHECK(dest_block_header.getByPosition(i) == res.getByPosition(i)); + // // } + // // } + // } } else + { + ostr_ptr = compress_chunk_codec_stream->getWriter(); + } + { size_t part_rows = std::accumulate(dest_columns[part_id].begin(), dest_columns[part_id].end(), 0, [](const auto & r, const auto & columns) { return r + columns[0]->size(); }); encoded_rows += part_rows; - compress_chunk_codec_stream->encodeHeader(dest_block_header, part_rows); + EncodeHeader(*ostr_ptr, dest_block_header, part_rows); for (size_t col_index = 0; col_index < dest_block_header.columns(); ++col_index) { auto && col_type_name = dest_block_header.getByPosition(col_index); for (auto && columns : dest_columns[part_id]) { - compress_chunk_codec_stream->encodeColumn(std::move(columns[col_index]), col_type_name); + EncodeColumn(*ostr_ptr, std::move(columns[col_index]), col_type_name); } } tracked_packets[part_id]->getPacket().add_chunks(compress_chunk_codec_stream->getString()); compress_chunk_codec_stream->reset(); - { - // decode - const auto & chunks = tracked_packets[part_id]->getPacket().chunks(); - const auto & dd = chunks[chunks.size() - 1]; - - ReadBufferFromString istr(dd); - auto && compress_buffer = CompressedCHBlockChunkCodec::CompressedReadBuffer(istr); - - size_t rows{}; - Block res = DecodeHeader(compress_buffer, dest_block_header, rows); - DecodeColumns(compress_buffer, res, res.columns(), rows); - RUNTIME_CHECK(res.rows() == part_rows, res.rows(), part_rows); - RUNTIME_CHECK(res.columns() == dest_block_header.columns(), res.columns(), dest_block_header.columns()); - res.checkNumberOfRows(); - for (size_t i = 0; i < res.columns(); ++i) - { - RUNTIME_CHECK(dest_block_header.getByPosition(i) == res.getByPosition(i)); - } - } + // { + // // decode + // const auto & chunks = tracked_packets[part_id]->getPacket().chunks(); + // const auto & dd = chunks[chunks.size() - 1]; + + // ReadBufferFromString istr(dd); + // auto && compress_buffer = CompressedCHBlockChunkCodec::CompressedReadBuffer(istr); + + // size_t rows{}; + // Block res = DecodeHeader(compress_buffer, dest_block_header, rows); + // DecodeColumns(compress_buffer, res, res.columns(), rows); + // RUNTIME_CHECK(res.rows() == part_rows, res.rows(), part_rows); + // RUNTIME_CHECK(res.columns() == dest_block_header.columns(), res.columns(), dest_block_header.columns()); + // res.checkNumberOfRows(); + // for (size_t i = 0; i < res.columns(); ++i) + // { + // RUNTIME_CHECK(dest_block_header.getByPosition(i) == res.getByPosition(i)); + // } + // } } } diff --git a/dbms/src/Flash/Mpp/HashPartitionWriterV1.h b/dbms/src/Flash/Mpp/HashPartitionWriterV1.h index 6184e4ae5c8..8dcef4d2d03 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriterV1.h +++ b/dbms/src/Flash/Mpp/HashPartitionWriterV1.h @@ -51,6 +51,7 @@ class HashPartitionWriterV1 : public DAGResponseWriter void sendExecutionSummary(); private: + uint16_t partition_num; Int64 batch_send_min_limit; bool should_send_exec_summary_at_last; ExchangeWriterPtr writer; @@ -58,7 +59,6 @@ class HashPartitionWriterV1 : public DAGResponseWriter std::vector partition_col_ids; TiDB::TiDBCollators collators; size_t rows_in_blocks; - uint16_t partition_num; DataTypes expected_types; mpp::CompressMethod compress_method{}; std::unique_ptr compress_chunk_codec_stream; diff --git a/dbms/src/Flash/Mpp/MppVersion.h b/dbms/src/Flash/Mpp/MppVersion.h index f0346361e36..bf675822de4 100644 --- a/dbms/src/Flash/Mpp/MppVersion.h +++ b/dbms/src/Flash/Mpp/MppVersion.h @@ -4,6 +4,13 @@ namespace TiDB { +enum MppVersion : int64_t +{ + MppVersionV0 = 0, + MppVersionV1, + MppVersionMAX, +}; + bool CheckMppVersion(int64_t mpp_version); std::string GenMppVersionErrorMessage(int64_t mpp_version); int64_t GetMppVersion(); diff --git a/dbms/src/Flash/Mpp/Utils.cpp b/dbms/src/Flash/Mpp/Utils.cpp index a2723b812bb..4a4ec261833 100644 --- a/dbms/src/Flash/Mpp/Utils.cpp +++ b/dbms/src/Flash/Mpp/Utils.cpp @@ -47,45 +47,32 @@ void trimStackTrace(String & message) namespace TiDB { - -enum class MppVersion : int64_t -{ - V0 = 0, - V1, - MAX, -}; - -static constexpr int64_t toInt64(MppVersion v) -{ - return static_cast(v); -} - // Latest mpp-version supported by TiFlash -constexpr MppVersion MPP_VERSION = MppVersion(toInt64(MppVersion::MAX) - 1); +constexpr MppVersion MPP_VERSION = MppVersion((MppVersion::MppVersionMAX)-1); // TODO: set version after committed -constexpr std::array MPP_TIFLASH_RELEASE_VERSION = {"", "?"}; +constexpr std::array MPP_TIFLASH_RELEASE_VERSION = {"", "?"}; // Check mpp-version is illegal bool CheckMppVersion(int64_t mpp_version) { - return mpp_version >= toInt64(MppVersion::V0) && mpp_version < toInt64(MppVersion::MAX); + return mpp_version >= (MppVersion::MppVersionV0) && mpp_version < (MppVersion::MppVersionMAX); } std::string GenMppVersionErrorMessage(int64_t mpp_version) { auto err_msg = fmt::format("invalid mpp version `{}`, expect version: min `{}`, max `{}` release version `{}`", mpp_version, - toInt64(MppVersion::V0), - toInt64(MPP_VERSION), - MPP_TIFLASH_RELEASE_VERSION[toInt64(MPP_VERSION)]); + (MppVersion::MppVersionV0), + (MPP_VERSION), + MPP_TIFLASH_RELEASE_VERSION[(MPP_VERSION)]); return err_msg; } // Get latest mpp-version supported by TiFlash int64_t GetMppVersion() { - return toInt64(MPP_VERSION); + return (MPP_VERSION); } std::string GetMppVersionReleaseInfo(int64_t mpp_version) diff --git a/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp b/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp index 1840b586dc4..873de5e2bde 100644 --- a/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp +++ b/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp @@ -475,7 +475,7 @@ try { const size_t block_rows = 64; const size_t block_num = 64; - const size_t batch_send_min_limit = 100; + const size_t batch_send_min_limit = 16; const uint16_t part_num = 4; // 1. Build Blocks. @@ -512,7 +512,7 @@ try size_t per_part_rows = block_rows * block_num / part_num; ASSERT_EQ(write_report.size(), part_num); - CHBlockChunkDecodeAndSquash decoder(header, std::numeric_limits::max()); + CHBlockChunkDecodeAndSquash decoder(header, 512); for (const auto & ele : write_report) { @@ -520,28 +520,21 @@ try for (const auto & tracked_packet : ele.second) { auto & packet = tracked_packet->getPacket(); + ASSERT_EQ(packet.version(), TiDB::MppVersion::MppVersionV1); - if (packet.version() && packet.compress().method() != mpp::CompressMethod::NONE) + for (auto && chunk : packet.chunks()) { - for (auto && chunk : packet.chunks()) - { - auto && result = decoder.decodeAndSquashWithCompress(chunk); - if (!result) - continue; - decoded_block_rows += result->rows(); - } - } - else - { - for (auto && chunk : packet.chunks()) - { - auto result = decoder.decodeAndSquash(chunk); - if (!result) - continue; - decoded_block_rows += result->rows(); - } + auto && result = decoder.decodeAndSquash(chunk, packet.compress().method() != mpp::CompressMethod::NONE); + if (!result) + continue; + decoded_block_rows += result->rows(); } } + { + auto result = decoder.flush(); + if (result) + decoded_block_rows += result->rows(); + } ASSERT_EQ(decoded_block_rows, per_part_rows); } } From 2fb53798d8d8cd924548d2367ae9f37faa8c85e6 Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Tue, 3 Jan 2023 13:58:10 +0800 Subject: [PATCH 34/93] 28 Signed-off-by: Zhigao Tong --- dbms/src/Flash/Coprocessor/CompressCHBlockChunkCodecStream.h | 2 +- dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/dbms/src/Flash/Coprocessor/CompressCHBlockChunkCodecStream.h b/dbms/src/Flash/Coprocessor/CompressCHBlockChunkCodecStream.h index 2fa179f47eb..463fdebce64 100644 --- a/dbms/src/Flash/Coprocessor/CompressCHBlockChunkCodecStream.h +++ b/dbms/src/Flash/Coprocessor/CompressCHBlockChunkCodecStream.h @@ -26,7 +26,7 @@ struct WriteBufferFromOwnStringList final void nextImpl() override { - buffs.emplace_back(std::string(DBMS_DEFAULT_BUFFER_SIZE, 0)); + buffs.emplace_back(std::string(128 * 1024, 0)); // 128KB WriteBuffer::set(buffs.back().data(), buffs.back().size()); }; diff --git a/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp b/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp index 873de5e2bde..8db87131689 100644 --- a/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp +++ b/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp @@ -520,6 +520,7 @@ try for (const auto & tracked_packet : ele.second) { auto & packet = tracked_packet->getPacket(); + ASSERT_EQ(packet.version(), TiDB::MppVersion::MppVersionV1); for (auto && chunk : packet.chunks()) From 77b36c5d26a7945feac8b051acfebf8a21f58abe Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Tue, 3 Jan 2023 14:06:15 +0800 Subject: [PATCH 35/93] 29 Signed-off-by: Zhigao Tong --- dbms/src/Flash/Mpp/ExchangeReceiver.cpp | 33 ++++++++++++------- .../Mpp/tests/gtest_mpp_exchange_writer.cpp | 2 +- .../Planner/plans/PhysicalExchangeSender.cpp | 3 +- 3 files changed, 25 insertions(+), 13 deletions(-) diff --git a/dbms/src/Flash/Mpp/ExchangeReceiver.cpp b/dbms/src/Flash/Mpp/ExchangeReceiver.cpp index bcbfe433694..ee3b0ced0ce 100644 --- a/dbms/src/Flash/Mpp/ExchangeReceiver.cpp +++ b/dbms/src/Flash/Mpp/ExchangeReceiver.cpp @@ -696,7 +696,24 @@ DecodeDetail ExchangeReceiverBase::decodeChunks( // Record total packet size even if fine grained shuffle is enabled. detail.packet_bytes = packet.ByteSizeLong(); - if (packet.version()) + switch (packet.version()) + { + case 0: + { + for (const String * chunk : recv_msg->chunks) + { + auto result = decoder_ptr->decodeAndSquash(*chunk); + if (!result) + continue; + detail.rows += result->rows(); + if likely (result->rows() > 0) + { + block_queue.push(std::move(result.value())); + } + } + return detail; + } + case 1: { for (auto && chunk : packet.chunks()) { @@ -711,17 +728,11 @@ DecodeDetail ExchangeReceiverBase::decodeChunks( } return detail; } - - for (const String * chunk : recv_msg->chunks) + default: { - auto result = decoder_ptr->decodeAndSquash(*chunk); - if (!result) - continue; - detail.rows += result->rows(); - if likely (result->rows() > 0) - { - block_queue.push(std::move(result.value())); - } + RUNTIME_CHECK_MSG(false, "Unknown mpp packet version {}", packet.version()); + break; + } } return detail; } diff --git a/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp b/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp index 8db87131689..567681becbe 100644 --- a/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp +++ b/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp @@ -521,7 +521,7 @@ try { auto & packet = tracked_packet->getPacket(); - ASSERT_EQ(packet.version(), TiDB::MppVersion::MppVersionV1); + ASSERT_EQ(packet.version(), 1); for (auto && chunk : packet.chunks()) { diff --git a/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp b/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp index a22e99636a2..030ce820ed0 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp +++ b/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp @@ -25,6 +25,7 @@ #include "Flash/Coprocessor/CompressCHBlockChunkCodecStream.h" #include "Flash/Mpp/HashPartitionWriterV1.h" +#include "Flash/Mpp/MppVersion.h" namespace DB { @@ -151,7 +152,7 @@ std::unique_ptr NewMPPExchangeWriter( else { auto && compress_method = dag_context.getExchangeSenderMeta().compress(); - if (compress_method == mpp::CompressMethod::NONE || !dag_context.getMPPTaskMeta().mpp_version()) + if (TiDB::MppVersion::MppVersionV0 == dag_context.getMPPTaskMeta().mpp_version()) return std::make_unique>( writer, partition_col_ids, From 436bb802563455db6ffe5cdd041800815a69b10c Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Tue, 3 Jan 2023 14:12:40 +0800 Subject: [PATCH 36/93] 30 --- contrib/tzg.mpp.proto.patch | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/contrib/tzg.mpp.proto.patch b/contrib/tzg.mpp.proto.patch index 239232a9903..776e7ca21f3 100644 --- a/contrib/tzg.mpp.proto.patch +++ b/contrib/tzg.mpp.proto.patch @@ -1085,7 +1085,7 @@ index fcd4691..783c2eb 100644 iNdEx = preIndex skippy, err := skipMpp(dAtA[iNdEx:]) diff --git a/proto/mpp.proto b/proto/mpp.proto -index 6739455..d693f8d 100644 +index 6739455..0fdf25d 100644 --- a/proto/mpp.proto +++ b/proto/mpp.proto @@ -11,6 +11,17 @@ option (gogoproto.unmarshaler_all) = true; @@ -1146,7 +1146,7 @@ index 6739455..d693f8d 100644 Error error = 2; repeated bytes chunks = 3; repeated uint64 stream_ids = 4; -+ int64 version = 5; // version of data format ++ int64 version = 5; // version of data format + DataPacketCompressMeta compress = 6; // data compression } From a8f1b89c97a8924b82c98f32da4299588a40c6a3 Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Wed, 4 Jan 2023 21:41:30 +0800 Subject: [PATCH 37/93] 31 --- .../Flash/Coprocessor/CHBlockChunkCodec.cpp | 12 +- .../Coprocessor/CHBlockChunkCodecStream.h | 2 +- .../Coprocessor/ChunkDecodeAndSquash.cpp | 4 +- .../CompressCHBlockChunkCodecStream.cpp | 107 +++++- .../CompressCHBlockChunkCodecStream.h | 4 +- dbms/src/Flash/Mpp/ExchangeReceiver.cpp | 18 +- dbms/src/Flash/Mpp/HashPartitionWriter.cpp | 30 +- dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp | 344 ++++++++++++------ dbms/src/Flash/Mpp/HashPartitionWriterV1.h | 18 +- .../Mpp/tests/gtest_mpp_exchange_writer.cpp | 4 +- .../Planner/plans/PhysicalExchangeSender.cpp | 4 +- metrics/grafana/tiflash_summary.json | 2 +- 12 files changed, 376 insertions(+), 173 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/CHBlockChunkCodec.cpp b/dbms/src/Flash/Coprocessor/CHBlockChunkCodec.cpp index 7de63acb975..acb21a8d314 100644 --- a/dbms/src/Flash/Coprocessor/CHBlockChunkCodec.cpp +++ b/dbms/src/Flash/Coprocessor/CHBlockChunkCodec.cpp @@ -74,15 +74,17 @@ CHBlockChunkCodec::CHBlockChunkCodec(const DAGSchema & schema) output_names.push_back(c.first); } -size_t getExtraInfoSize(const Block & block) +size_t GetExtraInfoSize(const Block & block) { - size_t size = 64; /// to hold some length of structures, such as column number, row number... + size_t size = 8 + 8; /// to hold some length of structures, such as column number, row number... size_t columns = block.columns(); for (size_t i = 0; i < columns; ++i) { const ColumnWithTypeAndName & column = block.safeGetByPosition(i); size += column.name.size(); + size += 8; size += column.type->getName().size(); + size += 8; if (column.column->isColumnConst()) { size += column.column->byteSize() * column.column->size(); @@ -93,7 +95,7 @@ size_t getExtraInfoSize(const Block & block) size_t ApproxBlockBytes(const Block & block) { - return block.bytes() + getExtraInfoSize(block); + return block.bytes() + GetExtraInfoSize(block); } CompressionMethod ToInternalCompressionMethod(mpp::CompressMethod compress_method) @@ -111,7 +113,7 @@ CompressionMethod ToInternalCompressionMethod(mpp::CompressMethod compress_metho } } -void writeData(const IDataType & type, const ColumnPtr & column, WriteBuffer & ostr, size_t offset, size_t limit) +void WriteColumnData(const IDataType & type, const ColumnPtr & column, WriteBuffer & ostr, size_t offset, size_t limit) { /** If there are columns-constants - then we materialize them. * (Since the data type does not know how to serialize / deserialize constants.) @@ -153,7 +155,7 @@ void EncodeCHBlockChunk(WriteBuffer * ostr_ptr, const Block & block) writeStringBinary(column.type->getName(), *ostr_ptr); if (rows) - writeData(*column.type, column.column, *ostr_ptr, 0, 0); + WriteColumnData(*column.type, column.column, *ostr_ptr, 0, 0); } } diff --git a/dbms/src/Flash/Coprocessor/CHBlockChunkCodecStream.h b/dbms/src/Flash/Coprocessor/CHBlockChunkCodecStream.h index cacf94778dc..34b482e90e3 100644 --- a/dbms/src/Flash/Coprocessor/CHBlockChunkCodecStream.h +++ b/dbms/src/Flash/Coprocessor/CHBlockChunkCodecStream.h @@ -43,7 +43,7 @@ class CHBlockChunkCodecStream : public ChunkCodecStream DataTypes expected_types; }; -size_t getExtraInfoSize(const Block & block); +size_t GetExtraInfoSize(const Block & block); size_t ApproxBlockBytes(const Block & block); CompressionMethod ToInternalCompressionMethod(mpp::CompressMethod compress_method); std::unique_ptr NewCHBlockChunkCodecStream(const std::vector & field_types); diff --git a/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.cpp b/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.cpp index b5c2691349b..1e0ccb8d8d4 100644 --- a/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.cpp +++ b/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.cpp @@ -57,7 +57,7 @@ std::optional CHBlockChunkDecodeAndSquash::decodeAndSquashWithCompressImp { size_t rows{}; Block block = DecodeHeader(istr, codec.header, rows); - DecodeColumns(istr, block, codec.header.columns(), rows, static_cast(rows_limit * 1.5)); + DecodeColumns(istr, block, rows, static_cast(rows_limit * 1.5)); if (block) accumulated_block.emplace(std::move(block)); } @@ -65,7 +65,7 @@ std::optional CHBlockChunkDecodeAndSquash::decodeAndSquashWithCompressImp { size_t rows{}; DecodeHeader(istr, codec.header, rows); - DecodeColumns(istr, *accumulated_block, codec.header.columns(), rows, 0); + DecodeColumns(istr, *accumulated_block, rows, 0); } if (accumulated_block && accumulated_block->rows() >= rows_limit) diff --git a/dbms/src/Flash/Coprocessor/CompressCHBlockChunkCodecStream.cpp b/dbms/src/Flash/Coprocessor/CompressCHBlockChunkCodecStream.cpp index caaeb5b583f..6a9c709321a 100644 --- a/dbms/src/Flash/Coprocessor/CompressCHBlockChunkCodecStream.cpp +++ b/dbms/src/Flash/Coprocessor/CompressCHBlockChunkCodecStream.cpp @@ -6,9 +6,11 @@ #include #include +#include namespace DB { +extern void WriteColumnData(const IDataType & type, const ColumnPtr & column, WriteBuffer & ostr, size_t offset, size_t limit); void EncodeHeader(WriteBuffer & ostr, const Block & header, size_t rows) { @@ -24,12 +26,10 @@ void EncodeHeader(WriteBuffer & ostr, const Block & header, size_t rows) } } -extern void writeData(const IDataType & type, const ColumnPtr & column, WriteBuffer & ostr, size_t offset, size_t limit); - -void EncodeColumn(WriteBuffer & ostr, const ColumnPtr & column, const ColumnWithTypeAndName & type_name) +void EncodeColumn__(WriteBuffer & ostr, const ColumnPtr & column, const ColumnWithTypeAndName & type_name) { writeVarUInt(column->size(), ostr); - writeData(*type_name.type, column, ostr, 0, 0); + WriteColumnData(*type_name.type, column, ostr, 0, 0); } std::unique_ptr NewCompressCHBlockChunkCodecStream(CompressionMethod compress_method) @@ -37,20 +37,19 @@ std::unique_ptr NewCompressCHBlockChunkCodecStr return std::make_unique(compress_method); } -Block DecodeHeader(ReadBuffer & istr, const Block & header, size_t & rows) +Block DecodeHeader(ReadBuffer & istr, const Block & header, size_t & total_rows) { Block res; - if (istr.eof()) - { - return res; - } + assert(!istr.eof()); size_t columns = 0; { readVarUInt(columns, istr); - readVarUInt(rows, istr); + readVarUInt(total_rows, istr); } + if (header) + CodecUtils::checkColumnSize(header.columns(), columns); for (size_t i = 0; i < columns; ++i) { @@ -78,7 +77,7 @@ Block DecodeHeader(ReadBuffer & istr, const Block & header, size_t & rows) return res; } -void DecodeColumns(ReadBuffer & istr, Block & res, size_t columns, size_t rows, size_t reserve_size) +void DecodeColumns___(ReadBuffer & istr, Block & res, size_t columns, size_t rows, size_t reserve_size) { if (!rows) return; @@ -116,4 +115,90 @@ void DecodeColumns(ReadBuffer & istr, Block & res, size_t columns, size_t rows, res.setColumns(std::move(mutable_columns)); } + +void DecodeColumns_bb(ReadBuffer & istr, Block & res, size_t rows_to_read, size_t reserve_size) +{ + if (!rows_to_read) + return; + + auto && mutable_columns = res.mutateColumns(); + for (auto && column : mutable_columns) + { + if (reserve_size > 0) + column->reserve(std::max(rows_to_read, reserve_size)); + else + column->reserve(rows_to_read + column->size()); + } + + size_t decode_rows = 0; + for (size_t sz = 0; decode_rows < rows_to_read; decode_rows += sz) + { + readVarUInt(sz, istr); + + for (size_t i = 0; i < res.columns(); ++i) + { + /// Data + res.getByPosition(i).type->deserializeBinaryBulkWithMultipleStreams( + *mutable_columns[i], + [&](const IDataType::SubstreamPath &) { + return &istr; + }, + sz, + 0, + {}, + {}); + } + } + + assert(decode_rows == rows_to_read); + + res.setColumns(std::move(mutable_columns)); +} + +void DecodeColumns(ReadBuffer & istr, Block & res, size_t rows_to_read, size_t reserve_size) +{ + if (!rows_to_read) + return; + + auto && mutable_columns = res.mutateColumns(); + for (auto && column : mutable_columns) + { + if (reserve_size > 0) + column->reserve(std::max(rows_to_read, reserve_size)); + else + column->reserve(rows_to_read + column->size()); + } + + std::vector column_batch; + { + size_t sz{}; + readVarUInt(sz, istr); + column_batch.resize(sz); + for (size_t i = 0; i < sz; ++i) + { + readVarUInt(column_batch[i], istr); + } + assert(std::accumulate(column_batch.begin(), column_batch.end(), 0, [](auto c, auto & e) { return c + e; }) == int(rows_to_read)); + } + + for (size_t i = 0; i < res.columns(); ++i) + { + for (const auto & sz : column_batch) + { + /// Data + res.getByPosition(i).type->deserializeBinaryBulkWithMultipleStreams( + *mutable_columns[i], + [&](const IDataType::SubstreamPath &) { + return &istr; + }, + sz, + 0, + {}, + {}); + } + } + + res.setColumns(std::move(mutable_columns)); +} + } // namespace DB \ No newline at end of file diff --git a/dbms/src/Flash/Coprocessor/CompressCHBlockChunkCodecStream.h b/dbms/src/Flash/Coprocessor/CompressCHBlockChunkCodecStream.h index 463fdebce64..341dce05d6b 100644 --- a/dbms/src/Flash/Coprocessor/CompressCHBlockChunkCodecStream.h +++ b/dbms/src/Flash/Coprocessor/CompressCHBlockChunkCodecStream.h @@ -118,8 +118,8 @@ struct CompressCHBlockChunkCodecStream }; void EncodeHeader(WriteBuffer & ostr, const Block & header, size_t rows); -void EncodeColumn(WriteBuffer & ostr, const ColumnPtr & column, const ColumnWithTypeAndName & type_name); -void DecodeColumns(ReadBuffer & istr, Block & res, size_t columns, size_t rows, size_t reserve_size = 0); +void EncodeColumn__(WriteBuffer & ostr, const ColumnPtr & column, const ColumnWithTypeAndName & type_name); +void DecodeColumns(ReadBuffer & istr, Block & res, size_t rows_to_read, size_t reserve_size = 0); Block DecodeHeader(ReadBuffer & istr, const Block & header, size_t & rows); std::unique_ptr NewCompressCHBlockChunkCodecStream(CompressionMethod compress_method); diff --git a/dbms/src/Flash/Mpp/ExchangeReceiver.cpp b/dbms/src/Flash/Mpp/ExchangeReceiver.cpp index ee3b0ced0ce..3d1d12bcf70 100644 --- a/dbms/src/Flash/Mpp/ExchangeReceiver.cpp +++ b/dbms/src/Flash/Mpp/ExchangeReceiver.cpp @@ -26,6 +26,7 @@ #include #include +#include "Flash/Mpp/HashPartitionWriterV1.h" #include "mpp.pb.h" namespace DB @@ -698,7 +699,7 @@ DecodeDetail ExchangeReceiverBase::decodeChunks( switch (packet.version()) { - case 0: + case HashPartitionWriterV0: { for (const String * chunk : recv_msg->chunks) { @@ -713,24 +714,25 @@ DecodeDetail ExchangeReceiverBase::decodeChunks( } return detail; } - case 1: + case HashPartitionWriterV1: { + RUNTIME_CHECK(packet.chunks().size() == int(recv_msg->chunks.size()), + packet.chunks().size(), + recv_msg->chunks.size()); + for (auto && chunk : packet.chunks()) { auto && result = decoder_ptr->decodeAndSquash(chunk, packet.compress().method() != mpp::CompressMethod::NONE); - if (!result) + if (!result || !result->rows()) continue; detail.rows += result->rows(); - if likely (result->rows() > 0) - { - block_queue.push(std::move(*result)); - } + block_queue.push(std::move(*result)); } return detail; } default: { - RUNTIME_CHECK_MSG(false, "Unknown mpp packet version {}", packet.version()); + RUNTIME_CHECK_MSG(false, "Unknown mpp packet version {}, please update TiFlash instance", packet.version()); break; } } diff --git a/dbms/src/Flash/Mpp/HashPartitionWriter.cpp b/dbms/src/Flash/Mpp/HashPartitionWriter.cpp index 874352699c6..ea9dc0f8640 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriter.cpp +++ b/dbms/src/Flash/Mpp/HashPartitionWriter.cpp @@ -180,33 +180,15 @@ void HashPartitionWriter::writePackets(const TrackedMppDataPa { writer->partitionWrite(packet, part_id); - auto sz = inner_packet.ByteSizeLong(); - switch (inner_packet.compress().method()) - { - case mpp::NONE: - { - if (writer->isLocal(part_id)) - { - GET_METRIC(tiflash_exchange_data_bytes, type_hash_none_local).Increment(sz); - } - else - { - GET_METRIC(tiflash_exchange_data_bytes, type_hash_none).Increment(sz); - } - break; - } - case mpp::LZ4: + assert(inner_packet.compress().method() == mpp::CompressMethod::NONE); + + if (auto sz = inner_packet.ByteSizeLong(); writer->isLocal(part_id)) { - GET_METRIC(tiflash_exchange_data_bytes, type_hash_lz4).Increment(sz); - break; + GET_METRIC(tiflash_exchange_data_bytes, type_hash_none_local).Increment(sz); } - case mpp::ZSTD: + else { - GET_METRIC(tiflash_exchange_data_bytes, type_hash_zstd).Increment(sz); - break; - } - default: - break; + GET_METRIC(tiflash_exchange_data_bytes, type_hash_none).Increment(sz); } } } diff --git a/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp b/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp index bf90cb39663..f52a1352107 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp +++ b/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp @@ -22,6 +22,7 @@ #include #include +#include #include #include "Common/Exception.h" @@ -31,23 +32,24 @@ #include "Flash/Coprocessor/CompressedCHBlockChunkCodec.h" #include "Flash/Mpp/MppVersion.h" #include "IO/CompressedStream.h" +#include "IO/CompressionSettings.h" #include "common/logger_useful.h" #include "ext/scope_guard.h" namespace DB { template -HashPartitionWriterV1::HashPartitionWriterV1( +HashPartitionWriterImplV1::HashPartitionWriterImplV1( ExchangeWriterPtr writer_, std::vector partition_col_ids_, TiDB::TiDBCollators collators_, - Int64 batch_send_min_limit_, + Int64 partition_batch_limit_, bool should_send_exec_summary_at_last_, DAGContext & dag_context_, mpp::CompressMethod compress_method_) : DAGResponseWriter(/*records_per_chunk=*/-1, dag_context_) , partition_num(writer_->getPartitionNum()) - , batch_send_min_limit(batch_send_min_limit_ * partition_num) + , partition_batch_limit(partition_batch_limit_ * partition_num) , should_send_exec_summary_at_last(should_send_exec_summary_at_last_) , writer(writer_) , partition_col_ids(std::move(partition_col_ids_)) @@ -68,7 +70,7 @@ HashPartitionWriterV1::HashPartitionWriterV1( } template -void HashPartitionWriterV1::finishWrite() +void HashPartitionWriterImplV1::finishWrite() { assert(0 == rows_in_blocks); if (should_send_exec_summary_at_last) @@ -76,7 +78,7 @@ void HashPartitionWriterV1::finishWrite() } template -void HashPartitionWriterV1::sendExecutionSummary() +void HashPartitionWriterImplV1::sendExecutionSummary() { tipb::SelectResponse response; summary_collector.addExecuteSummaries(response); @@ -84,14 +86,14 @@ void HashPartitionWriterV1::sendExecutionSummary() } template -void HashPartitionWriterV1::flush() +void HashPartitionWriterImplV1::flush() { if (rows_in_blocks > 0) partitionAndEncodeThenWriteBlocks(); } template -void HashPartitionWriterV1::write(const Block & block) +void HashPartitionWriterImplV1::write(const Block & block) { RUNTIME_CHECK_MSG( block.columns() == dag_context.result_field_types.size(), @@ -103,27 +105,30 @@ void HashPartitionWriterV1::write(const Block & block) blocks.push_back(block); } - if (static_cast(rows_in_blocks) > batch_send_min_limit) + if (static_cast(rows_in_blocks) > partition_batch_limit) partitionAndEncodeThenWriteBlocks(); } +extern void WriteColumnData(const IDataType & type, const ColumnPtr & column, WriteBuffer & ostr, size_t offset, size_t limit); + template -void HashPartitionWriterV1::partitionAndEncodeThenWriteBlocks() +void HashPartitionWriterImplV1::partitionAndEncodeThenWriteBlocks() { + return partitionAndEncodeThenWriteBlocksTest(); + assert(compress_chunk_codec_stream); - auto tracked_packets = HashBaseWriterHelper::createPackets(partition_num, 1); + // Set mpp packet data version to `1` + auto tracked_packets = HashBaseWriterHelper::createPackets(partition_num, HashPartitionWriterV1); + // Do NOT enable data compression when using local tunnel for (size_t part_id = 0; part_id < partition_num; ++part_id) { - auto method = compress_method; - if (writer->isLocal(part_id)) - { - method = mpp::CompressMethod::NONE; - } + auto method = writer->isLocal(part_id) ? mpp::CompressMethod::NONE : compress_method; tracked_packets[part_id]->getPacket().mutable_compress()->set_method(method); } + // Sum of all approximate block data memory size size_t ori_block_mem_size = 0; if (!blocks.empty()) @@ -131,12 +136,13 @@ void HashPartitionWriterV1::partitionAndEncodeThenWriteBlocks assert(rows_in_blocks > 0); HashBaseWriterHelper::materializeBlocks(blocks); - Block dest_block_header = blocks[0].cloneEmpty(); - assertBlockSchema(expected_types, dest_block_header, "HashPartitionWriterV1"); + + // All blocks are same, use one block's meta info as header + Block dest_block_header = blocks.back().cloneEmpty(); std::vector partition_key_containers(collators.size()); std::vector> dest_columns(partition_num); - size_t total_rows = 0, encoded_rows = 0; + [[maybe_unused]] size_t total_rows = 0, encoded_rows = 0; while (!blocks.empty()) { @@ -147,7 +153,7 @@ void HashPartitionWriterV1::partitionAndEncodeThenWriteBlocks ori_block_mem_size += ApproxBlockBytes(block); total_rows += block.rows(); - auto dest_tbl_cols = HashBaseWriterHelper::createDestColumns(block, partition_num); + auto && dest_tbl_cols = HashBaseWriterHelper::createDestColumns(block, partition_num); HashBaseWriterHelper::scatterColumns(block, partition_num, collators, partition_key_containers, partition_col_ids, dest_tbl_cols); blocks.pop_back(); @@ -158,136 +164,254 @@ void HashPartitionWriterV1::partitionAndEncodeThenWriteBlocks } } + size_t header_size = GetExtraInfoSize(dest_block_header); + for (size_t part_id = 0; part_id < partition_num; ++part_id) { - WriteBuffer * ostr_ptr{}; - if (tracked_packets[part_id]->getPacket().compress().method() == mpp::NONE) - { - ostr_ptr = compress_chunk_codec_stream->getWriterWithoutCompress(); - // for (auto && columns : dest_columns[part_id]) - // { - // dest_block_header.setColumns(std::move(columns)); - // encoded_rows += dest_block_header.rows(); - - // EncodeCHBlockChunk(ostr_ptr, dest_block_header); - // tracked_packets[part_id]->getPacket().add_chunks(ostr_ptr->getString()); - // ostr_ptr->reset(); - - // // { - // // const auto & chunks = tracked_packets[part_id]->getPacket().chunks(); - // // const auto & dd = chunks[chunks.size() - 1]; - - // // auto res = CHBlockChunkCodec::decode(dd, dest_block_header); - // // RUNTIME_CHECK(res.rows() == dest_block_header.rows(), res.rows(), dest_block_header.rows()); - // // RUNTIME_CHECK(res.columns() == dest_block_header.columns(), res.columns(), dest_block_header.columns()); - // // res.checkNumberOfRows(); - // // for (size_t i = 0; i < res.columns(); ++i) - // // { - // // RUNTIME_CHECK(dest_block_header.getByPosition(i) == res.getByPosition(i)); - // // } - // // } - // } - } - else + size_t part_rows = std::accumulate(dest_columns[part_id].begin(), dest_columns[part_id].end(), 0, [](const auto & r, const auto & columns) { return r + columns[0]->size(); }); + + if (!part_rows) + continue; + + size_t part_column_bytes = std::accumulate(dest_columns[part_id].begin(), dest_columns[part_id].end(), 0, [](auto res, const auto & columns) { + for (const auto & elem : columns) + res += elem->byteSize(); + return res + 8; + }); + + // Each partition encode format: + // header meta(include all row count of the partition); + // repeated: + // row count; + // columns data; + size_t init_size = part_column_bytes + header_size; + + // Reserve enough memory buffer size + auto output_buffer = std::make_unique(init_size); + std::unique_ptr compress_codec{}; + WriteBuffer * ostr_ptr = output_buffer.get(); + + // Init compression writer + if (tracked_packets[part_id]->getPacket().compress().method() != mpp::NONE) { - ostr_ptr = compress_chunk_codec_stream->getWriter(); + compress_codec = std::make_unique( + *output_buffer, + CompressionSettings(ToInternalCompressionMethod(compress_method)), + init_size); + ostr_ptr = compress_codec.get(); } + // Encode header + EncodeHeader(*ostr_ptr, dest_block_header, part_rows); + + for (auto && columns : dest_columns[part_id]) { - size_t part_rows = std::accumulate(dest_columns[part_id].begin(), dest_columns[part_id].end(), 0, [](const auto & r, const auto & columns) { return r + columns[0]->size(); }); - encoded_rows += part_rows; + size_t rows = columns[0]->size(); + if (!rows) + continue; - EncodeHeader(*ostr_ptr, dest_block_header, part_rows); + // Encode row count for next columns + writeVarUInt(rows, *ostr_ptr); + encoded_rows += rows; + + // Encode columns data for (size_t col_index = 0; col_index < dest_block_header.columns(); ++col_index) { auto && col_type_name = dest_block_header.getByPosition(col_index); - for (auto && columns : dest_columns[part_id]) - { - EncodeColumn(*ostr_ptr, std::move(columns[col_index]), col_type_name); - } + WriteColumnData(*col_type_name.type, std::move(columns[col_index]), *ostr_ptr, 0, 0); } - tracked_packets[part_id]->getPacket().add_chunks(compress_chunk_codec_stream->getString()); - compress_chunk_codec_stream->reset(); - - // { - // // decode - // const auto & chunks = tracked_packets[part_id]->getPacket().chunks(); - // const auto & dd = chunks[chunks.size() - 1]; - - // ReadBufferFromString istr(dd); - // auto && compress_buffer = CompressedCHBlockChunkCodec::CompressedReadBuffer(istr); - - // size_t rows{}; - // Block res = DecodeHeader(compress_buffer, dest_block_header, rows); - // DecodeColumns(compress_buffer, res, res.columns(), rows); - // RUNTIME_CHECK(res.rows() == part_rows, res.rows(), part_rows); - // RUNTIME_CHECK(res.columns() == dest_block_header.columns(), res.columns(), dest_block_header.columns()); - // res.checkNumberOfRows(); - // for (size_t i = 0; i < res.columns(); ++i) - // { - // RUNTIME_CHECK(dest_block_header.getByPosition(i) == res.getByPosition(i)); - // } - // } + columns.clear(); } - } - RUNTIME_CHECK(encoded_rows == total_rows, encoded_rows, total_rows); + // Flush rest buffer + if (compress_codec) + compress_codec->next(); + tracked_packets[part_id]->getPacket().add_chunks(output_buffer->releaseStr()); + } + assert(encoded_rows == total_rows); assert(blocks.empty()); rows_in_blocks = 0; } - writePackets(tracked_packets); + writePackets(std::move(tracked_packets)); GET_METRIC(tiflash_exchange_data_bytes, type_hash_original_all).Increment(ori_block_mem_size); } + template -void HashPartitionWriterV1::writePackets(const TrackedMppDataPacketPtrs & packets) +void HashPartitionWriterImplV1::partitionAndEncodeThenWriteBlocksTest() { - for (size_t part_id = 0; part_id < packets.size(); ++part_id) + // Set mpp packet data version to `1` + auto tracked_packets = HashBaseWriterHelper::createPackets(partition_num, HashPartitionWriterV1); + + // Do NOT enable data compression when using local tunnel + for (size_t part_id = 0; part_id < partition_num; ++part_id) { - const auto & packet = packets[part_id]; - assert(packet); + auto method = writer->isLocal(part_id) ? mpp::CompressMethod::NONE : compress_method; + tracked_packets[part_id]->getPacket().mutable_compress()->set_method(method); + } - auto & inner_packet = packet->getPacket(); - if (likely(inner_packet.chunks_size() > 0)) + // Sum of all approximate block data memory size + size_t ori_block_mem_size = 0; + + if (!blocks.empty()) + { + assert(rows_in_blocks > 0); + + HashBaseWriterHelper::materializeBlocks(blocks); + Block dest_block_header = blocks.back().cloneEmpty(); + + std::vector partition_key_containers(collators.size()); + std::vector> dest_columns(partition_num); + size_t total_rows = 0, encoded_rows = 0; + + while (!blocks.empty()) { - writer->partitionWrite(packet, part_id); + const auto & block = blocks.back(); + block.checkNumberOfRows(); + assertBlockSchema(expected_types, block, "HashPartitionWriterV1"); - auto sz = inner_packet.ByteSizeLong(); - switch (inner_packet.compress().method()) - { - case mpp::NONE: + ori_block_mem_size += ApproxBlockBytes(block); + total_rows += block.rows(); + + auto dest_tbl_cols = HashBaseWriterHelper::createDestColumns(block, partition_num); + HashBaseWriterHelper::scatterColumns(block, partition_num, collators, partition_key_containers, partition_col_ids, dest_tbl_cols); + blocks.pop_back(); + + for (size_t part_id = 0; part_id < partition_num; ++part_id) { - if (writer->isLocal(part_id)) - { - GET_METRIC(tiflash_exchange_data_bytes, type_hash_none_local).Increment(sz); - } - else - { - GET_METRIC(tiflash_exchange_data_bytes, type_hash_none).Increment(sz); - } - break; + auto & columns = dest_tbl_cols[part_id]; + dest_columns[part_id].emplace_back(std::move(columns)); } - case mpp::LZ4: + } + + size_t header_size = GetExtraInfoSize(dest_block_header) + 8; + + for (size_t part_id = 0; part_id < partition_num; ++part_id) + { + auto & part_columns = dest_columns[part_id]; + size_t part_rows = std::accumulate(part_columns.begin(), part_columns.end(), 0, [](const auto & r, const auto & columns) { return r + columns.front()->size(); }); + + if (!part_rows) + continue; + + encoded_rows += part_rows; + + size_t part_column_bytes = std::accumulate( + part_columns.begin(), + part_columns.end(), + 0, + [](auto res, const auto & columns) { + for (const auto & elem : columns) + res += elem->byteSize(); + return res + 8; + }); + + size_t init_size = part_column_bytes + header_size; + + // Reserve enough memory buffer size + auto output_buffer = std::make_unique(init_size); + std::unique_ptr compress_codec{}; + WriteBuffer * ostr_ptr = output_buffer.get(); + + // Init compression writer + if (tracked_packets[part_id]->getPacket().compress().method() != mpp::NONE) { - GET_METRIC(tiflash_exchange_data_bytes, type_hash_lz4).Increment(sz); - break; + compress_codec = std::make_unique( + *output_buffer, + CompressionSettings(ToInternalCompressionMethod(compress_method)), + init_size); + ostr_ptr = compress_codec.get(); } - case mpp::ZSTD: + + // Encode header + EncodeHeader(*ostr_ptr, dest_block_header, part_rows); + writeVarUInt(part_columns.size(), *ostr_ptr); + for (auto && columns : part_columns) { - GET_METRIC(tiflash_exchange_data_bytes, type_hash_zstd).Increment(sz); - break; + writeVarUInt(columns.front()->size(), *ostr_ptr); } - default: - break; + + for (size_t col_index = 0; col_index < dest_block_header.columns(); ++col_index) + { + auto && col_type_name = dest_block_header.getByPosition(col_index); + for (auto && columns : part_columns) + { + WriteColumnData(*col_type_name.type, std::move(columns[col_index]), *ostr_ptr, 0, 0); + } } + + // Flush rest buffer + if (compress_codec) + compress_codec->next(); + + tracked_packets[part_id]->getPacket().add_chunks(output_buffer->releaseStr()); + } + + RUNTIME_CHECK(encoded_rows == total_rows, encoded_rows, total_rows); + + assert(blocks.empty()); + rows_in_blocks = 0; + } + + writePackets(std::move(tracked_packets)); + + GET_METRIC(tiflash_exchange_data_bytes, type_hash_original_all).Increment(ori_block_mem_size); +} + +static void updateHashPartitionWriterMetrics(mpp::CompressMethod method, size_t sz, bool is_local) +{ + switch (method) + { + case mpp::NONE: + { + if (is_local) + { + GET_METRIC(tiflash_exchange_data_bytes, type_hash_none_local).Increment(sz); + } + else + { + GET_METRIC(tiflash_exchange_data_bytes, type_hash_none).Increment(sz); + } + break; + } + case mpp::LZ4: + { + GET_METRIC(tiflash_exchange_data_bytes, type_hash_lz4).Increment(sz); + break; + } + case mpp::ZSTD: + { + GET_METRIC(tiflash_exchange_data_bytes, type_hash_zstd).Increment(sz); + break; + } + default: + break; + } +} + +template +void HashPartitionWriterImplV1::writePackets(TrackedMppDataPacketPtrs && packets) +{ + for (size_t part_id = 0; part_id < packets.size(); ++part_id) + { + auto & packet = packets[part_id]; + assert(packet); + + auto & inner_packet = packet->getPacket(); + + if (auto sz = inner_packet.ByteSizeLong(); likely(inner_packet.chunks_size() > 0)) + { + auto method = inner_packet.compress().method(); + writer->partitionWrite(std::move(packet), part_id); + updateHashPartitionWriterMetrics(method, sz, writer->isLocal(part_id)); } } } -template class HashPartitionWriterV1; +template class HashPartitionWriterImplV1; } // namespace DB diff --git a/dbms/src/Flash/Mpp/HashPartitionWriterV1.h b/dbms/src/Flash/Mpp/HashPartitionWriterV1.h index 8dcef4d2d03..8fbd6889357 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriterV1.h +++ b/dbms/src/Flash/Mpp/HashPartitionWriterV1.h @@ -27,15 +27,22 @@ namespace DB struct CompressCHBlockChunkCodecStream; +enum HashPartitionWriterVersion : int64_t +{ + HashPartitionWriterV0 = 0, + HashPartitionWriterV1, + HashPartitionWriterVersionMax, +}; + template -class HashPartitionWriterV1 : public DAGResponseWriter +class HashPartitionWriterImplV1 : public DAGResponseWriter { public: - HashPartitionWriterV1( + HashPartitionWriterImplV1( ExchangeWriterPtr writer_, std::vector partition_col_ids_, TiDB::TiDBCollators collators_, - Int64 batch_send_min_limit_, + Int64 partition_batch_limit_, bool should_send_exec_summary_at_last, DAGContext & dag_context_, mpp::CompressMethod compress_method_); @@ -45,14 +52,15 @@ class HashPartitionWriterV1 : public DAGResponseWriter private: void partitionAndEncodeThenWriteBlocks(); + void partitionAndEncodeThenWriteBlocksTest(); - void writePackets(const TrackedMppDataPacketPtrs & packets); + void writePackets(TrackedMppDataPacketPtrs && packets); void sendExecutionSummary(); private: uint16_t partition_num; - Int64 batch_send_min_limit; + Int64 partition_batch_limit; bool should_send_exec_summary_at_last; ExchangeWriterPtr writer; std::vector blocks; diff --git a/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp b/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp index 567681becbe..b1f37d78c52 100644 --- a/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp +++ b/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp @@ -495,7 +495,7 @@ try auto mock_writer = std::make_shared(checker, part_num); // 3. Start to write. - auto dag_writer = std::make_shared>>( + auto dag_writer = std::make_shared>>( mock_writer, part_col_ids, part_col_collators, @@ -521,7 +521,7 @@ try { auto & packet = tracked_packet->getPacket(); - ASSERT_EQ(packet.version(), 1); + ASSERT_EQ(packet.version(), HashPartitionWriterV1); for (auto && chunk : packet.chunks()) { diff --git a/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp b/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp index 030ce820ed0..4fbce4e0780 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp +++ b/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp @@ -160,11 +160,11 @@ std::unique_ptr NewMPPExchangeWriter( batch_send_min_limit, should_send_exec_summary_at_last, dag_context); - return std::make_unique>( + return std::make_unique>( writer, partition_col_ids, partition_col_collators, - batch_send_min_limit, + 8192, should_send_exec_summary_at_last, dag_context, compress_method); diff --git a/metrics/grafana/tiflash_summary.json b/metrics/grafana/tiflash_summary.json index 86d7c052bff..af8ac96b809 100644 --- a/metrics/grafana/tiflash_summary.json +++ b/metrics/grafana/tiflash_summary.json @@ -3202,7 +3202,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tiflash_coprocessor_response_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance)", + "expr": "sum(rate(tiflash_coprocessor_response_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[30s])) by (instance)", "format": "time_series", "intervalFactor": 1, "legendFormat": "{{instance}}", From 09cb7d3233c5c3fd50a7a5170c1720031dbc9da0 Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Thu, 5 Jan 2023 13:15:02 +0800 Subject: [PATCH 38/93] 32 --- .../CompressCHBlockChunkCodecStream.cpp | 54 +++++-------------- dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp | 30 ++++++----- dbms/src/Flash/Mpp/HashPartitionWriterV1.h | 1 - .../Mpp/tests/gtest_mpp_exchange_writer.cpp | 14 ++++- 4 files changed, 41 insertions(+), 58 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/CompressCHBlockChunkCodecStream.cpp b/dbms/src/Flash/Coprocessor/CompressCHBlockChunkCodecStream.cpp index 6a9c709321a..6ba28c57d87 100644 --- a/dbms/src/Flash/Coprocessor/CompressCHBlockChunkCodecStream.cpp +++ b/dbms/src/Flash/Coprocessor/CompressCHBlockChunkCodecStream.cpp @@ -77,46 +77,7 @@ Block DecodeHeader(ReadBuffer & istr, const Block & header, size_t & total_rows) return res; } -void DecodeColumns___(ReadBuffer & istr, Block & res, size_t columns, size_t rows, size_t reserve_size) -{ - if (!rows) - return; - - auto && mutable_columns = res.mutateColumns(); - - for (size_t i = 0; i < columns; ++i) - { - /// Data - auto && read_column = mutable_columns[i]; - if (reserve_size > 0) - read_column->reserve(std::max(rows, reserve_size) + read_column->size()); - else - read_column->reserve(rows + read_column->size()); - - size_t read_rows = 0; - for (size_t sz = 0; read_rows < rows; read_rows += sz) - { - readVarUInt(sz, istr); - if (!sz) - continue; - res.getByPosition(i).type->deserializeBinaryBulkWithMultipleStreams( - *read_column, - [&](const IDataType::SubstreamPath &) { - return &istr; - }, - sz, - 0, - {}, - {}); - } - assert(read_rows == rows); - } - - res.setColumns(std::move(mutable_columns)); -} - - -void DecodeColumns_bb(ReadBuffer & istr, Block & res, size_t rows_to_read, size_t reserve_size) +[[maybe_unused]] static inline void DecodeColumns_by_block(ReadBuffer & istr, Block & res, size_t rows_to_read, size_t reserve_size) { if (!rows_to_read) return; @@ -130,11 +91,15 @@ void DecodeColumns_bb(ReadBuffer & istr, Block & res, size_t rows_to_read, size_ column->reserve(rows_to_read + column->size()); } + // Contain columns of multi blocks size_t decode_rows = 0; for (size_t sz = 0; decode_rows < rows_to_read; decode_rows += sz) { readVarUInt(sz, istr); + assert(sz > 0); + + // Decode columns of one block for (size_t i = 0; i < res.columns(); ++i) { /// Data @@ -155,7 +120,7 @@ void DecodeColumns_bb(ReadBuffer & istr, Block & res, size_t rows_to_read, size_ res.setColumns(std::move(mutable_columns)); } -void DecodeColumns(ReadBuffer & istr, Block & res, size_t rows_to_read, size_t reserve_size) +[[maybe_unused]] static inline void DecodeColumns_by_col(ReadBuffer & istr, Block & res, size_t rows_to_read, size_t reserve_size) { if (!rows_to_read) return; @@ -185,6 +150,8 @@ void DecodeColumns(ReadBuffer & istr, Block & res, size_t rows_to_read, size_t r { for (const auto & sz : column_batch) { + if (!sz) + continue; /// Data res.getByPosition(i).type->deserializeBinaryBulkWithMultipleStreams( *mutable_columns[i], @@ -201,4 +168,9 @@ void DecodeColumns(ReadBuffer & istr, Block & res, size_t rows_to_read, size_t r res.setColumns(std::move(mutable_columns)); } +void DecodeColumns(ReadBuffer & istr, Block & res, size_t rows_to_read, size_t reserve_size) +{ + return DecodeColumns_by_block(istr, res, rows_to_read, reserve_size); +} + } // namespace DB \ No newline at end of file diff --git a/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp b/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp index f52a1352107..a8037eadec8 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp +++ b/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp @@ -56,7 +56,6 @@ HashPartitionWriterImplV1::HashPartitionWriterImplV1( , collators(std::move(collators_)) , compress_method(compress_method_) { - assert(compress_method != mpp::CompressMethod::NONE); assert(dag_context.getMPPTaskMeta().mpp_version() > 0); rows_in_blocks = 0; @@ -66,7 +65,6 @@ HashPartitionWriterImplV1::HashPartitionWriterImplV1( { expected_types.emplace_back(getDataTypeByFieldTypeForComputingLayer(field_type)); } - compress_chunk_codec_stream = NewCompressCHBlockChunkCodecStream(ToInternalCompressionMethod(compress_method)); } template @@ -114,9 +112,10 @@ extern void WriteColumnData(const IDataType & type, const ColumnPtr & column, Wr template void HashPartitionWriterImplV1::partitionAndEncodeThenWriteBlocks() { - return partitionAndEncodeThenWriteBlocksTest(); + // return partitionAndEncodeThenWriteBlocksTest(); - assert(compress_chunk_codec_stream); + if (blocks.empty()) + return; // Set mpp packet data version to `1` auto tracked_packets = HashBaseWriterHelper::createPackets(partition_num, HashPartitionWriterV1); @@ -131,7 +130,6 @@ void HashPartitionWriterImplV1::partitionAndEncodeThenWriteBl // Sum of all approximate block data memory size size_t ori_block_mem_size = 0; - if (!blocks.empty()) { assert(rows_in_blocks > 0); @@ -168,15 +166,16 @@ void HashPartitionWriterImplV1::partitionAndEncodeThenWriteBl for (size_t part_id = 0; part_id < partition_num; ++part_id) { - size_t part_rows = std::accumulate(dest_columns[part_id].begin(), dest_columns[part_id].end(), 0, [](const auto & r, const auto & columns) { return r + columns[0]->size(); }); + auto & part_columns = dest_columns[part_id]; + size_t part_rows = std::accumulate(part_columns.begin(), part_columns.end(), 0, [](const auto & r, const auto & columns) { return r + columns.front()->size(); }); if (!part_rows) continue; - size_t part_column_bytes = std::accumulate(dest_columns[part_id].begin(), dest_columns[part_id].end(), 0, [](auto res, const auto & columns) { + size_t part_column_bytes = std::accumulate(part_columns.begin(), part_columns.end(), 0, [](auto res, const auto & columns) { for (const auto & elem : columns) res += elem->byteSize(); - return res + 8; + return res + 8 /*partition rows*/; }); // Each partition encode format: @@ -204,9 +203,9 @@ void HashPartitionWriterImplV1::partitionAndEncodeThenWriteBl // Encode header EncodeHeader(*ostr_ptr, dest_block_header, part_rows); - for (auto && columns : dest_columns[part_id]) + for (auto && columns : part_columns) { - size_t rows = columns[0]->size(); + size_t rows = columns.front()->size(); if (!rows) continue; @@ -244,6 +243,9 @@ void HashPartitionWriterImplV1::partitionAndEncodeThenWriteBl template void HashPartitionWriterImplV1::partitionAndEncodeThenWriteBlocksTest() { + if (blocks.empty()) + return; + // Set mpp packet data version to `1` auto tracked_packets = HashBaseWriterHelper::createPackets(partition_num, HashPartitionWriterV1); @@ -257,7 +259,6 @@ void HashPartitionWriterImplV1::partitionAndEncodeThenWriteBl // Sum of all approximate block data memory size size_t ori_block_mem_size = 0; - if (!blocks.empty()) { assert(rows_in_blocks > 0); @@ -266,7 +267,7 @@ void HashPartitionWriterImplV1::partitionAndEncodeThenWriteBl std::vector partition_key_containers(collators.size()); std::vector> dest_columns(partition_num); - size_t total_rows = 0, encoded_rows = 0; + [[maybe_unused]] size_t total_rows = 0, encoded_rows = 0; while (!blocks.empty()) { @@ -340,6 +341,8 @@ void HashPartitionWriterImplV1::partitionAndEncodeThenWriteBl auto && col_type_name = dest_block_header.getByPosition(col_index); for (auto && columns : part_columns) { + if (columns[col_index]->empty()) + continue; WriteColumnData(*col_type_name.type, std::move(columns[col_index]), *ostr_ptr, 0, 0); } } @@ -351,8 +354,7 @@ void HashPartitionWriterImplV1::partitionAndEncodeThenWriteBl tracked_packets[part_id]->getPacket().add_chunks(output_buffer->releaseStr()); } - RUNTIME_CHECK(encoded_rows == total_rows, encoded_rows, total_rows); - + assert(encoded_rows == total_rows); assert(blocks.empty()); rows_in_blocks = 0; } diff --git a/dbms/src/Flash/Mpp/HashPartitionWriterV1.h b/dbms/src/Flash/Mpp/HashPartitionWriterV1.h index 8fbd6889357..2d9cd8dc79d 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriterV1.h +++ b/dbms/src/Flash/Mpp/HashPartitionWriterV1.h @@ -69,7 +69,6 @@ class HashPartitionWriterImplV1 : public DAGResponseWriter size_t rows_in_blocks; DataTypes expected_types; mpp::CompressMethod compress_method{}; - std::unique_ptr compress_chunk_codec_stream; }; } // namespace DB diff --git a/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp b/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp index b1f37d78c52..2dd4e69a725 100644 --- a/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp +++ b/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp @@ -25,6 +25,7 @@ #include #include #include +#include #include #include "Flash/Mpp/MppVersion.h" @@ -514,10 +515,10 @@ try CHBlockChunkDecodeAndSquash decoder(header, 512); - for (const auto & ele : write_report) + for (size_t part_index = 0; part_index < part_num; ++part_index) { size_t decoded_block_rows = 0; - for (const auto & tracked_packet : ele.second) + for (const auto & tracked_packet : write_report[part_index]) { auto & packet = tracked_packet->getPacket(); @@ -525,6 +526,15 @@ try for (auto && chunk : packet.chunks()) { + if (part_index == 0) + { + ASSERT_EQ(packet.compress().method(), mpp::CompressMethod::NONE); + } + else + { + ASSERT_NE(packet.compress().method(), mpp::CompressMethod::NONE); + } + auto && result = decoder.decodeAndSquash(chunk, packet.compress().method() != mpp::CompressMethod::NONE); if (!result) continue; From 334654e689749fa3ff713a673e59fdac8544b2b3 Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Thu, 5 Jan 2023 16:37:20 +0800 Subject: [PATCH 39/93] 33 Signed-off-by: Zhigao Tong --- contrib/tzg.mpp.proto.patch | 195 +++++++++--------- .../Flash/Coprocessor/CHBlockChunkCodec.cpp | 4 +- dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp | 4 +- dbms/src/Flash/Mpp/MPPTask.cpp | 4 +- .../Mpp/tests/gtest_mpp_exchange_writer.cpp | 2 +- 5 files changed, 105 insertions(+), 104 deletions(-) diff --git a/contrib/tzg.mpp.proto.patch b/contrib/tzg.mpp.proto.patch index 776e7ca21f3..b90a69132ff 100644 --- a/contrib/tzg.mpp.proto.patch +++ b/contrib/tzg.mpp.proto.patch @@ -1,5 +1,5 @@ diff --git a/pkg/mpp/mpp.pb.go b/pkg/mpp/mpp.pb.go -index fcd4691..783c2eb 100644 +index fcd4691..be5dd41 100644 --- a/pkg/mpp/mpp.pb.go +++ b/pkg/mpp/mpp.pb.go @@ -26,6 +26,82 @@ var _ = math.Inf @@ -10,21 +10,21 @@ index fcd4691..783c2eb 100644 +type CompressMethod int32 + +const ( -+ CompressMethod_NONE CompressMethod = 0 -+ CompressMethod_LZ4 CompressMethod = 1 -+ CompressMethod_ZSTD CompressMethod = 2 ++ CompressMethod_NONE CompressMethod = 0 ++ CompressMethod_FAST CompressMethod = 1 ++ CompressMethod_HIGH_COMPRESSION CompressMethod = 2 +) + +var CompressMethod_name = map[int32]string{ + 0: "NONE", -+ 1: "LZ4", -+ 2: "ZSTD", ++ 1: "FAST", ++ 2: "HIGH_COMPRESSION", +} + +var CompressMethod_value = map[string]int32{ -+ "NONE": 0, -+ "LZ4": 1, -+ "ZSTD": 2, ++ "NONE": 0, ++ "FAST": 1, ++ "HIGH_COMPRESSION": 2, +} + +func (x CompressMethod) String() string { @@ -296,7 +296,7 @@ index fcd4691..783c2eb 100644 + Error *Error `protobuf:"bytes,2,opt,name=error,proto3" json:"error,omitempty"` + Chunks [][]byte `protobuf:"bytes,3,rep,name=chunks,proto3" json:"chunks,omitempty"` + StreamIds []uint64 `protobuf:"varint,4,rep,packed,name=stream_ids,json=streamIds,proto3" json:"stream_ids,omitempty"` -+ MppVersion int64 `protobuf:"varint,5,opt,name=mpp_version,json=mppVersion,proto3" json:"mpp_version,omitempty"` ++ Version int64 `protobuf:"varint,5,opt,name=version,proto3" json:"version,omitempty"` + Compress *DataPacketCompressMeta `protobuf:"bytes,6,opt,name=compress,proto3" json:"compress,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` @@ -316,9 +316,9 @@ index fcd4691..783c2eb 100644 return nil } -+func (m *MPPDataPacket) GetMppVersion() int64 { ++func (m *MPPDataPacket) GetVersion() int64 { + if m != nil { -+ return m.MppVersion ++ return m.Version + } + return 0 +} @@ -371,7 +371,7 @@ index fcd4691..783c2eb 100644 proto.RegisterType((*MPPDataPacket)(nil), "mpp.MPPDataPacket") proto.RegisterType((*Error)(nil), "mpp.Error") } -@@ -665,49 +840,91 @@ func init() { +@@ -665,49 +840,92 @@ func init() { func init() { proto.RegisterFile("mpp.proto", fileDescriptor_819623c7fa76fc55) } var fileDescriptor_819623c7fa76fc55 = []byte{ @@ -418,59 +418,60 @@ index fcd4691..783c2eb 100644 - 0xb1, 0xf3, 0xf5, 0x76, 0xec, 0x7c, 0xfa, 0x36, 0xfe, 0x07, 0xb6, 0xb9, 0x58, 0x84, 0x2a, 0x5d, - 0x55, 0xe1, 0xaa, 0x32, 0x5f, 0xa7, 0xab, 0xae, 0xf9, 0x39, 0xfd, 0x11, 0x00, 0x00, 0xff, 0xff, - 0xcd, 0x41, 0xdb, 0xe2, 0xed, 0x04, 0x00, 0x00, -+ // 831 bytes of a gzipped FileDescriptorProto -+ 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x55, 0xcf, 0x8f, 0x1b, 0x35, -+ 0x14, 0xee, 0x64, 0xf2, 0xf3, 0x65, 0x36, 0xa4, 0xee, 0xaa, 0x9b, 0xb6, 0x74, 0x49, 0x47, 0x1c, -+ 0x22, 0x10, 0xa9, 0xd8, 0x22, 0xb8, 0x21, 0xc1, 0x6e, 0x0e, 0x23, 0xb1, 0x4b, 0x70, 0xa3, 0x0a, -+ 0xf5, 0x32, 0xf2, 0x8e, 0x4d, 0x32, 0x4a, 0x66, 0x3c, 0xb5, 0x9d, 0x88, 0x9e, 0xf9, 0x23, 0xe0, -+ 0x4f, 0xe2, 0x88, 0x38, 0x70, 0x46, 0xcb, 0x3f, 0x82, 0xfc, 0x3c, 0x93, 0xcd, 0x66, 0x97, 0x1f, -+ 0x3d, 0x65, 0xde, 0xe7, 0xe7, 0xef, 0x7d, 0xef, 0x7b, 0xb6, 0x03, 0x9d, 0xac, 0x28, 0xc6, 0x85, -+ 0x92, 0x46, 0x12, 0x3f, 0x2b, 0x8a, 0xc7, 0x87, 0x73, 0x39, 0x97, 0x18, 0x3f, 0xb7, 0x5f, 0x6e, -+ 0xe9, 0xf1, 0xfd, 0xc4, 0x62, 0x89, 0xd0, 0x5a, 0xaa, 0x12, 0x0a, 0x32, 0x61, 0x58, 0x71, 0xe9, -+ 0xa2, 0x70, 0x02, 0x64, 0xf2, 0x63, 0xb2, 0x60, 0xf9, 0x5c, 0xbc, 0x14, 0x39, 0x17, 0xea, 0x5c, -+ 0x18, 0x46, 0x9e, 0x43, 0x3b, 0x91, 0x59, 0xa1, 0x84, 0xd6, 0x03, 0x6f, 0xe8, 0x8d, 0x7a, 0x27, -+ 0x0f, 0xc6, 0xb6, 0xde, 0x69, 0x09, 0x9e, 0x0b, 0xb3, 0x90, 0x9c, 0x6e, 0x93, 0xc2, 0x9f, 0x6b, -+ 0xd0, 0x9e, 0x31, 0xbd, 0xc4, 0xdd, 0x8f, 0xa0, 0xad, 0x0d, 0x53, 0x26, 0x36, 0x6e, 0x77, 0x9d, -+ 0xb6, 0x30, 0x9e, 0x69, 0x72, 0x04, 0x2d, 0xc3, 0xf4, 0x32, 0x4e, 0xf9, 0xa0, 0x36, 0xf4, 0x46, -+ 0x3e, 0x6d, 0xda, 0x30, 0xe2, 0xe4, 0x19, 0x04, 0x05, 0x53, 0x26, 0x35, 0xa9, 0xcc, 0xed, 0xaa, -+ 0x8f, 0xab, 0xdd, 0x2d, 0x16, 0x71, 0x32, 0x80, 0x16, 0xe3, 0x1c, 0x35, 0xd5, 0x87, 0xde, 0xa8, -+ 0x43, 0xab, 0x90, 0x3c, 0x81, 0xce, 0x9c, 0x99, 0x85, 0x50, 0x76, 0x67, 0x03, 0x2b, 0xb6, 0x1d, -+ 0x10, 0x71, 0xab, 0xe6, 0xcd, 0x5a, 0xa8, 0xb7, 0x56, 0x4d, 0xd3, 0xa9, 0xc1, 0x78, 0xa6, 0xc9, -+ 0x87, 0xd0, 0x5b, 0xc9, 0x84, 0xad, 0x62, 0x97, 0x90, 0xf2, 0x41, 0x0b, 0x13, 0x02, 0x44, 0xbf, -+ 0xb3, 0x60, 0xc4, 0x2d, 0xbb, 0x16, 0x6a, 0xe3, 0xd8, 0xdb, 0x8e, 0xdd, 0x01, 0x11, 0x27, 0x1f, -+ 0x40, 0x37, 0x2b, 0x8a, 0x78, 0x23, 0x94, 0x4e, 0x65, 0x3e, 0xe8, 0xa0, 0x6c, 0xc8, 0x8a, 0xe2, -+ 0x95, 0x43, 0xc2, 0x3e, 0xf4, 0x22, 0xfd, 0xd5, 0x2a, 0xdd, 0x08, 0x2a, 0xde, 0xac, 0x85, 0x36, -+ 0xe1, 0x14, 0xde, 0xdb, 0x22, 0xba, 0x90, 0xb9, 0x16, 0xe4, 0x7d, 0xe8, 0xb0, 0x0d, 0x4b, 0x57, -+ 0xec, 0x72, 0x25, 0xd0, 0xb2, 0x36, 0xbd, 0x06, 0xf6, 0x6b, 0xd4, 0x6e, 0xd5, 0xf8, 0xbd, 0x06, -+ 0x0f, 0xce, 0x52, 0x5d, 0x30, 0x93, 0x2c, 0xec, 0x14, 0xca, 0x4a, 0xe4, 0x19, 0xd4, 0xed, 0xb0, -+ 0x91, 0xb1, 0x7b, 0x72, 0x80, 0x23, 0xac, 0xa6, 0x44, 0x71, 0xc9, 0xfa, 0x2e, 0xf2, 0x44, 0x72, -+ 0xc1, 0xe3, 0x62, 0xc5, 0x1c, 0x79, 0x40, 0xbb, 0x25, 0x36, 0x5d, 0xb1, 0xdc, 0xfa, 0x6e, 0xd2, -+ 0x4c, 0xc8, 0xb5, 0x29, 0xa7, 0x52, 0x85, 0xe4, 0x53, 0x68, 0x29, 0x31, 0x4f, 0x65, 0x6e, 0x27, -+ 0xe2, 0x8f, 0xba, 0x27, 0x47, 0xe3, 0xdd, 0xf3, 0x46, 0x71, 0x2d, 0xca, 0x7f, 0x90, 0xb4, 0xca, -+ 0x23, 0x4f, 0x01, 0x74, 0xb2, 0x10, 0x19, 0xb3, 0xed, 0xe0, 0xac, 0x7c, 0xda, 0x71, 0xc8, 0x2b, -+ 0xa1, 0xc8, 0x97, 0x70, 0x60, 0x6c, 0xcf, 0x71, 0xc5, 0xdb, 0x44, 0xde, 0x47, 0x37, 0x78, 0x67, -+ 0x36, 0xc3, 0x91, 0x6b, 0x1a, 0x98, 0x9d, 0x88, 0x44, 0x70, 0x28, 0xca, 0xe3, 0x1c, 0x6b, 0x3c, -+ 0xcf, 0x31, 0x3a, 0xd0, 0x42, 0x07, 0x8e, 0xd0, 0x81, 0xdb, 0xe7, 0x9d, 0x12, 0x71, 0x0b, 0x0b, -+ 0x33, 0x38, 0xbc, 0xe9, 0x69, 0x39, 0xab, 0x21, 0x34, 0x84, 0x52, 0x52, 0x95, 0xae, 0x82, 0xe3, -+ 0xb4, 0x08, 0x75, 0x0b, 0xe4, 0x05, 0x1c, 0x28, 0x61, 0xd4, 0xdb, 0x6d, 0x13, 0x35, 0x6c, 0xa2, -+ 0x37, 0x2e, 0x6f, 0x9e, 0x13, 0x4b, 0x03, 0x4c, 0x2a, 0x95, 0x87, 0xdf, 0xc3, 0xfd, 0x53, 0x96, -+ 0x27, 0x62, 0xf5, 0x8e, 0x03, 0xdc, 0xca, 0xa9, 0xfd, 0x83, 0x9c, 0xf0, 0x73, 0x20, 0xbb, 0xcc, -+ 0xff, 0xb7, 0x8d, 0xf0, 0x27, 0x0f, 0x9e, 0x4e, 0xb4, 0xb5, 0x37, 0xd5, 0x8b, 0xf3, 0xe9, 0xf4, -+ 0x54, 0xe6, 0xb9, 0x48, 0xec, 0x65, 0xac, 0xe4, 0x8d, 0xa1, 0xbb, 0x6b, 0xf2, 0x9d, 0x2a, 0x41, -+ 0x5f, 0x3f, 0x2b, 0x27, 0xd6, 0x98, 0x44, 0xa4, 0x9b, 0x6a, 0x47, 0xed, 0xae, 0x1d, 0x41, 0x95, -+ 0x83, 0x63, 0x98, 0xc0, 0xc3, 0x33, 0x66, 0xd8, 0x94, 0x25, 0x4b, 0x61, 0x76, 0xde, 0x1f, 0x46, -+ 0x3e, 0x86, 0x66, 0x86, 0xef, 0xd0, 0xbf, 0x3d, 0x51, 0x65, 0x4a, 0xf8, 0x87, 0x07, 0x07, 0xe7, -+ 0xd3, 0xe9, 0x35, 0x15, 0x21, 0x50, 0xe7, 0xac, 0x54, 0x1d, 0x50, 0xfc, 0xfe, 0x6f, 0x33, 0xc9, -+ 0x43, 0x68, 0x26, 0x8b, 0x75, 0xbe, 0xd4, 0x03, 0x7f, 0xe8, 0x8f, 0x02, 0x5a, 0x46, 0x78, 0xae, -+ 0x8d, 0x12, 0x2c, 0x8b, 0x53, 0xee, 0x6e, 0x43, 0x9d, 0x76, 0x1c, 0x12, 0x71, 0xbd, 0x7f, 0x85, -+ 0x1b, 0xfb, 0x57, 0x98, 0x7c, 0xb1, 0xf3, 0xe2, 0x36, 0xb1, 0xf8, 0x13, 0x2c, 0x7e, 0x77, 0xef, -+ 0x3b, 0x2f, 0xef, 0x05, 0x34, 0x50, 0xa0, 0xed, 0xc7, 0xde, 0x59, 0xec, 0xa7, 0x41, 0xf1, 0x9b, -+ 0xf4, 0xc1, 0xcf, 0xf4, 0x1c, 0xbb, 0xe9, 0x50, 0xfb, 0xb9, 0x2f, 0xc4, 0xdf, 0x17, 0xf2, 0xd1, -+ 0x27, 0xd0, 0xbb, 0x69, 0x21, 0x69, 0x43, 0xfd, 0xe2, 0xdb, 0x8b, 0x49, 0xff, 0x1e, 0x69, 0x81, -+ 0xff, 0xcd, 0xeb, 0xcf, 0xfa, 0x9e, 0x85, 0x5e, 0xbf, 0x9c, 0x9d, 0xf5, 0x6b, 0x5f, 0x87, 0xbf, -+ 0x5e, 0x1d, 0x7b, 0xbf, 0x5d, 0x1d, 0x7b, 0x7f, 0x5e, 0x1d, 0x7b, 0xbf, 0xfc, 0x75, 0x7c, 0x0f, -+ 0xfa, 0x52, 0xcd, 0xc7, 0x26, 0x5d, 0x6e, 0xc6, 0xcb, 0x0d, 0xfe, 0xc7, 0x5c, 0x36, 0xf1, 0xe7, -+ 0xc5, 0xdf, 0x01, 0x00, 0x00, 0xff, 0xff, 0x68, 0x3e, 0x5c, 0x7f, 0xb3, 0x06, 0x00, 0x00, ++ // 847 bytes of a gzipped FileDescriptorProto ++ 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x55, 0xcd, 0x8e, 0xdb, 0x36, ++ 0x10, 0x8e, 0x2c, 0xaf, 0x7f, 0xc6, 0x5a, 0xd7, 0x61, 0x16, 0x59, 0x27, 0x69, 0xb6, 0x8e, 0xd0, ++ 0x83, 0xd1, 0x02, 0x0e, 0xba, 0x01, 0x5a, 0xa0, 0x87, 0x02, 0xe9, 0xc6, 0x6d, 0x74, 0xf0, 0xae, ++ 0xcb, 0x35, 0x82, 0xde, 0x04, 0xae, 0xc8, 0xda, 0x82, 0x2d, 0x51, 0x21, 0x69, 0xa3, 0x39, 0xf7, ++ 0x21, 0xda, 0x47, 0xea, 0xad, 0x45, 0x9f, 0xa0, 0xd8, 0xbe, 0x48, 0xc1, 0xa1, 0xe4, 0xf5, 0xfe, ++ 0xf4, 0x27, 0x27, 0x71, 0x3e, 0x0e, 0x67, 0xbe, 0xf9, 0x86, 0x1c, 0x41, 0x3b, 0x2b, 0x8a, 0x51, ++ 0xa1, 0xa4, 0x91, 0xc4, 0xcf, 0x8a, 0xe2, 0xf1, 0xc1, 0x5c, 0xce, 0x25, 0xda, 0xcf, 0xed, 0xca, ++ 0x6d, 0x3d, 0xbe, 0x9f, 0x58, 0x2c, 0x11, 0x5a, 0x4b, 0x55, 0x42, 0x41, 0x26, 0x0c, 0x2b, 0x2e, ++ 0x9c, 0x15, 0x8e, 0x81, 0x8c, 0x7f, 0x4c, 0x16, 0x2c, 0x9f, 0x8b, 0x73, 0x91, 0x73, 0xa1, 0x26, ++ 0xc2, 0x30, 0xf2, 0x1c, 0x5a, 0x89, 0xcc, 0x0a, 0x25, 0xb4, 0xee, 0x7b, 0x03, 0x6f, 0xd8, 0x3d, ++ 0x7e, 0x30, 0xb2, 0xf9, 0x4e, 0x4a, 0x70, 0x22, 0xcc, 0x42, 0x72, 0xba, 0x75, 0x0a, 0x7f, 0xae, ++ 0x41, 0x6b, 0xc6, 0xf4, 0x12, 0x4f, 0x3f, 0x82, 0x96, 0x36, 0x4c, 0x99, 0xd8, 0xb8, 0xd3, 0x75, ++ 0xda, 0x44, 0x7b, 0xa6, 0xc9, 0x21, 0x34, 0x0d, 0xd3, 0xcb, 0x38, 0xe5, 0xfd, 0xda, 0xc0, 0x1b, ++ 0xfa, 0xb4, 0x61, 0xcd, 0x88, 0x93, 0x67, 0x10, 0x14, 0x4c, 0x99, 0xd4, 0xa4, 0x32, 0xb7, 0xbb, ++ 0x3e, 0xee, 0x76, 0xb6, 0x58, 0xc4, 0x49, 0x1f, 0x9a, 0x8c, 0x73, 0xe4, 0x54, 0x1f, 0x78, 0xc3, ++ 0x36, 0xad, 0x4c, 0xf2, 0x04, 0xda, 0x73, 0x66, 0x16, 0x42, 0xd9, 0x93, 0x7b, 0x98, 0xb1, 0xe5, ++ 0x80, 0x88, 0x5b, 0x36, 0x6f, 0xd7, 0x42, 0xbd, 0xb3, 0x6c, 0x1a, 0x8e, 0x0d, 0xda, 0x33, 0x4d, ++ 0x3e, 0x86, 0xee, 0x4a, 0x26, 0x6c, 0x15, 0x3b, 0x87, 0x94, 0xf7, 0x9b, 0xe8, 0x10, 0x20, 0xfa, ++ 0x9d, 0x05, 0x23, 0x6e, 0xa3, 0x6b, 0xa1, 0x36, 0x2e, 0x7a, 0xcb, 0x45, 0x77, 0x40, 0xc4, 0xc9, ++ 0x47, 0xd0, 0xc9, 0x8a, 0x22, 0xde, 0x08, 0xa5, 0x53, 0x99, 0xf7, 0xdb, 0x48, 0x1b, 0xb2, 0xa2, ++ 0x78, 0xe3, 0x90, 0xb0, 0x07, 0xdd, 0x48, 0xbf, 0x5c, 0xa5, 0x1b, 0x41, 0xc5, 0xdb, 0xb5, 0xd0, ++ 0x26, 0x9c, 0xc2, 0x07, 0x5b, 0x44, 0x17, 0x32, 0xd7, 0x82, 0x7c, 0x08, 0x6d, 0xb6, 0x61, 0xe9, ++ 0x8a, 0x5d, 0xac, 0x04, 0x4a, 0xd6, 0xa2, 0x57, 0xc0, 0xcd, 0x1c, 0xb5, 0x5b, 0x39, 0xfe, 0xa8, ++ 0xc1, 0x83, 0x57, 0xa9, 0x2e, 0x98, 0x49, 0x16, 0xb6, 0x0b, 0x65, 0x26, 0xf2, 0x0c, 0xea, 0xb6, ++ 0xd9, 0x18, 0xb1, 0x73, 0xbc, 0x8f, 0x2d, 0xac, 0xba, 0x44, 0x71, 0xcb, 0xea, 0x2e, 0xf2, 0x44, ++ 0x72, 0xc1, 0xe3, 0x62, 0xc5, 0x5c, 0xf0, 0x80, 0x76, 0x4a, 0x6c, 0xba, 0x62, 0xb9, 0xd5, 0xdd, ++ 0xa4, 0x99, 0x90, 0x6b, 0x53, 0x76, 0xa5, 0x32, 0xc9, 0x67, 0xd0, 0x54, 0x62, 0x9e, 0xca, 0xdc, ++ 0x76, 0xc4, 0x1f, 0x76, 0x8e, 0x0f, 0x47, 0xbb, 0xf7, 0x8d, 0xe2, 0x5e, 0x94, 0xff, 0x20, 0x69, ++ 0xe5, 0x47, 0x9e, 0x02, 0xe8, 0x64, 0x21, 0x32, 0x66, 0xcb, 0xc1, 0x5e, 0xf9, 0xb4, 0xed, 0x90, ++ 0x37, 0x42, 0x91, 0xaf, 0x60, 0xdf, 0xd8, 0x9a, 0xe3, 0x2a, 0x6e, 0x03, 0xe3, 0x3e, 0xba, 0x16, ++ 0x77, 0x66, 0x3d, 0x5c, 0x70, 0x4d, 0x03, 0xb3, 0x63, 0x91, 0x08, 0x0e, 0x44, 0x79, 0x9d, 0x63, ++ 0x8d, 0xf7, 0x39, 0x46, 0x05, 0x9a, 0xa8, 0xc0, 0x21, 0x2a, 0x70, 0xfb, 0xbe, 0x53, 0x22, 0x6e, ++ 0x61, 0x61, 0x06, 0x07, 0xd7, 0x35, 0x2d, 0x7b, 0x35, 0x80, 0x3d, 0xa1, 0x94, 0x54, 0xa5, 0xaa, ++ 0xe0, 0x62, 0x5a, 0x84, 0xba, 0x0d, 0xf2, 0x02, 0xf6, 0x95, 0x30, 0xea, 0xdd, 0xb6, 0x88, 0x1a, ++ 0x16, 0xd1, 0x1d, 0x95, 0x2f, 0xcf, 0x91, 0xa5, 0x01, 0x3a, 0x95, 0xcc, 0xc3, 0xef, 0xe1, 0xfe, ++ 0x09, 0xcb, 0x13, 0xb1, 0x7a, 0xcf, 0x06, 0x6e, 0xe9, 0xd4, 0xfe, 0x81, 0x4e, 0xf8, 0x39, 0x90, ++ 0xdd, 0xc8, 0xff, 0xb7, 0x8c, 0xf0, 0x27, 0x0f, 0x9e, 0x8e, 0xb5, 0x95, 0x37, 0xd5, 0x8b, 0xc9, ++ 0x74, 0x7a, 0x22, 0xf3, 0x5c, 0x24, 0xf6, 0x31, 0x56, 0xf4, 0x46, 0xd0, 0xd9, 0x15, 0xf9, 0x4e, ++ 0x96, 0xa0, 0xaf, 0xc6, 0xca, 0xb1, 0x15, 0x26, 0x11, 0xe9, 0xa6, 0x3a, 0x51, 0xbb, 0xeb, 0x44, ++ 0x50, 0xf9, 0x60, 0x1b, 0xc6, 0xf0, 0xf0, 0x15, 0x33, 0x6c, 0xca, 0x92, 0xa5, 0x30, 0x3b, 0xf3, ++ 0x87, 0x91, 0x4f, 0xa1, 0x91, 0xe1, 0x1c, 0xfa, 0xb7, 0x11, 0x55, 0xba, 0x84, 0xbf, 0x79, 0xb0, ++ 0x3f, 0x99, 0x4e, 0xaf, 0x42, 0x11, 0x02, 0x75, 0xce, 0x4a, 0xd6, 0x01, 0xc5, 0xf5, 0x7f, 0x8b, ++ 0x49, 0x1e, 0x42, 0x23, 0x59, 0xac, 0xf3, 0xa5, 0xee, 0xfb, 0x03, 0x7f, 0x18, 0xd0, 0xd2, 0xc2, ++ 0x7b, 0x6d, 0x94, 0x60, 0x59, 0x9c, 0x72, 0xf7, 0x1a, 0xea, 0xb4, 0xed, 0x90, 0x88, 0x6b, 0xfb, ++ 0x86, 0xaa, 0xe7, 0xeb, 0xee, 0x7c, 0x65, 0x92, 0x2f, 0x76, 0x46, 0x6d, 0x03, 0xb3, 0x3e, 0xc1, ++ 0xac, 0x77, 0x17, 0xbd, 0x33, 0x72, 0x4f, 0x61, 0x0f, 0x99, 0xd9, 0x42, 0xec, 0x63, 0xc5, 0x42, ++ 0xf6, 0x28, 0xae, 0x49, 0x0f, 0xfc, 0x4c, 0xcf, 0xb1, 0x8c, 0x36, 0xb5, 0xcb, 0x9b, 0x43, 0xc4, ++ 0xbf, 0x39, 0x44, 0x3e, 0xf9, 0x12, 0xba, 0xd7, 0xb5, 0x23, 0x2d, 0xa8, 0x9f, 0x9e, 0x9d, 0x8e, ++ 0x7b, 0xf7, 0xec, 0xea, 0x9b, 0x97, 0xe7, 0xb3, 0x9e, 0x47, 0x0e, 0xa0, 0xf7, 0x3a, 0xfa, 0xf6, ++ 0x75, 0x7c, 0x72, 0x36, 0x99, 0xd2, 0xf1, 0xf9, 0x79, 0x74, 0x76, 0xda, 0xab, 0x7d, 0x1d, 0xfe, ++ 0x7a, 0x79, 0xe4, 0xfd, 0x7e, 0x79, 0xe4, 0xfd, 0x79, 0x79, 0xe4, 0xfd, 0xf2, 0xd7, 0xd1, 0x3d, ++ 0xe8, 0x49, 0x35, 0x1f, 0x99, 0x74, 0xb9, 0x19, 0x2d, 0x37, 0xf8, 0xa7, 0xb9, 0x68, 0xe0, 0xe7, ++ 0xc5, 0xdf, 0x01, 0x00, 0x00, 0xff, 0xff, 0x0e, 0xad, 0x33, 0x36, 0xb9, 0x06, 0x00, 0x00, +} + +func (m *ExchangeSenderMeta) Marshal() (dAtA []byte, err error) { @@ -506,7 +507,7 @@ index fcd4691..783c2eb 100644 } func (m *TaskMeta) Marshal() (dAtA []byte, err error) { -@@ -734,6 +951,11 @@ func (m *TaskMeta) MarshalToSizedBuffer(dAtA []byte) (int, error) { +@@ -734,6 +952,11 @@ func (m *TaskMeta) MarshalToSizedBuffer(dAtA []byte) (int, error) { i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } @@ -518,7 +519,7 @@ index fcd4691..783c2eb 100644 if m.ServerId != 0 { i = encodeVarintMpp(dAtA, i, uint64(m.ServerId)) i-- -@@ -830,6 +1052,11 @@ func (m *IsAliveResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { +@@ -830,6 +1053,11 @@ func (m *IsAliveResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } @@ -530,7 +531,7 @@ index fcd4691..783c2eb 100644 if m.Available { i-- if m.Available { -@@ -867,6 +1094,18 @@ func (m *DispatchTaskRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { +@@ -867,6 +1095,18 @@ func (m *DispatchTaskRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } @@ -549,7 +550,7 @@ index fcd4691..783c2eb 100644 if len(m.TableRegions) > 0 { for iNdEx := len(m.TableRegions) - 1; iNdEx >= 0; iNdEx-- { { -@@ -1121,6 +1360,38 @@ func (m *EstablishMPPConnectionRequest) MarshalToSizedBuffer(dAtA []byte) (int, +@@ -1121,6 +1361,38 @@ func (m *EstablishMPPConnectionRequest) MarshalToSizedBuffer(dAtA []byte) (int, return len(dAtA) - i, nil } @@ -588,7 +589,7 @@ index fcd4691..783c2eb 100644 func (m *MPPDataPacket) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) -@@ -1145,21 +1416,38 @@ func (m *MPPDataPacket) MarshalToSizedBuffer(dAtA []byte) (int, error) { +@@ -1145,21 +1417,38 @@ func (m *MPPDataPacket) MarshalToSizedBuffer(dAtA []byte) (int, error) { i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } @@ -604,8 +605,8 @@ index fcd4691..783c2eb 100644 + i-- + dAtA[i] = 0x32 + } -+ if m.MppVersion != 0 { -+ i = encodeVarintMpp(dAtA, i, uint64(m.MppVersion)) ++ if m.Version != 0 { ++ i = encodeVarintMpp(dAtA, i, uint64(m.Version)) + i-- + dAtA[i] = 0x28 + } @@ -636,7 +637,7 @@ index fcd4691..783c2eb 100644 i-- dAtA[i] = 0x22 } -@@ -1218,6 +1506,11 @@ func (m *Error) MarshalToSizedBuffer(dAtA []byte) (int, error) { +@@ -1218,6 +1507,11 @@ func (m *Error) MarshalToSizedBuffer(dAtA []byte) (int, error) { i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } @@ -648,7 +649,7 @@ index fcd4691..783c2eb 100644 if len(m.Msg) > 0 { i -= len(m.Msg) copy(dAtA[i:], m.Msg) -@@ -1244,6 +1537,21 @@ func encodeVarintMpp(dAtA []byte, offset int, v uint64) int { +@@ -1244,6 +1538,21 @@ func encodeVarintMpp(dAtA []byte, offset int, v uint64) int { dAtA[offset] = uint8(v) return base } @@ -670,7 +671,7 @@ index fcd4691..783c2eb 100644 func (m *TaskMeta) Size() (n int) { if m == nil { return 0 -@@ -1275,6 +1583,9 @@ func (m *TaskMeta) Size() (n int) { +@@ -1275,6 +1584,9 @@ func (m *TaskMeta) Size() (n int) { if m.ServerId != 0 { n += 1 + sovMpp(uint64(m.ServerId)) } @@ -680,7 +681,7 @@ index fcd4691..783c2eb 100644 if m.XXX_unrecognized != nil { n += len(m.XXX_unrecognized) } -@@ -1302,6 +1613,9 @@ func (m *IsAliveResponse) Size() (n int) { +@@ -1302,6 +1614,9 @@ func (m *IsAliveResponse) Size() (n int) { if m.Available { n += 2 } @@ -690,7 +691,7 @@ index fcd4691..783c2eb 100644 if m.XXX_unrecognized != nil { n += len(m.XXX_unrecognized) } -@@ -1340,6 +1654,10 @@ func (m *DispatchTaskRequest) Size() (n int) { +@@ -1340,6 +1655,10 @@ func (m *DispatchTaskRequest) Size() (n int) { n += 1 + l + sovMpp(uint64(l)) } } @@ -701,7 +702,7 @@ index fcd4691..783c2eb 100644 if m.XXX_unrecognized != nil { n += len(m.XXX_unrecognized) } -@@ -1424,6 +1742,21 @@ func (m *EstablishMPPConnectionRequest) Size() (n int) { +@@ -1424,6 +1743,21 @@ func (m *EstablishMPPConnectionRequest) Size() (n int) { return n } @@ -723,12 +724,12 @@ index fcd4691..783c2eb 100644 func (m *MPPDataPacket) Size() (n int) { if m == nil { return 0 -@@ -1451,6 +1784,13 @@ func (m *MPPDataPacket) Size() (n int) { +@@ -1451,6 +1785,13 @@ func (m *MPPDataPacket) Size() (n int) { } n += 1 + sovMpp(uint64(l)) + l } -+ if m.MppVersion != 0 { -+ n += 1 + sovMpp(uint64(m.MppVersion)) ++ if m.Version != 0 { ++ n += 1 + sovMpp(uint64(m.Version)) + } + if m.Compress != nil { + l = m.Compress.Size() @@ -737,7 +738,7 @@ index fcd4691..783c2eb 100644 if m.XXX_unrecognized != nil { n += len(m.XXX_unrecognized) } -@@ -1470,6 +1810,9 @@ func (m *Error) Size() (n int) { +@@ -1470,6 +1811,9 @@ func (m *Error) Size() (n int) { if l > 0 { n += 1 + l + sovMpp(uint64(l)) } @@ -747,7 +748,7 @@ index fcd4691..783c2eb 100644 if m.XXX_unrecognized != nil { n += len(m.XXX_unrecognized) } -@@ -1482,6 +1825,76 @@ func sovMpp(x uint64) (n int) { +@@ -1482,6 +1826,76 @@ func sovMpp(x uint64) (n int) { func sozMpp(x uint64) (n int) { return sovMpp(uint64((x << 1) ^ uint64((int64(x) >> 63)))) } @@ -824,7 +825,7 @@ index fcd4691..783c2eb 100644 func (m *TaskMeta) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 -@@ -1676,6 +2089,25 @@ func (m *TaskMeta) Unmarshal(dAtA []byte) error { +@@ -1676,6 +2090,25 @@ func (m *TaskMeta) Unmarshal(dAtA []byte) error { break } } @@ -850,7 +851,7 @@ index fcd4691..783c2eb 100644 default: iNdEx = preIndex skippy, err := skipMpp(dAtA[iNdEx:]) -@@ -1798,6 +2230,25 @@ func (m *IsAliveResponse) Unmarshal(dAtA []byte) error { +@@ -1798,6 +2231,25 @@ func (m *IsAliveResponse) Unmarshal(dAtA []byte) error { } } m.Available = bool(v != 0) @@ -876,7 +877,7 @@ index fcd4691..783c2eb 100644 default: iNdEx = preIndex skippy, err := skipMpp(dAtA[iNdEx:]) -@@ -2025,6 +2476,42 @@ func (m *DispatchTaskRequest) Unmarshal(dAtA []byte) error { +@@ -2025,6 +2477,42 @@ func (m *DispatchTaskRequest) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex @@ -919,7 +920,7 @@ index fcd4691..783c2eb 100644 default: iNdEx = preIndex skippy, err := skipMpp(dAtA[iNdEx:]) -@@ -2501,6 +2988,76 @@ func (m *EstablishMPPConnectionRequest) Unmarshal(dAtA []byte) error { +@@ -2501,6 +2989,76 @@ func (m *EstablishMPPConnectionRequest) Unmarshal(dAtA []byte) error { } return nil } @@ -996,15 +997,15 @@ index fcd4691..783c2eb 100644 func (m *MPPDataPacket) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 -@@ -2708,6 +3265,61 @@ func (m *MPPDataPacket) Unmarshal(dAtA []byte) error { +@@ -2708,6 +3266,61 @@ func (m *MPPDataPacket) Unmarshal(dAtA []byte) error { } else { return fmt.Errorf("proto: wrong wireType = %d for field StreamIds", wireType) } + case 5: + if wireType != 0 { -+ return fmt.Errorf("proto: wrong wireType = %d for field MppVersion", wireType) ++ return fmt.Errorf("proto: wrong wireType = %d for field Version", wireType) + } -+ m.MppVersion = 0 ++ m.Version = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMpp @@ -1014,7 +1015,7 @@ index fcd4691..783c2eb 100644 + } + b := dAtA[iNdEx] + iNdEx++ -+ m.MppVersion |= int64(b&0x7F) << shift ++ m.Version |= int64(b&0x7F) << shift + if b < 0x80 { + break + } @@ -1058,7 +1059,7 @@ index fcd4691..783c2eb 100644 default: iNdEx = preIndex skippy, err := skipMpp(dAtA[iNdEx:]) -@@ -2810,6 +3422,25 @@ func (m *Error) Unmarshal(dAtA []byte) error { +@@ -2810,6 +3423,25 @@ func (m *Error) Unmarshal(dAtA []byte) error { } m.Msg = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex @@ -1085,7 +1086,7 @@ index fcd4691..783c2eb 100644 iNdEx = preIndex skippy, err := skipMpp(dAtA[iNdEx:]) diff --git a/proto/mpp.proto b/proto/mpp.proto -index 6739455..0fdf25d 100644 +index 6739455..e3ec05f 100644 --- a/proto/mpp.proto +++ b/proto/mpp.proto @@ -11,6 +11,17 @@ option (gogoproto.unmarshaler_all) = true; @@ -1094,9 +1095,9 @@ index 6739455..0fdf25d 100644 +// Data compression method +enum CompressMethod { -+ NONE = 0; -+ LZ4 = 1; -+ ZSTD = 2; ++ NONE = 0; // no compression ++ FAST = 1; // fast compression/decompression speed, compression ratio is lower than HC mode ++ HIGH_COMPRESSION = 2; // high compression (HC) ratio mode +} + +message ExchangeSenderMeta { diff --git a/dbms/src/Flash/Coprocessor/CHBlockChunkCodec.cpp b/dbms/src/Flash/Coprocessor/CHBlockChunkCodec.cpp index acb21a8d314..55a7732aa4d 100644 --- a/dbms/src/Flash/Coprocessor/CHBlockChunkCodec.cpp +++ b/dbms/src/Flash/Coprocessor/CHBlockChunkCodec.cpp @@ -104,9 +104,9 @@ CompressionMethod ToInternalCompressionMethod(mpp::CompressMethod compress_metho { case mpp::NONE: return CompressionMethod::NONE; - case mpp::LZ4: + case mpp::CompressMethod::FAST: return CompressionMethod::LZ4; - case mpp::ZSTD: + case mpp::CompressMethod::HIGH_COMPRESSION: return CompressionMethod::ZSTD; default: throw Exception(ErrorCodes::LOGICAL_ERROR, "Unkown compress method {}", mpp::CompressMethod_Name(compress_method)); diff --git a/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp b/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp index a8037eadec8..7ea111d43df 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp +++ b/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp @@ -380,12 +380,12 @@ static void updateHashPartitionWriterMetrics(mpp::CompressMethod method, size_t } break; } - case mpp::LZ4: + case mpp::CompressMethod::FAST: { GET_METRIC(tiflash_exchange_data_bytes, type_hash_lz4).Increment(sz); break; } - case mpp::ZSTD: + case mpp::CompressMethod::HIGH_COMPRESSION: { GET_METRIC(tiflash_exchange_data_bytes, type_hash_zstd).Increment(sz); break; diff --git a/dbms/src/Flash/Mpp/MPPTask.cpp b/dbms/src/Flash/Mpp/MPPTask.cpp index c6904b7e517..1f5be8d548e 100644 --- a/dbms/src/Flash/Mpp/MPPTask.cpp +++ b/dbms/src/Flash/Mpp/MPPTask.cpp @@ -283,10 +283,10 @@ void MPPTask::prepare(const mpp::DispatchTaskRequest & task_request) // switch (tzg::SnappyStatistic::globalInstance().getMethod()) // { // case tzg::SnappyStatistic::CompressMethod::LZ4: - // compress_method = mpp::CompressMethod::LZ4; + // compress_method = mpp::CompressMethod::FAST; // break; // case tzg::SnappyStatistic::CompressMethod::ZSTD: - // compress_method = mpp::CompressMethod::ZSTD; + // compress_method = mpp::CompressMethod::HIGH_COMPRESSION; // break; // default: // compress_method = mpp::CompressMethod::NONE; diff --git a/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp b/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp index 2dd4e69a725..a55cf2d183a 100644 --- a/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp +++ b/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp @@ -503,7 +503,7 @@ try batch_send_min_limit, /*should_send_exec_summary_at_last=*/false, *dag_context_ptr, - mpp::CompressMethod::LZ4); + mpp::CompressMethod::FAST); for (const auto & block : blocks) dag_writer->write(block); dag_writer->flush(); From 1efc507c855bf69727c583de5b82ff046104232c Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Thu, 5 Jan 2023 17:32:25 +0800 Subject: [PATCH 40/93] 34 Signed-off-by: Zhigao Tong --- contrib/tzg.mpp.proto.patch | 254 +++++++++--------- .../Flash/Coprocessor/CHBlockChunkCodec.cpp | 14 +- .../Coprocessor/CHBlockChunkCodecStream.h | 2 +- dbms/src/Flash/Mpp/ExchangeReceiver.cpp | 2 +- dbms/src/Flash/Mpp/HashPartitionWriter.cpp | 38 +-- dbms/src/Flash/Mpp/HashPartitionWriter.h | 2 +- dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp | 28 +- dbms/src/Flash/Mpp/HashPartitionWriterV1.h | 4 +- dbms/src/Flash/Mpp/MPPTask.cpp | 12 +- .../Mpp/tests/gtest_mpp_exchange_writer.cpp | 8 +- .../Planner/plans/PhysicalExchangeSender.cpp | 6 +- 11 files changed, 177 insertions(+), 193 deletions(-) diff --git a/contrib/tzg.mpp.proto.patch b/contrib/tzg.mpp.proto.patch index b90a69132ff..74c4b08831d 100644 --- a/contrib/tzg.mpp.proto.patch +++ b/contrib/tzg.mpp.proto.patch @@ -1,45 +1,45 @@ diff --git a/pkg/mpp/mpp.pb.go b/pkg/mpp/mpp.pb.go -index fcd4691..be5dd41 100644 +index fcd4691..35cdbf2 100644 --- a/pkg/mpp/mpp.pb.go +++ b/pkg/mpp/mpp.pb.go @@ -26,6 +26,82 @@ var _ = math.Inf // proto package needs to be updated. const _ = proto.ProtoPackageIsVersion3 // please upgrade the proto package -+// Data compression method -+type CompressMethod int32 ++// Data compression mode ++type CompressionMode int32 + +const ( -+ CompressMethod_NONE CompressMethod = 0 -+ CompressMethod_FAST CompressMethod = 1 -+ CompressMethod_HIGH_COMPRESSION CompressMethod = 2 ++ CompressionMode_NONE CompressionMode = 0 ++ CompressionMode_FAST CompressionMode = 1 ++ CompressionMode_HIGH_COMPRESSION CompressionMode = 2 +) + -+var CompressMethod_name = map[int32]string{ ++var CompressionMode_name = map[int32]string{ + 0: "NONE", + 1: "FAST", + 2: "HIGH_COMPRESSION", +} + -+var CompressMethod_value = map[string]int32{ ++var CompressionMode_value = map[string]int32{ + "NONE": 0, + "FAST": 1, + "HIGH_COMPRESSION": 2, +} + -+func (x CompressMethod) String() string { -+ return proto.EnumName(CompressMethod_name, int32(x)) ++func (x CompressionMode) String() string { ++ return proto.EnumName(CompressionMode_name, int32(x)) +} + -+func (CompressMethod) EnumDescriptor() ([]byte, []int) { ++func (CompressionMode) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_819623c7fa76fc55, []int{0} +} + +type ExchangeSenderMeta struct { -+ Compress CompressMethod `protobuf:"varint,1,opt,name=compress,proto3,enum=mpp.CompressMethod" json:"compress,omitempty"` -+ XXX_NoUnkeyedLiteral struct{} `json:"-"` -+ XXX_unrecognized []byte `json:"-"` -+ XXX_sizecache int32 `json:"-"` ++ Compress CompressionMode `protobuf:"varint,1,opt,name=compress,proto3,enum=mpp.CompressionMode" json:"compress,omitempty"` ++ XXX_NoUnkeyedLiteral struct{} `json:"-"` ++ XXX_unrecognized []byte `json:"-"` ++ XXX_sizecache int32 `json:"-"` +} + +func (m *ExchangeSenderMeta) Reset() { *m = ExchangeSenderMeta{} } @@ -75,11 +75,11 @@ index fcd4691..be5dd41 100644 + +var xxx_messageInfo_ExchangeSenderMeta proto.InternalMessageInfo + -+func (m *ExchangeSenderMeta) GetCompress() CompressMethod { ++func (m *ExchangeSenderMeta) GetCompress() CompressionMode { + if m != nil { + return m.Compress + } -+ return CompressMethod_NONE ++ return CompressionMode_NONE +} + // TaskMeta contains meta of a mpp plan, including query's ts and task address. @@ -235,25 +235,25 @@ index fcd4691..be5dd41 100644 return nil } -+type DataPacketCompressMeta struct { -+ Method CompressMethod `protobuf:"varint,1,opt,name=method,proto3,enum=mpp.CompressMethod" json:"method,omitempty"` -+ XXX_NoUnkeyedLiteral struct{} `json:"-"` -+ XXX_unrecognized []byte `json:"-"` -+ XXX_sizecache int32 `json:"-"` ++type DataPacketCompressionMeta struct { ++ Mode CompressionMode `protobuf:"varint,1,opt,name=mode,proto3,enum=mpp.CompressionMode" json:"mode,omitempty"` ++ XXX_NoUnkeyedLiteral struct{} `json:"-"` ++ XXX_unrecognized []byte `json:"-"` ++ XXX_sizecache int32 `json:"-"` +} + -+func (m *DataPacketCompressMeta) Reset() { *m = DataPacketCompressMeta{} } -+func (m *DataPacketCompressMeta) String() string { return proto.CompactTextString(m) } -+func (*DataPacketCompressMeta) ProtoMessage() {} -+func (*DataPacketCompressMeta) Descriptor() ([]byte, []int) { ++func (m *DataPacketCompressionMeta) Reset() { *m = DataPacketCompressionMeta{} } ++func (m *DataPacketCompressionMeta) String() string { return proto.CompactTextString(m) } ++func (*DataPacketCompressionMeta) ProtoMessage() {} ++func (*DataPacketCompressionMeta) Descriptor() ([]byte, []int) { + return fileDescriptor_819623c7fa76fc55, []int{9} +} -+func (m *DataPacketCompressMeta) XXX_Unmarshal(b []byte) error { ++func (m *DataPacketCompressionMeta) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} -+func (m *DataPacketCompressMeta) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { ++func (m *DataPacketCompressionMeta) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { -+ return xxx_messageInfo_DataPacketCompressMeta.Marshal(b, m, deterministic) ++ return xxx_messageInfo_DataPacketCompressionMeta.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) @@ -263,23 +263,23 @@ index fcd4691..be5dd41 100644 + return b[:n], nil + } +} -+func (m *DataPacketCompressMeta) XXX_Merge(src proto.Message) { -+ xxx_messageInfo_DataPacketCompressMeta.Merge(m, src) ++func (m *DataPacketCompressionMeta) XXX_Merge(src proto.Message) { ++ xxx_messageInfo_DataPacketCompressionMeta.Merge(m, src) +} -+func (m *DataPacketCompressMeta) XXX_Size() int { ++func (m *DataPacketCompressionMeta) XXX_Size() int { + return m.Size() +} -+func (m *DataPacketCompressMeta) XXX_DiscardUnknown() { -+ xxx_messageInfo_DataPacketCompressMeta.DiscardUnknown(m) ++func (m *DataPacketCompressionMeta) XXX_DiscardUnknown() { ++ xxx_messageInfo_DataPacketCompressionMeta.DiscardUnknown(m) +} + -+var xxx_messageInfo_DataPacketCompressMeta proto.InternalMessageInfo ++var xxx_messageInfo_DataPacketCompressionMeta proto.InternalMessageInfo + -+func (m *DataPacketCompressMeta) GetMethod() CompressMethod { ++func (m *DataPacketCompressionMeta) GetMode() CompressionMode { + if m != nil { -+ return m.Method ++ return m.Mode + } -+ return CompressMethod_NONE ++ return CompressionMode_NONE +} + // when TiFlash sends data to TiDB, Data packets wrap tipb.SelectResponse, i.e., serialize tipb.SelectResponse into data; @@ -292,15 +292,15 @@ index fcd4691..be5dd41 100644 - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -+ Data []byte `protobuf:"bytes,1,opt,name=data,proto3" json:"data,omitempty"` -+ Error *Error `protobuf:"bytes,2,opt,name=error,proto3" json:"error,omitempty"` -+ Chunks [][]byte `protobuf:"bytes,3,rep,name=chunks,proto3" json:"chunks,omitempty"` -+ StreamIds []uint64 `protobuf:"varint,4,rep,packed,name=stream_ids,json=streamIds,proto3" json:"stream_ids,omitempty"` -+ Version int64 `protobuf:"varint,5,opt,name=version,proto3" json:"version,omitempty"` -+ Compress *DataPacketCompressMeta `protobuf:"bytes,6,opt,name=compress,proto3" json:"compress,omitempty"` -+ XXX_NoUnkeyedLiteral struct{} `json:"-"` -+ XXX_unrecognized []byte `json:"-"` -+ XXX_sizecache int32 `json:"-"` ++ Data []byte `protobuf:"bytes,1,opt,name=data,proto3" json:"data,omitempty"` ++ Error *Error `protobuf:"bytes,2,opt,name=error,proto3" json:"error,omitempty"` ++ Chunks [][]byte `protobuf:"bytes,3,rep,name=chunks,proto3" json:"chunks,omitempty"` ++ StreamIds []uint64 `protobuf:"varint,4,rep,packed,name=stream_ids,json=streamIds,proto3" json:"stream_ids,omitempty"` ++ Version int64 `protobuf:"varint,5,opt,name=version,proto3" json:"version,omitempty"` ++ Compress *DataPacketCompressionMeta `protobuf:"bytes,6,opt,name=compress,proto3" json:"compress,omitempty"` ++ XXX_NoUnkeyedLiteral struct{} `json:"-"` ++ XXX_unrecognized []byte `json:"-"` ++ XXX_sizecache int32 `json:"-"` } func (m *MPPDataPacket) Reset() { *m = MPPDataPacket{} } @@ -323,7 +323,7 @@ index fcd4691..be5dd41 100644 + return 0 +} + -+func (m *MPPDataPacket) GetCompress() *DataPacketCompressMeta { ++func (m *MPPDataPacket) GetCompress() *DataPacketCompressionMeta { + if m != nil { + return m.Compress + } @@ -358,7 +358,7 @@ index fcd4691..be5dd41 100644 +} + func init() { -+ proto.RegisterEnum("mpp.CompressMethod", CompressMethod_name, CompressMethod_value) ++ proto.RegisterEnum("mpp.CompressionMode", CompressionMode_name, CompressionMode_value) + proto.RegisterType((*ExchangeSenderMeta)(nil), "mpp.ExchangeSenderMeta") proto.RegisterType((*TaskMeta)(nil), "mpp.TaskMeta") proto.RegisterType((*IsAliveRequest)(nil), "mpp.IsAliveRequest") @@ -367,7 +367,7 @@ index fcd4691..be5dd41 100644 proto.RegisterType((*CancelTaskRequest)(nil), "mpp.CancelTaskRequest") proto.RegisterType((*CancelTaskResponse)(nil), "mpp.CancelTaskResponse") proto.RegisterType((*EstablishMPPConnectionRequest)(nil), "mpp.EstablishMPPConnectionRequest") -+ proto.RegisterType((*DataPacketCompressMeta)(nil), "mpp.DataPacketCompressMeta") ++ proto.RegisterType((*DataPacketCompressionMeta)(nil), "mpp.DataPacketCompressionMeta") proto.RegisterType((*MPPDataPacket)(nil), "mpp.MPPDataPacket") proto.RegisterType((*Error)(nil), "mpp.Error") } @@ -418,60 +418,60 @@ index fcd4691..be5dd41 100644 - 0xb1, 0xf3, 0xf5, 0x76, 0xec, 0x7c, 0xfa, 0x36, 0xfe, 0x07, 0xb6, 0xb9, 0x58, 0x84, 0x2a, 0x5d, - 0x55, 0xe1, 0xaa, 0x32, 0x5f, 0xa7, 0xab, 0xae, 0xf9, 0x39, 0xfd, 0x11, 0x00, 0x00, 0xff, 0xff, - 0xcd, 0x41, 0xdb, 0xe2, 0xed, 0x04, 0x00, 0x00, -+ // 847 bytes of a gzipped FileDescriptorProto -+ 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x55, 0xcd, 0x8e, 0xdb, 0x36, -+ 0x10, 0x8e, 0x2c, 0xaf, 0x7f, 0xc6, 0x5a, 0xd7, 0x61, 0x16, 0x59, 0x27, 0x69, 0xb6, 0x8e, 0xd0, -+ 0x83, 0xd1, 0x02, 0x0e, 0xba, 0x01, 0x5a, 0xa0, 0x87, 0x02, 0xe9, 0xc6, 0x6d, 0x74, 0xf0, 0xae, -+ 0xcb, 0x35, 0x82, 0xde, 0x04, 0xae, 0xc8, 0xda, 0x82, 0x2d, 0x51, 0x21, 0x69, 0xa3, 0x39, 0xf7, -+ 0x21, 0xda, 0x47, 0xea, 0xad, 0x45, 0x9f, 0xa0, 0xd8, 0xbe, 0x48, 0xc1, 0xa1, 0xe4, 0xf5, 0xfe, -+ 0xf4, 0x27, 0x27, 0x71, 0x3e, 0x0e, 0x67, 0xbe, 0xf9, 0x86, 0x1c, 0x41, 0x3b, 0x2b, 0x8a, 0x51, -+ 0xa1, 0xa4, 0x91, 0xc4, 0xcf, 0x8a, 0xe2, 0xf1, 0xc1, 0x5c, 0xce, 0x25, 0xda, 0xcf, 0xed, 0xca, -+ 0x6d, 0x3d, 0xbe, 0x9f, 0x58, 0x2c, 0x11, 0x5a, 0x4b, 0x55, 0x42, 0x41, 0x26, 0x0c, 0x2b, 0x2e, -+ 0x9c, 0x15, 0x8e, 0x81, 0x8c, 0x7f, 0x4c, 0x16, 0x2c, 0x9f, 0x8b, 0x73, 0x91, 0x73, 0xa1, 0x26, -+ 0xc2, 0x30, 0xf2, 0x1c, 0x5a, 0x89, 0xcc, 0x0a, 0x25, 0xb4, 0xee, 0x7b, 0x03, 0x6f, 0xd8, 0x3d, -+ 0x7e, 0x30, 0xb2, 0xf9, 0x4e, 0x4a, 0x70, 0x22, 0xcc, 0x42, 0x72, 0xba, 0x75, 0x0a, 0x7f, 0xae, -+ 0x41, 0x6b, 0xc6, 0xf4, 0x12, 0x4f, 0x3f, 0x82, 0x96, 0x36, 0x4c, 0x99, 0xd8, 0xb8, 0xd3, 0x75, -+ 0xda, 0x44, 0x7b, 0xa6, 0xc9, 0x21, 0x34, 0x0d, 0xd3, 0xcb, 0x38, 0xe5, 0xfd, 0xda, 0xc0, 0x1b, -+ 0xfa, 0xb4, 0x61, 0xcd, 0x88, 0x93, 0x67, 0x10, 0x14, 0x4c, 0x99, 0xd4, 0xa4, 0x32, 0xb7, 0xbb, -+ 0x3e, 0xee, 0x76, 0xb6, 0x58, 0xc4, 0x49, 0x1f, 0x9a, 0x8c, 0x73, 0xe4, 0x54, 0x1f, 0x78, 0xc3, -+ 0x36, 0xad, 0x4c, 0xf2, 0x04, 0xda, 0x73, 0x66, 0x16, 0x42, 0xd9, 0x93, 0x7b, 0x98, 0xb1, 0xe5, -+ 0x80, 0x88, 0x5b, 0x36, 0x6f, 0xd7, 0x42, 0xbd, 0xb3, 0x6c, 0x1a, 0x8e, 0x0d, 0xda, 0x33, 0x4d, -+ 0x3e, 0x86, 0xee, 0x4a, 0x26, 0x6c, 0x15, 0x3b, 0x87, 0x94, 0xf7, 0x9b, 0xe8, 0x10, 0x20, 0xfa, -+ 0x9d, 0x05, 0x23, 0x6e, 0xa3, 0x6b, 0xa1, 0x36, 0x2e, 0x7a, 0xcb, 0x45, 0x77, 0x40, 0xc4, 0xc9, -+ 0x47, 0xd0, 0xc9, 0x8a, 0x22, 0xde, 0x08, 0xa5, 0x53, 0x99, 0xf7, 0xdb, 0x48, 0x1b, 0xb2, 0xa2, -+ 0x78, 0xe3, 0x90, 0xb0, 0x07, 0xdd, 0x48, 0xbf, 0x5c, 0xa5, 0x1b, 0x41, 0xc5, 0xdb, 0xb5, 0xd0, -+ 0x26, 0x9c, 0xc2, 0x07, 0x5b, 0x44, 0x17, 0x32, 0xd7, 0x82, 0x7c, 0x08, 0x6d, 0xb6, 0x61, 0xe9, -+ 0x8a, 0x5d, 0xac, 0x04, 0x4a, 0xd6, 0xa2, 0x57, 0xc0, 0xcd, 0x1c, 0xb5, 0x5b, 0x39, 0xfe, 0xa8, -+ 0xc1, 0x83, 0x57, 0xa9, 0x2e, 0x98, 0x49, 0x16, 0xb6, 0x0b, 0x65, 0x26, 0xf2, 0x0c, 0xea, 0xb6, -+ 0xd9, 0x18, 0xb1, 0x73, 0xbc, 0x8f, 0x2d, 0xac, 0xba, 0x44, 0x71, 0xcb, 0xea, 0x2e, 0xf2, 0x44, -+ 0x72, 0xc1, 0xe3, 0x62, 0xc5, 0x5c, 0xf0, 0x80, 0x76, 0x4a, 0x6c, 0xba, 0x62, 0xb9, 0xd5, 0xdd, -+ 0xa4, 0x99, 0x90, 0x6b, 0x53, 0x76, 0xa5, 0x32, 0xc9, 0x67, 0xd0, 0x54, 0x62, 0x9e, 0xca, 0xdc, -+ 0x76, 0xc4, 0x1f, 0x76, 0x8e, 0x0f, 0x47, 0xbb, 0xf7, 0x8d, 0xe2, 0x5e, 0x94, 0xff, 0x20, 0x69, -+ 0xe5, 0x47, 0x9e, 0x02, 0xe8, 0x64, 0x21, 0x32, 0x66, 0xcb, 0xc1, 0x5e, 0xf9, 0xb4, 0xed, 0x90, -+ 0x37, 0x42, 0x91, 0xaf, 0x60, 0xdf, 0xd8, 0x9a, 0xe3, 0x2a, 0x6e, 0x03, 0xe3, 0x3e, 0xba, 0x16, -+ 0x77, 0x66, 0x3d, 0x5c, 0x70, 0x4d, 0x03, 0xb3, 0x63, 0x91, 0x08, 0x0e, 0x44, 0x79, 0x9d, 0x63, -+ 0x8d, 0xf7, 0x39, 0x46, 0x05, 0x9a, 0xa8, 0xc0, 0x21, 0x2a, 0x70, 0xfb, 0xbe, 0x53, 0x22, 0x6e, -+ 0x61, 0x61, 0x06, 0x07, 0xd7, 0x35, 0x2d, 0x7b, 0x35, 0x80, 0x3d, 0xa1, 0x94, 0x54, 0xa5, 0xaa, -+ 0xe0, 0x62, 0x5a, 0x84, 0xba, 0x0d, 0xf2, 0x02, 0xf6, 0x95, 0x30, 0xea, 0xdd, 0xb6, 0x88, 0x1a, -+ 0x16, 0xd1, 0x1d, 0x95, 0x2f, 0xcf, 0x91, 0xa5, 0x01, 0x3a, 0x95, 0xcc, 0xc3, 0xef, 0xe1, 0xfe, -+ 0x09, 0xcb, 0x13, 0xb1, 0x7a, 0xcf, 0x06, 0x6e, 0xe9, 0xd4, 0xfe, 0x81, 0x4e, 0xf8, 0x39, 0x90, -+ 0xdd, 0xc8, 0xff, 0xb7, 0x8c, 0xf0, 0x27, 0x0f, 0x9e, 0x8e, 0xb5, 0x95, 0x37, 0xd5, 0x8b, 0xc9, -+ 0x74, 0x7a, 0x22, 0xf3, 0x5c, 0x24, 0xf6, 0x31, 0x56, 0xf4, 0x46, 0xd0, 0xd9, 0x15, 0xf9, 0x4e, -+ 0x96, 0xa0, 0xaf, 0xc6, 0xca, 0xb1, 0x15, 0x26, 0x11, 0xe9, 0xa6, 0x3a, 0x51, 0xbb, 0xeb, 0x44, -+ 0x50, 0xf9, 0x60, 0x1b, 0xc6, 0xf0, 0xf0, 0x15, 0x33, 0x6c, 0xca, 0x92, 0xa5, 0x30, 0x3b, 0xf3, -+ 0x87, 0x91, 0x4f, 0xa1, 0x91, 0xe1, 0x1c, 0xfa, 0xb7, 0x11, 0x55, 0xba, 0x84, 0xbf, 0x79, 0xb0, -+ 0x3f, 0x99, 0x4e, 0xaf, 0x42, 0x11, 0x02, 0x75, 0xce, 0x4a, 0xd6, 0x01, 0xc5, 0xf5, 0x7f, 0x8b, -+ 0x49, 0x1e, 0x42, 0x23, 0x59, 0xac, 0xf3, 0xa5, 0xee, 0xfb, 0x03, 0x7f, 0x18, 0xd0, 0xd2, 0xc2, -+ 0x7b, 0x6d, 0x94, 0x60, 0x59, 0x9c, 0x72, 0xf7, 0x1a, 0xea, 0xb4, 0xed, 0x90, 0x88, 0x6b, 0xfb, -+ 0x86, 0xaa, 0xe7, 0xeb, 0xee, 0x7c, 0x65, 0x92, 0x2f, 0x76, 0x46, 0x6d, 0x03, 0xb3, 0x3e, 0xc1, -+ 0xac, 0x77, 0x17, 0xbd, 0x33, 0x72, 0x4f, 0x61, 0x0f, 0x99, 0xd9, 0x42, 0xec, 0x63, 0xc5, 0x42, -+ 0xf6, 0x28, 0xae, 0x49, 0x0f, 0xfc, 0x4c, 0xcf, 0xb1, 0x8c, 0x36, 0xb5, 0xcb, 0x9b, 0x43, 0xc4, -+ 0xbf, 0x39, 0x44, 0x3e, 0xf9, 0x12, 0xba, 0xd7, 0xb5, 0x23, 0x2d, 0xa8, 0x9f, 0x9e, 0x9d, 0x8e, -+ 0x7b, 0xf7, 0xec, 0xea, 0x9b, 0x97, 0xe7, 0xb3, 0x9e, 0x47, 0x0e, 0xa0, 0xf7, 0x3a, 0xfa, 0xf6, -+ 0x75, 0x7c, 0x72, 0x36, 0x99, 0xd2, 0xf1, 0xf9, 0x79, 0x74, 0x76, 0xda, 0xab, 0x7d, 0x1d, 0xfe, -+ 0x7a, 0x79, 0xe4, 0xfd, 0x7e, 0x79, 0xe4, 0xfd, 0x79, 0x79, 0xe4, 0xfd, 0xf2, 0xd7, 0xd1, 0x3d, -+ 0xe8, 0x49, 0x35, 0x1f, 0x99, 0x74, 0xb9, 0x19, 0x2d, 0x37, 0xf8, 0xa7, 0xb9, 0x68, 0xe0, 0xe7, -+ 0xc5, 0xdf, 0x01, 0x00, 0x00, 0xff, 0xff, 0x0e, 0xad, 0x33, 0x36, 0xb9, 0x06, 0x00, 0x00, ++ // 844 bytes of a gzipped FileDescriptorProto ++ 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x55, 0xcd, 0x6e, 0x1b, 0x37, ++ 0x10, 0xce, 0x6a, 0x65, 0xfd, 0x8c, 0xd6, 0x8e, 0xc2, 0x0a, 0xb5, 0x9c, 0x36, 0xaa, 0xb2, 0xe8, ++ 0x41, 0xe8, 0x41, 0x6d, 0x1d, 0xa0, 0x87, 0x16, 0x28, 0x90, 0x3a, 0x4a, 0xb3, 0x07, 0xd9, 0x2a, ++ 0x2d, 0x04, 0xbd, 0x2d, 0xe8, 0x25, 0x2b, 0x2d, 0xa4, 0x5d, 0x6e, 0x48, 0x5a, 0x68, 0xce, 0x7d, ++ 0x88, 0xf6, 0x91, 0x7a, 0x2c, 0xf2, 0x04, 0x85, 0xfb, 0x22, 0x05, 0x87, 0xbb, 0xb2, 0x22, 0x27, ++ 0xfd, 0x39, 0x2d, 0xe7, 0xe3, 0x70, 0xe6, 0x9b, 0x6f, 0xc8, 0x59, 0x68, 0x67, 0x45, 0x31, 0x2e, ++ 0x94, 0x34, 0x92, 0xf8, 0x59, 0x51, 0x3c, 0xec, 0x2d, 0xe4, 0x42, 0xa2, 0xfd, 0xb9, 0x5d, 0xb9, ++ 0xad, 0x87, 0x0f, 0x12, 0x8b, 0x25, 0x42, 0x6b, 0xa9, 0x4a, 0x28, 0xc8, 0x84, 0x61, 0xc5, 0x95, ++ 0xb3, 0xc2, 0xe7, 0x40, 0x26, 0x3f, 0x27, 0x4b, 0x96, 0x2f, 0xc4, 0xa5, 0xc8, 0xb9, 0x50, 0x53, ++ 0x61, 0x18, 0xf9, 0x02, 0x5a, 0x89, 0xcc, 0x0a, 0x25, 0xb4, 0xee, 0x7b, 0x43, 0x6f, 0x74, 0x74, ++ 0xda, 0x1b, 0xdb, 0x7c, 0x67, 0x25, 0x98, 0xca, 0x7c, 0x2a, 0xb9, 0xa0, 0x5b, 0xaf, 0xf0, 0xd7, ++ 0x1a, 0xb4, 0xe6, 0x4c, 0xaf, 0xf0, 0xf8, 0x09, 0xb4, 0xb4, 0x61, 0xca, 0xc4, 0xc6, 0x1d, 0xaf, ++ 0xd3, 0x26, 0xda, 0x73, 0x4d, 0x8e, 0xa1, 0x69, 0x98, 0x5e, 0xc5, 0x29, 0xef, 0xd7, 0x86, 0xde, ++ 0xc8, 0xa7, 0x0d, 0x6b, 0x46, 0x9c, 0x3c, 0x86, 0xa0, 0x60, 0xca, 0xa4, 0x26, 0x95, 0xb9, 0xdd, ++ 0xf5, 0x71, 0xb7, 0xb3, 0xc5, 0x22, 0x4e, 0xfa, 0xd0, 0x64, 0x9c, 0x23, 0xa9, 0xfa, 0xd0, 0x1b, ++ 0xb5, 0x69, 0x65, 0x92, 0x8f, 0xa0, 0xbd, 0x60, 0x66, 0x29, 0x94, 0x3d, 0x79, 0x80, 0x19, 0x5b, ++ 0x0e, 0x88, 0xb8, 0x65, 0xf3, 0xea, 0x5a, 0xa8, 0xd7, 0x96, 0x4d, 0xc3, 0xb1, 0x41, 0x7b, 0xae, ++ 0xc9, 0xa7, 0x70, 0xb4, 0x96, 0x09, 0x5b, 0xc7, 0xce, 0x21, 0xe5, 0xfd, 0x26, 0x3a, 0x04, 0x88, ++ 0xfe, 0x60, 0xc1, 0x88, 0xdb, 0xe8, 0x5a, 0xa8, 0x8d, 0x8b, 0xde, 0x72, 0xd1, 0x1d, 0x10, 0x71, ++ 0xf2, 0x09, 0x74, 0xb2, 0xa2, 0x88, 0x37, 0x42, 0x59, 0x55, 0xfa, 0x6d, 0xa4, 0x0d, 0x59, 0x51, ++ 0xbc, 0x74, 0x48, 0xd8, 0x85, 0xa3, 0x48, 0x3f, 0x5d, 0xa7, 0x1b, 0x41, 0xc5, 0xab, 0x6b, 0xa1, ++ 0x4d, 0x38, 0x83, 0xfb, 0x5b, 0x44, 0x17, 0x32, 0xd7, 0x82, 0x7c, 0x0c, 0x6d, 0xb6, 0x61, 0xe9, ++ 0x9a, 0x5d, 0xad, 0x05, 0x4a, 0xd6, 0xa2, 0xb7, 0xc0, 0x7e, 0x8e, 0xda, 0x9d, 0x1c, 0x6f, 0x6a, ++ 0xf0, 0xc1, 0xb3, 0x54, 0x17, 0xcc, 0x24, 0x4b, 0xdb, 0x85, 0x32, 0x13, 0x79, 0x0c, 0x75, 0xdb, ++ 0x6d, 0x8c, 0xd8, 0x39, 0x3d, 0xc4, 0x1e, 0x56, 0x5d, 0xa2, 0xb8, 0x65, 0x75, 0x17, 0x79, 0x22, ++ 0xb9, 0xe0, 0x71, 0xb1, 0x66, 0x2e, 0x78, 0x40, 0x3b, 0x25, 0x36, 0x5b, 0xb3, 0xdc, 0xea, 0x6e, ++ 0xd2, 0x4c, 0xc8, 0x6b, 0x53, 0x76, 0xa5, 0x32, 0xc9, 0x97, 0xd0, 0x54, 0x62, 0x91, 0xca, 0xdc, ++ 0x76, 0xc4, 0x1f, 0x75, 0x4e, 0x8f, 0xc7, 0xbb, 0x17, 0x8e, 0xe2, 0x5e, 0x94, 0xff, 0x24, 0x69, ++ 0xe5, 0x47, 0x1e, 0x01, 0xe8, 0x64, 0x29, 0x32, 0x66, 0xcb, 0xc1, 0x5e, 0xf9, 0xb4, 0xed, 0x90, ++ 0x97, 0x42, 0x91, 0x6f, 0xe1, 0xd0, 0xd8, 0x9a, 0xe3, 0x2a, 0x6e, 0x03, 0xe3, 0x9e, 0xbc, 0x15, ++ 0x77, 0x6e, 0x3d, 0x5c, 0x70, 0x4d, 0x03, 0xb3, 0x63, 0x91, 0x08, 0x7a, 0xa2, 0xbc, 0xcf, 0xb1, ++ 0xc6, 0x0b, 0x1d, 0xa3, 0x02, 0x4d, 0x54, 0xe0, 0x18, 0x15, 0xb8, 0x7b, 0xe1, 0x29, 0x11, 0x77, ++ 0xb0, 0x30, 0x83, 0xde, 0xdb, 0x9a, 0x96, 0xbd, 0x1a, 0xc2, 0x81, 0x50, 0x4a, 0xaa, 0x52, 0x55, ++ 0x70, 0x31, 0x2d, 0x42, 0xdd, 0x06, 0x79, 0x02, 0x87, 0x4a, 0x18, 0xf5, 0x7a, 0x5b, 0x44, 0x0d, ++ 0x8b, 0x38, 0x1a, 0x97, 0x4f, 0xcf, 0x91, 0xa5, 0x01, 0x3a, 0x95, 0xcc, 0xc3, 0x1f, 0xe1, 0xc1, ++ 0x19, 0xcb, 0x13, 0xb1, 0xfe, 0x9f, 0x0d, 0xdc, 0xd2, 0xa9, 0xbd, 0x87, 0x4e, 0xf8, 0x15, 0x90, ++ 0xdd, 0xc8, 0xff, 0xb5, 0x8c, 0xf0, 0x17, 0x0f, 0x1e, 0x4d, 0xb4, 0x95, 0x37, 0xd5, 0xcb, 0xe9, ++ 0x6c, 0x76, 0x26, 0xf3, 0x5c, 0x24, 0xf6, 0x31, 0x56, 0xf4, 0xc6, 0xd0, 0xd9, 0x15, 0xf9, 0x9d, ++ 0x2c, 0x41, 0xdf, 0xce, 0x95, 0x53, 0x2b, 0x4c, 0x22, 0xd2, 0x4d, 0x75, 0xa2, 0xf6, 0xae, 0x13, ++ 0x41, 0xe5, 0x83, 0x6d, 0x98, 0xc0, 0xc9, 0x33, 0x66, 0xd8, 0x8c, 0x25, 0x2b, 0x61, 0x76, 0x07, ++ 0x90, 0x0d, 0x38, 0x82, 0x7a, 0x26, 0xb9, 0xf8, 0xc7, 0x21, 0x85, 0x1e, 0xe1, 0x1b, 0x0f, 0x0e, ++ 0xa7, 0xb3, 0xd9, 0x6d, 0x28, 0x42, 0xa0, 0xce, 0x59, 0xc9, 0x3a, 0xa0, 0xb8, 0xfe, 0x77, 0x31, ++ 0xc9, 0x87, 0xd0, 0x48, 0x96, 0xd7, 0xf9, 0x4a, 0xf7, 0xfd, 0xa1, 0x3f, 0x0a, 0x68, 0x69, 0xe1, ++ 0xbd, 0x36, 0x4a, 0xb0, 0x2c, 0x4e, 0xb9, 0x7b, 0x0d, 0x75, 0xda, 0x76, 0x48, 0xc4, 0xb5, 0x7d, ++ 0x43, 0xd5, 0xf3, 0x75, 0x77, 0xbe, 0x32, 0xc9, 0xd7, 0x3b, 0xb3, 0xb6, 0x81, 0x59, 0x07, 0x98, ++ 0xf5, 0xbd, 0x45, 0xef, 0x4c, 0xdd, 0x73, 0x38, 0x40, 0x72, 0xb6, 0x96, 0xa4, 0xd2, 0xe1, 0x80, ++ 0xe2, 0x9a, 0x74, 0xc1, 0xcf, 0xf4, 0x02, 0x2b, 0x69, 0x53, 0xbb, 0xdc, 0x9f, 0x23, 0xfe, 0xfe, ++ 0x1c, 0xf9, 0xec, 0x1b, 0xb8, 0xbf, 0xa7, 0x1e, 0x69, 0x41, 0xfd, 0xfc, 0xe2, 0x7c, 0xd2, 0xbd, ++ 0x67, 0x57, 0xcf, 0x9f, 0x5e, 0xce, 0xbb, 0x1e, 0xe9, 0x41, 0xf7, 0x45, 0xf4, 0xfd, 0x8b, 0xf8, ++ 0xec, 0x62, 0x3a, 0xa3, 0x93, 0xcb, 0xcb, 0xe8, 0xe2, 0xbc, 0x5b, 0xfb, 0x2e, 0xfc, 0xfd, 0x66, ++ 0xe0, 0xfd, 0x71, 0x33, 0xf0, 0xfe, 0xbc, 0x19, 0x78, 0xbf, 0xfd, 0x35, 0xb8, 0x07, 0x5d, 0xa9, ++ 0x16, 0x63, 0x93, 0xae, 0x36, 0xe3, 0xd5, 0x06, 0x7f, 0x37, 0x57, 0x0d, 0xfc, 0x3c, 0xf9, 0x3b, ++ 0x00, 0x00, 0xff, 0xff, 0x40, 0xa5, 0xdf, 0xfa, 0xbe, 0x06, 0x00, 0x00, +} + +func (m *ExchangeSenderMeta) Marshal() (dAtA []byte, err error) { @@ -554,7 +554,7 @@ index fcd4691..be5dd41 100644 return len(dAtA) - i, nil } -+func (m *DataPacketCompressMeta) Marshal() (dAtA []byte, err error) { ++func (m *DataPacketCompressionMeta) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) @@ -564,12 +564,12 @@ index fcd4691..be5dd41 100644 + return dAtA[:n], nil +} + -+func (m *DataPacketCompressMeta) MarshalTo(dAtA []byte) (int, error) { ++func (m *DataPacketCompressionMeta) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + -+func (m *DataPacketCompressMeta) MarshalToSizedBuffer(dAtA []byte) (int, error) { ++func (m *DataPacketCompressionMeta) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int @@ -578,8 +578,8 @@ index fcd4691..be5dd41 100644 + i -= len(m.XXX_unrecognized) + copy(dAtA[i:], m.XXX_unrecognized) + } -+ if m.Method != 0 { -+ i = encodeVarintMpp(dAtA, i, uint64(m.Method)) ++ if m.Mode != 0 { ++ i = encodeVarintMpp(dAtA, i, uint64(m.Mode)) + i-- + dAtA[i] = 0x8 + } @@ -706,14 +706,14 @@ index fcd4691..be5dd41 100644 return n } -+func (m *DataPacketCompressMeta) Size() (n int) { ++func (m *DataPacketCompressionMeta) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l -+ if m.Method != 0 { -+ n += 1 + sovMpp(uint64(m.Method)) ++ if m.Mode != 0 { ++ n += 1 + sovMpp(uint64(m.Mode)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) @@ -795,7 +795,7 @@ index fcd4691..be5dd41 100644 + } + b := dAtA[iNdEx] + iNdEx++ -+ m.Compress |= CompressMethod(b&0x7F) << shift ++ m.Compress |= CompressionMode(b&0x7F) << shift + if b < 0x80 { + break + } @@ -924,7 +924,7 @@ index fcd4691..be5dd41 100644 } return nil } -+func (m *DataPacketCompressMeta) Unmarshal(dAtA []byte) error { ++func (m *DataPacketCompressionMeta) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { @@ -947,17 +947,17 @@ index fcd4691..be5dd41 100644 + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { -+ return fmt.Errorf("proto: DataPacketCompressMeta: wiretype end group for non-group") ++ return fmt.Errorf("proto: DataPacketCompressionMeta: wiretype end group for non-group") + } + if fieldNum <= 0 { -+ return fmt.Errorf("proto: DataPacketCompressMeta: illegal tag %d (wire type %d)", fieldNum, wire) ++ return fmt.Errorf("proto: DataPacketCompressionMeta: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { -+ return fmt.Errorf("proto: wrong wireType = %d for field Method", wireType) ++ return fmt.Errorf("proto: wrong wireType = %d for field Mode", wireType) + } -+ m.Method = 0 ++ m.Mode = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMpp @@ -967,7 +967,7 @@ index fcd4691..be5dd41 100644 + } + b := dAtA[iNdEx] + iNdEx++ -+ m.Method |= CompressMethod(b&0x7F) << shift ++ m.Mode |= CompressionMode(b&0x7F) << shift + if b < 0x80 { + break + } @@ -1050,7 +1050,7 @@ index fcd4691..be5dd41 100644 + return io.ErrUnexpectedEOF + } + if m.Compress == nil { -+ m.Compress = &DataPacketCompressMeta{} ++ m.Compress = &DataPacketCompressionMeta{} + } + if err := m.Compress.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err @@ -1086,22 +1086,22 @@ index fcd4691..be5dd41 100644 iNdEx = preIndex skippy, err := skipMpp(dAtA[iNdEx:]) diff --git a/proto/mpp.proto b/proto/mpp.proto -index 6739455..e3ec05f 100644 +index 6739455..9cb2b90 100644 --- a/proto/mpp.proto +++ b/proto/mpp.proto @@ -11,6 +11,17 @@ option (gogoproto.unmarshaler_all) = true; option java_package = "org.tikv.kvproto"; -+// Data compression method -+enum CompressMethod { ++// Data compression mode ++enum CompressionMode { + NONE = 0; // no compression + FAST = 1; // fast compression/decompression speed, compression ratio is lower than HC mode + HIGH_COMPRESSION = 2; // high compression (HC) ratio mode +} + +message ExchangeSenderMeta { -+ CompressMethod compress = 1; ++ CompressionMode compress = 1; +} + // TaskMeta contains meta of a mpp plan, including query's ts and task address. @@ -1136,8 +1136,8 @@ index 6739455..e3ec05f 100644 TaskMeta receiver_meta = 2; // node closer to the tidb mpp gather. } -+message DataPacketCompressMeta { -+ CompressMethod method = 1; ++message DataPacketCompressionMeta { ++ CompressionMode mode = 1; +} + // when TiFlash sends data to TiDB, Data packets wrap tipb.SelectResponse, i.e., serialize tipb.SelectResponse into data; @@ -1148,7 +1148,7 @@ index 6739455..e3ec05f 100644 repeated bytes chunks = 3; repeated uint64 stream_ids = 4; + int64 version = 5; // version of data format -+ DataPacketCompressMeta compress = 6; // data compression ++ DataPacketCompressionMeta compress = 6; // data compression meta } message Error { diff --git a/dbms/src/Flash/Coprocessor/CHBlockChunkCodec.cpp b/dbms/src/Flash/Coprocessor/CHBlockChunkCodec.cpp index 55a7732aa4d..3c26a01d062 100644 --- a/dbms/src/Flash/Coprocessor/CHBlockChunkCodec.cpp +++ b/dbms/src/Flash/Coprocessor/CHBlockChunkCodec.cpp @@ -98,18 +98,18 @@ size_t ApproxBlockBytes(const Block & block) return block.bytes() + GetExtraInfoSize(block); } -CompressionMethod ToInternalCompressionMethod(mpp::CompressMethod compress_method) +CompressionMethod ToInternalCompressionMethod(mpp::CompressionMode compress_method) { switch (compress_method) { - case mpp::NONE: + case mpp::CompressionMode::NONE: return CompressionMethod::NONE; - case mpp::CompressMethod::FAST: - return CompressionMethod::LZ4; - case mpp::CompressMethod::HIGH_COMPRESSION: - return CompressionMethod::ZSTD; + case mpp::CompressionMode::FAST: + return CompressionMethod::LZ4; // use LZ4 method as fast mode + case mpp::CompressionMode::HIGH_COMPRESSION: + return CompressionMethod::ZSTD; // use ZSTD method as HC mode default: - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unkown compress method {}", mpp::CompressMethod_Name(compress_method)); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unkown compresssion mode {}", mpp::CompressionMode_Name(compress_method)); } } diff --git a/dbms/src/Flash/Coprocessor/CHBlockChunkCodecStream.h b/dbms/src/Flash/Coprocessor/CHBlockChunkCodecStream.h index 34b482e90e3..ab3c5127e84 100644 --- a/dbms/src/Flash/Coprocessor/CHBlockChunkCodecStream.h +++ b/dbms/src/Flash/Coprocessor/CHBlockChunkCodecStream.h @@ -45,7 +45,7 @@ class CHBlockChunkCodecStream : public ChunkCodecStream size_t GetExtraInfoSize(const Block & block); size_t ApproxBlockBytes(const Block & block); -CompressionMethod ToInternalCompressionMethod(mpp::CompressMethod compress_method); +CompressionMethod ToInternalCompressionMethod(mpp::CompressionMode compress_method); std::unique_ptr NewCHBlockChunkCodecStream(const std::vector & field_types); void EncodeCHBlockChunk(WriteBuffer * ostr_ptr, const Block & block); diff --git a/dbms/src/Flash/Mpp/ExchangeReceiver.cpp b/dbms/src/Flash/Mpp/ExchangeReceiver.cpp index 3d1d12bcf70..b83a91aca94 100644 --- a/dbms/src/Flash/Mpp/ExchangeReceiver.cpp +++ b/dbms/src/Flash/Mpp/ExchangeReceiver.cpp @@ -722,7 +722,7 @@ DecodeDetail ExchangeReceiverBase::decodeChunks( for (auto && chunk : packet.chunks()) { - auto && result = decoder_ptr->decodeAndSquash(chunk, packet.compress().method() != mpp::CompressMethod::NONE); + auto && result = decoder_ptr->decodeAndSquash(chunk, packet.compress().mode() != mpp::CompressionMode::NONE); if (!result || !result->rows()) continue; detail.rows += result->rows(); diff --git a/dbms/src/Flash/Mpp/HashPartitionWriter.cpp b/dbms/src/Flash/Mpp/HashPartitionWriter.cpp index ea9dc0f8640..5c3e3c84f4f 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriter.cpp +++ b/dbms/src/Flash/Mpp/HashPartitionWriter.cpp @@ -107,20 +107,8 @@ void HashPartitionWriter::partitionAndEncodeThenWriteBlocks() auto tracked_packets = HashBaseWriterHelper::createPackets(partition_num); - for (size_t part_id = 0; part_id < partition_num; ++part_id) - { - // auto method = compress_method; - // if (writer->getTunnels()[part_id]->isLocal()) - // { - // method = mpp::CompressMethod::NONE; - // } - // tracked_packets[part_id]->getPacket().mutable_compress()->set_method(method); - // tracked_packets[part_id]->getPacket().set_mpp_version(TiDB::GetMppVersion()); - } - size_t ori_block_mem_size = 0; - if (!blocks.empty()) { assert(rows_in_blocks > 0); @@ -145,14 +133,7 @@ void HashPartitionWriter::partitionAndEncodeThenWriteBlocks() if (dest_block_rows > 0) { auto * codec_stream = chunk_codec_stream.get(); - // if (tracked_packets[part_id]->getPacket().compress().method() != mpp::CompressMethod::NONE) - // { - // assert(compress_chunk_codec_stream); - // // no need compress - // codec_stream = compress_chunk_codec_stream.get(); - // } codec_stream->encode(dest_block, 0, dest_block_rows); - // ori_block_mem_size += ApproxBlockBytes(dest_block); tracked_packets[part_id]->addChunk(codec_stream->getString()); codec_stream->clear(); } @@ -180,15 +161,18 @@ void HashPartitionWriter::writePackets(const TrackedMppDataPa { writer->partitionWrite(packet, part_id); - assert(inner_packet.compress().method() == mpp::CompressMethod::NONE); - - if (auto sz = inner_packet.ByteSizeLong(); writer->isLocal(part_id)) - { - GET_METRIC(tiflash_exchange_data_bytes, type_hash_none_local).Increment(sz); - } - else + // Update metrics about exchange hash partition { - GET_METRIC(tiflash_exchange_data_bytes, type_hash_none).Increment(sz); + assert(inner_packet.compress().mode() == mpp::CompressionMode::NONE); + + if (auto sz = inner_packet.ByteSizeLong(); writer->isLocal(part_id)) + { + GET_METRIC(tiflash_exchange_data_bytes, type_hash_none_local).Increment(sz); + } + else + { + GET_METRIC(tiflash_exchange_data_bytes, type_hash_none).Increment(sz); + } } } } diff --git a/dbms/src/Flash/Mpp/HashPartitionWriter.h b/dbms/src/Flash/Mpp/HashPartitionWriter.h index 3e01661adf7..81cf9323e42 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriter.h +++ b/dbms/src/Flash/Mpp/HashPartitionWriter.h @@ -56,7 +56,7 @@ class HashPartitionWriter : public DAGResponseWriter size_t rows_in_blocks; uint16_t partition_num; - // mpp::CompressMethod compress_method{}; + // mpp::CompressionMode compress_method{}; std::unique_ptr chunk_codec_stream; // std::unique_ptr compress_chunk_codec_stream; }; diff --git a/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp b/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp index 7ea111d43df..7639d1ef0bc 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp +++ b/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp @@ -46,7 +46,7 @@ HashPartitionWriterImplV1::HashPartitionWriterImplV1( Int64 partition_batch_limit_, bool should_send_exec_summary_at_last_, DAGContext & dag_context_, - mpp::CompressMethod compress_method_) + mpp::CompressionMode compress_method_) : DAGResponseWriter(/*records_per_chunk=*/-1, dag_context_) , partition_num(writer_->getPartitionNum()) , partition_batch_limit(partition_batch_limit_ * partition_num) @@ -123,8 +123,8 @@ void HashPartitionWriterImplV1::partitionAndEncodeThenWriteBl // Do NOT enable data compression when using local tunnel for (size_t part_id = 0; part_id < partition_num; ++part_id) { - auto method = writer->isLocal(part_id) ? mpp::CompressMethod::NONE : compress_method; - tracked_packets[part_id]->getPacket().mutable_compress()->set_method(method); + auto mode = writer->isLocal(part_id) ? mpp::CompressionMode::NONE : compress_method; + tracked_packets[part_id]->getPacket().mutable_compress()->set_mode(mode); } // Sum of all approximate block data memory size @@ -191,7 +191,7 @@ void HashPartitionWriterImplV1::partitionAndEncodeThenWriteBl WriteBuffer * ostr_ptr = output_buffer.get(); // Init compression writer - if (tracked_packets[part_id]->getPacket().compress().method() != mpp::NONE) + if (tracked_packets[part_id]->getPacket().compress().mode() != mpp::CompressionMode::NONE) { compress_codec = std::make_unique( *output_buffer, @@ -252,8 +252,8 @@ void HashPartitionWriterImplV1::partitionAndEncodeThenWriteBl // Do NOT enable data compression when using local tunnel for (size_t part_id = 0; part_id < partition_num; ++part_id) { - auto method = writer->isLocal(part_id) ? mpp::CompressMethod::NONE : compress_method; - tracked_packets[part_id]->getPacket().mutable_compress()->set_method(method); + auto mode = writer->isLocal(part_id) ? mpp::CompressionMode::NONE : compress_method; + tracked_packets[part_id]->getPacket().mutable_compress()->set_mode(mode); } // Sum of all approximate block data memory size @@ -319,7 +319,7 @@ void HashPartitionWriterImplV1::partitionAndEncodeThenWriteBl WriteBuffer * ostr_ptr = output_buffer.get(); // Init compression writer - if (tracked_packets[part_id]->getPacket().compress().method() != mpp::NONE) + if (tracked_packets[part_id]->getPacket().compress().mode() != mpp::CompressionMode::NONE) { compress_codec = std::make_unique( *output_buffer, @@ -364,11 +364,11 @@ void HashPartitionWriterImplV1::partitionAndEncodeThenWriteBl GET_METRIC(tiflash_exchange_data_bytes, type_hash_original_all).Increment(ori_block_mem_size); } -static void updateHashPartitionWriterMetrics(mpp::CompressMethod method, size_t sz, bool is_local) +static void updateHashPartitionWriterMetrics(mpp::CompressionMode mode, size_t sz, bool is_local) { - switch (method) + switch (mode) { - case mpp::NONE: + case mpp::CompressionMode::NONE: { if (is_local) { @@ -380,12 +380,12 @@ static void updateHashPartitionWriterMetrics(mpp::CompressMethod method, size_t } break; } - case mpp::CompressMethod::FAST: + case mpp::CompressionMode::FAST: { GET_METRIC(tiflash_exchange_data_bytes, type_hash_lz4).Increment(sz); break; } - case mpp::CompressMethod::HIGH_COMPRESSION: + case mpp::CompressionMode::HIGH_COMPRESSION: { GET_METRIC(tiflash_exchange_data_bytes, type_hash_zstd).Increment(sz); break; @@ -407,9 +407,9 @@ void HashPartitionWriterImplV1::writePackets(TrackedMppDataPa if (auto sz = inner_packet.ByteSizeLong(); likely(inner_packet.chunks_size() > 0)) { - auto method = inner_packet.compress().method(); + auto mode = inner_packet.compress().mode(); writer->partitionWrite(std::move(packet), part_id); - updateHashPartitionWriterMetrics(method, sz, writer->isLocal(part_id)); + updateHashPartitionWriterMetrics(mode, sz, writer->isLocal(part_id)); } } } diff --git a/dbms/src/Flash/Mpp/HashPartitionWriterV1.h b/dbms/src/Flash/Mpp/HashPartitionWriterV1.h index 2d9cd8dc79d..ff6125f21ac 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriterV1.h +++ b/dbms/src/Flash/Mpp/HashPartitionWriterV1.h @@ -45,7 +45,7 @@ class HashPartitionWriterImplV1 : public DAGResponseWriter Int64 partition_batch_limit_, bool should_send_exec_summary_at_last, DAGContext & dag_context_, - mpp::CompressMethod compress_method_); + mpp::CompressionMode compress_method_); void write(const Block & block) override; void flush() override; void finishWrite() override; @@ -68,7 +68,7 @@ class HashPartitionWriterImplV1 : public DAGResponseWriter TiDB::TiDBCollators collators; size_t rows_in_blocks; DataTypes expected_types; - mpp::CompressMethod compress_method{}; + mpp::CompressionMode compress_method{}; }; } // namespace DB diff --git a/dbms/src/Flash/Mpp/MPPTask.cpp b/dbms/src/Flash/Mpp/MPPTask.cpp index 1f5be8d548e..7ca6e6adb5f 100644 --- a/dbms/src/Flash/Mpp/MPPTask.cpp +++ b/dbms/src/Flash/Mpp/MPPTask.cpp @@ -279,26 +279,26 @@ void MPPTask::prepare(const mpp::DispatchTaskRequest & task_request) // if (true) // nolint // { // auto exchange_sender_meta = task_request.exchange_sender_meta(); - // mpp::CompressMethod compress_method{}; + // mpp::CompressionMode compress_method{}; // switch (tzg::SnappyStatistic::globalInstance().getMethod()) // { // case tzg::SnappyStatistic::CompressMethod::LZ4: - // compress_method = mpp::CompressMethod::FAST; + // compress_method = mpp::CompressionMode::FAST; // break; // case tzg::SnappyStatistic::CompressMethod::ZSTD: - // compress_method = mpp::CompressMethod::HIGH_COMPRESSION; + // compress_method = mpp::CompressionMode::HIGH_COMPRESSION; // break; // default: - // compress_method = mpp::CompressMethod::NONE; + // compress_method = mpp::CompressionMode::NONE; // break; // } // exchange_sender_meta.set_compress(compress_method); - // LOG_DEBUG(log, "DAGContext use compress method {}", mpp::CompressMethod_Name(compress_method)); + // LOG_DEBUG(log, "DAGContext use compress method {}", mpp::CompressionMode_Name(compress_method)); // dag_context = std::make_unique(dag_req, task_request.meta(), exchange_sender_meta, is_root_mpp_task); // } // else { - LOG_DEBUG(log, "DAGContext use compress method {}", mpp::CompressMethod_Name(task_request.exchange_sender_meta().compress())); + LOG_DEBUG(log, "DAGContext use compress method {}", mpp::CompressionMode_Name(task_request.exchange_sender_meta().compress())); dag_context = std::make_unique(dag_req, task_request.meta(), task_request.exchange_sender_meta(), is_root_mpp_task); } diff --git a/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp b/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp index a55cf2d183a..6897772f8cd 100644 --- a/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp +++ b/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp @@ -503,7 +503,7 @@ try batch_send_min_limit, /*should_send_exec_summary_at_last=*/false, *dag_context_ptr, - mpp::CompressMethod::FAST); + mpp::CompressionMode::FAST); for (const auto & block : blocks) dag_writer->write(block); dag_writer->flush(); @@ -528,14 +528,14 @@ try { if (part_index == 0) { - ASSERT_EQ(packet.compress().method(), mpp::CompressMethod::NONE); + ASSERT_EQ(packet.compress().mode(), mpp::CompressionMode::NONE); } else { - ASSERT_NE(packet.compress().method(), mpp::CompressMethod::NONE); + ASSERT_NE(packet.compress().mode(), mpp::CompressionMode::NONE); } - auto && result = decoder.decodeAndSquash(chunk, packet.compress().method() != mpp::CompressMethod::NONE); + auto && result = decoder.decodeAndSquash(chunk, packet.compress().mode() != mpp::CompressionMode::NONE); if (!result) continue; decoded_block_rows += result->rows(); diff --git a/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp b/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp index 4fbce4e0780..32c05646558 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp +++ b/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp @@ -120,7 +120,7 @@ std::unique_ptr NewMPPExchangeWriter( if (dag_context.isRootMPPTask()) { // No need to use use data compression - RUNTIME_CHECK(dag_context.getExchangeSenderMeta().compress() == mpp::CompressMethod::NONE); + RUNTIME_CHECK(dag_context.getExchangeSenderMeta().compress() == mpp::CompressionMode::NONE); RUNTIME_CHECK(!enable_fine_grained_shuffle); RUNTIME_CHECK(exchange_type == tipb::ExchangeType::PassThrough); @@ -138,7 +138,7 @@ std::unique_ptr NewMPPExchangeWriter( if (enable_fine_grained_shuffle) { // TODO: support data compression if necessary - RUNTIME_CHECK(dag_context.getExchangeSenderMeta().compress() == mpp::CompressMethod::NONE); + RUNTIME_CHECK(dag_context.getExchangeSenderMeta().compress() == mpp::CompressionMode::NONE); return std::make_unique>( writer, @@ -173,7 +173,7 @@ std::unique_ptr NewMPPExchangeWriter( else { // TODO: support data compression if necessary - RUNTIME_CHECK(dag_context.getExchangeSenderMeta().compress() == mpp::CompressMethod::NONE); + RUNTIME_CHECK(dag_context.getExchangeSenderMeta().compress() == mpp::CompressionMode::NONE); RUNTIME_CHECK(!enable_fine_grained_shuffle); return std::make_unique>( From 357843ff589bb81dc0a555c2e4b5294fcd30bcfa Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Thu, 5 Jan 2023 17:35:23 +0800 Subject: [PATCH 41/93] 35 Signed-off-by: Zhigao Tong --- dbms/src/Flash/Mpp/MPPTask.cpp | 28 ++-------------------------- 1 file changed, 2 insertions(+), 26 deletions(-) diff --git a/dbms/src/Flash/Mpp/MPPTask.cpp b/dbms/src/Flash/Mpp/MPPTask.cpp index 7ca6e6adb5f..f95b892aa99 100644 --- a/dbms/src/Flash/Mpp/MPPTask.cpp +++ b/dbms/src/Flash/Mpp/MPPTask.cpp @@ -276,32 +276,8 @@ void MPPTask::prepare(const mpp::DispatchTaskRequest & task_request) is_root_mpp_task = task_meta.task_id() == -1; } - // if (true) // nolint - // { - // auto exchange_sender_meta = task_request.exchange_sender_meta(); - // mpp::CompressionMode compress_method{}; - // switch (tzg::SnappyStatistic::globalInstance().getMethod()) - // { - // case tzg::SnappyStatistic::CompressMethod::LZ4: - // compress_method = mpp::CompressionMode::FAST; - // break; - // case tzg::SnappyStatistic::CompressMethod::ZSTD: - // compress_method = mpp::CompressionMode::HIGH_COMPRESSION; - // break; - // default: - // compress_method = mpp::CompressionMode::NONE; - // break; - // } - // exchange_sender_meta.set_compress(compress_method); - // LOG_DEBUG(log, "DAGContext use compress method {}", mpp::CompressionMode_Name(compress_method)); - // dag_context = std::make_unique(dag_req, task_request.meta(), exchange_sender_meta, is_root_mpp_task); - // } - // else - { - LOG_DEBUG(log, "DAGContext use compress method {}", mpp::CompressionMode_Name(task_request.exchange_sender_meta().compress())); - dag_context = std::make_unique(dag_req, task_request.meta(), task_request.exchange_sender_meta(), is_root_mpp_task); - } - + LOG_TRACE(log, "DAGContext use compression mode {}", mpp::CompressionMode_Name(task_request.exchange_sender_meta().compress())); + dag_context = std::make_unique(dag_req, task_request.meta(), task_request.exchange_sender_meta(), is_root_mpp_task); dag_context->log = log; dag_context->tables_regions_info = std::move(tables_regions_info); dag_context->tidb_host = context->getClientInfo().current_address.toString(); From 950503ce880dbca3a301b3a93fc86bbabc4e869f Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Thu, 5 Jan 2023 17:36:54 +0800 Subject: [PATCH 42/93] 36 Signed-off-by: Zhigao Tong --- dbms/src/Flash/Mpp/MPPTask.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/dbms/src/Flash/Mpp/MPPTask.cpp b/dbms/src/Flash/Mpp/MPPTask.cpp index f95b892aa99..4f8650b8457 100644 --- a/dbms/src/Flash/Mpp/MPPTask.cpp +++ b/dbms/src/Flash/Mpp/MPPTask.cpp @@ -276,7 +276,6 @@ void MPPTask::prepare(const mpp::DispatchTaskRequest & task_request) is_root_mpp_task = task_meta.task_id() == -1; } - LOG_TRACE(log, "DAGContext use compression mode {}", mpp::CompressionMode_Name(task_request.exchange_sender_meta().compress())); dag_context = std::make_unique(dag_req, task_request.meta(), task_request.exchange_sender_meta(), is_root_mpp_task); dag_context->log = log; dag_context->tables_regions_info = std::move(tables_regions_info); From 78cb77c72b24a8825a6aa944cc27301a45ffde59 Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Thu, 5 Jan 2023 19:08:38 +0800 Subject: [PATCH 43/93] 37 --- contrib/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index f4346e0b69d..4400614f887 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -112,7 +112,7 @@ if (USE_INTERNAL_GRPC_LIBRARY) endif () execute_process( - COMMAND grep "CompressMethod compress" "${TiFlash_SOURCE_DIR}/contrib/kvproto/proto/mpp.proto" + COMMAND grep "CompressionMode compress" "${TiFlash_SOURCE_DIR}/contrib/kvproto/proto/mpp.proto" RESULT_VARIABLE HAVE_APPLY_PATCH) # grep - Normally, the exit status is 0 if selected lines are found and 1 otherwise. But the exit status is 2 if an error occurred. From 2c4558c596569334df4e1d1d6271116ead7eb3dc Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Thu, 5 Jan 2023 20:17:09 +0800 Subject: [PATCH 44/93] 38 --- dbms/src/Flash/Coprocessor/CHBlockChunkCodec.cpp | 6 +++--- dbms/src/Flash/Coprocessor/CHBlockChunkCodecStream.h | 7 +++++-- .../Coprocessor/CompressCHBlockChunkCodecStream.cpp | 4 ++-- .../Coprocessor/CompressCHBlockChunkCodecStream.h | 10 +++++----- dbms/src/Flash/Mpp/HashPartitionWriter.cpp | 5 ----- dbms/src/Flash/Mpp/HashPartitionWriter.h | 2 -- dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp | 12 ++++++------ dbms/src/Flash/Mpp/HashPartitionWriterV1.h | 4 ++-- .../Flash/Planner/plans/PhysicalExchangeSender.cpp | 4 ++-- 9 files changed, 25 insertions(+), 29 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/CHBlockChunkCodec.cpp b/dbms/src/Flash/Coprocessor/CHBlockChunkCodec.cpp index 3c26a01d062..519526eaa59 100644 --- a/dbms/src/Flash/Coprocessor/CHBlockChunkCodec.cpp +++ b/dbms/src/Flash/Coprocessor/CHBlockChunkCodec.cpp @@ -98,9 +98,9 @@ size_t ApproxBlockBytes(const Block & block) return block.bytes() + GetExtraInfoSize(block); } -CompressionMethod ToInternalCompressionMethod(mpp::CompressionMode compress_method) +CompressionMethod ToInternalCompressionMethod(mpp::CompressionMode compression_mode) { - switch (compress_method) + switch (compression_mode) { case mpp::CompressionMode::NONE: return CompressionMethod::NONE; @@ -109,7 +109,7 @@ CompressionMethod ToInternalCompressionMethod(mpp::CompressionMode compress_meth case mpp::CompressionMode::HIGH_COMPRESSION: return CompressionMethod::ZSTD; // use ZSTD method as HC mode default: - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unkown compresssion mode {}", mpp::CompressionMode_Name(compress_method)); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unkown compresssion mode {}", mpp::CompressionMode_Name(compression_mode)); } } diff --git a/dbms/src/Flash/Coprocessor/CHBlockChunkCodecStream.h b/dbms/src/Flash/Coprocessor/CHBlockChunkCodecStream.h index ab3c5127e84..fe4d1b70343 100644 --- a/dbms/src/Flash/Coprocessor/CHBlockChunkCodecStream.h +++ b/dbms/src/Flash/Coprocessor/CHBlockChunkCodecStream.h @@ -25,7 +25,10 @@ #include #include - +namespace mpp +{ +enum CompressionMode : int; +} namespace DB { class CHBlockChunkCodecStream : public ChunkCodecStream @@ -45,7 +48,7 @@ class CHBlockChunkCodecStream : public ChunkCodecStream size_t GetExtraInfoSize(const Block & block); size_t ApproxBlockBytes(const Block & block); -CompressionMethod ToInternalCompressionMethod(mpp::CompressionMode compress_method); +CompressionMethod ToInternalCompressionMethod(mpp::CompressionMode); std::unique_ptr NewCHBlockChunkCodecStream(const std::vector & field_types); void EncodeCHBlockChunk(WriteBuffer * ostr_ptr, const Block & block); diff --git a/dbms/src/Flash/Coprocessor/CompressCHBlockChunkCodecStream.cpp b/dbms/src/Flash/Coprocessor/CompressCHBlockChunkCodecStream.cpp index 6ba28c57d87..92452268608 100644 --- a/dbms/src/Flash/Coprocessor/CompressCHBlockChunkCodecStream.cpp +++ b/dbms/src/Flash/Coprocessor/CompressCHBlockChunkCodecStream.cpp @@ -32,9 +32,9 @@ void EncodeColumn__(WriteBuffer & ostr, const ColumnPtr & column, const ColumnWi WriteColumnData(*type_name.type, column, ostr, 0, 0); } -std::unique_ptr NewCompressCHBlockChunkCodecStream(CompressionMethod compress_method) +std::unique_ptr NewCompressCHBlockChunkCodecStream(CompressionMethod compression_method) { - return std::make_unique(compress_method); + return std::make_unique(compression_method); } Block DecodeHeader(ReadBuffer & istr, const Block & header, size_t & total_rows) diff --git a/dbms/src/Flash/Coprocessor/CompressCHBlockChunkCodecStream.h b/dbms/src/Flash/Coprocessor/CompressCHBlockChunkCodecStream.h index 341dce05d6b..be27090e224 100644 --- a/dbms/src/Flash/Coprocessor/CompressCHBlockChunkCodecStream.h +++ b/dbms/src/Flash/Coprocessor/CompressCHBlockChunkCodecStream.h @@ -74,13 +74,13 @@ struct WriteBufferFromOwnStringList final struct CompressCHBlockChunkCodecStream { - explicit CompressCHBlockChunkCodecStream(CompressionMethod compress_method_) - : compress_method(compress_method_) + explicit CompressCHBlockChunkCodecStream(CompressionMethod compression_method_) + : compression_method(compression_method_) { output_buffer = std::make_unique(); compress_write_buffer = std::make_unique( *output_buffer, - CompressionSettings(compress_method), + CompressionSettings(compression_method), DBMS_DEFAULT_BUFFER_SIZE); } @@ -112,7 +112,7 @@ struct CompressCHBlockChunkCodecStream ~CompressCHBlockChunkCodecStream() = default; // bool enable_compress{true}; - CompressionMethod compress_method; + CompressionMethod compression_method; std::unique_ptr output_buffer{}; std::unique_ptr compress_write_buffer{}; }; @@ -122,5 +122,5 @@ void EncodeColumn__(WriteBuffer & ostr, const ColumnPtr & column, const ColumnWi void DecodeColumns(ReadBuffer & istr, Block & res, size_t rows_to_read, size_t reserve_size = 0); Block DecodeHeader(ReadBuffer & istr, const Block & header, size_t & rows); -std::unique_ptr NewCompressCHBlockChunkCodecStream(CompressionMethod compress_method); +std::unique_ptr NewCompressCHBlockChunkCodecStream(CompressionMethod compression_method); } // namespace DB \ No newline at end of file diff --git a/dbms/src/Flash/Mpp/HashPartitionWriter.cpp b/dbms/src/Flash/Mpp/HashPartitionWriter.cpp index 5c3e3c84f4f..5ea23322118 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriter.cpp +++ b/dbms/src/Flash/Mpp/HashPartitionWriter.cpp @@ -52,11 +52,6 @@ HashPartitionWriter::HashPartitionWriter( partition_num = writer_->getPartitionNum(); RUNTIME_CHECK(partition_num > 0); RUNTIME_CHECK(dag_context.encode_type == tipb::EncodeType::TypeCHBlock); - - // if (auto method = ToInternalCompressionMethod(compress_method); method != CompressionMethod::NONE) - // { - // compress_chunk_codec_stream = CompressedCHBlockChunkCodec::newCodecStream(dag_context.result_field_types, method); - // } chunk_codec_stream = std::make_unique()->newCodecStream(dag_context.result_field_types); } diff --git a/dbms/src/Flash/Mpp/HashPartitionWriter.h b/dbms/src/Flash/Mpp/HashPartitionWriter.h index 81cf9323e42..e2d18efb4f4 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriter.h +++ b/dbms/src/Flash/Mpp/HashPartitionWriter.h @@ -56,9 +56,7 @@ class HashPartitionWriter : public DAGResponseWriter size_t rows_in_blocks; uint16_t partition_num; - // mpp::CompressionMode compress_method{}; std::unique_ptr chunk_codec_stream; - // std::unique_ptr compress_chunk_codec_stream; }; } // namespace DB diff --git a/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp b/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp index 7639d1ef0bc..ad0cd096ac3 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp +++ b/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp @@ -46,7 +46,7 @@ HashPartitionWriterImplV1::HashPartitionWriterImplV1( Int64 partition_batch_limit_, bool should_send_exec_summary_at_last_, DAGContext & dag_context_, - mpp::CompressionMode compress_method_) + mpp::CompressionMode compression_mode_) : DAGResponseWriter(/*records_per_chunk=*/-1, dag_context_) , partition_num(writer_->getPartitionNum()) , partition_batch_limit(partition_batch_limit_ * partition_num) @@ -54,7 +54,7 @@ HashPartitionWriterImplV1::HashPartitionWriterImplV1( , writer(writer_) , partition_col_ids(std::move(partition_col_ids_)) , collators(std::move(collators_)) - , compress_method(compress_method_) + , compression_mode(compression_mode_) { assert(dag_context.getMPPTaskMeta().mpp_version() > 0); @@ -123,7 +123,7 @@ void HashPartitionWriterImplV1::partitionAndEncodeThenWriteBl // Do NOT enable data compression when using local tunnel for (size_t part_id = 0; part_id < partition_num; ++part_id) { - auto mode = writer->isLocal(part_id) ? mpp::CompressionMode::NONE : compress_method; + auto mode = writer->isLocal(part_id) ? mpp::CompressionMode::NONE : compression_mode; tracked_packets[part_id]->getPacket().mutable_compress()->set_mode(mode); } @@ -195,7 +195,7 @@ void HashPartitionWriterImplV1::partitionAndEncodeThenWriteBl { compress_codec = std::make_unique( *output_buffer, - CompressionSettings(ToInternalCompressionMethod(compress_method)), + CompressionSettings(ToInternalCompressionMethod(compression_mode)), init_size); ostr_ptr = compress_codec.get(); } @@ -252,7 +252,7 @@ void HashPartitionWriterImplV1::partitionAndEncodeThenWriteBl // Do NOT enable data compression when using local tunnel for (size_t part_id = 0; part_id < partition_num; ++part_id) { - auto mode = writer->isLocal(part_id) ? mpp::CompressionMode::NONE : compress_method; + auto mode = writer->isLocal(part_id) ? mpp::CompressionMode::NONE : compression_mode; tracked_packets[part_id]->getPacket().mutable_compress()->set_mode(mode); } @@ -323,7 +323,7 @@ void HashPartitionWriterImplV1::partitionAndEncodeThenWriteBl { compress_codec = std::make_unique( *output_buffer, - CompressionSettings(ToInternalCompressionMethod(compress_method)), + CompressionSettings(ToInternalCompressionMethod(compression_mode)), init_size); ostr_ptr = compress_codec.get(); } diff --git a/dbms/src/Flash/Mpp/HashPartitionWriterV1.h b/dbms/src/Flash/Mpp/HashPartitionWriterV1.h index ff6125f21ac..4cd96b18ea9 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriterV1.h +++ b/dbms/src/Flash/Mpp/HashPartitionWriterV1.h @@ -45,7 +45,7 @@ class HashPartitionWriterImplV1 : public DAGResponseWriter Int64 partition_batch_limit_, bool should_send_exec_summary_at_last, DAGContext & dag_context_, - mpp::CompressionMode compress_method_); + mpp::CompressionMode compression_mode_); void write(const Block & block) override; void flush() override; void finishWrite() override; @@ -68,7 +68,7 @@ class HashPartitionWriterImplV1 : public DAGResponseWriter TiDB::TiDBCollators collators; size_t rows_in_blocks; DataTypes expected_types; - mpp::CompressionMode compress_method{}; + mpp::CompressionMode compression_mode{}; }; } // namespace DB diff --git a/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp b/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp index 32c05646558..2486db8faac 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp +++ b/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp @@ -151,7 +151,7 @@ std::unique_ptr NewMPPExchangeWriter( } else { - auto && compress_method = dag_context.getExchangeSenderMeta().compress(); + auto && compression_mode = dag_context.getExchangeSenderMeta().compress(); if (TiDB::MppVersion::MppVersionV0 == dag_context.getMPPTaskMeta().mpp_version()) return std::make_unique>( writer, @@ -167,7 +167,7 @@ std::unique_ptr NewMPPExchangeWriter( 8192, should_send_exec_summary_at_last, dag_context, - compress_method); + compression_mode); } } else From 0478c3fcf8a07d39673b699fd3a8e0cc54daf155 Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Fri, 6 Jan 2023 00:18:33 +0800 Subject: [PATCH 45/93] 39 --- .../Flash/Coprocessor/CHBlockChunkCodec.cpp | 15 ------------ .../Coprocessor/CHBlockChunkCodecStream.h | 1 - dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp | 24 +++++++++++++++---- dbms/src/Flash/Mpp/HashPartitionWriterV1.h | 2 ++ 4 files changed, 22 insertions(+), 20 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/CHBlockChunkCodec.cpp b/dbms/src/Flash/Coprocessor/CHBlockChunkCodec.cpp index 519526eaa59..938bf71618c 100644 --- a/dbms/src/Flash/Coprocessor/CHBlockChunkCodec.cpp +++ b/dbms/src/Flash/Coprocessor/CHBlockChunkCodec.cpp @@ -98,21 +98,6 @@ size_t ApproxBlockBytes(const Block & block) return block.bytes() + GetExtraInfoSize(block); } -CompressionMethod ToInternalCompressionMethod(mpp::CompressionMode compression_mode) -{ - switch (compression_mode) - { - case mpp::CompressionMode::NONE: - return CompressionMethod::NONE; - case mpp::CompressionMode::FAST: - return CompressionMethod::LZ4; // use LZ4 method as fast mode - case mpp::CompressionMode::HIGH_COMPRESSION: - return CompressionMethod::ZSTD; // use ZSTD method as HC mode - default: - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unkown compresssion mode {}", mpp::CompressionMode_Name(compression_mode)); - } -} - void WriteColumnData(const IDataType & type, const ColumnPtr & column, WriteBuffer & ostr, size_t offset, size_t limit) { /** If there are columns-constants - then we materialize them. diff --git a/dbms/src/Flash/Coprocessor/CHBlockChunkCodecStream.h b/dbms/src/Flash/Coprocessor/CHBlockChunkCodecStream.h index fe4d1b70343..a9e81a530de 100644 --- a/dbms/src/Flash/Coprocessor/CHBlockChunkCodecStream.h +++ b/dbms/src/Flash/Coprocessor/CHBlockChunkCodecStream.h @@ -48,7 +48,6 @@ class CHBlockChunkCodecStream : public ChunkCodecStream size_t GetExtraInfoSize(const Block & block); size_t ApproxBlockBytes(const Block & block); -CompressionMethod ToInternalCompressionMethod(mpp::CompressionMode); std::unique_ptr NewCHBlockChunkCodecStream(const std::vector & field_types); void EncodeCHBlockChunk(WriteBuffer * ostr_ptr, const Block & block); diff --git a/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp b/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp index ad0cd096ac3..0a0591f40f0 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp +++ b/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp @@ -38,6 +38,22 @@ namespace DB { + +CompressionMethod ToInternalCompressionMethod(mpp::CompressionMode compression_mode) +{ + switch (compression_mode) + { + case mpp::CompressionMode::NONE: + return CompressionMethod::NONE; + case mpp::CompressionMode::FAST: + return CompressionMethod::LZ4; // use LZ4 method as fast mode + case mpp::CompressionMode::HIGH_COMPRESSION: + return CompressionMethod::ZSTD; // use ZSTD method as HC mode + default: + return CompressionMethod::NONE; + } +} + template HashPartitionWriterImplV1::HashPartitionWriterImplV1( ExchangeWriterPtr writer_, @@ -366,9 +382,9 @@ void HashPartitionWriterImplV1::partitionAndEncodeThenWriteBl static void updateHashPartitionWriterMetrics(mpp::CompressionMode mode, size_t sz, bool is_local) { - switch (mode) + switch (ToInternalCompressionMethod(mode)) { - case mpp::CompressionMode::NONE: + case CompressionMethod::NONE: { if (is_local) { @@ -380,12 +396,12 @@ static void updateHashPartitionWriterMetrics(mpp::CompressionMode mode, size_t s } break; } - case mpp::CompressionMode::FAST: + case CompressionMethod::LZ4: { GET_METRIC(tiflash_exchange_data_bytes, type_hash_lz4).Increment(sz); break; } - case mpp::CompressionMode::HIGH_COMPRESSION: + case CompressionMethod::ZSTD: { GET_METRIC(tiflash_exchange_data_bytes, type_hash_zstd).Increment(sz); break; diff --git a/dbms/src/Flash/Mpp/HashPartitionWriterV1.h b/dbms/src/Flash/Mpp/HashPartitionWriterV1.h index 4cd96b18ea9..6b6bf50d2a4 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriterV1.h +++ b/dbms/src/Flash/Mpp/HashPartitionWriterV1.h @@ -71,4 +71,6 @@ class HashPartitionWriterImplV1 : public DAGResponseWriter mpp::CompressionMode compression_mode{}; }; +CompressionMethod ToInternalCompressionMethod(mpp::CompressionMode compression_mode); + } // namespace DB From f3ad485e19d308d11269f1cba50a355c81248db9 Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Fri, 6 Jan 2023 09:52:40 +0800 Subject: [PATCH 46/93] 40 Signed-off-by: Zhigao Tong --- contrib/CMakeLists.txt | 2 +- contrib/tzg.mpp.proto.patch | 158 +++++++++--------- dbms/src/Flash/Mpp/ExchangeReceiver.cpp | 2 +- dbms/src/Flash/Mpp/HashPartitionWriter.cpp | 2 +- dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp | 10 +- .../Mpp/tests/gtest_mpp_exchange_writer.cpp | 6 +- .../Planner/plans/PhysicalExchangeSender.cpp | 8 +- 7 files changed, 94 insertions(+), 94 deletions(-) diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 4400614f887..a31bbb3c2a2 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -112,7 +112,7 @@ if (USE_INTERNAL_GRPC_LIBRARY) endif () execute_process( - COMMAND grep "CompressionMode compress" "${TiFlash_SOURCE_DIR}/contrib/kvproto/proto/mpp.proto" + COMMAND grep "CompressionMode compression" "${TiFlash_SOURCE_DIR}/contrib/kvproto/proto/mpp.proto" RESULT_VARIABLE HAVE_APPLY_PATCH) # grep - Normally, the exit status is 0 if selected lines are found and 1 otherwise. But the exit status is 2 if an error occurred. diff --git a/contrib/tzg.mpp.proto.patch b/contrib/tzg.mpp.proto.patch index 74c4b08831d..5d4ba9ea7ed 100644 --- a/contrib/tzg.mpp.proto.patch +++ b/contrib/tzg.mpp.proto.patch @@ -1,5 +1,5 @@ diff --git a/pkg/mpp/mpp.pb.go b/pkg/mpp/mpp.pb.go -index fcd4691..35cdbf2 100644 +index fcd4691..f48ae58 100644 --- a/pkg/mpp/mpp.pb.go +++ b/pkg/mpp/mpp.pb.go @@ -26,6 +26,82 @@ var _ = math.Inf @@ -36,7 +36,7 @@ index fcd4691..35cdbf2 100644 +} + +type ExchangeSenderMeta struct { -+ Compress CompressionMode `protobuf:"varint,1,opt,name=compress,proto3,enum=mpp.CompressionMode" json:"compress,omitempty"` ++ Compression CompressionMode `protobuf:"varint,1,opt,name=compression,proto3,enum=mpp.CompressionMode" json:"compression,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` @@ -75,9 +75,9 @@ index fcd4691..35cdbf2 100644 + +var xxx_messageInfo_ExchangeSenderMeta proto.InternalMessageInfo + -+func (m *ExchangeSenderMeta) GetCompress() CompressionMode { ++func (m *ExchangeSenderMeta) GetCompression() CompressionMode { + if m != nil { -+ return m.Compress ++ return m.Compression + } + return CompressionMode_NONE +} @@ -297,7 +297,7 @@ index fcd4691..35cdbf2 100644 + Chunks [][]byte `protobuf:"bytes,3,rep,name=chunks,proto3" json:"chunks,omitempty"` + StreamIds []uint64 `protobuf:"varint,4,rep,packed,name=stream_ids,json=streamIds,proto3" json:"stream_ids,omitempty"` + Version int64 `protobuf:"varint,5,opt,name=version,proto3" json:"version,omitempty"` -+ Compress *DataPacketCompressionMeta `protobuf:"bytes,6,opt,name=compress,proto3" json:"compress,omitempty"` ++ Compression *DataPacketCompressionMeta `protobuf:"bytes,6,opt,name=compression,proto3" json:"compression,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` @@ -323,9 +323,9 @@ index fcd4691..35cdbf2 100644 + return 0 +} + -+func (m *MPPDataPacket) GetCompress() *DataPacketCompressionMeta { ++func (m *MPPDataPacket) GetCompression() *DataPacketCompressionMeta { + if m != nil { -+ return m.Compress ++ return m.Compression + } + return nil +} @@ -418,60 +418,60 @@ index fcd4691..35cdbf2 100644 - 0xb1, 0xf3, 0xf5, 0x76, 0xec, 0x7c, 0xfa, 0x36, 0xfe, 0x07, 0xb6, 0xb9, 0x58, 0x84, 0x2a, 0x5d, - 0x55, 0xe1, 0xaa, 0x32, 0x5f, 0xa7, 0xab, 0xae, 0xf9, 0x39, 0xfd, 0x11, 0x00, 0x00, 0xff, 0xff, - 0xcd, 0x41, 0xdb, 0xe2, 0xed, 0x04, 0x00, 0x00, -+ // 844 bytes of a gzipped FileDescriptorProto -+ 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x55, 0xcd, 0x6e, 0x1b, 0x37, -+ 0x10, 0xce, 0x6a, 0x65, 0xfd, 0x8c, 0xd6, 0x8e, 0xc2, 0x0a, 0xb5, 0x9c, 0x36, 0xaa, 0xb2, 0xe8, -+ 0x41, 0xe8, 0x41, 0x6d, 0x1d, 0xa0, 0x87, 0x16, 0x28, 0x90, 0x3a, 0x4a, 0xb3, 0x07, 0xd9, 0x2a, -+ 0x2d, 0x04, 0xbd, 0x2d, 0xe8, 0x25, 0x2b, 0x2d, 0xa4, 0x5d, 0x6e, 0x48, 0x5a, 0x68, 0xce, 0x7d, -+ 0x88, 0xf6, 0x91, 0x7a, 0x2c, 0xf2, 0x04, 0x85, 0xfb, 0x22, 0x05, 0x87, 0xbb, 0xb2, 0x22, 0x27, -+ 0xfd, 0x39, 0x2d, 0xe7, 0xe3, 0x70, 0xe6, 0x9b, 0x6f, 0xc8, 0x59, 0x68, 0x67, 0x45, 0x31, 0x2e, -+ 0x94, 0x34, 0x92, 0xf8, 0x59, 0x51, 0x3c, 0xec, 0x2d, 0xe4, 0x42, 0xa2, 0xfd, 0xb9, 0x5d, 0xb9, -+ 0xad, 0x87, 0x0f, 0x12, 0x8b, 0x25, 0x42, 0x6b, 0xa9, 0x4a, 0x28, 0xc8, 0x84, 0x61, 0xc5, 0x95, -+ 0xb3, 0xc2, 0xe7, 0x40, 0x26, 0x3f, 0x27, 0x4b, 0x96, 0x2f, 0xc4, 0xa5, 0xc8, 0xb9, 0x50, 0x53, -+ 0x61, 0x18, 0xf9, 0x02, 0x5a, 0x89, 0xcc, 0x0a, 0x25, 0xb4, 0xee, 0x7b, 0x43, 0x6f, 0x74, 0x74, -+ 0xda, 0x1b, 0xdb, 0x7c, 0x67, 0x25, 0x98, 0xca, 0x7c, 0x2a, 0xb9, 0xa0, 0x5b, 0xaf, 0xf0, 0xd7, -+ 0x1a, 0xb4, 0xe6, 0x4c, 0xaf, 0xf0, 0xf8, 0x09, 0xb4, 0xb4, 0x61, 0xca, 0xc4, 0xc6, 0x1d, 0xaf, -+ 0xd3, 0x26, 0xda, 0x73, 0x4d, 0x8e, 0xa1, 0x69, 0x98, 0x5e, 0xc5, 0x29, 0xef, 0xd7, 0x86, 0xde, -+ 0xc8, 0xa7, 0x0d, 0x6b, 0x46, 0x9c, 0x3c, 0x86, 0xa0, 0x60, 0xca, 0xa4, 0x26, 0x95, 0xb9, 0xdd, -+ 0xf5, 0x71, 0xb7, 0xb3, 0xc5, 0x22, 0x4e, 0xfa, 0xd0, 0x64, 0x9c, 0x23, 0xa9, 0xfa, 0xd0, 0x1b, -+ 0xb5, 0x69, 0x65, 0x92, 0x8f, 0xa0, 0xbd, 0x60, 0x66, 0x29, 0x94, 0x3d, 0x79, 0x80, 0x19, 0x5b, -+ 0x0e, 0x88, 0xb8, 0x65, 0xf3, 0xea, 0x5a, 0xa8, 0xd7, 0x96, 0x4d, 0xc3, 0xb1, 0x41, 0x7b, 0xae, -+ 0xc9, 0xa7, 0x70, 0xb4, 0x96, 0x09, 0x5b, 0xc7, 0xce, 0x21, 0xe5, 0xfd, 0x26, 0x3a, 0x04, 0x88, -+ 0xfe, 0x60, 0xc1, 0x88, 0xdb, 0xe8, 0x5a, 0xa8, 0x8d, 0x8b, 0xde, 0x72, 0xd1, 0x1d, 0x10, 0x71, -+ 0xf2, 0x09, 0x74, 0xb2, 0xa2, 0x88, 0x37, 0x42, 0x59, 0x55, 0xfa, 0x6d, 0xa4, 0x0d, 0x59, 0x51, -+ 0xbc, 0x74, 0x48, 0xd8, 0x85, 0xa3, 0x48, 0x3f, 0x5d, 0xa7, 0x1b, 0x41, 0xc5, 0xab, 0x6b, 0xa1, -+ 0x4d, 0x38, 0x83, 0xfb, 0x5b, 0x44, 0x17, 0x32, 0xd7, 0x82, 0x7c, 0x0c, 0x6d, 0xb6, 0x61, 0xe9, -+ 0x9a, 0x5d, 0xad, 0x05, 0x4a, 0xd6, 0xa2, 0xb7, 0xc0, 0x7e, 0x8e, 0xda, 0x9d, 0x1c, 0x6f, 0x6a, -+ 0xf0, 0xc1, 0xb3, 0x54, 0x17, 0xcc, 0x24, 0x4b, 0xdb, 0x85, 0x32, 0x13, 0x79, 0x0c, 0x75, 0xdb, -+ 0x6d, 0x8c, 0xd8, 0x39, 0x3d, 0xc4, 0x1e, 0x56, 0x5d, 0xa2, 0xb8, 0x65, 0x75, 0x17, 0x79, 0x22, -+ 0xb9, 0xe0, 0x71, 0xb1, 0x66, 0x2e, 0x78, 0x40, 0x3b, 0x25, 0x36, 0x5b, 0xb3, 0xdc, 0xea, 0x6e, -+ 0xd2, 0x4c, 0xc8, 0x6b, 0x53, 0x76, 0xa5, 0x32, 0xc9, 0x97, 0xd0, 0x54, 0x62, 0x91, 0xca, 0xdc, -+ 0x76, 0xc4, 0x1f, 0x75, 0x4e, 0x8f, 0xc7, 0xbb, 0x17, 0x8e, 0xe2, 0x5e, 0x94, 0xff, 0x24, 0x69, -+ 0xe5, 0x47, 0x1e, 0x01, 0xe8, 0x64, 0x29, 0x32, 0x66, 0xcb, 0xc1, 0x5e, 0xf9, 0xb4, 0xed, 0x90, -+ 0x97, 0x42, 0x91, 0x6f, 0xe1, 0xd0, 0xd8, 0x9a, 0xe3, 0x2a, 0x6e, 0x03, 0xe3, 0x9e, 0xbc, 0x15, -+ 0x77, 0x6e, 0x3d, 0x5c, 0x70, 0x4d, 0x03, 0xb3, 0x63, 0x91, 0x08, 0x7a, 0xa2, 0xbc, 0xcf, 0xb1, -+ 0xc6, 0x0b, 0x1d, 0xa3, 0x02, 0x4d, 0x54, 0xe0, 0x18, 0x15, 0xb8, 0x7b, 0xe1, 0x29, 0x11, 0x77, -+ 0xb0, 0x30, 0x83, 0xde, 0xdb, 0x9a, 0x96, 0xbd, 0x1a, 0xc2, 0x81, 0x50, 0x4a, 0xaa, 0x52, 0x55, -+ 0x70, 0x31, 0x2d, 0x42, 0xdd, 0x06, 0x79, 0x02, 0x87, 0x4a, 0x18, 0xf5, 0x7a, 0x5b, 0x44, 0x0d, -+ 0x8b, 0x38, 0x1a, 0x97, 0x4f, 0xcf, 0x91, 0xa5, 0x01, 0x3a, 0x95, 0xcc, 0xc3, 0x1f, 0xe1, 0xc1, -+ 0x19, 0xcb, 0x13, 0xb1, 0xfe, 0x9f, 0x0d, 0xdc, 0xd2, 0xa9, 0xbd, 0x87, 0x4e, 0xf8, 0x15, 0x90, -+ 0xdd, 0xc8, 0xff, 0xb5, 0x8c, 0xf0, 0x17, 0x0f, 0x1e, 0x4d, 0xb4, 0x95, 0x37, 0xd5, 0xcb, 0xe9, -+ 0x6c, 0x76, 0x26, 0xf3, 0x5c, 0x24, 0xf6, 0x31, 0x56, 0xf4, 0xc6, 0xd0, 0xd9, 0x15, 0xf9, 0x9d, -+ 0x2c, 0x41, 0xdf, 0xce, 0x95, 0x53, 0x2b, 0x4c, 0x22, 0xd2, 0x4d, 0x75, 0xa2, 0xf6, 0xae, 0x13, -+ 0x41, 0xe5, 0x83, 0x6d, 0x98, 0xc0, 0xc9, 0x33, 0x66, 0xd8, 0x8c, 0x25, 0x2b, 0x61, 0x76, 0x07, -+ 0x90, 0x0d, 0x38, 0x82, 0x7a, 0x26, 0xb9, 0xf8, 0xc7, 0x21, 0x85, 0x1e, 0xe1, 0x1b, 0x0f, 0x0e, -+ 0xa7, 0xb3, 0xd9, 0x6d, 0x28, 0x42, 0xa0, 0xce, 0x59, 0xc9, 0x3a, 0xa0, 0xb8, 0xfe, 0x77, 0x31, -+ 0xc9, 0x87, 0xd0, 0x48, 0x96, 0xd7, 0xf9, 0x4a, 0xf7, 0xfd, 0xa1, 0x3f, 0x0a, 0x68, 0x69, 0xe1, -+ 0xbd, 0x36, 0x4a, 0xb0, 0x2c, 0x4e, 0xb9, 0x7b, 0x0d, 0x75, 0xda, 0x76, 0x48, 0xc4, 0xb5, 0x7d, -+ 0x43, 0xd5, 0xf3, 0x75, 0x77, 0xbe, 0x32, 0xc9, 0xd7, 0x3b, 0xb3, 0xb6, 0x81, 0x59, 0x07, 0x98, -+ 0xf5, 0xbd, 0x45, 0xef, 0x4c, 0xdd, 0x73, 0x38, 0x40, 0x72, 0xb6, 0x96, 0xa4, 0xd2, 0xe1, 0x80, -+ 0xe2, 0x9a, 0x74, 0xc1, 0xcf, 0xf4, 0x02, 0x2b, 0x69, 0x53, 0xbb, 0xdc, 0x9f, 0x23, 0xfe, 0xfe, -+ 0x1c, 0xf9, 0xec, 0x1b, 0xb8, 0xbf, 0xa7, 0x1e, 0x69, 0x41, 0xfd, 0xfc, 0xe2, 0x7c, 0xd2, 0xbd, -+ 0x67, 0x57, 0xcf, 0x9f, 0x5e, 0xce, 0xbb, 0x1e, 0xe9, 0x41, 0xf7, 0x45, 0xf4, 0xfd, 0x8b, 0xf8, -+ 0xec, 0x62, 0x3a, 0xa3, 0x93, 0xcb, 0xcb, 0xe8, 0xe2, 0xbc, 0x5b, 0xfb, 0x2e, 0xfc, 0xfd, 0x66, -+ 0xe0, 0xfd, 0x71, 0x33, 0xf0, 0xfe, 0xbc, 0x19, 0x78, 0xbf, 0xfd, 0x35, 0xb8, 0x07, 0x5d, 0xa9, -+ 0x16, 0x63, 0x93, 0xae, 0x36, 0xe3, 0xd5, 0x06, 0x7f, 0x37, 0x57, 0x0d, 0xfc, 0x3c, 0xf9, 0x3b, -+ 0x00, 0x00, 0xff, 0xff, 0x40, 0xa5, 0xdf, 0xfa, 0xbe, 0x06, 0x00, 0x00, ++ // 846 bytes of a gzipped FileDescriptorProto ++ 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x55, 0xcd, 0x6e, 0x23, 0x45, ++ 0x10, 0xde, 0xf1, 0x38, 0xfe, 0x29, 0x4f, 0xb2, 0xde, 0xc6, 0x22, 0xce, 0xc2, 0x1a, 0xef, 0x88, ++ 0x83, 0xc5, 0xc1, 0x88, 0xac, 0xb4, 0x17, 0x24, 0xc4, 0x92, 0x35, 0xec, 0x48, 0x38, 0x31, 0x1d, ++ 0x6b, 0xc5, 0x6d, 0xd4, 0x99, 0x6e, 0xec, 0x91, 0x3d, 0xd3, 0xb3, 0xdd, 0x1d, 0x8b, 0x3d, 0xf3, ++ 0x10, 0xf0, 0x48, 0x1c, 0x11, 0x27, 0x8e, 0x28, 0xbc, 0x08, 0xea, 0xea, 0x99, 0xc4, 0x71, 0x36, ++ 0xfc, 0x9c, 0xdc, 0xf5, 0x75, 0xcd, 0x57, 0x55, 0x5f, 0x55, 0x97, 0xa1, 0x9d, 0x15, 0xc5, 0xb8, ++ 0x50, 0xd2, 0x48, 0xe2, 0x67, 0x45, 0xf1, 0xb8, 0xb7, 0x90, 0x0b, 0x89, 0xf6, 0xa7, 0xf6, 0xe4, ++ 0xae, 0x1e, 0x3f, 0x4a, 0x2c, 0x96, 0x08, 0xad, 0xa5, 0x2a, 0xa1, 0x20, 0x13, 0x86, 0x15, 0x17, ++ 0xce, 0x0a, 0xbf, 0x05, 0x32, 0xf9, 0x31, 0x59, 0xb2, 0x7c, 0x21, 0xce, 0x45, 0xce, 0x85, 0x9a, ++ 0x0a, 0xc3, 0xc8, 0x73, 0xe8, 0x24, 0x32, 0x2b, 0x94, 0xd0, 0x3a, 0x95, 0x79, 0xdf, 0x1b, 0x7a, ++ 0xa3, 0x83, 0xe3, 0xde, 0xd8, 0x86, 0x3c, 0xb9, 0xc1, 0xa7, 0x92, 0x0b, 0xba, 0xed, 0x18, 0xfe, ++ 0x5c, 0x83, 0xd6, 0x9c, 0xe9, 0x15, 0x92, 0x1c, 0x41, 0x4b, 0x1b, 0xa6, 0x4c, 0x6c, 0x34, 0x32, ++ 0xd4, 0x69, 0x13, 0xed, 0xb9, 0x26, 0x87, 0xd0, 0x34, 0x4c, 0xaf, 0xe2, 0x94, 0xf7, 0x6b, 0x43, ++ 0x6f, 0xe4, 0xd3, 0x86, 0x35, 0x23, 0x4e, 0x9e, 0x42, 0x50, 0x30, 0x65, 0x52, 0x93, 0xca, 0xdc, ++ 0xde, 0xfa, 0x78, 0xdb, 0xb9, 0xc6, 0x22, 0x4e, 0xfa, 0xd0, 0x64, 0x9c, 0xdb, 0x88, 0xfd, 0xfa, ++ 0xd0, 0x1b, 0xb5, 0x69, 0x65, 0x92, 0x0f, 0xa0, 0xbd, 0x60, 0x66, 0x29, 0x94, 0xfd, 0x72, 0x0f, ++ 0x23, 0xb6, 0x1c, 0x10, 0x71, 0x9b, 0xcd, 0x9b, 0x4b, 0xa1, 0xde, 0xda, 0x6c, 0x1a, 0x2e, 0x1b, ++ 0xb4, 0xe7, 0x9a, 0x7c, 0x0c, 0x07, 0x6b, 0x99, 0xb0, 0x75, 0xec, 0x1c, 0x52, 0xde, 0x6f, 0xa2, ++ 0x43, 0x80, 0xe8, 0x77, 0x16, 0x8c, 0xb8, 0x65, 0xd7, 0x42, 0x6d, 0x1c, 0x7b, 0xcb, 0xb1, 0x3b, ++ 0x20, 0xe2, 0xe4, 0x23, 0xe8, 0x64, 0x45, 0x11, 0x6f, 0x84, 0x42, 0xc1, 0xda, 0x98, 0x36, 0x64, ++ 0x45, 0xf1, 0xda, 0x21, 0x61, 0x17, 0x0e, 0x22, 0xfd, 0x62, 0x9d, 0x6e, 0x04, 0x15, 0x6f, 0x2e, ++ 0x85, 0x36, 0xe1, 0x0c, 0x1e, 0x5e, 0x23, 0xba, 0x90, 0xb9, 0x16, 0xe4, 0x43, 0x68, 0xb3, 0x0d, ++ 0x4b, 0xd7, 0xec, 0x62, 0x2d, 0x50, 0xb2, 0x16, 0xbd, 0x01, 0x76, 0x63, 0xd4, 0xee, 0xc4, 0xf8, ++ 0xbd, 0x06, 0xef, 0xbd, 0x4c, 0x75, 0xc1, 0x4c, 0xb2, 0xb4, 0x5d, 0x28, 0x23, 0x91, 0xa7, 0x50, ++ 0xb7, 0x3d, 0x47, 0xc6, 0xce, 0xf1, 0x3e, 0xb6, 0xb1, 0xea, 0x12, 0xc5, 0x2b, 0xab, 0xbb, 0xc8, ++ 0x13, 0xc9, 0x05, 0x8f, 0x8b, 0x35, 0x73, 0xe4, 0x01, 0xed, 0x94, 0xd8, 0x6c, 0xcd, 0x72, 0xab, ++ 0xbb, 0x49, 0x33, 0x21, 0x2f, 0x4d, 0xd9, 0x95, 0xca, 0x24, 0x9f, 0x41, 0x53, 0x89, 0x45, 0x2a, ++ 0x73, 0xdb, 0x11, 0x7f, 0xd4, 0x39, 0x3e, 0x1c, 0x6f, 0x8f, 0x1d, 0xc5, 0xbb, 0x28, 0xff, 0x41, ++ 0xd2, 0xca, 0x8f, 0x3c, 0x01, 0xd0, 0xc9, 0x52, 0x64, 0xcc, 0x96, 0x83, 0xbd, 0xf2, 0x69, 0xdb, ++ 0x21, 0xaf, 0x85, 0x22, 0x5f, 0xc0, 0xbe, 0xb1, 0x35, 0xc7, 0x15, 0x6f, 0x03, 0x79, 0x8f, 0x6e, ++ 0xf1, 0xce, 0xad, 0x87, 0x23, 0xd7, 0x34, 0x30, 0x5b, 0x16, 0x89, 0xa0, 0x27, 0xca, 0xa9, 0x8e, ++ 0x35, 0x8e, 0x75, 0x8c, 0x0a, 0x34, 0x51, 0x81, 0x43, 0x54, 0xe0, 0xee, 0xd8, 0x53, 0x22, 0xee, ++ 0x60, 0x61, 0x06, 0xbd, 0xdb, 0x9a, 0x96, 0xbd, 0x1a, 0xc2, 0x9e, 0x50, 0x4a, 0xaa, 0x52, 0x55, ++ 0x70, 0x9c, 0x16, 0xa1, 0xee, 0x82, 0x3c, 0x83, 0x7d, 0x25, 0x8c, 0x7a, 0x7b, 0x5d, 0x44, 0x0d, ++ 0x8b, 0x38, 0x18, 0x97, 0x0f, 0xd0, 0x25, 0x4b, 0x03, 0x74, 0x2a, 0x33, 0x0f, 0xbf, 0x87, 0x47, ++ 0x27, 0x2c, 0x4f, 0xc4, 0xfa, 0x7f, 0x36, 0xf0, 0x3a, 0x9d, 0xda, 0x3d, 0xe9, 0x84, 0xcf, 0x81, ++ 0x6c, 0x33, 0xff, 0xd7, 0x32, 0xc2, 0x9f, 0x3c, 0x78, 0x32, 0xd1, 0x56, 0xde, 0x54, 0x2f, 0xa7, ++ 0xb3, 0xd9, 0x89, 0xcc, 0x73, 0x91, 0xd8, 0xc7, 0x58, 0xa5, 0x37, 0x86, 0xce, 0xb6, 0xc8, 0xef, ++ 0xcc, 0x12, 0xf4, 0xcd, 0x76, 0x39, 0xb6, 0xc2, 0x24, 0x22, 0xdd, 0x54, 0x5f, 0xd4, 0xde, 0xf5, ++ 0x45, 0x50, 0xf9, 0x60, 0x1b, 0x26, 0x70, 0xf4, 0x92, 0x19, 0x36, 0x63, 0xc9, 0x4a, 0x98, 0xed, ++ 0x1d, 0x64, 0x09, 0x47, 0x50, 0xcf, 0x24, 0x17, 0xff, 0xb8, 0xa7, 0xd0, 0x23, 0xfc, 0xc3, 0x83, ++ 0xfd, 0xe9, 0x6c, 0x76, 0x43, 0x45, 0x08, 0xd4, 0x39, 0x2b, 0xb3, 0x0e, 0x28, 0x9e, 0xff, 0x5d, ++ 0x4c, 0xf2, 0x3e, 0x34, 0x92, 0xe5, 0x65, 0xbe, 0xd2, 0x7d, 0x7f, 0xe8, 0x8f, 0x02, 0x5a, 0x5a, ++ 0x38, 0xd7, 0x46, 0x09, 0x96, 0xc5, 0x29, 0x77, 0xaf, 0xa1, 0x4e, 0xdb, 0x0e, 0x89, 0xb8, 0xb6, ++ 0x6f, 0xa8, 0x7a, 0xbe, 0x6e, 0xe6, 0x2b, 0x93, 0x7c, 0x79, 0x7b, 0xe3, 0x36, 0x30, 0xf0, 0x00, ++ 0x03, 0xdf, 0x5b, 0xf7, 0xed, 0xdd, 0x7b, 0x0a, 0x7b, 0x98, 0xa2, 0xad, 0x28, 0xa9, 0xd4, 0xd8, ++ 0xa3, 0x78, 0x26, 0x5d, 0xf0, 0x33, 0xbd, 0xc0, 0x7a, 0xda, 0xd4, 0x1e, 0x77, 0xb7, 0x89, 0xbf, ++ 0xbb, 0x4d, 0x3e, 0xf9, 0x1c, 0x1e, 0xee, 0x68, 0x48, 0x5a, 0x50, 0x3f, 0x3d, 0x3b, 0x9d, 0x74, ++ 0x1f, 0xd8, 0xd3, 0xd7, 0x2f, 0xce, 0xe7, 0x5d, 0x8f, 0xf4, 0xa0, 0xfb, 0x2a, 0xfa, 0xe6, 0x55, ++ 0x7c, 0x72, 0x36, 0x9d, 0xd1, 0xc9, 0xf9, 0x79, 0x74, 0x76, 0xda, 0xad, 0x7d, 0x15, 0xfe, 0x7a, ++ 0x35, 0xf0, 0x7e, 0xbb, 0x1a, 0x78, 0x7f, 0x5e, 0x0d, 0xbc, 0x5f, 0xfe, 0x1a, 0x3c, 0x80, 0xae, ++ 0x54, 0x8b, 0xb1, 0x49, 0x57, 0x9b, 0xf1, 0x6a, 0x83, 0x7f, 0x3d, 0x17, 0x0d, 0xfc, 0x79, 0xf6, ++ 0x77, 0x00, 0x00, 0x00, 0xff, 0xff, 0xd0, 0x35, 0x9b, 0x00, 0xca, 0x06, 0x00, 0x00, +} + +func (m *ExchangeSenderMeta) Marshal() (dAtA []byte, err error) { @@ -498,8 +498,8 @@ index fcd4691..35cdbf2 100644 + i -= len(m.XXX_unrecognized) + copy(dAtA[i:], m.XXX_unrecognized) + } -+ if m.Compress != 0 { -+ i = encodeVarintMpp(dAtA, i, uint64(m.Compress)) ++ if m.Compression != 0 { ++ i = encodeVarintMpp(dAtA, i, uint64(m.Compression)) + i-- + dAtA[i] = 0x8 + } @@ -593,9 +593,9 @@ index fcd4691..35cdbf2 100644 i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } -+ if m.Compress != nil { ++ if m.Compression != nil { + { -+ size, err := m.Compress.MarshalToSizedBuffer(dAtA[:i]) ++ size, err := m.Compression.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } @@ -659,8 +659,8 @@ index fcd4691..35cdbf2 100644 + } + var l int + _ = l -+ if m.Compress != 0 { -+ n += 1 + sovMpp(uint64(m.Compress)) ++ if m.Compression != 0 { ++ n += 1 + sovMpp(uint64(m.Compression)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) @@ -731,8 +731,8 @@ index fcd4691..35cdbf2 100644 + if m.Version != 0 { + n += 1 + sovMpp(uint64(m.Version)) + } -+ if m.Compress != nil { -+ l = m.Compress.Size() ++ if m.Compression != nil { ++ l = m.Compression.Size() + n += 1 + l + sovMpp(uint64(l)) + } if m.XXX_unrecognized != nil { @@ -783,9 +783,9 @@ index fcd4691..35cdbf2 100644 + switch fieldNum { + case 1: + if wireType != 0 { -+ return fmt.Errorf("proto: wrong wireType = %d for field Compress", wireType) ++ return fmt.Errorf("proto: wrong wireType = %d for field Compression", wireType) + } -+ m.Compress = 0 ++ m.Compression = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMpp @@ -795,7 +795,7 @@ index fcd4691..35cdbf2 100644 + } + b := dAtA[iNdEx] + iNdEx++ -+ m.Compress |= CompressionMode(b&0x7F) << shift ++ m.Compression |= CompressionMode(b&0x7F) << shift + if b < 0x80 { + break + } @@ -1022,7 +1022,7 @@ index fcd4691..35cdbf2 100644 + } + case 6: + if wireType != 2 { -+ return fmt.Errorf("proto: wrong wireType = %d for field Compress", wireType) ++ return fmt.Errorf("proto: wrong wireType = %d for field Compression", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { @@ -1049,10 +1049,10 @@ index fcd4691..35cdbf2 100644 + if postIndex > l { + return io.ErrUnexpectedEOF + } -+ if m.Compress == nil { -+ m.Compress = &DataPacketCompressionMeta{} ++ if m.Compression == nil { ++ m.Compression = &DataPacketCompressionMeta{} + } -+ if err := m.Compress.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { ++ if err := m.Compression.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex @@ -1086,7 +1086,7 @@ index fcd4691..35cdbf2 100644 iNdEx = preIndex skippy, err := skipMpp(dAtA[iNdEx:]) diff --git a/proto/mpp.proto b/proto/mpp.proto -index 6739455..9cb2b90 100644 +index 6739455..0943734 100644 --- a/proto/mpp.proto +++ b/proto/mpp.proto @@ -11,6 +11,17 @@ option (gogoproto.unmarshaler_all) = true; @@ -1101,7 +1101,7 @@ index 6739455..9cb2b90 100644 +} + +message ExchangeSenderMeta { -+ CompressionMode compress = 1; ++ CompressionMode compression = 1; +} + // TaskMeta contains meta of a mpp plan, including query's ts and task address. @@ -1148,7 +1148,7 @@ index 6739455..9cb2b90 100644 repeated bytes chunks = 3; repeated uint64 stream_ids = 4; + int64 version = 5; // version of data format -+ DataPacketCompressionMeta compress = 6; // data compression meta ++ DataPacketCompressionMeta compression = 6; // data compression meta } message Error { diff --git a/dbms/src/Flash/Mpp/ExchangeReceiver.cpp b/dbms/src/Flash/Mpp/ExchangeReceiver.cpp index b83a91aca94..532b2954325 100644 --- a/dbms/src/Flash/Mpp/ExchangeReceiver.cpp +++ b/dbms/src/Flash/Mpp/ExchangeReceiver.cpp @@ -722,7 +722,7 @@ DecodeDetail ExchangeReceiverBase::decodeChunks( for (auto && chunk : packet.chunks()) { - auto && result = decoder_ptr->decodeAndSquash(chunk, packet.compress().mode() != mpp::CompressionMode::NONE); + auto && result = decoder_ptr->decodeAndSquash(chunk, packet.compression().mode() != mpp::CompressionMode::NONE); if (!result || !result->rows()) continue; detail.rows += result->rows(); diff --git a/dbms/src/Flash/Mpp/HashPartitionWriter.cpp b/dbms/src/Flash/Mpp/HashPartitionWriter.cpp index 5ea23322118..73afe1de372 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriter.cpp +++ b/dbms/src/Flash/Mpp/HashPartitionWriter.cpp @@ -158,7 +158,7 @@ void HashPartitionWriter::writePackets(const TrackedMppDataPa // Update metrics about exchange hash partition { - assert(inner_packet.compress().mode() == mpp::CompressionMode::NONE); + assert(inner_packet.compression().mode() == mpp::CompressionMode::NONE); if (auto sz = inner_packet.ByteSizeLong(); writer->isLocal(part_id)) { diff --git a/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp b/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp index 0a0591f40f0..55545d5a836 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp +++ b/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp @@ -140,7 +140,7 @@ void HashPartitionWriterImplV1::partitionAndEncodeThenWriteBl for (size_t part_id = 0; part_id < partition_num; ++part_id) { auto mode = writer->isLocal(part_id) ? mpp::CompressionMode::NONE : compression_mode; - tracked_packets[part_id]->getPacket().mutable_compress()->set_mode(mode); + tracked_packets[part_id]->getPacket().mutable_compression()->set_mode(mode); } // Sum of all approximate block data memory size @@ -207,7 +207,7 @@ void HashPartitionWriterImplV1::partitionAndEncodeThenWriteBl WriteBuffer * ostr_ptr = output_buffer.get(); // Init compression writer - if (tracked_packets[part_id]->getPacket().compress().mode() != mpp::CompressionMode::NONE) + if (tracked_packets[part_id]->getPacket().compression().mode() != mpp::CompressionMode::NONE) { compress_codec = std::make_unique( *output_buffer, @@ -269,7 +269,7 @@ void HashPartitionWriterImplV1::partitionAndEncodeThenWriteBl for (size_t part_id = 0; part_id < partition_num; ++part_id) { auto mode = writer->isLocal(part_id) ? mpp::CompressionMode::NONE : compression_mode; - tracked_packets[part_id]->getPacket().mutable_compress()->set_mode(mode); + tracked_packets[part_id]->getPacket().mutable_compression()->set_mode(mode); } // Sum of all approximate block data memory size @@ -335,7 +335,7 @@ void HashPartitionWriterImplV1::partitionAndEncodeThenWriteBl WriteBuffer * ostr_ptr = output_buffer.get(); // Init compression writer - if (tracked_packets[part_id]->getPacket().compress().mode() != mpp::CompressionMode::NONE) + if (tracked_packets[part_id]->getPacket().compression().mode() != mpp::CompressionMode::NONE) { compress_codec = std::make_unique( *output_buffer, @@ -423,7 +423,7 @@ void HashPartitionWriterImplV1::writePackets(TrackedMppDataPa if (auto sz = inner_packet.ByteSizeLong(); likely(inner_packet.chunks_size() > 0)) { - auto mode = inner_packet.compress().mode(); + auto mode = inner_packet.compression().mode(); writer->partitionWrite(std::move(packet), part_id); updateHashPartitionWriterMetrics(mode, sz, writer->isLocal(part_id)); } diff --git a/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp b/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp index 6897772f8cd..c115954dfda 100644 --- a/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp +++ b/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp @@ -528,14 +528,14 @@ try { if (part_index == 0) { - ASSERT_EQ(packet.compress().mode(), mpp::CompressionMode::NONE); + ASSERT_EQ(packet.compression().mode(), mpp::CompressionMode::NONE); } else { - ASSERT_NE(packet.compress().mode(), mpp::CompressionMode::NONE); + ASSERT_NE(packet.compression().mode(), mpp::CompressionMode::NONE); } - auto && result = decoder.decodeAndSquash(chunk, packet.compress().mode() != mpp::CompressionMode::NONE); + auto && result = decoder.decodeAndSquash(chunk, packet.compression().mode() != mpp::CompressionMode::NONE); if (!result) continue; decoded_block_rows += result->rows(); diff --git a/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp b/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp index 2486db8faac..94d704f3618 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp +++ b/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp @@ -120,7 +120,7 @@ std::unique_ptr NewMPPExchangeWriter( if (dag_context.isRootMPPTask()) { // No need to use use data compression - RUNTIME_CHECK(dag_context.getExchangeSenderMeta().compress() == mpp::CompressionMode::NONE); + RUNTIME_CHECK(dag_context.getExchangeSenderMeta().compression() == mpp::CompressionMode::NONE); RUNTIME_CHECK(!enable_fine_grained_shuffle); RUNTIME_CHECK(exchange_type == tipb::ExchangeType::PassThrough); @@ -138,7 +138,7 @@ std::unique_ptr NewMPPExchangeWriter( if (enable_fine_grained_shuffle) { // TODO: support data compression if necessary - RUNTIME_CHECK(dag_context.getExchangeSenderMeta().compress() == mpp::CompressionMode::NONE); + RUNTIME_CHECK(dag_context.getExchangeSenderMeta().compression() == mpp::CompressionMode::NONE); return std::make_unique>( writer, @@ -151,7 +151,7 @@ std::unique_ptr NewMPPExchangeWriter( } else { - auto && compression_mode = dag_context.getExchangeSenderMeta().compress(); + auto && compression_mode = dag_context.getExchangeSenderMeta().compression(); if (TiDB::MppVersion::MppVersionV0 == dag_context.getMPPTaskMeta().mpp_version()) return std::make_unique>( writer, @@ -173,7 +173,7 @@ std::unique_ptr NewMPPExchangeWriter( else { // TODO: support data compression if necessary - RUNTIME_CHECK(dag_context.getExchangeSenderMeta().compress() == mpp::CompressionMode::NONE); + RUNTIME_CHECK(dag_context.getExchangeSenderMeta().compression() == mpp::CompressionMode::NONE); RUNTIME_CHECK(!enable_fine_grained_shuffle); return std::make_unique>( From 12e2692bbeb968bab86da6e6421206f7e20136c9 Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Fri, 6 Jan 2023 10:01:09 +0800 Subject: [PATCH 47/93] 41 --- contrib/tzg.mpp.proto.patch | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/contrib/tzg.mpp.proto.patch b/contrib/tzg.mpp.proto.patch index 5d4ba9ea7ed..eebd4a931cc 100644 --- a/contrib/tzg.mpp.proto.patch +++ b/contrib/tzg.mpp.proto.patch @@ -1086,7 +1086,7 @@ index fcd4691..f48ae58 100644 iNdEx = preIndex skippy, err := skipMpp(dAtA[iNdEx:]) diff --git a/proto/mpp.proto b/proto/mpp.proto -index 6739455..0943734 100644 +index 6739455..6fa315e 100644 --- a/proto/mpp.proto +++ b/proto/mpp.proto @@ -11,6 +11,17 @@ option (gogoproto.unmarshaler_all) = true; @@ -1095,9 +1095,9 @@ index 6739455..0943734 100644 +// Data compression mode +enum CompressionMode { -+ NONE = 0; // no compression -+ FAST = 1; // fast compression/decompression speed, compression ratio is lower than HC mode -+ HIGH_COMPRESSION = 2; // high compression (HC) ratio mode ++ NONE = 0; // no compression ++ FAST = 1; // fast compression/decompression speed, compression ratio is lower than HC mode ++ HIGH_COMPRESSION = 2; // high compression (HC) ratio mode +} + +message ExchangeSenderMeta { From 45fdeb3125f4e3a207719d73def285444f3b0799 Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Fri, 6 Jan 2023 10:10:54 +0800 Subject: [PATCH 48/93] 42 --- contrib/CMakeLists.txt | 25 ------------------------- contrib/kvproto | 2 +- 2 files changed, 1 insertion(+), 26 deletions(-) diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index a31bbb3c2a2..cf5c6d5704d 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -111,31 +111,6 @@ if (USE_INTERNAL_GRPC_LIBRARY) unset (CMAKE_POLICY_DEFAULT_CMP0077) endif () -execute_process( - COMMAND grep "CompressionMode compression" "${TiFlash_SOURCE_DIR}/contrib/kvproto/proto/mpp.proto" - RESULT_VARIABLE HAVE_APPLY_PATCH) - -# grep - Normally, the exit status is 0 if selected lines are found and 1 otherwise. But the exit status is 2 if an error occurred. -if (HAVE_APPLY_PATCH EQUAL 1) - message(STATUS "mpp compress patch not apply: ${HAVE_APPLY_PATCH}, patching...") - set (MPP_PATCH_FILE "${TiFlash_SOURCE_DIR}/contrib/tzg.mpp.proto.patch") - # apply the patch - execute_process( - COMMAND git apply -v "${MPP_PATCH_FILE}" - WORKING_DIRECTORY "${TiFlash_SOURCE_DIR}/contrib/kvproto" - COMMAND_ECHO STDOUT - RESULT_VARIABLE PATCH_SUCC) - if (NOT PATCH_SUCC EQUAL 0) - message(FATAL_ERROR "Can not apply mpp compress patch ${MPP_PATCH_FILE}") - else () - message(STATUS "mpp compress patch done") - endif () -elseif (HAVE_APPLY_PATCH EQUAL 0) - message(STATUS "mpp compress patch have been applied: ${HAVE_APPLY_PATCH}") -else () - message(FATAL_ERROR "Can not check the mpp compress patch status") -endif () - add_subdirectory (kvproto/cpp) target_no_warning(kvproto deprecated-declarations) target_no_warning(kvproto non-c-typedef-for-linkage) diff --git a/contrib/kvproto b/contrib/kvproto index 0a9b14f1fc2..4f3ab10cd53 160000 --- a/contrib/kvproto +++ b/contrib/kvproto @@ -1 +1 @@ -Subproject commit 0a9b14f1fc265bb5ad5a9ccc91f278c9f3716fda +Subproject commit 4f3ab10cd530a0e832c790e157b8b784208d3829 From f0b805eadd6c41696814cadd45d1d249b8754c46 Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Fri, 6 Jan 2023 10:11:30 +0800 Subject: [PATCH 49/93] 43 --- .gitmodules | 3 +- contrib/tzg.mpp.proto.patch | 1158 ----------------------------------- 2 files changed, 1 insertion(+), 1160 deletions(-) delete mode 100644 contrib/tzg.mpp.proto.patch diff --git a/.gitmodules b/.gitmodules index 9665bc2c5da..8ca783471b0 100644 --- a/.gitmodules +++ b/.gitmodules @@ -24,8 +24,7 @@ url = https://github.com/pingcap/boost-extra.git [submodule "contrib/kvproto"] path = contrib/kvproto - url = https://github.com/pingcap/kvproto.git - ignore = dirty + url = https://github.com/solotzg/kvproto.git [submodule "contrib/tipb"] path = contrib/tipb url = https://github.com/pingcap/tipb.git diff --git a/contrib/tzg.mpp.proto.patch b/contrib/tzg.mpp.proto.patch deleted file mode 100644 index eebd4a931cc..00000000000 --- a/contrib/tzg.mpp.proto.patch +++ /dev/null @@ -1,1158 +0,0 @@ -diff --git a/pkg/mpp/mpp.pb.go b/pkg/mpp/mpp.pb.go -index fcd4691..f48ae58 100644 ---- a/pkg/mpp/mpp.pb.go -+++ b/pkg/mpp/mpp.pb.go -@@ -26,6 +26,82 @@ var _ = math.Inf - // proto package needs to be updated. - const _ = proto.ProtoPackageIsVersion3 // please upgrade the proto package - -+// Data compression mode -+type CompressionMode int32 -+ -+const ( -+ CompressionMode_NONE CompressionMode = 0 -+ CompressionMode_FAST CompressionMode = 1 -+ CompressionMode_HIGH_COMPRESSION CompressionMode = 2 -+) -+ -+var CompressionMode_name = map[int32]string{ -+ 0: "NONE", -+ 1: "FAST", -+ 2: "HIGH_COMPRESSION", -+} -+ -+var CompressionMode_value = map[string]int32{ -+ "NONE": 0, -+ "FAST": 1, -+ "HIGH_COMPRESSION": 2, -+} -+ -+func (x CompressionMode) String() string { -+ return proto.EnumName(CompressionMode_name, int32(x)) -+} -+ -+func (CompressionMode) EnumDescriptor() ([]byte, []int) { -+ return fileDescriptor_819623c7fa76fc55, []int{0} -+} -+ -+type ExchangeSenderMeta struct { -+ Compression CompressionMode `protobuf:"varint,1,opt,name=compression,proto3,enum=mpp.CompressionMode" json:"compression,omitempty"` -+ XXX_NoUnkeyedLiteral struct{} `json:"-"` -+ XXX_unrecognized []byte `json:"-"` -+ XXX_sizecache int32 `json:"-"` -+} -+ -+func (m *ExchangeSenderMeta) Reset() { *m = ExchangeSenderMeta{} } -+func (m *ExchangeSenderMeta) String() string { return proto.CompactTextString(m) } -+func (*ExchangeSenderMeta) ProtoMessage() {} -+func (*ExchangeSenderMeta) Descriptor() ([]byte, []int) { -+ return fileDescriptor_819623c7fa76fc55, []int{0} -+} -+func (m *ExchangeSenderMeta) XXX_Unmarshal(b []byte) error { -+ return m.Unmarshal(b) -+} -+func (m *ExchangeSenderMeta) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { -+ if deterministic { -+ return xxx_messageInfo_ExchangeSenderMeta.Marshal(b, m, deterministic) -+ } else { -+ b = b[:cap(b)] -+ n, err := m.MarshalToSizedBuffer(b) -+ if err != nil { -+ return nil, err -+ } -+ return b[:n], nil -+ } -+} -+func (m *ExchangeSenderMeta) XXX_Merge(src proto.Message) { -+ xxx_messageInfo_ExchangeSenderMeta.Merge(m, src) -+} -+func (m *ExchangeSenderMeta) XXX_Size() int { -+ return m.Size() -+} -+func (m *ExchangeSenderMeta) XXX_DiscardUnknown() { -+ xxx_messageInfo_ExchangeSenderMeta.DiscardUnknown(m) -+} -+ -+var xxx_messageInfo_ExchangeSenderMeta proto.InternalMessageInfo -+ -+func (m *ExchangeSenderMeta) GetCompression() CompressionMode { -+ if m != nil { -+ return m.Compression -+ } -+ return CompressionMode_NONE -+} -+ - // TaskMeta contains meta of a mpp plan, including query's ts and task address. - type TaskMeta struct { - StartTs uint64 `protobuf:"varint,1,opt,name=start_ts,json=startTs,proto3" json:"start_ts,omitempty"` -@@ -36,6 +112,7 @@ type TaskMeta struct { - QueryTs uint64 `protobuf:"varint,6,opt,name=query_ts,json=queryTs,proto3" json:"query_ts,omitempty"` - LocalQueryId uint64 `protobuf:"varint,7,opt,name=local_query_id,json=localQueryId,proto3" json:"local_query_id,omitempty"` - ServerId uint64 `protobuf:"varint,8,opt,name=server_id,json=serverId,proto3" json:"server_id,omitempty"` -+ MppVersion int64 `protobuf:"varint,9,opt,name=mpp_version,json=mppVersion,proto3" json:"mpp_version,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -@@ -45,7 +122,7 @@ func (m *TaskMeta) Reset() { *m = TaskMeta{} } - func (m *TaskMeta) String() string { return proto.CompactTextString(m) } - func (*TaskMeta) ProtoMessage() {} - func (*TaskMeta) Descriptor() ([]byte, []int) { -- return fileDescriptor_819623c7fa76fc55, []int{0} -+ return fileDescriptor_819623c7fa76fc55, []int{1} - } - func (m *TaskMeta) XXX_Unmarshal(b []byte) error { - return m.Unmarshal(b) -@@ -130,6 +207,13 @@ func (m *TaskMeta) GetServerId() uint64 { - return 0 - } - -+func (m *TaskMeta) GetMppVersion() int64 { -+ if m != nil { -+ return m.MppVersion -+ } -+ return 0 -+} -+ - type IsAliveRequest struct { - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` -@@ -140,7 +224,7 @@ func (m *IsAliveRequest) Reset() { *m = IsAliveRequest{} } - func (m *IsAliveRequest) String() string { return proto.CompactTextString(m) } - func (*IsAliveRequest) ProtoMessage() {} - func (*IsAliveRequest) Descriptor() ([]byte, []int) { -- return fileDescriptor_819623c7fa76fc55, []int{1} -+ return fileDescriptor_819623c7fa76fc55, []int{2} - } - func (m *IsAliveRequest) XXX_Unmarshal(b []byte) error { - return m.Unmarshal(b) -@@ -171,6 +255,7 @@ var xxx_messageInfo_IsAliveRequest proto.InternalMessageInfo - - type IsAliveResponse struct { - Available bool `protobuf:"varint,1,opt,name=available,proto3" json:"available,omitempty"` -+ MppVersion int64 `protobuf:"varint,2,opt,name=mpp_version,json=mppVersion,proto3" json:"mpp_version,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -@@ -180,7 +265,7 @@ func (m *IsAliveResponse) Reset() { *m = IsAliveResponse{} } - func (m *IsAliveResponse) String() string { return proto.CompactTextString(m) } - func (*IsAliveResponse) ProtoMessage() {} - func (*IsAliveResponse) Descriptor() ([]byte, []int) { -- return fileDescriptor_819623c7fa76fc55, []int{2} -+ return fileDescriptor_819623c7fa76fc55, []int{3} - } - func (m *IsAliveResponse) XXX_Unmarshal(b []byte) error { - return m.Unmarshal(b) -@@ -216,6 +301,13 @@ func (m *IsAliveResponse) GetAvailable() bool { - return false - } - -+func (m *IsAliveResponse) GetMppVersion() int64 { -+ if m != nil { -+ return m.MppVersion -+ } -+ return 0 -+} -+ - // Dipsatch the task request to different tiflash servers. - type DispatchTaskRequest struct { - Meta *TaskMeta `protobuf:"bytes,1,opt,name=meta,proto3" json:"meta,omitempty"` -@@ -225,17 +317,19 @@ type DispatchTaskRequest struct { - // If this task contains table scan, we still need their region info. - SchemaVer int64 `protobuf:"varint,5,opt,name=schema_ver,json=schemaVer,proto3" json:"schema_ver,omitempty"` - // Used for partition table scan -- TableRegions []*coprocessor.TableRegions `protobuf:"bytes,6,rep,name=table_regions,json=tableRegions,proto3" json:"table_regions,omitempty"` -- XXX_NoUnkeyedLiteral struct{} `json:"-"` -- XXX_unrecognized []byte `json:"-"` -- XXX_sizecache int32 `json:"-"` -+ TableRegions []*coprocessor.TableRegions `protobuf:"bytes,6,rep,name=table_regions,json=tableRegions,proto3" json:"table_regions,omitempty"` -+ // Used for exchange sender -+ ExchangeSenderMeta *ExchangeSenderMeta `protobuf:"bytes,7,opt,name=exchange_sender_meta,json=exchangeSenderMeta,proto3" json:"exchange_sender_meta,omitempty"` -+ XXX_NoUnkeyedLiteral struct{} `json:"-"` -+ XXX_unrecognized []byte `json:"-"` -+ XXX_sizecache int32 `json:"-"` - } - - func (m *DispatchTaskRequest) Reset() { *m = DispatchTaskRequest{} } - func (m *DispatchTaskRequest) String() string { return proto.CompactTextString(m) } - func (*DispatchTaskRequest) ProtoMessage() {} - func (*DispatchTaskRequest) Descriptor() ([]byte, []int) { -- return fileDescriptor_819623c7fa76fc55, []int{3} -+ return fileDescriptor_819623c7fa76fc55, []int{4} - } - func (m *DispatchTaskRequest) XXX_Unmarshal(b []byte) error { - return m.Unmarshal(b) -@@ -306,6 +400,13 @@ func (m *DispatchTaskRequest) GetTableRegions() []*coprocessor.TableRegions { - return nil - } - -+func (m *DispatchTaskRequest) GetExchangeSenderMeta() *ExchangeSenderMeta { -+ if m != nil { -+ return m.ExchangeSenderMeta -+ } -+ return nil -+} -+ - // Get response of DispatchTaskRequest. - type DispatchTaskResponse struct { - Error *Error `protobuf:"bytes,1,opt,name=error,proto3" json:"error,omitempty"` -@@ -319,7 +420,7 @@ func (m *DispatchTaskResponse) Reset() { *m = DispatchTaskResponse{} } - func (m *DispatchTaskResponse) String() string { return proto.CompactTextString(m) } - func (*DispatchTaskResponse) ProtoMessage() {} - func (*DispatchTaskResponse) Descriptor() ([]byte, []int) { -- return fileDescriptor_819623c7fa76fc55, []int{4} -+ return fileDescriptor_819623c7fa76fc55, []int{5} - } - func (m *DispatchTaskResponse) XXX_Unmarshal(b []byte) error { - return m.Unmarshal(b) -@@ -375,7 +476,7 @@ func (m *CancelTaskRequest) Reset() { *m = CancelTaskRequest{} } - func (m *CancelTaskRequest) String() string { return proto.CompactTextString(m) } - func (*CancelTaskRequest) ProtoMessage() {} - func (*CancelTaskRequest) Descriptor() ([]byte, []int) { -- return fileDescriptor_819623c7fa76fc55, []int{5} -+ return fileDescriptor_819623c7fa76fc55, []int{6} - } - func (m *CancelTaskRequest) XXX_Unmarshal(b []byte) error { - return m.Unmarshal(b) -@@ -429,7 +530,7 @@ func (m *CancelTaskResponse) Reset() { *m = CancelTaskResponse{} } - func (m *CancelTaskResponse) String() string { return proto.CompactTextString(m) } - func (*CancelTaskResponse) ProtoMessage() {} - func (*CancelTaskResponse) Descriptor() ([]byte, []int) { -- return fileDescriptor_819623c7fa76fc55, []int{6} -+ return fileDescriptor_819623c7fa76fc55, []int{7} - } - func (m *CancelTaskResponse) XXX_Unmarshal(b []byte) error { - return m.Unmarshal(b) -@@ -478,7 +579,7 @@ func (m *EstablishMPPConnectionRequest) Reset() { *m = EstablishMPPConne - func (m *EstablishMPPConnectionRequest) String() string { return proto.CompactTextString(m) } - func (*EstablishMPPConnectionRequest) ProtoMessage() {} - func (*EstablishMPPConnectionRequest) Descriptor() ([]byte, []int) { -- return fileDescriptor_819623c7fa76fc55, []int{7} -+ return fileDescriptor_819623c7fa76fc55, []int{8} - } - func (m *EstablishMPPConnectionRequest) XXX_Unmarshal(b []byte) error { - return m.Unmarshal(b) -@@ -521,23 +622,72 @@ func (m *EstablishMPPConnectionRequest) GetReceiverMeta() *TaskMeta { - return nil - } - -+type DataPacketCompressionMeta struct { -+ Mode CompressionMode `protobuf:"varint,1,opt,name=mode,proto3,enum=mpp.CompressionMode" json:"mode,omitempty"` -+ XXX_NoUnkeyedLiteral struct{} `json:"-"` -+ XXX_unrecognized []byte `json:"-"` -+ XXX_sizecache int32 `json:"-"` -+} -+ -+func (m *DataPacketCompressionMeta) Reset() { *m = DataPacketCompressionMeta{} } -+func (m *DataPacketCompressionMeta) String() string { return proto.CompactTextString(m) } -+func (*DataPacketCompressionMeta) ProtoMessage() {} -+func (*DataPacketCompressionMeta) Descriptor() ([]byte, []int) { -+ return fileDescriptor_819623c7fa76fc55, []int{9} -+} -+func (m *DataPacketCompressionMeta) XXX_Unmarshal(b []byte) error { -+ return m.Unmarshal(b) -+} -+func (m *DataPacketCompressionMeta) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { -+ if deterministic { -+ return xxx_messageInfo_DataPacketCompressionMeta.Marshal(b, m, deterministic) -+ } else { -+ b = b[:cap(b)] -+ n, err := m.MarshalToSizedBuffer(b) -+ if err != nil { -+ return nil, err -+ } -+ return b[:n], nil -+ } -+} -+func (m *DataPacketCompressionMeta) XXX_Merge(src proto.Message) { -+ xxx_messageInfo_DataPacketCompressionMeta.Merge(m, src) -+} -+func (m *DataPacketCompressionMeta) XXX_Size() int { -+ return m.Size() -+} -+func (m *DataPacketCompressionMeta) XXX_DiscardUnknown() { -+ xxx_messageInfo_DataPacketCompressionMeta.DiscardUnknown(m) -+} -+ -+var xxx_messageInfo_DataPacketCompressionMeta proto.InternalMessageInfo -+ -+func (m *DataPacketCompressionMeta) GetMode() CompressionMode { -+ if m != nil { -+ return m.Mode -+ } -+ return CompressionMode_NONE -+} -+ - // when TiFlash sends data to TiDB, Data packets wrap tipb.SelectResponse, i.e., serialize tipb.SelectResponse into data; - // when TiFlash sends data to TiFlash, data blocks are serialized into chunks, and the execution_summaries in tipb.SelectResponse are serialized into data only for the last packet. - type MPPDataPacket struct { -- Data []byte `protobuf:"bytes,1,opt,name=data,proto3" json:"data,omitempty"` -- Error *Error `protobuf:"bytes,2,opt,name=error,proto3" json:"error,omitempty"` -- Chunks [][]byte `protobuf:"bytes,3,rep,name=chunks,proto3" json:"chunks,omitempty"` -- StreamIds []uint64 `protobuf:"varint,4,rep,packed,name=stream_ids,json=streamIds,proto3" json:"stream_ids,omitempty"` -- XXX_NoUnkeyedLiteral struct{} `json:"-"` -- XXX_unrecognized []byte `json:"-"` -- XXX_sizecache int32 `json:"-"` -+ Data []byte `protobuf:"bytes,1,opt,name=data,proto3" json:"data,omitempty"` -+ Error *Error `protobuf:"bytes,2,opt,name=error,proto3" json:"error,omitempty"` -+ Chunks [][]byte `protobuf:"bytes,3,rep,name=chunks,proto3" json:"chunks,omitempty"` -+ StreamIds []uint64 `protobuf:"varint,4,rep,packed,name=stream_ids,json=streamIds,proto3" json:"stream_ids,omitempty"` -+ Version int64 `protobuf:"varint,5,opt,name=version,proto3" json:"version,omitempty"` -+ Compression *DataPacketCompressionMeta `protobuf:"bytes,6,opt,name=compression,proto3" json:"compression,omitempty"` -+ XXX_NoUnkeyedLiteral struct{} `json:"-"` -+ XXX_unrecognized []byte `json:"-"` -+ XXX_sizecache int32 `json:"-"` - } - - func (m *MPPDataPacket) Reset() { *m = MPPDataPacket{} } - func (m *MPPDataPacket) String() string { return proto.CompactTextString(m) } - func (*MPPDataPacket) ProtoMessage() {} - func (*MPPDataPacket) Descriptor() ([]byte, []int) { -- return fileDescriptor_819623c7fa76fc55, []int{8} -+ return fileDescriptor_819623c7fa76fc55, []int{10} - } - func (m *MPPDataPacket) XXX_Unmarshal(b []byte) error { - return m.Unmarshal(b) -@@ -594,9 +744,24 @@ func (m *MPPDataPacket) GetStreamIds() []uint64 { - return nil - } - -+func (m *MPPDataPacket) GetVersion() int64 { -+ if m != nil { -+ return m.Version -+ } -+ return 0 -+} -+ -+func (m *MPPDataPacket) GetCompression() *DataPacketCompressionMeta { -+ if m != nil { -+ return m.Compression -+ } -+ return nil -+} -+ - type Error struct { - Code int32 `protobuf:"varint,1,opt,name=code,proto3" json:"code,omitempty"` - Msg string `protobuf:"bytes,2,opt,name=msg,proto3" json:"msg,omitempty"` -+ MppVersion int64 `protobuf:"varint,3,opt,name=mpp_version,json=mppVersion,proto3" json:"mpp_version,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -@@ -606,7 +771,7 @@ func (m *Error) Reset() { *m = Error{} } - func (m *Error) String() string { return proto.CompactTextString(m) } - func (*Error) ProtoMessage() {} - func (*Error) Descriptor() ([]byte, []int) { -- return fileDescriptor_819623c7fa76fc55, []int{9} -+ return fileDescriptor_819623c7fa76fc55, []int{11} - } - func (m *Error) XXX_Unmarshal(b []byte) error { - return m.Unmarshal(b) -@@ -649,7 +814,16 @@ func (m *Error) GetMsg() string { - return "" - } - -+func (m *Error) GetMppVersion() int64 { -+ if m != nil { -+ return m.MppVersion -+ } -+ return 0 -+} -+ - func init() { -+ proto.RegisterEnum("mpp.CompressionMode", CompressionMode_name, CompressionMode_value) -+ proto.RegisterType((*ExchangeSenderMeta)(nil), "mpp.ExchangeSenderMeta") - proto.RegisterType((*TaskMeta)(nil), "mpp.TaskMeta") - proto.RegisterType((*IsAliveRequest)(nil), "mpp.IsAliveRequest") - proto.RegisterType((*IsAliveResponse)(nil), "mpp.IsAliveResponse") -@@ -658,6 +832,7 @@ func init() { - proto.RegisterType((*CancelTaskRequest)(nil), "mpp.CancelTaskRequest") - proto.RegisterType((*CancelTaskResponse)(nil), "mpp.CancelTaskResponse") - proto.RegisterType((*EstablishMPPConnectionRequest)(nil), "mpp.EstablishMPPConnectionRequest") -+ proto.RegisterType((*DataPacketCompressionMeta)(nil), "mpp.DataPacketCompressionMeta") - proto.RegisterType((*MPPDataPacket)(nil), "mpp.MPPDataPacket") - proto.RegisterType((*Error)(nil), "mpp.Error") - } -@@ -665,49 +840,92 @@ func init() { - func init() { proto.RegisterFile("mpp.proto", fileDescriptor_819623c7fa76fc55) } - - var fileDescriptor_819623c7fa76fc55 = []byte{ -- // 664 bytes of a gzipped FileDescriptorProto -- 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x54, 0xcd, 0x6e, 0xd3, 0x4c, -- 0x14, 0xfd, 0x1c, 0xe7, 0xcf, 0x37, 0x4e, 0xbf, 0x76, 0xbe, 0xea, 0x6b, 0x5a, 0x68, 0x94, 0x5a, -- 0x2c, 0xb2, 0xc1, 0x15, 0xad, 0xc4, 0x12, 0x09, 0xda, 0x2e, 0xbc, 0xa8, 0x14, 0x46, 0x11, 0x62, -- 0x67, 0x4d, 0x3d, 0x43, 0x62, 0xc5, 0xf6, 0xb8, 0x33, 0x13, 0x4b, 0x95, 0xd8, 0xb1, 0xe0, 0x15, -- 0x78, 0x24, 0x96, 0x3c, 0x02, 0x2a, 0xef, 0xc0, 0x1a, 0xcd, 0x8c, 0x1d, 0x5a, 0x89, 0x0a, 0x58, -- 0x65, 0xce, 0xb9, 0x77, 0xce, 0xb9, 0x3e, 0xbe, 0x31, 0x78, 0x79, 0x59, 0x86, 0xa5, 0xe0, 0x8a, -- 0x23, 0x37, 0x2f, 0xcb, 0x83, 0xdd, 0x05, 0x5f, 0x70, 0x83, 0x8f, 0xf5, 0xc9, 0x96, 0x0e, 0x76, -- 0x12, 0xcd, 0x25, 0x4c, 0x4a, 0x2e, 0x6a, 0xca, 0xcf, 0x99, 0x22, 0xe5, 0x95, 0x45, 0xc1, 0x77, -- 0x07, 0xfa, 0x73, 0x22, 0x57, 0x97, 0x4c, 0x11, 0xb4, 0x0f, 0x7d, 0xa9, 0x88, 0x50, 0xb1, 0x92, -- 0x23, 0x67, 0xe2, 0x4c, 0xdb, 0xb8, 0x67, 0xf0, 0x5c, 0xa2, 0x3d, 0xe8, 0x29, 0x22, 0x57, 0x71, -- 0x4a, 0x47, 0xad, 0x89, 0x33, 0x75, 0x71, 0x57, 0xc3, 0x88, 0xa2, 0x23, 0xf0, 0x4b, 0x22, 0x54, -- 0xaa, 0x52, 0x5e, 0xe8, 0xaa, 0x6b, 0xaa, 0x83, 0x0d, 0x17, 0x51, 0x34, 0x82, 0x1e, 0xa1, 0x54, -- 0x30, 0x29, 0x47, 0xed, 0x89, 0x33, 0xf5, 0x70, 0x03, 0xd1, 0x23, 0xf0, 0x16, 0x44, 0x2d, 0x99, -- 0xd0, 0x37, 0x3b, 0xc6, 0xb1, 0x6f, 0x89, 0x88, 0xea, 0x69, 0xae, 0xd7, 0x4c, 0xdc, 0xe8, 0x69, -- 0xba, 0x76, 0x1a, 0x83, 0xe7, 0x12, 0x3d, 0x81, 0xad, 0x8c, 0x27, 0x24, 0x8b, 0x6d, 0x43, 0x4a, -- 0x47, 0x3d, 0xd3, 0xe0, 0x1b, 0xf6, 0xb5, 0x26, 0x23, 0xaa, 0xd5, 0x25, 0x13, 0x95, 0x55, 0xef, -- 0x5b, 0x75, 0x4b, 0x44, 0x34, 0xd8, 0x86, 0xad, 0x48, 0xbe, 0xcc, 0xd2, 0x8a, 0x61, 0x76, 0xbd, -- 0x66, 0x52, 0x05, 0xc7, 0xf0, 0xef, 0x86, 0x91, 0x25, 0x2f, 0x24, 0x43, 0x8f, 0xc1, 0x23, 0x15, -- 0x49, 0x33, 0x72, 0x95, 0x31, 0x93, 0x48, 0x1f, 0xff, 0x24, 0x82, 0x8f, 0x2d, 0xf8, 0xef, 0x3c, -- 0x95, 0x25, 0x51, 0xc9, 0x52, 0x67, 0x58, 0x0b, 0xa1, 0x23, 0x68, 0xeb, 0x8c, 0xcd, 0x85, 0xc1, -- 0xc9, 0x30, 0xd4, 0x6f, 0xaa, 0xc9, 0x18, 0x9b, 0x92, 0x4e, 0x8d, 0x15, 0x09, 0xa7, 0x8c, 0xc6, -- 0x65, 0x46, 0x0a, 0x93, 0xa9, 0x8f, 0x07, 0x35, 0x37, 0xcb, 0x48, 0xa1, 0x53, 0x53, 0x69, 0xce, -- 0xf8, 0x5a, 0xd5, 0x99, 0x36, 0x10, 0x3d, 0x83, 0x9e, 0x60, 0x8b, 0x94, 0x17, 0x3a, 0x4f, 0x77, -- 0x3a, 0x38, 0xd9, 0x0b, 0xef, 0xbe, 0x66, 0x6c, 0x6a, 0x51, 0xf1, 0x8e, 0xe3, 0xa6, 0x0f, 0x1d, -- 0x02, 0xc8, 0x64, 0xc9, 0x72, 0x12, 0x57, 0x4c, 0x98, 0xa4, 0x5d, 0xec, 0x59, 0xe6, 0x0d, 0x13, -- 0xe8, 0x05, 0x0c, 0x95, 0x7e, 0xa4, 0xb8, 0xd1, 0xed, 0x1a, 0xdd, 0xfd, 0x7b, 0xba, 0x73, 0xdd, -- 0x61, 0xc5, 0x25, 0xf6, 0xd5, 0x1d, 0x14, 0xe4, 0xb0, 0x7b, 0x3f, 0x88, 0x3a, 0xbf, 0x09, 0x74, -- 0x98, 0x10, 0x5c, 0xd4, 0x51, 0x80, 0x89, 0xe2, 0x42, 0x33, 0xd8, 0x16, 0xd0, 0x29, 0x0c, 0x05, -- 0x53, 0xe2, 0x66, 0xe3, 0xdc, 0x32, 0xce, 0x5b, 0x61, 0xbd, 0xa5, 0xd6, 0x01, 0xfb, 0xa6, 0xa9, -- 0xb1, 0x7b, 0x0b, 0x3b, 0x67, 0xa4, 0x48, 0x58, 0xf6, 0x97, 0xa9, 0x6f, 0xc6, 0x69, 0x3d, 0x30, -- 0x4e, 0xf0, 0x1c, 0xd0, 0x5d, 0xe5, 0x3f, 0x7d, 0x8c, 0xe0, 0x83, 0x03, 0x87, 0x17, 0x52, 0x67, -- 0x92, 0xca, 0xe5, 0xe5, 0x6c, 0x76, 0xc6, 0x8b, 0x82, 0x25, 0x7a, 0xff, 0x9b, 0xf1, 0x42, 0x18, -- 0x48, 0x56, 0x50, 0x26, 0xe2, 0x87, 0xa7, 0x04, 0xdb, 0x61, 0xfe, 0x8b, 0x27, 0x3a, 0x98, 0x84, -- 0xa5, 0x55, 0x73, 0xa3, 0xf5, 0xab, 0x1b, 0x7e, 0xd3, 0xa3, 0x51, 0xf0, 0x1e, 0x86, 0x97, 0xb3, -- 0xd9, 0x39, 0x51, 0x64, 0x46, 0x92, 0x15, 0x53, 0x08, 0x41, 0x9b, 0x92, 0xda, 0xcd, 0xc7, 0xe6, -- 0xfc, 0xfb, 0x10, 0xd0, 0xff, 0xd0, 0x4d, 0x96, 0xeb, 0x62, 0x25, 0x47, 0xee, 0xc4, 0x9d, 0xfa, -- 0xb8, 0x46, 0x66, 0x89, 0x94, 0x60, 0x24, 0x8f, 0x53, 0x6a, 0x57, 0xaf, 0x8d, 0x3d, 0xcb, 0x44, -- 0x54, 0x06, 0x4f, 0xa1, 0x63, 0x64, 0xb4, 0xab, 0x5e, 0x63, 0xe3, 0xda, 0xc1, 0xe6, 0x8c, 0xb6, -- 0xc1, 0xcd, 0xe5, 0xc2, 0x78, 0x7a, 0x58, 0x1f, 0x5f, 0x05, 0x9f, 0x6f, 0xc7, 0xce, 0x97, 0xdb, -- 0xb1, 0xf3, 0xf5, 0x76, 0xec, 0x7c, 0xfa, 0x36, 0xfe, 0x07, 0xb6, 0xb9, 0x58, 0x84, 0x2a, 0x5d, -- 0x55, 0xe1, 0xaa, 0x32, 0x5f, 0xa7, 0xab, 0xae, 0xf9, 0x39, 0xfd, 0x11, 0x00, 0x00, 0xff, 0xff, -- 0xcd, 0x41, 0xdb, 0xe2, 0xed, 0x04, 0x00, 0x00, -+ // 846 bytes of a gzipped FileDescriptorProto -+ 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x55, 0xcd, 0x6e, 0x23, 0x45, -+ 0x10, 0xde, 0xf1, 0x38, 0xfe, 0x29, 0x4f, 0xb2, 0xde, 0xc6, 0x22, 0xce, 0xc2, 0x1a, 0xef, 0x88, -+ 0x83, 0xc5, 0xc1, 0x88, 0xac, 0xb4, 0x17, 0x24, 0xc4, 0x92, 0x35, 0xec, 0x48, 0x38, 0x31, 0x1d, -+ 0x6b, 0xc5, 0x6d, 0xd4, 0x99, 0x6e, 0xec, 0x91, 0x3d, 0xd3, 0xb3, 0xdd, 0x1d, 0x8b, 0x3d, 0xf3, -+ 0x10, 0xf0, 0x48, 0x1c, 0x11, 0x27, 0x8e, 0x28, 0xbc, 0x08, 0xea, 0xea, 0x99, 0xc4, 0x71, 0x36, -+ 0xfc, 0x9c, 0xdc, 0xf5, 0x75, 0xcd, 0x57, 0x55, 0x5f, 0x55, 0x97, 0xa1, 0x9d, 0x15, 0xc5, 0xb8, -+ 0x50, 0xd2, 0x48, 0xe2, 0x67, 0x45, 0xf1, 0xb8, 0xb7, 0x90, 0x0b, 0x89, 0xf6, 0xa7, 0xf6, 0xe4, -+ 0xae, 0x1e, 0x3f, 0x4a, 0x2c, 0x96, 0x08, 0xad, 0xa5, 0x2a, 0xa1, 0x20, 0x13, 0x86, 0x15, 0x17, -+ 0xce, 0x0a, 0xbf, 0x05, 0x32, 0xf9, 0x31, 0x59, 0xb2, 0x7c, 0x21, 0xce, 0x45, 0xce, 0x85, 0x9a, -+ 0x0a, 0xc3, 0xc8, 0x73, 0xe8, 0x24, 0x32, 0x2b, 0x94, 0xd0, 0x3a, 0x95, 0x79, 0xdf, 0x1b, 0x7a, -+ 0xa3, 0x83, 0xe3, 0xde, 0xd8, 0x86, 0x3c, 0xb9, 0xc1, 0xa7, 0x92, 0x0b, 0xba, 0xed, 0x18, 0xfe, -+ 0x5c, 0x83, 0xd6, 0x9c, 0xe9, 0x15, 0x92, 0x1c, 0x41, 0x4b, 0x1b, 0xa6, 0x4c, 0x6c, 0x34, 0x32, -+ 0xd4, 0x69, 0x13, 0xed, 0xb9, 0x26, 0x87, 0xd0, 0x34, 0x4c, 0xaf, 0xe2, 0x94, 0xf7, 0x6b, 0x43, -+ 0x6f, 0xe4, 0xd3, 0x86, 0x35, 0x23, 0x4e, 0x9e, 0x42, 0x50, 0x30, 0x65, 0x52, 0x93, 0xca, 0xdc, -+ 0xde, 0xfa, 0x78, 0xdb, 0xb9, 0xc6, 0x22, 0x4e, 0xfa, 0xd0, 0x64, 0x9c, 0xdb, 0x88, 0xfd, 0xfa, -+ 0xd0, 0x1b, 0xb5, 0x69, 0x65, 0x92, 0x0f, 0xa0, 0xbd, 0x60, 0x66, 0x29, 0x94, 0xfd, 0x72, 0x0f, -+ 0x23, 0xb6, 0x1c, 0x10, 0x71, 0x9b, 0xcd, 0x9b, 0x4b, 0xa1, 0xde, 0xda, 0x6c, 0x1a, 0x2e, 0x1b, -+ 0xb4, 0xe7, 0x9a, 0x7c, 0x0c, 0x07, 0x6b, 0x99, 0xb0, 0x75, 0xec, 0x1c, 0x52, 0xde, 0x6f, 0xa2, -+ 0x43, 0x80, 0xe8, 0x77, 0x16, 0x8c, 0xb8, 0x65, 0xd7, 0x42, 0x6d, 0x1c, 0x7b, 0xcb, 0xb1, 0x3b, -+ 0x20, 0xe2, 0xe4, 0x23, 0xe8, 0x64, 0x45, 0x11, 0x6f, 0x84, 0x42, 0xc1, 0xda, 0x98, 0x36, 0x64, -+ 0x45, 0xf1, 0xda, 0x21, 0x61, 0x17, 0x0e, 0x22, 0xfd, 0x62, 0x9d, 0x6e, 0x04, 0x15, 0x6f, 0x2e, -+ 0x85, 0x36, 0xe1, 0x0c, 0x1e, 0x5e, 0x23, 0xba, 0x90, 0xb9, 0x16, 0xe4, 0x43, 0x68, 0xb3, 0x0d, -+ 0x4b, 0xd7, 0xec, 0x62, 0x2d, 0x50, 0xb2, 0x16, 0xbd, 0x01, 0x76, 0x63, 0xd4, 0xee, 0xc4, 0xf8, -+ 0xbd, 0x06, 0xef, 0xbd, 0x4c, 0x75, 0xc1, 0x4c, 0xb2, 0xb4, 0x5d, 0x28, 0x23, 0x91, 0xa7, 0x50, -+ 0xb7, 0x3d, 0x47, 0xc6, 0xce, 0xf1, 0x3e, 0xb6, 0xb1, 0xea, 0x12, 0xc5, 0x2b, 0xab, 0xbb, 0xc8, -+ 0x13, 0xc9, 0x05, 0x8f, 0x8b, 0x35, 0x73, 0xe4, 0x01, 0xed, 0x94, 0xd8, 0x6c, 0xcd, 0x72, 0xab, -+ 0xbb, 0x49, 0x33, 0x21, 0x2f, 0x4d, 0xd9, 0x95, 0xca, 0x24, 0x9f, 0x41, 0x53, 0x89, 0x45, 0x2a, -+ 0x73, 0xdb, 0x11, 0x7f, 0xd4, 0x39, 0x3e, 0x1c, 0x6f, 0x8f, 0x1d, 0xc5, 0xbb, 0x28, 0xff, 0x41, -+ 0xd2, 0xca, 0x8f, 0x3c, 0x01, 0xd0, 0xc9, 0x52, 0x64, 0xcc, 0x96, 0x83, 0xbd, 0xf2, 0x69, 0xdb, -+ 0x21, 0xaf, 0x85, 0x22, 0x5f, 0xc0, 0xbe, 0xb1, 0x35, 0xc7, 0x15, 0x6f, 0x03, 0x79, 0x8f, 0x6e, -+ 0xf1, 0xce, 0xad, 0x87, 0x23, 0xd7, 0x34, 0x30, 0x5b, 0x16, 0x89, 0xa0, 0x27, 0xca, 0xa9, 0x8e, -+ 0x35, 0x8e, 0x75, 0x8c, 0x0a, 0x34, 0x51, 0x81, 0x43, 0x54, 0xe0, 0xee, 0xd8, 0x53, 0x22, 0xee, -+ 0x60, 0x61, 0x06, 0xbd, 0xdb, 0x9a, 0x96, 0xbd, 0x1a, 0xc2, 0x9e, 0x50, 0x4a, 0xaa, 0x52, 0x55, -+ 0x70, 0x9c, 0x16, 0xa1, 0xee, 0x82, 0x3c, 0x83, 0x7d, 0x25, 0x8c, 0x7a, 0x7b, 0x5d, 0x44, 0x0d, -+ 0x8b, 0x38, 0x18, 0x97, 0x0f, 0xd0, 0x25, 0x4b, 0x03, 0x74, 0x2a, 0x33, 0x0f, 0xbf, 0x87, 0x47, -+ 0x27, 0x2c, 0x4f, 0xc4, 0xfa, 0x7f, 0x36, 0xf0, 0x3a, 0x9d, 0xda, 0x3d, 0xe9, 0x84, 0xcf, 0x81, -+ 0x6c, 0x33, 0xff, 0xd7, 0x32, 0xc2, 0x9f, 0x3c, 0x78, 0x32, 0xd1, 0x56, 0xde, 0x54, 0x2f, 0xa7, -+ 0xb3, 0xd9, 0x89, 0xcc, 0x73, 0x91, 0xd8, 0xc7, 0x58, 0xa5, 0x37, 0x86, 0xce, 0xb6, 0xc8, 0xef, -+ 0xcc, 0x12, 0xf4, 0xcd, 0x76, 0x39, 0xb6, 0xc2, 0x24, 0x22, 0xdd, 0x54, 0x5f, 0xd4, 0xde, 0xf5, -+ 0x45, 0x50, 0xf9, 0x60, 0x1b, 0x26, 0x70, 0xf4, 0x92, 0x19, 0x36, 0x63, 0xc9, 0x4a, 0x98, 0xed, -+ 0x1d, 0x64, 0x09, 0x47, 0x50, 0xcf, 0x24, 0x17, 0xff, 0xb8, 0xa7, 0xd0, 0x23, 0xfc, 0xc3, 0x83, -+ 0xfd, 0xe9, 0x6c, 0x76, 0x43, 0x45, 0x08, 0xd4, 0x39, 0x2b, 0xb3, 0x0e, 0x28, 0x9e, 0xff, 0x5d, -+ 0x4c, 0xf2, 0x3e, 0x34, 0x92, 0xe5, 0x65, 0xbe, 0xd2, 0x7d, 0x7f, 0xe8, 0x8f, 0x02, 0x5a, 0x5a, -+ 0x38, 0xd7, 0x46, 0x09, 0x96, 0xc5, 0x29, 0x77, 0xaf, 0xa1, 0x4e, 0xdb, 0x0e, 0x89, 0xb8, 0xb6, -+ 0x6f, 0xa8, 0x7a, 0xbe, 0x6e, 0xe6, 0x2b, 0x93, 0x7c, 0x79, 0x7b, 0xe3, 0x36, 0x30, 0xf0, 0x00, -+ 0x03, 0xdf, 0x5b, 0xf7, 0xed, 0xdd, 0x7b, 0x0a, 0x7b, 0x98, 0xa2, 0xad, 0x28, 0xa9, 0xd4, 0xd8, -+ 0xa3, 0x78, 0x26, 0x5d, 0xf0, 0x33, 0xbd, 0xc0, 0x7a, 0xda, 0xd4, 0x1e, 0x77, 0xb7, 0x89, 0xbf, -+ 0xbb, 0x4d, 0x3e, 0xf9, 0x1c, 0x1e, 0xee, 0x68, 0x48, 0x5a, 0x50, 0x3f, 0x3d, 0x3b, 0x9d, 0x74, -+ 0x1f, 0xd8, 0xd3, 0xd7, 0x2f, 0xce, 0xe7, 0x5d, 0x8f, 0xf4, 0xa0, 0xfb, 0x2a, 0xfa, 0xe6, 0x55, -+ 0x7c, 0x72, 0x36, 0x9d, 0xd1, 0xc9, 0xf9, 0x79, 0x74, 0x76, 0xda, 0xad, 0x7d, 0x15, 0xfe, 0x7a, -+ 0x35, 0xf0, 0x7e, 0xbb, 0x1a, 0x78, 0x7f, 0x5e, 0x0d, 0xbc, 0x5f, 0xfe, 0x1a, 0x3c, 0x80, 0xae, -+ 0x54, 0x8b, 0xb1, 0x49, 0x57, 0x9b, 0xf1, 0x6a, 0x83, 0x7f, 0x3d, 0x17, 0x0d, 0xfc, 0x79, 0xf6, -+ 0x77, 0x00, 0x00, 0x00, 0xff, 0xff, 0xd0, 0x35, 0x9b, 0x00, 0xca, 0x06, 0x00, 0x00, -+} -+ -+func (m *ExchangeSenderMeta) Marshal() (dAtA []byte, err error) { -+ size := m.Size() -+ dAtA = make([]byte, size) -+ n, err := m.MarshalToSizedBuffer(dAtA[:size]) -+ if err != nil { -+ return nil, err -+ } -+ return dAtA[:n], nil -+} -+ -+func (m *ExchangeSenderMeta) MarshalTo(dAtA []byte) (int, error) { -+ size := m.Size() -+ return m.MarshalToSizedBuffer(dAtA[:size]) -+} -+ -+func (m *ExchangeSenderMeta) MarshalToSizedBuffer(dAtA []byte) (int, error) { -+ i := len(dAtA) -+ _ = i -+ var l int -+ _ = l -+ if m.XXX_unrecognized != nil { -+ i -= len(m.XXX_unrecognized) -+ copy(dAtA[i:], m.XXX_unrecognized) -+ } -+ if m.Compression != 0 { -+ i = encodeVarintMpp(dAtA, i, uint64(m.Compression)) -+ i-- -+ dAtA[i] = 0x8 -+ } -+ return len(dAtA) - i, nil - } - - func (m *TaskMeta) Marshal() (dAtA []byte, err error) { -@@ -734,6 +952,11 @@ func (m *TaskMeta) MarshalToSizedBuffer(dAtA []byte) (int, error) { - i -= len(m.XXX_unrecognized) - copy(dAtA[i:], m.XXX_unrecognized) - } -+ if m.MppVersion != 0 { -+ i = encodeVarintMpp(dAtA, i, uint64(m.MppVersion)) -+ i-- -+ dAtA[i] = 0x48 -+ } - if m.ServerId != 0 { - i = encodeVarintMpp(dAtA, i, uint64(m.ServerId)) - i-- -@@ -830,6 +1053,11 @@ func (m *IsAliveResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { - i -= len(m.XXX_unrecognized) - copy(dAtA[i:], m.XXX_unrecognized) - } -+ if m.MppVersion != 0 { -+ i = encodeVarintMpp(dAtA, i, uint64(m.MppVersion)) -+ i-- -+ dAtA[i] = 0x10 -+ } - if m.Available { - i-- - if m.Available { -@@ -867,6 +1095,18 @@ func (m *DispatchTaskRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { - i -= len(m.XXX_unrecognized) - copy(dAtA[i:], m.XXX_unrecognized) - } -+ if m.ExchangeSenderMeta != nil { -+ { -+ size, err := m.ExchangeSenderMeta.MarshalToSizedBuffer(dAtA[:i]) -+ if err != nil { -+ return 0, err -+ } -+ i -= size -+ i = encodeVarintMpp(dAtA, i, uint64(size)) -+ } -+ i-- -+ dAtA[i] = 0x3a -+ } - if len(m.TableRegions) > 0 { - for iNdEx := len(m.TableRegions) - 1; iNdEx >= 0; iNdEx-- { - { -@@ -1121,6 +1361,38 @@ func (m *EstablishMPPConnectionRequest) MarshalToSizedBuffer(dAtA []byte) (int, - return len(dAtA) - i, nil - } - -+func (m *DataPacketCompressionMeta) Marshal() (dAtA []byte, err error) { -+ size := m.Size() -+ dAtA = make([]byte, size) -+ n, err := m.MarshalToSizedBuffer(dAtA[:size]) -+ if err != nil { -+ return nil, err -+ } -+ return dAtA[:n], nil -+} -+ -+func (m *DataPacketCompressionMeta) MarshalTo(dAtA []byte) (int, error) { -+ size := m.Size() -+ return m.MarshalToSizedBuffer(dAtA[:size]) -+} -+ -+func (m *DataPacketCompressionMeta) MarshalToSizedBuffer(dAtA []byte) (int, error) { -+ i := len(dAtA) -+ _ = i -+ var l int -+ _ = l -+ if m.XXX_unrecognized != nil { -+ i -= len(m.XXX_unrecognized) -+ copy(dAtA[i:], m.XXX_unrecognized) -+ } -+ if m.Mode != 0 { -+ i = encodeVarintMpp(dAtA, i, uint64(m.Mode)) -+ i-- -+ dAtA[i] = 0x8 -+ } -+ return len(dAtA) - i, nil -+} -+ - func (m *MPPDataPacket) Marshal() (dAtA []byte, err error) { - size := m.Size() - dAtA = make([]byte, size) -@@ -1145,21 +1417,38 @@ func (m *MPPDataPacket) MarshalToSizedBuffer(dAtA []byte) (int, error) { - i -= len(m.XXX_unrecognized) - copy(dAtA[i:], m.XXX_unrecognized) - } -+ if m.Compression != nil { -+ { -+ size, err := m.Compression.MarshalToSizedBuffer(dAtA[:i]) -+ if err != nil { -+ return 0, err -+ } -+ i -= size -+ i = encodeVarintMpp(dAtA, i, uint64(size)) -+ } -+ i-- -+ dAtA[i] = 0x32 -+ } -+ if m.Version != 0 { -+ i = encodeVarintMpp(dAtA, i, uint64(m.Version)) -+ i-- -+ dAtA[i] = 0x28 -+ } - if len(m.StreamIds) > 0 { -- dAtA9 := make([]byte, len(m.StreamIds)*10) -- var j8 int -+ dAtA11 := make([]byte, len(m.StreamIds)*10) -+ var j10 int - for _, num := range m.StreamIds { - for num >= 1<<7 { -- dAtA9[j8] = uint8(uint64(num)&0x7f | 0x80) -+ dAtA11[j10] = uint8(uint64(num)&0x7f | 0x80) - num >>= 7 -- j8++ -+ j10++ - } -- dAtA9[j8] = uint8(num) -- j8++ -+ dAtA11[j10] = uint8(num) -+ j10++ - } -- i -= j8 -- copy(dAtA[i:], dAtA9[:j8]) -- i = encodeVarintMpp(dAtA, i, uint64(j8)) -+ i -= j10 -+ copy(dAtA[i:], dAtA11[:j10]) -+ i = encodeVarintMpp(dAtA, i, uint64(j10)) - i-- - dAtA[i] = 0x22 - } -@@ -1218,6 +1507,11 @@ func (m *Error) MarshalToSizedBuffer(dAtA []byte) (int, error) { - i -= len(m.XXX_unrecognized) - copy(dAtA[i:], m.XXX_unrecognized) - } -+ if m.MppVersion != 0 { -+ i = encodeVarintMpp(dAtA, i, uint64(m.MppVersion)) -+ i-- -+ dAtA[i] = 0x18 -+ } - if len(m.Msg) > 0 { - i -= len(m.Msg) - copy(dAtA[i:], m.Msg) -@@ -1244,6 +1538,21 @@ func encodeVarintMpp(dAtA []byte, offset int, v uint64) int { - dAtA[offset] = uint8(v) - return base - } -+func (m *ExchangeSenderMeta) Size() (n int) { -+ if m == nil { -+ return 0 -+ } -+ var l int -+ _ = l -+ if m.Compression != 0 { -+ n += 1 + sovMpp(uint64(m.Compression)) -+ } -+ if m.XXX_unrecognized != nil { -+ n += len(m.XXX_unrecognized) -+ } -+ return n -+} -+ - func (m *TaskMeta) Size() (n int) { - if m == nil { - return 0 -@@ -1275,6 +1584,9 @@ func (m *TaskMeta) Size() (n int) { - if m.ServerId != 0 { - n += 1 + sovMpp(uint64(m.ServerId)) - } -+ if m.MppVersion != 0 { -+ n += 1 + sovMpp(uint64(m.MppVersion)) -+ } - if m.XXX_unrecognized != nil { - n += len(m.XXX_unrecognized) - } -@@ -1302,6 +1614,9 @@ func (m *IsAliveResponse) Size() (n int) { - if m.Available { - n += 2 - } -+ if m.MppVersion != 0 { -+ n += 1 + sovMpp(uint64(m.MppVersion)) -+ } - if m.XXX_unrecognized != nil { - n += len(m.XXX_unrecognized) - } -@@ -1340,6 +1655,10 @@ func (m *DispatchTaskRequest) Size() (n int) { - n += 1 + l + sovMpp(uint64(l)) - } - } -+ if m.ExchangeSenderMeta != nil { -+ l = m.ExchangeSenderMeta.Size() -+ n += 1 + l + sovMpp(uint64(l)) -+ } - if m.XXX_unrecognized != nil { - n += len(m.XXX_unrecognized) - } -@@ -1424,6 +1743,21 @@ func (m *EstablishMPPConnectionRequest) Size() (n int) { - return n - } - -+func (m *DataPacketCompressionMeta) Size() (n int) { -+ if m == nil { -+ return 0 -+ } -+ var l int -+ _ = l -+ if m.Mode != 0 { -+ n += 1 + sovMpp(uint64(m.Mode)) -+ } -+ if m.XXX_unrecognized != nil { -+ n += len(m.XXX_unrecognized) -+ } -+ return n -+} -+ - func (m *MPPDataPacket) Size() (n int) { - if m == nil { - return 0 -@@ -1451,6 +1785,13 @@ func (m *MPPDataPacket) Size() (n int) { - } - n += 1 + sovMpp(uint64(l)) + l - } -+ if m.Version != 0 { -+ n += 1 + sovMpp(uint64(m.Version)) -+ } -+ if m.Compression != nil { -+ l = m.Compression.Size() -+ n += 1 + l + sovMpp(uint64(l)) -+ } - if m.XXX_unrecognized != nil { - n += len(m.XXX_unrecognized) - } -@@ -1470,6 +1811,9 @@ func (m *Error) Size() (n int) { - if l > 0 { - n += 1 + l + sovMpp(uint64(l)) - } -+ if m.MppVersion != 0 { -+ n += 1 + sovMpp(uint64(m.MppVersion)) -+ } - if m.XXX_unrecognized != nil { - n += len(m.XXX_unrecognized) - } -@@ -1482,6 +1826,76 @@ func sovMpp(x uint64) (n int) { - func sozMpp(x uint64) (n int) { - return sovMpp(uint64((x << 1) ^ uint64((int64(x) >> 63)))) - } -+func (m *ExchangeSenderMeta) Unmarshal(dAtA []byte) error { -+ l := len(dAtA) -+ iNdEx := 0 -+ for iNdEx < l { -+ preIndex := iNdEx -+ var wire uint64 -+ for shift := uint(0); ; shift += 7 { -+ if shift >= 64 { -+ return ErrIntOverflowMpp -+ } -+ if iNdEx >= l { -+ return io.ErrUnexpectedEOF -+ } -+ b := dAtA[iNdEx] -+ iNdEx++ -+ wire |= uint64(b&0x7F) << shift -+ if b < 0x80 { -+ break -+ } -+ } -+ fieldNum := int32(wire >> 3) -+ wireType := int(wire & 0x7) -+ if wireType == 4 { -+ return fmt.Errorf("proto: ExchangeSenderMeta: wiretype end group for non-group") -+ } -+ if fieldNum <= 0 { -+ return fmt.Errorf("proto: ExchangeSenderMeta: illegal tag %d (wire type %d)", fieldNum, wire) -+ } -+ switch fieldNum { -+ case 1: -+ if wireType != 0 { -+ return fmt.Errorf("proto: wrong wireType = %d for field Compression", wireType) -+ } -+ m.Compression = 0 -+ for shift := uint(0); ; shift += 7 { -+ if shift >= 64 { -+ return ErrIntOverflowMpp -+ } -+ if iNdEx >= l { -+ return io.ErrUnexpectedEOF -+ } -+ b := dAtA[iNdEx] -+ iNdEx++ -+ m.Compression |= CompressionMode(b&0x7F) << shift -+ if b < 0x80 { -+ break -+ } -+ } -+ default: -+ iNdEx = preIndex -+ skippy, err := skipMpp(dAtA[iNdEx:]) -+ if err != nil { -+ return err -+ } -+ if (skippy < 0) || (iNdEx+skippy) < 0 { -+ return ErrInvalidLengthMpp -+ } -+ if (iNdEx + skippy) > l { -+ return io.ErrUnexpectedEOF -+ } -+ m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...) -+ iNdEx += skippy -+ } -+ } -+ -+ if iNdEx > l { -+ return io.ErrUnexpectedEOF -+ } -+ return nil -+} - func (m *TaskMeta) Unmarshal(dAtA []byte) error { - l := len(dAtA) - iNdEx := 0 -@@ -1676,6 +2090,25 @@ func (m *TaskMeta) Unmarshal(dAtA []byte) error { - break - } - } -+ case 9: -+ if wireType != 0 { -+ return fmt.Errorf("proto: wrong wireType = %d for field MppVersion", wireType) -+ } -+ m.MppVersion = 0 -+ for shift := uint(0); ; shift += 7 { -+ if shift >= 64 { -+ return ErrIntOverflowMpp -+ } -+ if iNdEx >= l { -+ return io.ErrUnexpectedEOF -+ } -+ b := dAtA[iNdEx] -+ iNdEx++ -+ m.MppVersion |= int64(b&0x7F) << shift -+ if b < 0x80 { -+ break -+ } -+ } - default: - iNdEx = preIndex - skippy, err := skipMpp(dAtA[iNdEx:]) -@@ -1798,6 +2231,25 @@ func (m *IsAliveResponse) Unmarshal(dAtA []byte) error { - } - } - m.Available = bool(v != 0) -+ case 2: -+ if wireType != 0 { -+ return fmt.Errorf("proto: wrong wireType = %d for field MppVersion", wireType) -+ } -+ m.MppVersion = 0 -+ for shift := uint(0); ; shift += 7 { -+ if shift >= 64 { -+ return ErrIntOverflowMpp -+ } -+ if iNdEx >= l { -+ return io.ErrUnexpectedEOF -+ } -+ b := dAtA[iNdEx] -+ iNdEx++ -+ m.MppVersion |= int64(b&0x7F) << shift -+ if b < 0x80 { -+ break -+ } -+ } - default: - iNdEx = preIndex - skippy, err := skipMpp(dAtA[iNdEx:]) -@@ -2025,6 +2477,42 @@ func (m *DispatchTaskRequest) Unmarshal(dAtA []byte) error { - return err - } - iNdEx = postIndex -+ case 7: -+ if wireType != 2 { -+ return fmt.Errorf("proto: wrong wireType = %d for field ExchangeSenderMeta", wireType) -+ } -+ var msglen int -+ for shift := uint(0); ; shift += 7 { -+ if shift >= 64 { -+ return ErrIntOverflowMpp -+ } -+ if iNdEx >= l { -+ return io.ErrUnexpectedEOF -+ } -+ b := dAtA[iNdEx] -+ iNdEx++ -+ msglen |= int(b&0x7F) << shift -+ if b < 0x80 { -+ break -+ } -+ } -+ if msglen < 0 { -+ return ErrInvalidLengthMpp -+ } -+ postIndex := iNdEx + msglen -+ if postIndex < 0 { -+ return ErrInvalidLengthMpp -+ } -+ if postIndex > l { -+ return io.ErrUnexpectedEOF -+ } -+ if m.ExchangeSenderMeta == nil { -+ m.ExchangeSenderMeta = &ExchangeSenderMeta{} -+ } -+ if err := m.ExchangeSenderMeta.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { -+ return err -+ } -+ iNdEx = postIndex - default: - iNdEx = preIndex - skippy, err := skipMpp(dAtA[iNdEx:]) -@@ -2501,6 +2989,76 @@ func (m *EstablishMPPConnectionRequest) Unmarshal(dAtA []byte) error { - } - return nil - } -+func (m *DataPacketCompressionMeta) Unmarshal(dAtA []byte) error { -+ l := len(dAtA) -+ iNdEx := 0 -+ for iNdEx < l { -+ preIndex := iNdEx -+ var wire uint64 -+ for shift := uint(0); ; shift += 7 { -+ if shift >= 64 { -+ return ErrIntOverflowMpp -+ } -+ if iNdEx >= l { -+ return io.ErrUnexpectedEOF -+ } -+ b := dAtA[iNdEx] -+ iNdEx++ -+ wire |= uint64(b&0x7F) << shift -+ if b < 0x80 { -+ break -+ } -+ } -+ fieldNum := int32(wire >> 3) -+ wireType := int(wire & 0x7) -+ if wireType == 4 { -+ return fmt.Errorf("proto: DataPacketCompressionMeta: wiretype end group for non-group") -+ } -+ if fieldNum <= 0 { -+ return fmt.Errorf("proto: DataPacketCompressionMeta: illegal tag %d (wire type %d)", fieldNum, wire) -+ } -+ switch fieldNum { -+ case 1: -+ if wireType != 0 { -+ return fmt.Errorf("proto: wrong wireType = %d for field Mode", wireType) -+ } -+ m.Mode = 0 -+ for shift := uint(0); ; shift += 7 { -+ if shift >= 64 { -+ return ErrIntOverflowMpp -+ } -+ if iNdEx >= l { -+ return io.ErrUnexpectedEOF -+ } -+ b := dAtA[iNdEx] -+ iNdEx++ -+ m.Mode |= CompressionMode(b&0x7F) << shift -+ if b < 0x80 { -+ break -+ } -+ } -+ default: -+ iNdEx = preIndex -+ skippy, err := skipMpp(dAtA[iNdEx:]) -+ if err != nil { -+ return err -+ } -+ if (skippy < 0) || (iNdEx+skippy) < 0 { -+ return ErrInvalidLengthMpp -+ } -+ if (iNdEx + skippy) > l { -+ return io.ErrUnexpectedEOF -+ } -+ m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...) -+ iNdEx += skippy -+ } -+ } -+ -+ if iNdEx > l { -+ return io.ErrUnexpectedEOF -+ } -+ return nil -+} - func (m *MPPDataPacket) Unmarshal(dAtA []byte) error { - l := len(dAtA) - iNdEx := 0 -@@ -2708,6 +3266,61 @@ func (m *MPPDataPacket) Unmarshal(dAtA []byte) error { - } else { - return fmt.Errorf("proto: wrong wireType = %d for field StreamIds", wireType) - } -+ case 5: -+ if wireType != 0 { -+ return fmt.Errorf("proto: wrong wireType = %d for field Version", wireType) -+ } -+ m.Version = 0 -+ for shift := uint(0); ; shift += 7 { -+ if shift >= 64 { -+ return ErrIntOverflowMpp -+ } -+ if iNdEx >= l { -+ return io.ErrUnexpectedEOF -+ } -+ b := dAtA[iNdEx] -+ iNdEx++ -+ m.Version |= int64(b&0x7F) << shift -+ if b < 0x80 { -+ break -+ } -+ } -+ case 6: -+ if wireType != 2 { -+ return fmt.Errorf("proto: wrong wireType = %d for field Compression", wireType) -+ } -+ var msglen int -+ for shift := uint(0); ; shift += 7 { -+ if shift >= 64 { -+ return ErrIntOverflowMpp -+ } -+ if iNdEx >= l { -+ return io.ErrUnexpectedEOF -+ } -+ b := dAtA[iNdEx] -+ iNdEx++ -+ msglen |= int(b&0x7F) << shift -+ if b < 0x80 { -+ break -+ } -+ } -+ if msglen < 0 { -+ return ErrInvalidLengthMpp -+ } -+ postIndex := iNdEx + msglen -+ if postIndex < 0 { -+ return ErrInvalidLengthMpp -+ } -+ if postIndex > l { -+ return io.ErrUnexpectedEOF -+ } -+ if m.Compression == nil { -+ m.Compression = &DataPacketCompressionMeta{} -+ } -+ if err := m.Compression.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { -+ return err -+ } -+ iNdEx = postIndex - default: - iNdEx = preIndex - skippy, err := skipMpp(dAtA[iNdEx:]) -@@ -2810,6 +3423,25 @@ func (m *Error) Unmarshal(dAtA []byte) error { - } - m.Msg = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex -+ case 3: -+ if wireType != 0 { -+ return fmt.Errorf("proto: wrong wireType = %d for field MppVersion", wireType) -+ } -+ m.MppVersion = 0 -+ for shift := uint(0); ; shift += 7 { -+ if shift >= 64 { -+ return ErrIntOverflowMpp -+ } -+ if iNdEx >= l { -+ return io.ErrUnexpectedEOF -+ } -+ b := dAtA[iNdEx] -+ iNdEx++ -+ m.MppVersion |= int64(b&0x7F) << shift -+ if b < 0x80 { -+ break -+ } -+ } - default: - iNdEx = preIndex - skippy, err := skipMpp(dAtA[iNdEx:]) -diff --git a/proto/mpp.proto b/proto/mpp.proto -index 6739455..6fa315e 100644 ---- a/proto/mpp.proto -+++ b/proto/mpp.proto -@@ -11,6 +11,17 @@ option (gogoproto.unmarshaler_all) = true; - - option java_package = "org.tikv.kvproto"; - -+// Data compression mode -+enum CompressionMode { -+ NONE = 0; // no compression -+ FAST = 1; // fast compression/decompression speed, compression ratio is lower than HC mode -+ HIGH_COMPRESSION = 2; // high compression (HC) ratio mode -+} -+ -+message ExchangeSenderMeta { -+ CompressionMode compression = 1; -+} -+ - // TaskMeta contains meta of a mpp plan, including query's ts and task address. - message TaskMeta { - uint64 start_ts = 1; // start ts of a query -@@ -21,6 +32,7 @@ message TaskMeta { - uint64 query_ts = 6; // timestamp when start to execute query, used for TiFlash miniTSO schedule. - uint64 local_query_id = 7; // unique local query_id if tidb don't restart. So we can use query_ts + local_query_id + server_id to represent a global unique query. - uint64 server_id = 8; // TiDB server id -+ int64 mpp_version = 9; // mpp version - } - - message IsAliveRequest { -@@ -28,6 +40,7 @@ message IsAliveRequest { - - message IsAliveResponse { - bool available = 1; -+ int64 mpp_version = 2; - } - - // Dipsatch the task request to different tiflash servers. -@@ -40,6 +53,8 @@ message DispatchTaskRequest { - int64 schema_ver = 5; - // Used for partition table scan - repeated coprocessor.TableRegions table_regions = 6; -+ // Used for exchange sender -+ ExchangeSenderMeta exchange_sender_meta = 7; - } - - // Get response of DispatchTaskRequest. -@@ -64,6 +79,10 @@ message EstablishMPPConnectionRequest { - TaskMeta receiver_meta = 2; // node closer to the tidb mpp gather. - } - -+message DataPacketCompressionMeta { -+ CompressionMode mode = 1; -+} -+ - // when TiFlash sends data to TiDB, Data packets wrap tipb.SelectResponse, i.e., serialize tipb.SelectResponse into data; - // when TiFlash sends data to TiFlash, data blocks are serialized into chunks, and the execution_summaries in tipb.SelectResponse are serialized into data only for the last packet. - message MPPDataPacket { -@@ -71,9 +90,12 @@ message MPPDataPacket { - Error error = 2; - repeated bytes chunks = 3; - repeated uint64 stream_ids = 4; -+ int64 version = 5; // version of data format -+ DataPacketCompressionMeta compression = 6; // data compression meta - } - - message Error { - int32 code = 1; - string msg = 2; -+ int64 mpp_version = 3; - } From 94d5646c3c202e7cad97e0830c2514e30a6ad994 Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Fri, 6 Jan 2023 10:35:39 +0800 Subject: [PATCH 50/93] 44 --- contrib/kvproto | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/kvproto b/contrib/kvproto index 4f3ab10cd53..f918cbbbbd8 160000 --- a/contrib/kvproto +++ b/contrib/kvproto @@ -1 +1 @@ -Subproject commit 4f3ab10cd530a0e832c790e157b8b784208d3829 +Subproject commit f918cbbbbd89cacd7ec24fb66e8f35eea0abec19 From e70fcb50e37c34221f5532a1dce92dee84a36f0a Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Fri, 6 Jan 2023 14:11:34 +0800 Subject: [PATCH 51/93] 45 --- dbms/src/Flash/Mpp/HashPartitionWriter.cpp | 25 ++++---------------- dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp | 6 ----- dbms/src/Flash/Mpp/HashPartitionWriterV1.h | 1 - 3 files changed, 5 insertions(+), 27 deletions(-) diff --git a/dbms/src/Flash/Mpp/HashPartitionWriter.cpp b/dbms/src/Flash/Mpp/HashPartitionWriter.cpp index 21be6cc1dd0..c82220d5f23 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriter.cpp +++ b/dbms/src/Flash/Mpp/HashPartitionWriter.cpp @@ -13,25 +13,11 @@ // limitations under the License. #include -#include #include -#include #include #include #include -#include -#include - -#include "Common/Exception.h" -#include "Common/Stopwatch.h" -#include "Flash/Coprocessor/CHBlockChunkCodecStream.h" -#include "Flash/Coprocessor/CompressedCHBlockChunkCodec.h" -#include "Flash/Mpp/MppVersion.h" -#include "IO/CompressedStream.h" -#include "common/logger_useful.h" -#include "ext/scope_guard.h" -#include "mpp.pb.h" namespace DB { template @@ -78,11 +64,11 @@ void HashPartitionWriter::write(const Block & block) partitionAndEncodeThenWriteBlocks(); } +extern size_t ApproxBlockBytes(const Block & block); + template void HashPartitionWriter::partitionAndEncodeThenWriteBlocks() { - assert(chunk_codec_stream); - auto tracked_packets = HashBaseWriterHelper::createPackets(partition_num); size_t ori_block_mem_size = 0; @@ -110,10 +96,9 @@ void HashPartitionWriter::partitionAndEncodeThenWriteBlocks() size_t dest_block_rows = dest_block.rows(); if (dest_block_rows > 0) { - auto * codec_stream = chunk_codec_stream.get(); - codec_stream->encode(dest_block, 0, dest_block_rows); - tracked_packets[part_id]->addChunk(codec_stream->getString()); - codec_stream->clear(); + chunk_codec_stream->encode(dest_block, 0, dest_block_rows); + tracked_packets[part_id]->addChunk(chunk_codec_stream->getString()); + chunk_codec_stream->clear(); } } } diff --git a/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp b/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp index 187357e4849..05556da00f2 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp +++ b/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp @@ -75,10 +75,6 @@ HashPartitionWriterImplV1::HashPartitionWriterImplV1( rows_in_blocks = 0; RUNTIME_CHECK(partition_num > 0); RUNTIME_CHECK(dag_context.encode_type == tipb::EncodeType::TypeCHBlock); - for (const auto & field_type : dag_context.result_field_types) - { - expected_types.emplace_back(getDataTypeByFieldTypeForComputingLayer(field_type)); - } } template @@ -144,7 +140,6 @@ void HashPartitionWriterImplV1::partitionAndEncodeThenWriteBl { const auto & block = blocks.back(); block.checkNumberOfRows(); - assertBlockSchema(expected_types, block, "HashPartitionWriterV1"); ori_block_mem_size += ApproxBlockBytes(block); total_rows += block.rows(); @@ -271,7 +266,6 @@ void HashPartitionWriterImplV1::partitionAndEncodeThenWriteBl { const auto & block = blocks.back(); block.checkNumberOfRows(); - assertBlockSchema(expected_types, block, "HashPartitionWriterV1"); ori_block_mem_size += ApproxBlockBytes(block); total_rows += block.rows(); diff --git a/dbms/src/Flash/Mpp/HashPartitionWriterV1.h b/dbms/src/Flash/Mpp/HashPartitionWriterV1.h index 5be76ae308f..55e937320b2 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriterV1.h +++ b/dbms/src/Flash/Mpp/HashPartitionWriterV1.h @@ -62,7 +62,6 @@ class HashPartitionWriterImplV1 : public DAGResponseWriter std::vector partition_col_ids; TiDB::TiDBCollators collators; size_t rows_in_blocks; - DataTypes expected_types; mpp::CompressionMode compression_mode{}; }; From 1cf3ccde02ffb9b79e8dd57200616944c3b0fb04 Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Fri, 6 Jan 2023 14:13:25 +0800 Subject: [PATCH 52/93] 46 --- dbms/src/Flash/Mpp/HashPartitionWriter.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/dbms/src/Flash/Mpp/HashPartitionWriter.cpp b/dbms/src/Flash/Mpp/HashPartitionWriter.cpp index c82220d5f23..9125c0a1566 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriter.cpp +++ b/dbms/src/Flash/Mpp/HashPartitionWriter.cpp @@ -14,6 +14,7 @@ #include #include +#include #include #include #include From 7e2aaadc109c4c7fc812e0fdb132c4fb028f6579 Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Fri, 6 Jan 2023 14:15:13 +0800 Subject: [PATCH 53/93] 47 --- dbms/src/Flash/Mpp/HashPartitionWriter.h | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/dbms/src/Flash/Mpp/HashPartitionWriter.h b/dbms/src/Flash/Mpp/HashPartitionWriter.h index 52928b99e0f..eb82c54592e 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriter.h +++ b/dbms/src/Flash/Mpp/HashPartitionWriter.h @@ -15,15 +15,13 @@ #pragma once #include +#include #include #include #include - namespace DB { -class DAGContext; - template class HashPartitionWriter : public DAGResponseWriter { @@ -50,7 +48,6 @@ class HashPartitionWriter : public DAGResponseWriter TiDB::TiDBCollators collators; size_t rows_in_blocks; uint16_t partition_num; - std::unique_ptr chunk_codec_stream; }; From 1719462a2fab6f30e5a42d2d3e23f34d48cdb8c8 Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Fri, 6 Jan 2023 14:16:28 +0800 Subject: [PATCH 54/93] 48 --- dbms/src/Flash/Mpp/HashPartitionWriter.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/dbms/src/Flash/Mpp/HashPartitionWriter.cpp b/dbms/src/Flash/Mpp/HashPartitionWriter.cpp index 9125c0a1566..c82220d5f23 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriter.cpp +++ b/dbms/src/Flash/Mpp/HashPartitionWriter.cpp @@ -14,7 +14,6 @@ #include #include -#include #include #include #include From c0be2b40323210082d37ac0f5069f5595d6d09de Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Fri, 6 Jan 2023 18:02:05 +0800 Subject: [PATCH 55/93] 49 Signed-off-by: Zhigao Tong --- .gitmodules | 2 +- CMakeLists.txt | 15 +- cmake/cpu_features.cmake | 2 + contrib/kvproto | 2 +- contrib/tipb | 2 +- dbms/CMakeLists.txt | 15 +- dbms/src/Common/FailPoint.cpp | 1 + dbms/src/Common/TiFlashBuildInfo.cpp | 4 +- dbms/src/Common/TiFlashMetrics.h | 7 +- .../Flash/Coprocessor/CHBlockChunkCodec.cpp | 107 ++++----- .../Coprocessor/CHBlockChunkCodecStream.h | 54 ----- ...odecStream.cpp => CHBlockChunkCodecV1.cpp} | 38 ++-- ...lockChunkCodec.h => CHBlockChunkCodecV1.h} | 19 +- .../Coprocessor/ChunkDecodeAndSquash.cpp | 26 +-- .../Flash/Coprocessor/ChunkDecodeAndSquash.h | 8 +- .../CompressCHBlockChunkCodecStream.h | 126 ----------- dbms/src/Flash/Coprocessor/DAGContext.h | 5 +- .../Coprocessor/DAGQueryBlockInterpreter.cpp | 4 +- dbms/src/Flash/Coprocessor/tzg-metrics.h | 121 ---------- dbms/src/Flash/FlashService.cpp | 21 +- .../Mpp/BroadcastOrPassThroughWriter.cpp | 12 + dbms/src/Flash/Mpp/ExchangeReceiver.cpp | 11 +- .../Flash/Mpp/FineGrainedShuffleWriter.cpp | 11 +- dbms/src/Flash/Mpp/HashPartitionWriter.cpp | 13 +- dbms/src/Flash/Mpp/HashPartitionWriter.h | 3 +- dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp | 208 ++++-------------- dbms/src/Flash/Mpp/HashPartitionWriterV1.h | 27 +-- dbms/src/Flash/Mpp/MPPTask.cpp | 5 +- dbms/src/Flash/Mpp/MPPTunnel.cpp | 9 +- dbms/src/Flash/Mpp/MPPTunnel.h | 2 - dbms/src/Flash/Mpp/MppVersion.h | 14 +- dbms/src/Flash/Mpp/Utils.cpp | 48 ++-- dbms/src/Flash/Mpp/newMPPExchangeWriter.h | 17 +- .../Mpp/tests/gtest_mpp_exchange_writer.cpp | 138 +++++++----- .../Planner/plans/PhysicalExchangeSender.cpp | 11 +- .../Planner/plans/PhysicalExchangeSender.h | 5 +- dbms/src/Functions/CMakeLists.txt | 7 +- dbms/src/IO/CompressedReadBufferBase.cpp | 13 +- dbms/src/IO/CompressedStream.h | 2 - dbms/src/IO/CompressedWriteBuffer.cpp | 7 +- dbms/src/IO/CompressedWriteBuffer.h | 2 +- dbms/src/Interpreters/Settings.h | 1 + dbms/src/Storages/Transaction/PDTiKVClient.h | 2 +- .../Transaction/ProxyFFIStatusService.cpp | 141 +----------- .../Storages/Transaction/ReadIndexWorker.cpp | 2 +- libs/libcommon/CMakeLists.txt | 19 +- libs/libcommon/include/common/avx2_memcpy.h | 2 +- libs/libcommon/include/common/sse2_memcpy.h | 134 ++++------- libs/libmemcpy/CMakeLists.txt | 6 +- libs/libsnappy/CMakeLists.txt | 27 --- tests/fullstack-test2/mpp/mpp-version.test | 48 ++++ tiflash-tools/.gitignore | 5 - tiflash-tools/define.py | 18 -- tiflash-tools/flash_cluster_manager.py | 138 ------------ tiflash-tools/flash_http_client.py | 51 ----- tiflash-tools/pd_client.py | 126 ----------- tiflash-tools/util.py | 138 ------------ 57 files changed, 454 insertions(+), 1548 deletions(-) delete mode 100644 dbms/src/Flash/Coprocessor/CHBlockChunkCodecStream.h rename dbms/src/Flash/Coprocessor/{CompressCHBlockChunkCodecStream.cpp => CHBlockChunkCodecV1.cpp} (86%) rename dbms/src/Flash/Coprocessor/{CompressedCHBlockChunkCodec.h => CHBlockChunkCodecV1.h} (62%) delete mode 100644 dbms/src/Flash/Coprocessor/CompressCHBlockChunkCodecStream.h delete mode 100644 dbms/src/Flash/Coprocessor/tzg-metrics.h delete mode 100644 libs/libsnappy/CMakeLists.txt create mode 100644 tests/fullstack-test2/mpp/mpp-version.test delete mode 100644 tiflash-tools/.gitignore delete mode 100644 tiflash-tools/define.py delete mode 100644 tiflash-tools/flash_cluster_manager.py delete mode 100644 tiflash-tools/flash_http_client.py delete mode 100644 tiflash-tools/pd_client.py delete mode 100644 tiflash-tools/util.py diff --git a/.gitmodules b/.gitmodules index 8ca783471b0..44bb7d920ab 100644 --- a/.gitmodules +++ b/.gitmodules @@ -24,7 +24,7 @@ url = https://github.com/pingcap/boost-extra.git [submodule "contrib/kvproto"] path = contrib/kvproto - url = https://github.com/solotzg/kvproto.git + url = https://github.com/pingcap/kvproto.git [submodule "contrib/tipb"] path = contrib/tipb url = https://github.com/pingcap/tipb.git diff --git a/CMakeLists.txt b/CMakeLists.txt index 6d4fb10d919..a851788034e 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -144,15 +144,16 @@ if (USE_STATIC_LIBRARIES) list(REVERSE CMAKE_FIND_LIBRARY_SUFFIXES) endif () -if (OS_LINUX AND ARCH_AMD64 AND NOT UNBUNDLED AND MAKE_STATIC_LIBRARIES AND NOT SPLIT_SHARED_LIBRARIES) - # Only for Linux (x86_64). - option(GLIBC_COMPATIBILITY "Enable compatibility with older glibc libraries." ON) -elseif (GLIBC_COMPATIBILITY) - message (STATUS "Glibc compatibility cannot be enabled in current configuration") +if (CMAKE_LIBRARY_ARCHITECTURE MATCHES "amd64.*|x86_64.*|AMD64.*") + option (USE_INTERNAL_MEMCPY "Use internal implementation of 'memcpy' function instead of provided by libc. Only for x86_64." ON) + + if (OS_LINUX) + option (GLIBC_COMPATIBILITY "Set to TRUE to enable compatibility with older glibc libraries. Only for x86_64, Linux. Implies USE_INTERNAL_MEMCPY." ON) + endif() endif () -if (OS_LINUX AND ARCH_AMD64) - option (USE_INTERNAL_MEMCPY "Use internal implementation of 'memcpy' function instead of provided by libc. Only for x86_64." ON) +if (GLIBC_COMPATIBILITY) + set (USE_INTERNAL_MEMCPY ON) endif () option (PIPE "-pipe compiler option [less /tmp usage, more ram usage]" ON) diff --git a/cmake/cpu_features.cmake b/cmake/cpu_features.cmake index 603d2c2e878..3f6aad2ed05 100644 --- a/cmake/cpu_features.cmake +++ b/cmake/cpu_features.cmake @@ -213,6 +213,8 @@ elseif (ARCH_AMD64) check_cxx_compiler_flag("${TIFLASH_COMPILER_MOVBE_FLAG}" TIFLASH_COMPILER_MOVBE_SUPPORT) set (TIFLASH_COMPILER_BMI2_FLAG "-mbmi2") check_cxx_compiler_flag("${TIFLASH_COMPILER_BMI2_FLAG}" TIFLASH_COMPILER_BMI2_SUPPORT) + + # `haswell` was released since 2013 with cpu feature avx2, bmi2. It's a practical arch for optimizer set (TIFLASH_COMPILER_ARCH_HASWELL_FLAG "-march=haswell") check_cxx_compiler_flag("${TIFLASH_COMPILER_ARCH_HASWELL_FLAG}" TIFLASH_COMPILER_ARCH_HASWELL_SUPPORT) else () diff --git a/contrib/kvproto b/contrib/kvproto index f918cbbbbd8..7cd28226c2a 160000 --- a/contrib/kvproto +++ b/contrib/kvproto @@ -1 +1 @@ -Subproject commit f918cbbbbd89cacd7ec24fb66e8f35eea0abec19 +Subproject commit 7cd28226c2a21e489b077a87e1f5c9ba2c950944 diff --git a/contrib/tipb b/contrib/tipb index 2fb82891081..566ee9ba807 160000 --- a/contrib/tipb +++ b/contrib/tipb @@ -1 +1 @@ -Subproject commit 2fb8289108131ccd5faf86d3e92fb05d4ff3b326 +Subproject commit 566ee9ba807d80382a2cdf2b0c0ef267ef5b85e8 diff --git a/dbms/CMakeLists.txt b/dbms/CMakeLists.txt index b8613754165..c30eb79b09b 100644 --- a/dbms/CMakeLists.txt +++ b/dbms/CMakeLists.txt @@ -105,15 +105,10 @@ add_headers_and_sources(dbms src/Client) add_headers_only(dbms src/Flash/Coprocessor) add_headers_only(dbms src/Server) -add_sources_compile_flag_avx2 ( - src/Columns/ColumnString.cpp - src/Columns/ColumnsCommon.cpp - src/Columns/ColumnVector.cpp - src/DataTypes/DataTypeString.cpp -) check_then_add_sources_compile_flag ( - TIFLASH_COMPILER_BMI2_SUPPORT - "${TIFLASH_COMPILER_BMI2_FLAG}" + TIFLASH_COMPILER_ARCH_HASWELL_SUPPORT + "${TIFLASH_COMPILER_ARCH_HASWELL_FLAG}" + src/Columns/ColumnString.cpp src/Columns/ColumnsCommon.cpp src/Columns/ColumnVector.cpp src/DataTypes/DataTypeString.cpp @@ -211,8 +206,6 @@ target_link_libraries (clickhouse_common_io libsymbolization ) -# target_link_libraries (clickhouse_common_io PRIVATE snappy) - target_include_directories (clickhouse_common_io BEFORE PRIVATE ${kvClient_SOURCE_DIR}/include) target_compile_definitions(clickhouse_common_io PUBLIC -DTIFLASH_SOURCE_PREFIX=\"${TiFlash_SOURCE_DIR}\") target_link_libraries (dbms @@ -309,7 +302,7 @@ if (ENABLE_TESTS) macro(grep_gtest_sources BASE_DIR DST_VAR) # Cold match files that are not in tests/ directories - file(GLOB_RECURSE "${DST_VAR}" RELATIVE "${BASE_DIR}" "gtest_mpp_exchange_writer.cpp") + file(GLOB_RECURSE "${DST_VAR}" RELATIVE "${BASE_DIR}" "gtest*.cpp") endmacro() macro(grep_bench_sources BASE_DIR DST_VAR) diff --git a/dbms/src/Common/FailPoint.cpp b/dbms/src/Common/FailPoint.cpp index 7bca6f8c6ad..26d9e6802b9 100644 --- a/dbms/src/Common/FailPoint.cpp +++ b/dbms/src/Common/FailPoint.cpp @@ -95,6 +95,7 @@ std::unordered_map> FailPointHelper::f M(force_ingest_via_replace) \ M(unblock_query_init_after_write) \ M(exception_in_merged_task_init) \ + M(invalid_mpp_version) \ M(force_fail_in_flush_region_data) diff --git a/dbms/src/Common/TiFlashBuildInfo.cpp b/dbms/src/Common/TiFlashBuildInfo.cpp index cc558c2a43a..8e34e62655a 100644 --- a/dbms/src/Common/TiFlashBuildInfo.cpp +++ b/dbms/src/Common/TiFlashBuildInfo.cpp @@ -14,7 +14,6 @@ #include #include -#include #include #include #include @@ -146,7 +145,6 @@ void outputDetail(std::ostream & os) << "Git Branch: " << getGitBranch() << std::endl << "UTC Build Time: " << getUTCBuildTime() << std::endl << "Enable Features: " << getEnabledFeatures() << std::endl - << "Profile: " << getProfile() << std::endl - << "Mpp Version: " << fmt::format("{} (release version {})", TiDB::GetMppVersion(), TiDB::GetMppVersionReleaseInfo(TiDB::GetMppVersion())) << std::endl; + << "Profile: " << getProfile() << std::endl; } } // namespace TiFlashBuildInfo diff --git a/dbms/src/Common/TiFlashMetrics.h b/dbms/src/Common/TiFlashMetrics.h index e7ca8450bcd..f497b2960cb 100644 --- a/dbms/src/Common/TiFlashMetrics.h +++ b/dbms/src/Common/TiFlashMetrics.h @@ -88,10 +88,13 @@ namespace DB F(type_cancel_mpp_task, {{"type", "cancel_mpp_task"}})) \ M(tiflash_exchange_data_bytes, "Total bytes of exchange operator", Counter, \ F(type_hash_original_all, {"type", "hash_original_all"}), \ - F(type_hash_none, {"type", "hash_none"}), \ + F(type_hash_none_remote, {"type", "hash_none_remote"}), \ F(type_hash_none_local, {"type", "hash_none_local"}), \ F(type_hash_lz4, {"type", "hash_lz4"}), \ - F(type_hash_zstd, {"type", "hash_zstd"})) \ + F(type_hash_zstd, {"type", "hash_zstd"}), \ + F(type_broadcast_passthrough_original_all, {"type", "broadcast_passthrough_original_all"}), \ + F(type_broadcast_passthrough_none_local, {"type", "broadcast_passthrough_none_local"}), \ + ) \ M(tiflash_schema_version, "Current version of tiflash cached schema", Gauge) \ M(tiflash_schema_applying, "Whether the schema is applying or not (holding lock)", Gauge) \ M(tiflash_schema_apply_count, "Total number of each kinds of apply", Counter, F(type_diff, {"type", "diff"}), \ diff --git a/dbms/src/Flash/Coprocessor/CHBlockChunkCodec.cpp b/dbms/src/Flash/Coprocessor/CHBlockChunkCodec.cpp index 938bf71618c..a44da75c506 100644 --- a/dbms/src/Flash/Coprocessor/CHBlockChunkCodec.cpp +++ b/dbms/src/Flash/Coprocessor/CHBlockChunkCodec.cpp @@ -17,48 +17,37 @@ #include #include #include -#include -#include #include -#include -#include #include -#include "ext/scope_guard.h" -#include "mpp.pb.h" - namespace DB { - -CHBlockChunkCodecStream::CHBlockChunkCodecStream(const std::vector & field_types) - : ChunkCodecStream(field_types) +class CHBlockChunkCodecStream : public ChunkCodecStream { - for (const auto & field_type : field_types) +public: + explicit CHBlockChunkCodecStream(const std::vector & field_types) + : ChunkCodecStream(field_types) { - expected_types.emplace_back(getDataTypeByFieldTypeForComputingLayer(field_type)); + for (const auto & field_type : field_types) + { + expected_types.emplace_back(getDataTypeByFieldTypeForComputingLayer(field_type)); + } } -} -String CHBlockChunkCodecStream::getString() -{ - if (output == nullptr) + String getString() override { - throw Exception("The output should not be null in getString()"); + if (output == nullptr) + { + throw Exception("The output should not be null in getString()"); + } + return output->releaseStr(); } - return output->releaseStr(); -} - -WriteBuffer * CHBlockChunkCodecStream::initOutputBuffer(size_t init_size) -{ - assert(output == nullptr); - output = std::make_unique(init_size); - return output.get(); -} -void CHBlockChunkCodecStream::clear() -{ - output = nullptr; -} + void clear() override { output = nullptr; } + void encode(const Block & block, size_t start, size_t end) override; + std::unique_ptr output; + DataTypes expected_types; +}; CHBlockChunkCodec::CHBlockChunkCodec( const Block & header_) @@ -74,17 +63,15 @@ CHBlockChunkCodec::CHBlockChunkCodec(const DAGSchema & schema) output_names.push_back(c.first); } -size_t GetExtraInfoSize(const Block & block) +size_t getExtraInfoSize(const Block & block) { - size_t size = 8 + 8; /// to hold some length of structures, such as column number, row number... + size_t size = 64; /// to hold some length of structures, such as column number, row number... size_t columns = block.columns(); for (size_t i = 0; i < columns; ++i) { const ColumnWithTypeAndName & column = block.safeGetByPosition(i); size += column.name.size(); - size += 8; size += column.type->getName().size(); - size += 8; if (column.column->isColumnConst()) { size += column.column->byteSize() * column.column->size(); @@ -93,11 +80,6 @@ size_t GetExtraInfoSize(const Block & block) return size; } -size_t ApproxBlockBytes(const Block & block) -{ - return block.bytes() + GetExtraInfoSize(block); -} - void WriteColumnData(const IDataType & type, const ColumnPtr & column, WriteBuffer & ostr, size_t offset, size_t limit) { /** If there are columns-constants - then we materialize them. @@ -124,26 +106,6 @@ void CHBlockChunkCodec::readData(const IDataType & type, IColumn & column, ReadB type.deserializeBinaryBulkWithMultipleStreams(column, input_stream_getter, rows, 0, false, {}); } -void EncodeCHBlockChunk(WriteBuffer * ostr_ptr, const Block & block) -{ - size_t columns = block.columns(); - size_t rows = block.rows(); - - writeVarUInt(columns, *ostr_ptr); - writeVarUInt(rows, *ostr_ptr); - - for (size_t i = 0; i < columns; i++) - { - const ColumnWithTypeAndName & column = block.safeGetByPosition(i); - - writeStringBinary(column.name, *ostr_ptr); - writeStringBinary(column.type->getName(), *ostr_ptr); - - if (rows) - WriteColumnData(*column.type, column.column, *ostr_ptr, 0, 0); - } -} - void CHBlockChunkCodecStream::encode(const Block & block, size_t start, size_t end) { /// only check block schema in CHBlock codec because for both @@ -154,28 +116,36 @@ void CHBlockChunkCodecStream::encode(const Block & block, size_t start, size_t e if (start != 0 || end != block.rows()) throw TiFlashException("CHBlock encode only support encode whole block", Errors::Coprocessor::Internal); + assert(output == nullptr); + output = std::make_unique(block.bytes() + getExtraInfoSize(block)); + block.checkNumberOfRows(); + size_t columns = block.columns(); + size_t rows = block.rows(); - size_t init_size = ApproxBlockBytes(block); - WriteBuffer * ostr_ptr = initOutputBuffer(init_size); + writeVarUInt(columns, *output); + writeVarUInt(rows, *output); - return EncodeCHBlockChunk(ostr_ptr, block); -} + for (size_t i = 0; i < columns; i++) + { + const ColumnWithTypeAndName & column = block.safeGetByPosition(i); -std::unique_ptr NewCHBlockChunkCodecStream(const std::vector & field_types) -{ - return std::make_unique(field_types); + writeStringBinary(column.name, *output); + writeStringBinary(column.type->getName(), *output); + + if (rows) + WriteColumnData(*column.type, column.column, *output, 0, 0); + } } std::unique_ptr CHBlockChunkCodec::newCodecStream(const std::vector & field_types) { - return NewCHBlockChunkCodecStream(field_types); + return std::make_unique(field_types); } Block CHBlockChunkCodec::decodeImpl(ReadBuffer & istr, size_t reserve_size) { Block res; - if (istr.eof()) { return res; @@ -253,5 +223,4 @@ Block CHBlockChunkCodec::decode(const String & str, const Block & header) ReadBufferFromString read_buffer(str); return CHBlockChunkCodec(header).decodeImpl(read_buffer); } - } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/CHBlockChunkCodecStream.h b/dbms/src/Flash/Coprocessor/CHBlockChunkCodecStream.h deleted file mode 100644 index a9e81a530de..00000000000 --- a/dbms/src/Flash/Coprocessor/CHBlockChunkCodecStream.h +++ /dev/null @@ -1,54 +0,0 @@ -// Copyright 2022 PingCAP, Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -#pragma once - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace mpp -{ -enum CompressionMode : int; -} -namespace DB -{ -class CHBlockChunkCodecStream : public ChunkCodecStream -{ -public: - explicit CHBlockChunkCodecStream(const std::vector & field_types); - String getString() override; - void clear() override; - void encode(const Block & block, size_t start, size_t end) override; - ~CHBlockChunkCodecStream() override = default; - -private: - WriteBuffer * initOutputBuffer(size_t init_size); - std::unique_ptr output; - DataTypes expected_types; -}; - -size_t GetExtraInfoSize(const Block & block); -size_t ApproxBlockBytes(const Block & block); -std::unique_ptr NewCHBlockChunkCodecStream(const std::vector & field_types); -void EncodeCHBlockChunk(WriteBuffer * ostr_ptr, const Block & block); - -} // namespace DB diff --git a/dbms/src/Flash/Coprocessor/CompressCHBlockChunkCodecStream.cpp b/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.cpp similarity index 86% rename from dbms/src/Flash/Coprocessor/CompressCHBlockChunkCodecStream.cpp rename to dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.cpp index 92452268608..55725992b53 100644 --- a/dbms/src/Flash/Coprocessor/CompressCHBlockChunkCodecStream.cpp +++ b/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.cpp @@ -1,8 +1,7 @@ -#include "CompressCHBlockChunkCodecStream.h" - #include +#include #include #include @@ -12,6 +11,30 @@ namespace DB { extern void WriteColumnData(const IDataType & type, const ColumnPtr & column, WriteBuffer & ostr, size_t offset, size_t limit); +size_t GetExtraInfoSize(const Block & block) +{ + size_t size = 8 + 8; /// to hold some length of structures, such as column number, row number... + size_t columns = block.columns(); + for (size_t i = 0; i < columns; ++i) + { + const ColumnWithTypeAndName & column = block.safeGetByPosition(i); + size += column.name.size(); + size += 8; + size += column.type->getName().size(); + size += 8; + if (column.column->isColumnConst()) + { + size += column.column->byteSize() * column.column->size(); + } + } + return size; +} + +size_t ApproxBlockBytes(const Block & block) +{ + return block.bytes() + GetExtraInfoSize(block); +} + void EncodeHeader(WriteBuffer & ostr, const Block & header, size_t rows) { size_t columns = header.columns(); @@ -26,17 +49,6 @@ void EncodeHeader(WriteBuffer & ostr, const Block & header, size_t rows) } } -void EncodeColumn__(WriteBuffer & ostr, const ColumnPtr & column, const ColumnWithTypeAndName & type_name) -{ - writeVarUInt(column->size(), ostr); - WriteColumnData(*type_name.type, column, ostr, 0, 0); -} - -std::unique_ptr NewCompressCHBlockChunkCodecStream(CompressionMethod compression_method) -{ - return std::make_unique(compression_method); -} - Block DecodeHeader(ReadBuffer & istr, const Block & header, size_t & total_rows) { Block res; diff --git a/dbms/src/Flash/Coprocessor/CompressedCHBlockChunkCodec.h b/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.h similarity index 62% rename from dbms/src/Flash/Coprocessor/CompressedCHBlockChunkCodec.h rename to dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.h index 831ef536954..d5f6966097e 100644 --- a/dbms/src/Flash/Coprocessor/CompressedCHBlockChunkCodec.h +++ b/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.h @@ -19,19 +19,14 @@ #include #include -namespace mpp -{ -enum CompressMethod : int; -} - namespace DB { -class CompressedCHBlockChunkCodec -{ -public: - using CompressedReadBuffer = CompressedReadBuffer; - using CompressedWriteBuffer = CompressedWriteBuffer; -}; - +size_t ApproxBlockBytes(const Block & block); +size_t GetExtraInfoSize(const Block & block); +using CompressedCHBlockChunkReadBuffer = CompressedReadBuffer; +using CompressedCHBlockChunkWriteBuffer = CompressedWriteBuffer; +void EncodeHeader(WriteBuffer & ostr, const Block & header, size_t rows); +void DecodeColumns(ReadBuffer & istr, Block & res, size_t rows_to_read, size_t reserve_size = 0); +Block DecodeHeader(ReadBuffer & istr, const Block & header, size_t & rows); } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.cpp b/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.cpp index 1e0ccb8d8d4..776ba293c6f 100644 --- a/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.cpp +++ b/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.cpp @@ -12,17 +12,11 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include #include -#include +#include #include -#include -#include - -#include "Flash/Coprocessor/CHBlockChunkCodecStream.h" -#include "Flash/Coprocessor/CompressCHBlockChunkCodecStream.h" -#include "Flash/Mpp/TrackedMppDataPacket.h" - namespace DB { CHBlockChunkDecodeAndSquash::CHBlockChunkDecodeAndSquash( @@ -33,16 +27,20 @@ CHBlockChunkDecodeAndSquash::CHBlockChunkDecodeAndSquash( { } -std::optional CHBlockChunkDecodeAndSquash::decodeAndSquash(std::string_view sv, bool compress) +std::optional CHBlockChunkDecodeAndSquash::decodeAndSquashWithCompression(std::string_view sv) { + if (static_cast(sv[0]) == CompressionMethodByte::NONE) + { + ReadBufferFromString istr(sv.substr(1, sv.size() - 1)); + return decodeAndSquashWithCompressionImpl(istr); + } + ReadBufferFromString istr(sv); - if (!compress) - return decodeAndSquashWithCompressImpl(istr); - auto && compress_buffer = CompressedCHBlockChunkCodec::CompressedReadBuffer(istr); - return decodeAndSquashWithCompressImpl(compress_buffer); + auto && compress_buffer = CompressedCHBlockChunkReadBuffer(istr); + return decodeAndSquashWithCompressionImpl(compress_buffer); } -std::optional CHBlockChunkDecodeAndSquash::decodeAndSquashWithCompressImpl(ReadBuffer & istr) +std::optional CHBlockChunkDecodeAndSquash::decodeAndSquashWithCompressionImpl(ReadBuffer & istr) { std::optional res; diff --git a/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.h b/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.h index 340479361ba..2c10da769e0 100644 --- a/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.h +++ b/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.h @@ -18,10 +18,6 @@ #include #include -#include - -#include "mpp.pb.h" - namespace DB { @@ -31,11 +27,11 @@ class CHBlockChunkDecodeAndSquash CHBlockChunkDecodeAndSquash(const Block & header, size_t rows_limit_); ~CHBlockChunkDecodeAndSquash() = default; std::optional decodeAndSquash(const String &); - std::optional decodeAndSquash(std::string_view, bool compress); + std::optional decodeAndSquashWithCompression(std::string_view); std::optional flush(); private: - std::optional decodeAndSquashWithCompressImpl(ReadBuffer & istr); + std::optional decodeAndSquashWithCompressionImpl(ReadBuffer & istr); private: CHBlockChunkCodec codec; diff --git a/dbms/src/Flash/Coprocessor/CompressCHBlockChunkCodecStream.h b/dbms/src/Flash/Coprocessor/CompressCHBlockChunkCodecStream.h deleted file mode 100644 index be27090e224..00000000000 --- a/dbms/src/Flash/Coprocessor/CompressCHBlockChunkCodecStream.h +++ /dev/null @@ -1,126 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include -#include - -#include "Columns/IColumn.h" -#include "CompressedCHBlockChunkCodec.h" - -namespace DB -{ - -struct WriteBufferFromOwnStringList final - : public WriteBuffer - , public boost::noncopyable -{ - WriteBufferFromOwnStringList() - : WriteBuffer(nullptr, 0) - { - reset(); - } - - void nextImpl() override - { - buffs.emplace_back(std::string(128 * 1024, 0)); // 128KB - WriteBuffer::set(buffs.back().data(), buffs.back().size()); - }; - - std::string getString() - { - next(); - - if (!buffs.empty() && buffs[0].size() >= bytes) - { - buffs[0].resize(bytes); - return std::move(buffs[0]); - } - - std::string res; - res.resize(bytes); - for (size_t sz = 0; sz < bytes;) - { - for (auto && s : buffs) - { - if (sz + s.size() < bytes) - { - std::memcpy(res.data() + sz, s.data(), s.size()); - sz += s.size(); - } - else - { - std::memcpy(res.data() + sz, s.data(), bytes - sz); - sz = bytes; - break; - } - } - } - return res; - } - - void reset() - { - buffs.clear(); - nextImpl(); - bytes = 0; - } - - std::vector buffs; -}; - -struct CompressCHBlockChunkCodecStream -{ - explicit CompressCHBlockChunkCodecStream(CompressionMethod compression_method_) - : compression_method(compression_method_) - { - output_buffer = std::make_unique(); - compress_write_buffer = std::make_unique( - *output_buffer, - CompressionSettings(compression_method), - DBMS_DEFAULT_BUFFER_SIZE); - } - - void reset() const - { - compress_write_buffer->next(); - output_buffer->reset(); - } - - WriteBufferFromOwnStringList * getWriterWithoutCompress() const - { - return output_buffer.get(); - } - - CompressedCHBlockChunkCodec::CompressedWriteBuffer * getWriter() const - { - return compress_write_buffer.get(); - } - - std::string getString() const - { - if (compress_write_buffer == nullptr) - { - throw Exception("The output should not be null in getString()"); - } - compress_write_buffer->next(); - return output_buffer->getString(); - } - ~CompressCHBlockChunkCodecStream() = default; - - // bool enable_compress{true}; - CompressionMethod compression_method; - std::unique_ptr output_buffer{}; - std::unique_ptr compress_write_buffer{}; -}; - -void EncodeHeader(WriteBuffer & ostr, const Block & header, size_t rows); -void EncodeColumn__(WriteBuffer & ostr, const ColumnPtr & column, const ColumnWithTypeAndName & type_name); -void DecodeColumns(ReadBuffer & istr, Block & res, size_t rows_to_read, size_t reserve_size = 0); -Block DecodeHeader(ReadBuffer & istr, const Block & header, size_t & rows); - -std::unique_ptr NewCompressCHBlockChunkCodecStream(CompressionMethod compression_method); -} // namespace DB \ No newline at end of file diff --git a/dbms/src/Flash/Coprocessor/DAGContext.h b/dbms/src/Flash/Coprocessor/DAGContext.h index 00a6335ed0d..5c37300c99d 100644 --- a/dbms/src/Flash/Coprocessor/DAGContext.h +++ b/dbms/src/Flash/Coprocessor/DAGContext.h @@ -156,7 +156,7 @@ class DAGContext } // for mpp - DAGContext(const tipb::DAGRequest & dag_request_, const mpp::TaskMeta & meta_, const mpp::ExchangeSenderMeta & exchange_sender_meta_, bool is_root_mpp_task_) + DAGContext(const tipb::DAGRequest & dag_request_, const mpp::TaskMeta & meta_, bool is_root_mpp_task_) : dag_request(&dag_request_) , dummy_query_string(dag_request->DebugString()) , dummy_ast(makeDummyQuery()) @@ -167,7 +167,6 @@ class DAGContext , flags(dag_request->flags()) , sql_mode(dag_request->sql_mode()) , mpp_task_meta(meta_) - , exchange_sender_meta(exchange_sender_meta_) , mpp_task_id(mpp_task_meta) , max_recorded_error_count(getMaxErrorCount(*dag_request)) , warnings(max_recorded_error_count) @@ -257,7 +256,6 @@ class DAGContext } UInt64 getWarningCount() { return warning_count; } const mpp::TaskMeta & getMPPTaskMeta() const { return mpp_task_meta; } - const mpp::ExchangeSenderMeta & getExchangeSenderMeta() const { return exchange_sender_meta; } bool isBatchCop() const { return is_batch_cop; } bool isMPPTask() const { return is_mpp_task; } /// root mpp task means mpp task that send data back to TiDB @@ -409,7 +407,6 @@ class DAGContext UInt64 flags; UInt64 sql_mode; mpp::TaskMeta mpp_task_meta; - mpp::ExchangeSenderMeta exchange_sender_meta; const MPPTaskId mpp_task_id = MPPTaskId::unknown_mpp_task_id; /// max_recorded_error_count is the max error/warning need to be recorded in warnings UInt64 max_recorded_error_count; diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp index a13b4b08997..5bfae6cb52e 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp @@ -801,7 +801,9 @@ void DAGQueryBlockInterpreter::handleExchangeSender(DAGPipeline & pipeline) dagContext(), enable_fine_grained_shuffle, stream_count, - batch_size); + batch_size, + exchange_sender.compression(), + context.getSettingsRef().batch_send_min_limit_compression); stream = std::make_shared(stream, std::move(response_writer), log->identifier()); stream->setExtraInfo(extra_info); }); diff --git a/dbms/src/Flash/Coprocessor/tzg-metrics.h b/dbms/src/Flash/Coprocessor/tzg-metrics.h deleted file mode 100644 index f49179d93d6..00000000000 --- a/dbms/src/Flash/Coprocessor/tzg-metrics.h +++ /dev/null @@ -1,121 +0,0 @@ -#pragma once - -#include -#include -#include - -namespace tzg -{ -struct SnappyStatistic -{ - enum CompressMethod : int - { - NONE = 0, - LZ4 = 1, - ZSTD = 2, - }; - - mutable std::atomic_uint64_t compressed_size{}; - mutable std::atomic_uint64_t uncompressed_size{}; - mutable std::atomic_uint64_t package{}; - mutable std::atomic_int64_t chunck_stream_cnt{}, max_chunck_stream_cnt{}; - mutable CompressMethod method{}; - mutable std::atomic durations{}, has_write_dur{}; - mutable std::atomic_uint64_t encode_bytes{}, has_write_rows{}; - - SnappyStatistic(const SnappyStatistic &) = delete; - - void clear() - { - compressed_size = {}; - uncompressed_size = {}; - package = {}; - } - - uint64_t getCompressedSize() const - { - return compressed_size; - } - uint64_t getUncompressedSize() const - { - return uncompressed_size; - } - - CompressMethod getMethod() const - { - return method; - } - void setMethod(CompressMethod m) - { - method = m; - } - int64_t getChunckStreamCnt() const - { - return chunck_stream_cnt; - } - int64_t getMaxChunckStreamCnt() const - { - return max_chunck_stream_cnt; - } - void addChunckStreamCnt(int x = 1) const - { - chunck_stream_cnt += x; - max_chunck_stream_cnt = std::max(max_chunck_stream_cnt.load(), chunck_stream_cnt.load()); - } - - void update(uint64_t compressed_size_, uint64_t uncompressed_size_) - { - compressed_size += compressed_size_; - uncompressed_size += uncompressed_size_; - ++package; - } - - void load(uint64_t & compressed_size_, uint64_t & uncompressed_size_, uint64_t & package_, CompressMethod & m) const - { - compressed_size_ = getCompressedSize(); - uncompressed_size_ = getUncompressedSize(); - package_ = package; - m = getMethod(); - } - - static SnappyStatistic & globalInstance() - { - static SnappyStatistic data{}; - return data; - } - - void addEncodeInfo(std::chrono::steady_clock::duration d, uint64_t ec) - { - std::chrono::steady_clock::duration x = durations.load(); - std::chrono::steady_clock::duration y = x + d; - for (; !durations.compare_exchange_strong(x, y);) - { - y = x + d; - } - encode_bytes += ec; - } - void addHashPartitionWriter(std::chrono::steady_clock::duration d, uint64_t rows) - { - std::chrono::steady_clock::duration x = has_write_dur.load(); - std::chrono::steady_clock::duration y = x + d; - for (; !has_write_dur.compare_exchange_strong(x, y);) - { - y = x + d; - } - has_write_rows += rows; - } - void getEncodeInfo(std::chrono::steady_clock::duration & d, uint64_t & ec, std::chrono::steady_clock::duration & hash_dur, uint64_t & hash_rows) - { - hash_dur = has_write_dur, hash_rows = has_write_rows; - d = durations; - ec = encode_bytes; - } - void clearEncodeInfo() - { - durations.store({}); - has_write_dur.store({}); - encode_bytes = 0; - has_write_rows = 0; - } -}; -} // namespace tzg \ No newline at end of file diff --git a/dbms/src/Flash/FlashService.cpp b/dbms/src/Flash/FlashService.cpp index a027a4349b5..2d976ad1ff8 100644 --- a/dbms/src/Flash/FlashService.cpp +++ b/dbms/src/Flash/FlashService.cpp @@ -229,9 +229,10 @@ grpc::Status FlashService::DispatchMPPTask( if (!check_result.ok()) return check_result; - if (auto mpp_version = request->meta().mpp_version(); !TiDB::CheckMppVersion(mpp_version)) + // DO NOT register mpp task and return grpc error + if (auto mpp_version = request->meta().mpp_version(); !DB::CheckMppVersion(mpp_version)) { - auto && err_msg = fmt::format("Failed to handling mpp dispatch request, reason=`{}`", TiDB::GenMppVersionErrorMessage(mpp_version)); + auto && err_msg = fmt::format("Failed to handling mpp dispatch request, reason=`{}`", DB::GenMppVersionErrorMessage(mpp_version)); LOG_WARNING(log, err_msg); return grpc::Status(grpc::StatusCode::INTERNAL, std::move(err_msg)); } @@ -278,7 +279,7 @@ grpc::Status FlashService::IsAlive(grpc::ServerContext * grpc_context [[maybe_un auto & tmt_context = context->getTMTContext(); response->set_available(tmt_context.checkRunning()); - response->set_mpp_version(TiDB::GetMppVersion()); + response->set_mpp_version(DB::GetMppVersion()); return grpc::Status::OK; } @@ -289,13 +290,13 @@ static grpc::Status CheckMppVersionForEstablishMPPConnection(const mpp::Establis std::string && err_reason{}; - if (!TiDB::CheckMppVersion(sender_mpp_version)) + if (!DB::CheckMppVersion(sender_mpp_version)) { - err_reason += fmt::format("sender failed: {};", TiDB::GenMppVersionErrorMessage(sender_mpp_version)); + err_reason += fmt::format("sender failed: {};", DB::GenMppVersionErrorMessage(sender_mpp_version)); } - if (!TiDB::CheckMppVersion(receiver_mpp_version)) + if (!DB::CheckMppVersion(receiver_mpp_version)) { - err_reason += fmt::format("receiver failed: {};", TiDB::GenMppVersionErrorMessage(receiver_mpp_version)); + err_reason += fmt::format("receiver failed: {};", DB::GenMppVersionErrorMessage(receiver_mpp_version)); } if (!err_reason.empty()) @@ -407,9 +408,9 @@ grpc::Status FlashService::CancelMPPTask( if (!check_result.ok()) return check_result; - if (auto mpp_version = request->meta().mpp_version(); !TiDB::CheckMppVersion(mpp_version)) + if (auto mpp_version = request->meta().mpp_version(); !DB::CheckMppVersion(mpp_version)) { - auto && err_msg = fmt::format("Failed to cancel mpp task, reason=`{}`", TiDB::GenMppVersionErrorMessage(mpp_version)); + auto && err_msg = fmt::format("Failed to cancel mpp task, reason=`{}`", DB::GenMppVersionErrorMessage(mpp_version)); LOG_WARNING(log, err_msg); return grpc::Status(grpc::StatusCode::INTERNAL, std::move(err_msg)); } @@ -461,7 +462,7 @@ ::grpc::Status FlashService::cancelMPPTaskForTest(const ::mpp::CancelTaskRequest if (!status.ok()) { auto err = std::make_unique(); - err->set_mpp_version(TiDB::GetMppVersion()); + err->set_mpp_version(DB::GetMppVersion()); err->set_msg("error status"); response->set_allocated_error(err.release()); return status; diff --git a/dbms/src/Flash/Mpp/BroadcastOrPassThroughWriter.cpp b/dbms/src/Flash/Mpp/BroadcastOrPassThroughWriter.cpp index f8fea4c1c9e..21801afa986 100644 --- a/dbms/src/Flash/Mpp/BroadcastOrPassThroughWriter.cpp +++ b/dbms/src/Flash/Mpp/BroadcastOrPassThroughWriter.cpp @@ -75,7 +75,19 @@ void BroadcastOrPassThroughWriter::encodeThenWriteBlocks() } assert(blocks.empty()); rows_in_blocks = 0; + auto packet_bytes = tracked_packet->getPacket().ByteSizeLong(); writer->broadcastOrPassThroughWrite(std::move(tracked_packet)); + + { + auto tunnel_cnt = writer->getPartitionNum(); + size_t local_tunnel_cnt = 0; + for (size_t i = 0; i < tunnel_cnt; ++i) + { + local_tunnel_cnt += writer->isLocal(i); + } + GET_METRIC(tiflash_exchange_data_bytes, type_broadcast_passthrough_original_all).Increment(packet_bytes * tunnel_cnt); + GET_METRIC(tiflash_exchange_data_bytes, type_broadcast_passthrough_none_local).Increment(packet_bytes * local_tunnel_cnt); + } } template class BroadcastOrPassThroughWriter; diff --git a/dbms/src/Flash/Mpp/ExchangeReceiver.cpp b/dbms/src/Flash/Mpp/ExchangeReceiver.cpp index 40bb30b1286..9dd4f37ed4b 100644 --- a/dbms/src/Flash/Mpp/ExchangeReceiver.cpp +++ b/dbms/src/Flash/Mpp/ExchangeReceiver.cpp @@ -22,14 +22,14 @@ #include #include #include +#include #include #include #include #include -#include "Flash/Mpp/HashPartitionWriterV1.h" -#include "mpp.pb.h" +#include "IO/CompressedStream.h" namespace DB { @@ -711,7 +711,7 @@ DecodeDetail ExchangeReceiverBase::decodeChunks( switch (packet.version()) { - case HashPartitionWriterV0: + case DB::MPPDataPacketV0: { for (const String * chunk : recv_msg->chunks) { @@ -726,7 +726,7 @@ DecodeDetail ExchangeReceiverBase::decodeChunks( } return detail; } - case HashPartitionWriterV1: + case DB::MPPDataPacketV1: { RUNTIME_CHECK(packet.chunks().size() == int(recv_msg->chunks.size()), packet.chunks().size(), @@ -734,7 +734,8 @@ DecodeDetail ExchangeReceiverBase::decodeChunks( for (auto && chunk : packet.chunks()) { - auto && result = decoder_ptr->decodeAndSquash(chunk, packet.compression().mode() != mpp::CompressionMode::NONE); + assert(!chunk.empty()); + auto && result = decoder_ptr->decodeAndSquashWithCompression(chunk); if (!result || !result->rows()) continue; detail.rows += result->rows(); diff --git a/dbms/src/Flash/Mpp/FineGrainedShuffleWriter.cpp b/dbms/src/Flash/Mpp/FineGrainedShuffleWriter.cpp index b5bb5852c5e..c51d4ae8654 100644 --- a/dbms/src/Flash/Mpp/FineGrainedShuffleWriter.cpp +++ b/dbms/src/Flash/Mpp/FineGrainedShuffleWriter.cpp @@ -158,16 +158,13 @@ void FineGrainedShuffleWriter::batchWriteFineGrainedShuffle() writePackets(tracked_packets); } +template +extern void WritePackets(TrackedMppDataPacketPtrs & packets, ExchangeWriterPtr & writer); + template void FineGrainedShuffleWriter::writePackets(TrackedMppDataPacketPtrs & packets) { - for (size_t part_id = 0; part_id < packets.size(); ++part_id) - { - auto & packet = packets[part_id]; - assert(packet); - if (likely(packet->getPacket().chunks_size() > 0)) - writer->partitionWrite(std::move(packet), part_id); - } + WritePackets(packets, writer); } template class FineGrainedShuffleWriter; diff --git a/dbms/src/Flash/Mpp/HashPartitionWriter.cpp b/dbms/src/Flash/Mpp/HashPartitionWriter.cpp index c82220d5f23..565a0e1b148 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriter.cpp +++ b/dbms/src/Flash/Mpp/HashPartitionWriter.cpp @@ -14,6 +14,7 @@ #include #include +#include #include #include #include @@ -111,7 +112,6 @@ void HashPartitionWriter::partitionAndEncodeThenWriteBlocks() GET_METRIC(tiflash_exchange_data_bytes, type_hash_original_all).Increment(ori_block_mem_size); } - static void updateHashPartitionWriterMetrics(size_t sz, bool is_local) { if (is_local) @@ -120,12 +120,12 @@ static void updateHashPartitionWriterMetrics(size_t sz, bool is_local) } else { - GET_METRIC(tiflash_exchange_data_bytes, type_hash_none).Increment(sz); + GET_METRIC(tiflash_exchange_data_bytes, type_hash_none_remote).Increment(sz); } } template -void HashPartitionWriter::writePackets(TrackedMppDataPacketPtrs & packets) +void WritePackets(TrackedMppDataPacketPtrs & packets, ExchangeWriterPtr & writer) { for (size_t part_id = 0; part_id < packets.size(); ++part_id) { @@ -136,13 +136,18 @@ void HashPartitionWriter::writePackets(TrackedMppDataPacketPt if (auto sz = inner_packet.ByteSizeLong(); likely(inner_packet.chunks_size() > 0)) { - assert(inner_packet.compression().mode() == mpp::CompressionMode::NONE); writer->partitionWrite(std::move(packet), part_id); updateHashPartitionWriterMetrics(sz, writer->isLocal(part_id)); } } } +template +void HashPartitionWriter::writePackets(TrackedMppDataPacketPtrs & packets) +{ + WritePackets(packets, writer); +} + template class HashPartitionWriter; } // namespace DB diff --git a/dbms/src/Flash/Mpp/HashPartitionWriter.h b/dbms/src/Flash/Mpp/HashPartitionWriter.h index eb82c54592e..deebcd3dce7 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriter.h +++ b/dbms/src/Flash/Mpp/HashPartitionWriter.h @@ -15,13 +15,14 @@ #pragma once #include -#include #include #include #include namespace DB { +class DAGContext; + template class HashPartitionWriter : public DAGResponseWriter { diff --git a/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp b/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp index 05556da00f2..a801908aa1e 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp +++ b/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp @@ -15,39 +15,24 @@ #include #include #include +#include #include #include #include -#include - -#include -#include -#include -#include - -#include "Common/Exception.h" -#include "Common/Stopwatch.h" -#include "Flash/Coprocessor/CHBlockChunkCodecStream.h" -#include "Flash/Coprocessor/CompressCHBlockChunkCodecStream.h" -#include "Flash/Coprocessor/CompressedCHBlockChunkCodec.h" -#include "Flash/Mpp/MppVersion.h" -#include "IO/CompressedStream.h" -#include "IO/CompressionSettings.h" -#include "common/logger_useful.h" -#include "ext/scope_guard.h" +#include namespace DB { -CompressionMethod ToInternalCompressionMethod(mpp::CompressionMode compression_mode) +static inline CompressionMethod ToInternalCompressionMethod(tipb::CompressionMode compression_mode) { switch (compression_mode) { - case mpp::CompressionMode::NONE: + case tipb::CompressionMode::NONE: return CompressionMethod::NONE; - case mpp::CompressionMode::FAST: + case tipb::CompressionMode::FAST: return CompressionMethod::LZ4; // use LZ4 method as fast mode - case mpp::CompressionMode::HIGH_COMPRESSION: + case tipb::CompressionMode::HIGH_COMPRESSION: return CompressionMethod::ZSTD; // use ZSTD method as HC mode default: return CompressionMethod::NONE; @@ -55,37 +40,42 @@ CompressionMethod ToInternalCompressionMethod(mpp::CompressionMode compression_m } template -HashPartitionWriterImplV1::HashPartitionWriterImplV1( +HashPartitionWriterV1::HashPartitionWriterV1( ExchangeWriterPtr writer_, std::vector partition_col_ids_, TiDB::TiDBCollators collators_, Int64 partition_batch_limit_, DAGContext & dag_context_, - mpp::CompressionMode compression_mode_) + tipb::CompressionMode compression_mode_) : DAGResponseWriter(/*records_per_chunk=*/-1, dag_context_) , partition_num(writer_->getPartitionNum()) - , partition_batch_limit(partition_batch_limit_ * partition_num) + , partition_batch_limit(partition_batch_limit_) , writer(writer_) , partition_col_ids(std::move(partition_col_ids_)) , collators(std::move(collators_)) - , compression_mode(compression_mode_) + , compression_method(ToInternalCompressionMethod(compression_mode_)) { assert(dag_context.getMPPTaskMeta().mpp_version() > 0); + if (partition_batch_limit < 0) + { + partition_batch_limit = 8192LL * partition_num; + } + rows_in_blocks = 0; RUNTIME_CHECK(partition_num > 0); RUNTIME_CHECK(dag_context.encode_type == tipb::EncodeType::TypeCHBlock); } template -void HashPartitionWriterImplV1::flush() +void HashPartitionWriterV1::flush() { if (rows_in_blocks > 0) partitionAndEncodeThenWriteBlocks(); } template -void HashPartitionWriterImplV1::write(const Block & block) +void HashPartitionWriterV1::write(const Block & block) { RUNTIME_CHECK_MSG( block.columns() == dag_context.result_field_types.size(), @@ -104,7 +94,7 @@ void HashPartitionWriterImplV1::write(const Block & block) extern void WriteColumnData(const IDataType & type, const ColumnPtr & column, WriteBuffer & ostr, size_t offset, size_t limit); template -void HashPartitionWriterImplV1::partitionAndEncodeThenWriteBlocks() +void HashPartitionWriterV1::partitionAndEncodeThenWriteBlocks() { // return partitionAndEncodeThenWriteBlocksTest(); @@ -112,14 +102,7 @@ void HashPartitionWriterImplV1::partitionAndEncodeThenWriteBl return; // Set mpp packet data version to `1` - auto tracked_packets = HashBaseWriterHelper::createPackets(partition_num, HashPartitionWriterV1); - - // Do NOT enable data compression when using local tunnel - for (size_t part_id = 0; part_id < partition_num; ++part_id) - { - auto mode = writer->isLocal(part_id) ? mpp::CompressionMode::NONE : compression_mode; - tracked_packets[part_id]->getPacket().mutable_compression()->set_mode(mode); - } + auto tracked_packets = HashBaseWriterHelper::createPackets(partition_num, DB::MPPDataPacketV1); // Sum of all approximate block data memory size size_t ori_block_mem_size = 0; @@ -176,22 +159,28 @@ void HashPartitionWriterImplV1::partitionAndEncodeThenWriteBl // repeated: // row count; // columns data; - size_t init_size = part_column_bytes + header_size; + size_t init_size = part_column_bytes + header_size + 1 /*compression method*/; // Reserve enough memory buffer size auto output_buffer = std::make_unique(init_size); - std::unique_ptr compress_codec{}; + std::unique_ptr compress_codec{}; WriteBuffer * ostr_ptr = output_buffer.get(); // Init compression writer - if (tracked_packets[part_id]->getPacket().compression().mode() != mpp::CompressionMode::NONE) + if (!writer->isLocal(part_id) && compression_method != CompressionMethod::NONE) { - compress_codec = std::make_unique( + // CompressedWriteBuffer will encode compression method flag as first byte + compress_codec = std::make_unique( *output_buffer, - CompressionSettings(ToInternalCompressionMethod(compression_mode)), + CompressionSettings(compression_method), init_size); ostr_ptr = compress_codec.get(); } + else + { + // Write compression method flag + output_buffer->write(static_cast(CompressionMethodByte::NONE)); + } // Encode header EncodeHeader(*ostr_ptr, dest_block_header, part_rows); @@ -232,133 +221,14 @@ void HashPartitionWriterImplV1::partitionAndEncodeThenWriteBl GET_METRIC(tiflash_exchange_data_bytes, type_hash_original_all).Increment(ori_block_mem_size); } - -template -void HashPartitionWriterImplV1::partitionAndEncodeThenWriteBlocksTest() +static void updateHashPartitionWriterMetrics(CompressionMethod method, size_t sz, bool is_local) { - if (blocks.empty()) - return; - - // Set mpp packet data version to `1` - auto tracked_packets = HashBaseWriterHelper::createPackets(partition_num, HashPartitionWriterV1); - - // Do NOT enable data compression when using local tunnel - for (size_t part_id = 0; part_id < partition_num; ++part_id) + if (is_local) { - auto mode = writer->isLocal(part_id) ? mpp::CompressionMode::NONE : compression_mode; - tracked_packets[part_id]->getPacket().mutable_compression()->set_mode(mode); + method = CompressionMethod::NONE; } - // Sum of all approximate block data memory size - size_t ori_block_mem_size = 0; - - { - assert(rows_in_blocks > 0); - - HashBaseWriterHelper::materializeBlocks(blocks); - Block dest_block_header = blocks.back().cloneEmpty(); - - std::vector partition_key_containers(collators.size()); - std::vector> dest_columns(partition_num); - [[maybe_unused]] size_t total_rows = 0, encoded_rows = 0; - - while (!blocks.empty()) - { - const auto & block = blocks.back(); - block.checkNumberOfRows(); - - ori_block_mem_size += ApproxBlockBytes(block); - total_rows += block.rows(); - - auto dest_tbl_cols = HashBaseWriterHelper::createDestColumns(block, partition_num); - HashBaseWriterHelper::scatterColumns(block, partition_col_ids, collators, partition_key_containers, partition_num, dest_tbl_cols); - blocks.pop_back(); - - for (size_t part_id = 0; part_id < partition_num; ++part_id) - { - auto & columns = dest_tbl_cols[part_id]; - dest_columns[part_id].emplace_back(std::move(columns)); - } - } - - size_t header_size = GetExtraInfoSize(dest_block_header) + 8; - - for (size_t part_id = 0; part_id < partition_num; ++part_id) - { - auto & part_columns = dest_columns[part_id]; - size_t part_rows = std::accumulate(part_columns.begin(), part_columns.end(), 0, [](const auto & r, const auto & columns) { return r + columns.front()->size(); }); - - if (!part_rows) - continue; - - encoded_rows += part_rows; - - size_t part_column_bytes = std::accumulate( - part_columns.begin(), - part_columns.end(), - 0, - [](auto res, const auto & columns) { - for (const auto & elem : columns) - res += elem->byteSize(); - return res + 8; - }); - - size_t init_size = part_column_bytes + header_size; - - // Reserve enough memory buffer size - auto output_buffer = std::make_unique(init_size); - std::unique_ptr compress_codec{}; - WriteBuffer * ostr_ptr = output_buffer.get(); - - // Init compression writer - if (tracked_packets[part_id]->getPacket().compression().mode() != mpp::CompressionMode::NONE) - { - compress_codec = std::make_unique( - *output_buffer, - CompressionSettings(ToInternalCompressionMethod(compression_mode)), - init_size); - ostr_ptr = compress_codec.get(); - } - - // Encode header - EncodeHeader(*ostr_ptr, dest_block_header, part_rows); - writeVarUInt(part_columns.size(), *ostr_ptr); - for (auto && columns : part_columns) - { - writeVarUInt(columns.front()->size(), *ostr_ptr); - } - - for (size_t col_index = 0; col_index < dest_block_header.columns(); ++col_index) - { - auto && col_type_name = dest_block_header.getByPosition(col_index); - for (auto && columns : part_columns) - { - if (columns[col_index]->empty()) - continue; - WriteColumnData(*col_type_name.type, std::move(columns[col_index]), *ostr_ptr, 0, 0); - } - } - - // Flush rest buffer - if (compress_codec) - compress_codec->next(); - - tracked_packets[part_id]->getPacket().add_chunks(output_buffer->releaseStr()); - } - - assert(encoded_rows == total_rows); - assert(blocks.empty()); - rows_in_blocks = 0; - } - - writePackets(std::move(tracked_packets)); - - GET_METRIC(tiflash_exchange_data_bytes, type_hash_original_all).Increment(ori_block_mem_size); -} - -static void updateHashPartitionWriterMetrics(mpp::CompressionMode mode, size_t sz, bool is_local) -{ - switch (ToInternalCompressionMethod(mode)) + switch (method) { case CompressionMethod::NONE: { @@ -368,7 +238,7 @@ static void updateHashPartitionWriterMetrics(mpp::CompressionMode mode, size_t s } else { - GET_METRIC(tiflash_exchange_data_bytes, type_hash_none).Increment(sz); + GET_METRIC(tiflash_exchange_data_bytes, type_hash_none_remote).Increment(sz); } break; } @@ -388,7 +258,7 @@ static void updateHashPartitionWriterMetrics(mpp::CompressionMode mode, size_t s } template -void HashPartitionWriterImplV1::writePackets(TrackedMppDataPacketPtrs && packets) +void HashPartitionWriterV1::writePackets(TrackedMppDataPacketPtrs && packets) { for (size_t part_id = 0; part_id < packets.size(); ++part_id) { @@ -396,16 +266,16 @@ void HashPartitionWriterImplV1::writePackets(TrackedMppDataPa assert(packet); auto & inner_packet = packet->getPacket(); + inner_packet.chunks(); if (auto sz = inner_packet.ByteSizeLong(); likely(inner_packet.chunks_size() > 0)) { - auto mode = inner_packet.compression().mode(); writer->partitionWrite(std::move(packet), part_id); - updateHashPartitionWriterMetrics(mode, sz, writer->isLocal(part_id)); + updateHashPartitionWriterMetrics(compression_method, sz, writer->isLocal(part_id)); } } } -template class HashPartitionWriterImplV1; +template class HashPartitionWriterV1; } // namespace DB diff --git a/dbms/src/Flash/Mpp/HashPartitionWriterV1.h b/dbms/src/Flash/Mpp/HashPartitionWriterV1.h index 55e937320b2..028e909ba08 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriterV1.h +++ b/dbms/src/Flash/Mpp/HashPartitionWriterV1.h @@ -15,36 +15,31 @@ #pragma once #include -#include #include #include #include #include - -namespace DB +namespace tipb { +enum CompressionMode : int; +} -struct CompressCHBlockChunkCodecStream; - -enum HashPartitionWriterVersion : int64_t +namespace DB { - HashPartitionWriterV0 = 0, - HashPartitionWriterV1, - HashPartitionWriterVersionMax, -}; - +enum class CompressionMethod; template -class HashPartitionWriterImplV1 : public DAGResponseWriter +class HashPartitionWriterV1 : public DAGResponseWriter { public: - HashPartitionWriterImplV1( + // If `partition_batch_limit_` is LT 0, `partition_batch_limit` will be set to `8192 * partition_num` + HashPartitionWriterV1( ExchangeWriterPtr writer_, std::vector partition_col_ids_, TiDB::TiDBCollators collators_, Int64 partition_batch_limit_, DAGContext & dag_context_, - mpp::CompressionMode compression_mode_); + tipb::CompressionMode compression_mode_); void write(const Block & block) override; void flush() override; @@ -62,9 +57,7 @@ class HashPartitionWriterImplV1 : public DAGResponseWriter std::vector partition_col_ids; TiDB::TiDBCollators collators; size_t rows_in_blocks; - mpp::CompressionMode compression_mode{}; + CompressionMethod compression_method{}; }; -CompressionMethod ToInternalCompressionMethod(mpp::CompressionMode compression_mode); - } // namespace DB diff --git a/dbms/src/Flash/Mpp/MPPTask.cpp b/dbms/src/Flash/Mpp/MPPTask.cpp index c8600fced4f..f6e5d942823 100644 --- a/dbms/src/Flash/Mpp/MPPTask.cpp +++ b/dbms/src/Flash/Mpp/MPPTask.cpp @@ -40,9 +40,6 @@ #include #include -#include "Flash/Coprocessor/tzg-metrics.h" -#include "mpp.pb.h" - namespace DB { namespace FailPoints @@ -283,7 +280,7 @@ void MPPTask::prepare(const mpp::DispatchTaskRequest & task_request) is_root_mpp_task = task_meta.task_id() == -1; } - dag_context = std::make_unique(dag_req, task_request.meta(), task_request.exchange_sender_meta(), is_root_mpp_task); + dag_context = std::make_unique(dag_req, task_request.meta(), is_root_mpp_task); dag_context->log = log; dag_context->tables_regions_info = std::move(tables_regions_info); dag_context->tidb_host = context->getClientInfo().current_address.toString(); diff --git a/dbms/src/Flash/Mpp/MPPTunnel.cpp b/dbms/src/Flash/Mpp/MPPTunnel.cpp index fd97c4c5f06..431b473f4e8 100644 --- a/dbms/src/Flash/Mpp/MPPTunnel.cpp +++ b/dbms/src/Flash/Mpp/MPPTunnel.cpp @@ -20,9 +20,6 @@ #include #include -#include "common/logger_useful.h" -#include "ext/scope_guard.h" - namespace DB { namespace FailPoints @@ -88,7 +85,7 @@ MPPTunnel::MPPTunnel( , timeout(timeout_) , tunnel_id(tunnel_id_) , mem_tracker(current_memory_tracker ? current_memory_tracker->shared_from_this() : nullptr) - , queue_size(std::max(5, input_steams_num_ * 10)) // MPMCQueue can benefit from a slightly larger queue size + , queue_size(std::max(5, input_steams_num_ * 5)) // MPMCQueue can benefit from a slightly larger queue size , log(Logger::get(req_id, tunnel_id)) , data_size_in_queue(0) { @@ -155,8 +152,6 @@ void MPPTunnel::close(const String & reason, bool wait_sender_finish) void MPPTunnel::write(TrackedMppDataPacketPtr && data) { - Stopwatch watch{}; - LOG_TRACE(log, "ready to write"); { std::unique_lock lk(mu); @@ -222,7 +217,7 @@ void MPPTunnel::connect(PacketWriter * writer) status = TunnelStatus::Connected; cv_for_status_changed.notify_all(); } - LOG_DEBUG(log, "Tunnel connected in {} mode", tunnelSenderModeToString(mode)); + LOG_DEBUG(log, "connected"); } void MPPTunnel::connectAsync(IAsyncCallData * call_data) diff --git a/dbms/src/Flash/Mpp/MPPTunnel.h b/dbms/src/Flash/Mpp/MPPTunnel.h index ce30cc1d3ff..953d40be3c2 100644 --- a/dbms/src/Flash/Mpp/MPPTunnel.h +++ b/dbms/src/Flash/Mpp/MPPTunnel.h @@ -128,8 +128,6 @@ class TunnelSender : private boost::noncopyable return memory_tracker != nullptr ? memory_tracker.get() : nullptr; } - size_t queue_size{}; - protected: /// TunnelSender use consumer state to inform tunnel that whether sender has finished its work class ConsumerState diff --git a/dbms/src/Flash/Mpp/MppVersion.h b/dbms/src/Flash/Mpp/MppVersion.h index bf675822de4..aeeb17b889d 100644 --- a/dbms/src/Flash/Mpp/MppVersion.h +++ b/dbms/src/Flash/Mpp/MppVersion.h @@ -2,17 +2,25 @@ #include -namespace TiDB +namespace DB { enum MppVersion : int64_t { MppVersionV0 = 0, MppVersionV1, + // MppVersionMAX, }; +enum MPPDataPacketVersion : int64_t +{ + MPPDataPacketV0 = 0, + MPPDataPacketV1, + MPPDataPacketMAX, +}; + bool CheckMppVersion(int64_t mpp_version); std::string GenMppVersionErrorMessage(int64_t mpp_version); int64_t GetMppVersion(); -std::string GetMppVersionReleaseInfo(int64_t mpp_version); -} // namespace TiDB \ No newline at end of file + +} // namespace DB \ No newline at end of file diff --git a/dbms/src/Flash/Mpp/Utils.cpp b/dbms/src/Flash/Mpp/Utils.cpp index 4a4ec261833..0ddfb7ed958 100644 --- a/dbms/src/Flash/Mpp/Utils.cpp +++ b/dbms/src/Flash/Mpp/Utils.cpp @@ -16,6 +16,7 @@ #include #include #include +#include #include #include @@ -23,11 +24,17 @@ namespace DB { + +namespace FailPoints +{ +extern const char invalid_mpp_version[]; +} // namespace FailPoints + mpp::MPPDataPacket getPacketWithError(String reason) { mpp::MPPDataPacket data; auto err = std::make_unique(); - err->set_mpp_version(TiDB::GetMppVersion()); + err->set_mpp_version(DB::GetMppVersion()); err->set_msg(std::move(reason)); data.set_allocated_error(err.release()); return data; @@ -43,45 +50,36 @@ void trimStackTrace(String & message) } } -} // namespace DB - -namespace TiDB -{ // Latest mpp-version supported by TiFlash -constexpr MppVersion MPP_VERSION = MppVersion((MppVersion::MppVersionMAX)-1); - -// TODO: set version after committed -constexpr std::array MPP_TIFLASH_RELEASE_VERSION = {"", "?"}; +static MppVersion NewestMppVersion = MppVersion(MppVersion::MppVersionMAX - 1); +static MppVersion MinMppVersion = MppVersion::MppVersionV0; // Check mpp-version is illegal bool CheckMppVersion(int64_t mpp_version) { - return mpp_version >= (MppVersion::MppVersionV0) && mpp_version < (MppVersion::MppVersionMAX); + fiu_do_on(FailPoints::invalid_mpp_version, { + mpp_version = -1; + }); + return mpp_version >= MinMppVersion && mpp_version <= NewestMppVersion; } std::string GenMppVersionErrorMessage(int64_t mpp_version) { - auto err_msg = fmt::format("invalid mpp version `{}`, expect version: min `{}`, max `{}` release version `{}`", + fiu_do_on(FailPoints::invalid_mpp_version, { + mpp_version = -1; + }); + auto err_msg = fmt::format("Invalid mpp version {}, TiFlash expects version: min {}, max {}, should upgrade {}", mpp_version, - (MppVersion::MppVersionV0), - (MPP_VERSION), - MPP_TIFLASH_RELEASE_VERSION[(MPP_VERSION)]); + MinMppVersion, + NewestMppVersion, + (mpp_version < MinMppVersion) ? "TiDB/planner" : "TiFlash"); return err_msg; } // Get latest mpp-version supported by TiFlash int64_t GetMppVersion() { - return (MPP_VERSION); -} - -std::string GetMppVersionReleaseInfo(int64_t mpp_version) -{ - if (CheckMppVersion(mpp_version)) - { - return MPP_TIFLASH_RELEASE_VERSION[mpp_version]; - } - return "unknown"; + return (NewestMppVersion); } -} // namespace TiDB \ No newline at end of file +} // namespace DB \ No newline at end of file diff --git a/dbms/src/Flash/Mpp/newMPPExchangeWriter.h b/dbms/src/Flash/Mpp/newMPPExchangeWriter.h index f8f949278d0..80e1bf0e320 100644 --- a/dbms/src/Flash/Mpp/newMPPExchangeWriter.h +++ b/dbms/src/Flash/Mpp/newMPPExchangeWriter.h @@ -34,13 +34,15 @@ std::unique_ptr NewMPPExchangeWriter( DAGContext & dag_context, bool enable_fine_grained_shuffle, UInt64 fine_grained_shuffle_stream_count, - UInt64 fine_grained_shuffle_batch_size) + UInt64 fine_grained_shuffle_batch_size, + tipb::CompressionMode compression_mode, + Int64 batch_send_min_limit_compression) { RUNTIME_CHECK(dag_context.isMPPTask()); if (dag_context.isRootMPPTask()) { // No need to use use data compression - RUNTIME_CHECK(dag_context.getExchangeSenderMeta().compression() == mpp::CompressionMode::NONE); + RUNTIME_CHECK(compression_mode == tipb::CompressionMode::NONE); RUNTIME_CHECK(!enable_fine_grained_shuffle); RUNTIME_CHECK(exchange_type == tipb::ExchangeType::PassThrough); @@ -57,7 +59,7 @@ std::unique_ptr NewMPPExchangeWriter( if (enable_fine_grained_shuffle) { // TODO: support data compression if necessary - RUNTIME_CHECK(dag_context.getExchangeSenderMeta().compression() == mpp::CompressionMode::NONE); + RUNTIME_CHECK(compression_mode == tipb::CompressionMode::NONE); return std::make_unique>( writer, @@ -69,8 +71,7 @@ std::unique_ptr NewMPPExchangeWriter( } else { - auto && compression_mode = dag_context.getExchangeSenderMeta().compression(); - if (TiDB::MppVersion::MppVersionV0 == dag_context.getMPPTaskMeta().mpp_version()) + if (DB::MppVersion::MppVersionV0 == dag_context.getMPPTaskMeta().mpp_version()) return std::make_unique>( writer, partition_col_ids, @@ -78,11 +79,11 @@ std::unique_ptr NewMPPExchangeWriter( batch_send_min_limit, dag_context); else - return std::make_unique>( + return std::make_unique>( writer, partition_col_ids, partition_col_collators, - 8192, + batch_send_min_limit_compression, dag_context, compression_mode); } @@ -90,7 +91,7 @@ std::unique_ptr NewMPPExchangeWriter( else { // TODO: support data compression if necessary - RUNTIME_CHECK(dag_context.getExchangeSenderMeta().compression() == mpp::CompressionMode::NONE); + RUNTIME_CHECK(compression_mode == tipb::CompressionMode::NONE); RUNTIME_CHECK(!enable_fine_grained_shuffle); return std::make_unique>( diff --git a/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp b/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp index e29cff70655..d0f921f56d3 100644 --- a/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp +++ b/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp @@ -31,6 +31,7 @@ #include #include "Flash/Mpp/MppVersion.h" +#include "IO/CompressedStream.h" #include "ext/scope_guard.h" #include "mpp.pb.h" @@ -48,7 +49,7 @@ class TestMPPExchangeWriter : public testing::Test dag_context_ptr->is_mpp_task = true; dag_context_ptr->is_root_mpp_task = false; dag_context_ptr->result_field_types = makeFields(); - dag_context_ptr->mpp_task_meta.set_mpp_version(TiDB::GetMppVersion()); + dag_context_ptr->mpp_task_meta.set_mpp_version(DB::GetMppVersion()); context.setDAGContext(dag_context_ptr.get()); } @@ -376,6 +377,22 @@ try } CATCH +static CompressionMethodByte ToCompressionMethodByte(CompressionMethod m) +{ + switch (m) + { + case CompressionMethod::LZ4: + return CompressionMethodByte::LZ4; + case CompressionMethod::NONE: + return CompressionMethodByte::NONE; + case CompressionMethod::ZSTD: + return CompressionMethodByte::ZSTD; + default: + RUNTIME_CHECK(false); + } + return CompressionMethodByte::NONE; +} + TEST_F(TestMPPExchangeWriter, testHashPartitionWriterV1) try { @@ -384,72 +401,75 @@ try const size_t batch_send_min_limit = 16; const uint16_t part_num = 4; - // 1. Build Blocks. - std::vector blocks; - for (size_t i = 0; i < block_num; ++i) + for (auto mode : {tipb::CompressionMode::NONE, tipb::CompressionMode::FAST, tipb::CompressionMode::HIGH_COMPRESSION}) { - blocks.emplace_back(prepareUniformBlock(block_rows)); - blocks.emplace_back(prepareUniformBlock(0)); - } - Block header = blocks.back(); - - // 2. Build MockExchangeWriter. - std::unordered_map write_report; - auto checker = [&write_report](const TrackedMppDataPacketPtr & packet, uint16_t part_id) { - write_report[part_id].emplace_back(packet); - }; - auto mock_writer = std::make_shared(checker, part_num); - - // 3. Start to write. - auto dag_writer = std::make_shared>>( - mock_writer, - part_col_ids, - part_col_collators, - batch_send_min_limit, - *dag_context_ptr, - mpp::CompressionMode::FAST); - for (const auto & block : blocks) - dag_writer->write(block); - dag_writer->flush(); - - // 4. Start to check write_report. - size_t per_part_rows = block_rows * block_num / part_num; - ASSERT_EQ(write_report.size(), part_num); - - CHBlockChunkDecodeAndSquash decoder(header, 512); - - for (size_t part_index = 0; part_index < part_num; ++part_index) - { - size_t decoded_block_rows = 0; - for (const auto & tracked_packet : write_report[part_index]) + // 1. Build Blocks. + std::vector blocks; + for (size_t i = 0; i < block_num; ++i) { - auto & packet = tracked_packet->getPacket(); + blocks.emplace_back(prepareUniformBlock(block_rows)); + blocks.emplace_back(prepareUniformBlock(0)); + } + Block header = blocks.back(); + + // 2. Build MockExchangeWriter. + std::unordered_map write_report; + auto checker = [&write_report](const TrackedMppDataPacketPtr & packet, uint16_t part_id) { + write_report[part_id].emplace_back(packet); + }; + auto mock_writer = std::make_shared(checker, part_num); + + // 3. Start to write. + auto dag_writer = std::make_shared>>( + mock_writer, + part_col_ids, + part_col_collators, + batch_send_min_limit, + *dag_context_ptr, + mode); + for (const auto & block : blocks) + dag_writer->write(block); + dag_writer->flush(); + + // 4. Start to check write_report. + size_t per_part_rows = block_rows * block_num / part_num; + ASSERT_EQ(write_report.size(), part_num); + + CHBlockChunkDecodeAndSquash decoder(header, 512); + + for (size_t part_index = 0; part_index < part_num; ++part_index) + { + size_t decoded_block_rows = 0; + for (const auto & tracked_packet : write_report[part_index]) + { + auto & packet = tracked_packet->getPacket(); - ASSERT_EQ(packet.version(), HashPartitionWriterV1); + ASSERT_EQ(packet.version(), DB::MPPDataPacketV1); - for (auto && chunk : packet.chunks()) - { - if (part_index == 0) - { - ASSERT_EQ(packet.compression().mode(), mpp::CompressionMode::NONE); - } - else + for (auto && chunk : packet.chunks()) { - ASSERT_NE(packet.compression().mode(), mpp::CompressionMode::NONE); + if (part_index == 0) + { + ASSERT_EQ(CompressionMethodByte(chunk[0]), CompressionMethodByte::NONE); + } + else + { + ASSERT_EQ(CompressionMethodByte(chunk[0]), ToCompressionMethodByte(ToInternalCompressionMethod(mode))); + } + + auto && result = decoder.decodeAndSquashWithCompression(chunk); + if (!result) + continue; + decoded_block_rows += result->rows(); } - - auto && result = decoder.decodeAndSquash(chunk, packet.compression().mode() != mpp::CompressionMode::NONE); - if (!result) - continue; - decoded_block_rows += result->rows(); } + { + auto result = decoder.flush(); + if (result) + decoded_block_rows += result->rows(); + } + ASSERT_EQ(decoded_block_rows, per_part_rows); } - { - auto result = decoder.flush(); - if (result) - decoded_block_rows += result->rows(); - } - ASSERT_EQ(decoded_block_rows, per_part_rows); } } CATCH diff --git a/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp b/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp index 406081622fe..e6e6a1c4ac7 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp +++ b/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp @@ -18,14 +18,10 @@ #include #include #include -#include #include #include #include -#include "Flash/Coprocessor/CompressCHBlockChunkCodecStream.h" -#include "Flash/Mpp/HashPartitionWriterV1.h" -#include "Flash/Mpp/MppVersion.h" namespace DB { @@ -49,7 +45,8 @@ PhysicalPlanNodePtr PhysicalExchangeSender::build( partition_col_ids, partition_col_collators, exchange_sender.tp(), - fine_grained_shuffle); + fine_grained_shuffle, + exchange_sender.compression()); // executeUnion will be call after sender.transform, so don't need to restore concurrency. physical_exchange_sender->disableRestoreConcurrency(); return physical_exchange_sender; @@ -83,7 +80,9 @@ void PhysicalExchangeSender::transformImpl(DAGPipeline & pipeline, Context & con dag_context, fine_grained_shuffle.enable(), fine_grained_shuffle.stream_count, - fine_grained_shuffle.batch_size); + fine_grained_shuffle.batch_size, + compression_mode, + context.getSettingsRef().batch_send_min_limit_compression); stream = std::make_shared(stream, std::move(response_writer), log->identifier()); stream->setExtraInfo(extra_info); }); diff --git a/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.h b/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.h index 536cd1e3164..aa545e5e0b4 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.h +++ b/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.h @@ -39,11 +39,13 @@ class PhysicalExchangeSender : public PhysicalUnary const std::vector & partition_col_ids_, const TiDB::TiDBCollators & collators_, const tipb::ExchangeType & exchange_type_, - const FineGrainedShuffle & fine_grained_shuffle_) + const FineGrainedShuffle & fine_grained_shuffle_, + const tipb::CompressionMode & compression_mode_) : PhysicalUnary(executor_id_, PlanType::ExchangeSender, schema_, req_id, child_) , partition_col_ids(partition_col_ids_) , partition_col_collators(collators_) , exchange_type(exchange_type_) + , compression_mode(compression_mode_) , fine_grained_shuffle(fine_grained_shuffle_) {} @@ -57,6 +59,7 @@ class PhysicalExchangeSender : public PhysicalUnary std::vector partition_col_ids; TiDB::TiDBCollators partition_col_collators; tipb::ExchangeType exchange_type; + tipb::CompressionMode compression_mode; FineGrainedShuffle fine_grained_shuffle; }; diff --git a/dbms/src/Functions/CMakeLists.txt b/dbms/src/Functions/CMakeLists.txt index 4a6cedca496..71769cdd96f 100644 --- a/dbms/src/Functions/CMakeLists.txt +++ b/dbms/src/Functions/CMakeLists.txt @@ -19,13 +19,12 @@ add_headers_and_sources(clickhouse_functions ./GatherUtils) add_headers_and_sources(clickhouse_functions ./Conditional) add_headers_and_sources(clickhouse_functions ${TiFlash_BINARY_DIR}/dbms/src/Functions) -check_then_add_sources_compile_flag (TIFLASH_ENABLE_AVX_SUPPORT "${TIFLASH_COMPILER_AVX2_FLAG}" CollationStringOptimized.cpp) -check_then_add_sources_compile_flag (TIFLASH_COMPILER_MOVBE_SUPPORT "${TIFLASH_COMPILER_MOVBE_FLAG}" CollationStringOptimized.cpp) check_then_add_sources_compile_flag ( - TIFLASH_COMPILER_BMI2_SUPPORT - "${TIFLASH_COMPILER_BMI2_FLAG}" + TIFLASH_COMPILER_ARCH_HASWELL_SUPPORT + "${TIFLASH_COMPILER_ARCH_HASWELL_FLAG}" CollationStringOptimized.cpp ) + list(REMOVE_ITEM clickhouse_functions_sources IFunction.cpp FunctionFactory.cpp FunctionHelpers.cpp) list(REMOVE_ITEM clickhouse_functions_headers IFunction.h FunctionFactory.h FunctionHelpers.h) diff --git a/dbms/src/IO/CompressedReadBufferBase.cpp b/dbms/src/IO/CompressedReadBufferBase.cpp index 40efeb1208a..f5ce5ed41e7 100644 --- a/dbms/src/IO/CompressedReadBufferBase.cpp +++ b/dbms/src/IO/CompressedReadBufferBase.cpp @@ -23,7 +23,6 @@ #include #include #include -// #include #include #include @@ -64,10 +63,8 @@ size_t CompressedReadBufferBase::readCompressedData(size_t & size_ size_t & size_compressed = size_compressed_without_checksum; - if (method == static_cast(CompressionMethodByte::LZ4) - || method == static_cast(CompressionMethodByte::ZSTD) - || method == static_cast(CompressionMethodByte::NONE) - || method == static_cast(CompressionMethodByte::SNAPPY)) + if (method == static_cast(CompressionMethodByte::LZ4) || method == static_cast(CompressionMethodByte::ZSTD) + || method == static_cast(CompressionMethodByte::NONE)) { size_compressed = unalignedLoad(&own_compressed_buffer[1]); size_decompressed = unalignedLoad(&own_compressed_buffer[5]); @@ -122,12 +119,6 @@ void CompressedReadBufferBase::decompress(char * to, size_t size_d if (ZSTD_isError(res)) throw Exception("Cannot ZSTD_decompress: " + std::string(ZSTD_getErrorName(res)), ErrorCodes::CANNOT_DECOMPRESS); } - // else if (method == static_cast(CompressionMethodByte::SNAPPY)) - // { - // auto res = snappy::RawUncompress(compressed_buffer + COMPRESSED_BLOCK_HEADER_SIZE, size_compressed_without_checksum - COMPRESSED_BLOCK_HEADER_SIZE, to); - // if (!res) - // throw Exception("Cannot decompress by SNAPPY", ErrorCodes::CANNOT_DECOMPRESS); - // } else if (method == static_cast(CompressionMethodByte::NONE)) { memcpy(to, &compressed_buffer[COMPRESSED_BLOCK_HEADER_SIZE], size_decompressed); diff --git a/dbms/src/IO/CompressedStream.h b/dbms/src/IO/CompressedStream.h index c55c5975370..9c4cc9fc7d3 100644 --- a/dbms/src/IO/CompressedStream.h +++ b/dbms/src/IO/CompressedStream.h @@ -32,7 +32,6 @@ enum class CompressionMethod LZ4HC = 2, /// The format is the same as for LZ4. The difference is only in compression. ZSTD = 3, /// Experimental algorithm: https://github.com/Cyan4973/zstd NONE = 4, /// No compression - SNAPPY = 5, /// Snappy: https://github.com/google/snappy }; /** The compressed block format is as follows: @@ -63,7 +62,6 @@ enum class CompressionMethodByte : uint8_t ZSTD = 0x90, // COL_END is not a compreesion method, but a flag of column end used in compact file. COL_END = 0x66, - SNAPPY = 0x67, }; } // namespace DB diff --git a/dbms/src/IO/CompressedWriteBuffer.cpp b/dbms/src/IO/CompressedWriteBuffer.cpp index 92bf73c1dc8..738830217c0 100644 --- a/dbms/src/IO/CompressedWriteBuffer.cpp +++ b/dbms/src/IO/CompressedWriteBuffer.cpp @@ -18,14 +18,11 @@ #include #include #include -// #include #include #include #include -#include "Flash/Coprocessor/tzg-metrics.h" - namespace DB { @@ -56,7 +53,10 @@ void CompressedWriteBuffer::nextImpl() { static constexpr size_t header_size = 1 + sizeof(UInt32) + sizeof(UInt32); +#pragma GCC diagnostic push +#pragma GCC diagnostic ignored "-Wold-style-cast" compressed_buffer.resize(header_size + LZ4_COMPRESSBOUND(uncompressed_size)); +#pragma GCC diagnostic pop compressed_buffer[0] = static_cast(CompressionMethodByte::LZ4); @@ -131,7 +131,6 @@ void CompressedWriteBuffer::nextImpl() CityHash_v1_0_2::uint128 checksum = CityHash_v1_0_2::CityHash128(compressed_buffer_ptr, compressed_size); out.write(reinterpret_cast(&checksum), sizeof(checksum)); } - tzg::SnappyStatistic::globalInstance().update(compressed_size, uncompressed_size); out.write(compressed_buffer_ptr, compressed_size); } diff --git a/dbms/src/IO/CompressedWriteBuffer.h b/dbms/src/IO/CompressedWriteBuffer.h index 1fc169cffb4..69bd87c6956 100644 --- a/dbms/src/IO/CompressedWriteBuffer.h +++ b/dbms/src/IO/CompressedWriteBuffer.h @@ -36,7 +36,7 @@ class CompressedWriteBuffer : public BufferWithOwnMemory void nextImpl() override; public: - explicit CompressedWriteBuffer( + CompressedWriteBuffer( WriteBuffer & out_, CompressionSettings compression_settings = CompressionSettings(), size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE); diff --git a/dbms/src/Interpreters/Settings.h b/dbms/src/Interpreters/Settings.h index ea6d1c89cd7..7546fd3928a 100644 --- a/dbms/src/Interpreters/Settings.h +++ b/dbms/src/Interpreters/Settings.h @@ -50,6 +50,7 @@ struct Settings M(SettingUInt64, read_tso, DEFAULT_MAX_READ_TSO, "tmt read tso.") \ M(SettingInt64, dag_records_per_chunk, DEFAULT_DAG_RECORDS_PER_CHUNK, "default chunk size of a DAG response.") \ M(SettingInt64, batch_send_min_limit, DEFAULT_BATCH_SEND_MIN_LIMIT, "default minimal chunk size of exchanging data among TiFlash.") \ + M(SettingInt64, batch_send_min_limit_compression, -1, "default minimal chunk size of exchanging data among TiFlash when using data compression.") \ M(SettingInt64, schema_version, DEFAULT_UNSPECIFIED_SCHEMA_VERSION, "tmt schema version.") \ M(SettingUInt64, mpp_task_timeout, DEFAULT_MPP_TASK_TIMEOUT, "mpp task max endurable time.") \ M(SettingUInt64, mpp_task_running_timeout, DEFAULT_MPP_TASK_RUNNING_TIMEOUT, "mpp task max time that running without any progress.") \ diff --git a/dbms/src/Storages/Transaction/PDTiKVClient.h b/dbms/src/Storages/Transaction/PDTiKVClient.h index ad4da83a3fe..f1be9a0a5c0 100644 --- a/dbms/src/Storages/Transaction/PDTiKVClient.h +++ b/dbms/src/Storages/Transaction/PDTiKVClient.h @@ -45,7 +45,7 @@ struct PDClientHelper if (!ignore_cache) { // In case we cost too much to update safe point from PD. - std::chrono::time_point now = std::chrono::steady_clock::now(); + auto now = std::chrono::steady_clock::now(); const auto duration = std::chrono::duration_cast(now - safe_point_last_update_time.load()); const auto min_interval = std::max(Int64(1), safe_point_update_interval_seconds); // at least one second if (duration.count() < min_interval) diff --git a/dbms/src/Storages/Transaction/ProxyFFIStatusService.cpp b/dbms/src/Storages/Transaction/ProxyFFIStatusService.cpp index 6895398c4a0..f3551bed94e 100644 --- a/dbms/src/Storages/Transaction/ProxyFFIStatusService.cpp +++ b/dbms/src/Storages/Transaction/ProxyFFIStatusService.cpp @@ -19,14 +19,6 @@ #include #include #include -#include - -#include -#include - -#include "Flash/Coprocessor/tzg-metrics.h" -#include "magic_enum.hpp" -#include "mpp.pb.h" namespace DB { @@ -121,142 +113,11 @@ HttpRequestRes HandleHttpRequestStoreStatus( .view = BaseBuffView{name->data(), name->size()}}}; } -HttpRequestRes HandleHttpRequestCompressStatus( - EngineStoreServerWrap *, - std::string_view path, - const std::string &, - std::string_view, - std::string_view) -{ - uint64_t compressed_size; - uint64_t uncompressed_size; - uint64_t package; - tzg::SnappyStatistic::CompressMethod method; - tzg::SnappyStatistic::globalInstance().load(compressed_size, uncompressed_size, package, method); - - // double f_compressed_size_mb = compressed_size * 1.0 / 1024 / 1024; - // double f_uncompressed_size_mb = uncompressed_size * 1.0 / 1024 / 1024; - auto * res = RawCppString::New(fmt::format("package: {}, compressed_size: {}, uncompressed_size: {}, compress_rate: {:.2f}, method: {}", - package, - compressed_size, - uncompressed_size, - uncompressed_size ? double(compressed_size) / uncompressed_size : 0.0, - magic_enum::enum_name(method))); - static const std::string_view clean_str = "tzg-compress-and-clean"; - if (path.find(clean_str) != path.npos) - { - tzg::SnappyStatistic::globalInstance().clear(); - res->append(", clean statistic"); - } - return HttpRequestRes{ - .status = HttpRequestStatus::Ok, - .res = CppStrWithView{ - .inner = GenRawCppPtr(res, RawCppPtrTypeImpl::String), - .view = BaseBuffView{res->data(), res->size()}}}; -} - -static inline void ClearCompress() -{ - tzg::SnappyStatistic::globalInstance().clear(); - tzg::SnappyStatistic::globalInstance().clearEncodeInfo(); -} - -HttpRequestRes HandleHttpRequestCompressClean( - EngineStoreServerWrap *, - std::string_view, - const std::string &, - std::string_view, - std::string_view) -{ - ClearCompress(); - auto * res = RawCppString::New(fmt::format("Clean compress info")); - - return HttpRequestRes{ - .status = HttpRequestStatus::Ok, - .res = CppStrWithView{ - .inner = GenRawCppPtr(res, RawCppPtrTypeImpl::String), - .view = BaseBuffView{res->data(), res->size()}}}; -} - -HttpRequestRes HandleHttpRequestStreamCnt( - EngineStoreServerWrap *, - std::string_view, - const std::string &, - std::string_view, - std::string_view) -{ - auto cnt = tzg::SnappyStatistic::globalInstance().getChunckStreamCnt(); - auto max_cnt = tzg::SnappyStatistic::globalInstance().getMaxChunckStreamCnt(); - auto * res = RawCppString::New(fmt::format("chunck-stream-cnt: {}, max-chunck-stream-cnt: {}", cnt, max_cnt)); - return HttpRequestRes{ - .status = HttpRequestStatus::Ok, - .res = CppStrWithView{ - .inner = GenRawCppPtr(res, RawCppPtrTypeImpl::String), - .view = BaseBuffView{res->data(), res->size()}}}; -} - -HttpRequestRes HandleHttpRequestEncodeInfo( - EngineStoreServerWrap *, - std::string_view, - const std::string &, - std::string_view, - std::string_view) -{ - std::chrono::steady_clock::duration d; - uint64_t ec; - std::chrono::steady_clock::duration hash_dur; - uint64_t hash_rows; - tzg::SnappyStatistic::globalInstance().getEncodeInfo(d, ec, hash_dur, hash_rows); - auto * res = RawCppString::New(fmt::format("uncompress-bytes: {}, time : {}, hash-part-write: {}, hash-part-time: {}", ec, d.count(), hash_rows, hash_dur.count())); - - return HttpRequestRes{ - .status = HttpRequestStatus::Ok, - .res = CppStrWithView{ - .inner = GenRawCppPtr(res, RawCppPtrTypeImpl::String), - .view = BaseBuffView{res->data(), res->size()}}}; -} - -HttpRequestRes HandleHttpRequestSetCompressMethod( - EngineStoreServerWrap *, - std::string_view path, - const std::string & api_name, - std::string_view, - std::string_view) -{ - auto method_str(path.substr(api_name.size())); - auto method = magic_enum::enum_cast(method_str); - if (method) - { - if (tzg::SnappyStatistic::globalInstance().getMethod() != *method) - { - ClearCompress(); - } - tzg::SnappyStatistic::globalInstance().setMethod(*method); - auto * res = RawCppString::New(fmt::format("Set compress method to {}", method_str)); - return HttpRequestRes{ - .status = HttpRequestStatus::Ok, - .res = CppStrWithView{ - .inner = GenRawCppPtr(res, RawCppPtrTypeImpl::String), - .view = BaseBuffView{res->data(), res->size()}}}; - } - else - { - return HttpRequestRes{.status = HttpRequestStatus::ErrorParam, .res = CppStrWithView{.inner = GenRawCppPtr(), .view = BaseBuffView{}}}; - } -} - - using HANDLE_HTTP_URI_METHOD = HttpRequestRes (*)(EngineStoreServerWrap *, std::string_view, const std::string &, std::string_view, std::string_view); static const std::map AVAILABLE_HTTP_URI = { {"/tiflash/sync-status/", HandleHttpRequestSyncStatus}, - {"/tiflash/store-status", HandleHttpRequestStoreStatus}, - {"/tiflash/tzg-compress", HandleHttpRequestCompressStatus}, - {"/tiflash/tzg-clean-compress", HandleHttpRequestCompressClean}, - {"/tiflash/set-tzg-compress-method/", HandleHttpRequestSetCompressMethod}, - {"/tiflash/get-tzg-compress-stream-cnt", HandleHttpRequestStreamCnt}, - {"/tiflash/get-tzg-encode-info", HandleHttpRequestEncodeInfo}, -}; + {"/tiflash/store-status", HandleHttpRequestStoreStatus}}; uint8_t CheckHttpUriAvailable(BaseBuffView path_) { diff --git a/dbms/src/Storages/Transaction/ReadIndexWorker.cpp b/dbms/src/Storages/Transaction/ReadIndexWorker.cpp index 374d646ccfc..27ad9e14d81 100644 --- a/dbms/src/Storages/Transaction/ReadIndexWorker.cpp +++ b/dbms/src/Storages/Transaction/ReadIndexWorker.cpp @@ -720,7 +720,7 @@ void ReadIndexWorker::consumeRegionNotifies(std::chrono::steady_clock::duration node->runOneRound(proxy_helper, read_index_notify_ctrl); } - TEST_LOG_FMT("worker {} set last run time {}", getID(), std::chrono::steady_clock::now()); + TEST_LOG_FMT("worker {} set last run time {}", getID(), Clock::now()); last_run_time.store(std::chrono::steady_clock::now(), std::memory_order_release); } diff --git a/libs/libcommon/CMakeLists.txt b/libs/libcommon/CMakeLists.txt index dbda79e250d..7632250b512 100644 --- a/libs/libcommon/CMakeLists.txt +++ b/libs/libcommon/CMakeLists.txt @@ -158,27 +158,18 @@ endif () # Region for specialized CPU flags tuning -check_then_add_sources_compile_flag ( - TIFLASH_ENABLE_AVX_SUPPORT - "${TIFLASH_COMPILER_AVX2_FLAG}" - src/mem_utils_avx2.cpp - src/crc64_avx2.cpp - src/avx2_mem_utils_impl.cpp -) check_then_add_sources_compile_flag ( TIFLASH_COMPILER_VPCLMULQDQ_SUPPORT "-mvpclmulqdq;-Wno-ignored-attributes" src/crc64_avx2.cpp src/crc64_avx512.cpp ) + check_then_add_sources_compile_flag ( - TIFLASH_COMPILER_MOVBE_SUPPORT - "${TIFLASH_COMPILER_MOVBE_FLAG}" - src/avx2_mem_utils_impl.cpp -) -check_then_add_sources_compile_flag ( - TIFLASH_COMPILER_BMI2_SUPPORT - "${TIFLASH_COMPILER_BMI2_FLAG}" + TIFLASH_COMPILER_ARCH_HASWELL_SUPPORT + "${TIFLASH_COMPILER_ARCH_HASWELL_FLAG}" + src/mem_utils_avx2.cpp + src/crc64_avx2.cpp src/avx2_mem_utils_impl.cpp ) diff --git a/libs/libcommon/include/common/avx2_memcpy.h b/libs/libcommon/include/common/avx2_memcpy.h index 9e2556dee3d..02f4ec0aa1d 100644 --- a/libs/libcommon/include/common/avx2_memcpy.h +++ b/libs/libcommon/include/common/avx2_memcpy.h @@ -95,7 +95,7 @@ ALWAYS_INLINE static inline void * avx2_inline_memcpy(void * __restrict dst_, co assert(size_t(dst) % block32_size == 0); } - // TODO: use non-temporal way(mark data unlikely to be used again soon) to minimize caching for large memory size(bigger than L3 cache size) if necessary. + // TODO: use non-temporal way(mark data unlikely to be used again soon) to minimize caching for large memory size(bigger than L2/L3 cache size) if necessary. // TODO: check whether source address is aligned to 32 and use specific aligned instructions if necessary. /// Aligned unrolled copy. diff --git a/libs/libcommon/include/common/sse2_memcpy.h b/libs/libcommon/include/common/sse2_memcpy.h index 3c75d9c225a..5c57275b39d 100644 --- a/libs/libcommon/include/common/sse2_memcpy.h +++ b/libs/libcommon/include/common/sse2_memcpy.h @@ -17,98 +17,14 @@ #include #include -#include #include #include -template -ALWAYS_INLINE inline void memcpy_ignore_overlap(char * __restrict dst, const char * __restrict src, size_t size); - -template -ALWAYS_INLINE inline void memcpy_block32_ignore_overlap(char * __restrict dst, const char * __restrict src, size_t size); - -template -ALWAYS_INLINE inline void memcpy_ignore_overlap(char * __restrict dst, const char * __restrict src, size_t size) -{ - assert(size >= sizeof(T)); - auto a = *reinterpret_cast(src); - auto b = *reinterpret_cast(src + size - sizeof(T)); - *reinterpret_cast(dst) = a; - *reinterpret_cast(dst + size - sizeof(T)) = b; -} - -template <> -ALWAYS_INLINE inline void memcpy_ignore_overlap<2>(char * __restrict dst, const char * __restrict src, size_t size) -{ - assert(size >= 2 && size <= 4); - using T = uint16_t; - static_assert(sizeof(T) == 2); - memcpy_ignore_overlap(dst, src, size); -} -template <> -ALWAYS_INLINE inline void memcpy_ignore_overlap<4>(char * __restrict dst, const char * __restrict src, size_t size) -{ - assert(size >= 4 && size <= 8); - using T = uint32_t; - static_assert(sizeof(T) == 4); - memcpy_ignore_overlap(dst, src, size); -} -template <> -ALWAYS_INLINE inline void memcpy_ignore_overlap<8>(char * __restrict dst, const char * __restrict src, size_t size) -{ - assert(size >= 8 && size <= 16); - using T = uint64_t; - static_assert(sizeof(T) == 8); - memcpy_ignore_overlap(dst, src, size); -} -template <> -ALWAYS_INLINE inline void memcpy_ignore_overlap<16>(char * __restrict dst, const char * __restrict src, size_t size) -{ - assert(size >= 16 && size <= 32); - auto c0 = _mm_loadu_si128(reinterpret_cast(src)); - auto c1 = _mm_loadu_si128(reinterpret_cast(src + size - 16)); - _mm_storeu_si128(reinterpret_cast<__m128i *>(dst), c0); - _mm_storeu_si128(reinterpret_cast<__m128i *>(dst + size - 16), c1); -} - -ALWAYS_INLINE static inline void sse2_inline_memcpy_small(void * __restrict dst_, const void * __restrict src_, size_t size) -{ - char * __restrict dst = reinterpret_cast(dst_); - const char * __restrict src = reinterpret_cast(src_); - - assert(size <= 32); - - if unlikely (size <= 1) - { - if likely (size == 1) - { - /// A single byte. - *dst = *src; - } - /// No bytes remaining. - } - else if unlikely (size <= 4) // sse2_inline_memcpy(_,_, 4 ) should use 4 bytes register directly - { - /// Chunks of 2..4 bytes. - memcpy_ignore_overlap<2>(dst, src, size); - } - else if unlikely (size <= 8) // sse2_inline_memcpy(_,_, 8 ) should use 8 bytes register directly - { - /// Chunks of 4..8 bytes. - memcpy_ignore_overlap<4>(dst, src, size); - } - else if unlikely (size <= 16) - { - /// Chunks of 8..16 bytes. - memcpy_ignore_overlap<8>(dst, src, size); - } - else - { - /// Chunks of 17..32 bytes. - memcpy_ignore_overlap<16>(dst, src, size); - } -} - +// Custom inline memcpy implementation for TiFlash. +// - it is recommended to use for inline function with `sse2` supported +// - it perform better than `legacy::inline_memcpy`(from clickhouse) according to `libs/libcommon/src/tests/bench_memcpy.cpp` +// - like `std::memcpy`, the behavior is undefined when the source and the destination objects overlap +// - moving data from register to memory costs more than the reversed way, so it's useful to reduce times about memory copying. ALWAYS_INLINE static inline void * sse2_inline_memcpy(void * __restrict dst_, const void * __restrict src_, size_t size) { char * __restrict dst = reinterpret_cast(dst_); @@ -116,12 +32,48 @@ ALWAYS_INLINE static inline void * sse2_inline_memcpy(void * __restrict dst_, co void * ret = dst; +#if defined(MCP) || defined(MCP_END) + static_assert(false); +#endif + #define MCP_END(n) tiflash_compiler_builtin_memcpy(dst + size - (n), src + size - (n), (n)); #define MCP(n) tiflash_compiler_builtin_memcpy(dst, src, (n)); - if likely (size <= 32) + if (likely(size <= 32)) { - sse2_inline_memcpy_small(dst, src, size); + if (unlikely(size <= 1)) + { + if (likely(size == 1)) + { + /// A single byte. + *dst = *src; + } + /// No bytes remaining. + } + else if (unlikely(size < 4)) // sse2_inline_memcpy(_,_, 4 ) should use 4 bytes register directly + { + /// Chunks of 2..3 bytes. + MCP(2); + MCP_END(2); + } + else if (unlikely(size < 8)) // sse2_inline_memcpy(_,_, 8 ) should use 8 bytes register directly + { + /// Chunks of 4..7 bytes. + MCP(4); + MCP_END(4); + } + else if (unlikely(size <= 16)) + { + /// Chunks of 8..16 bytes. + MCP(8); + MCP_END(8); + } + else + { + /// Chunks of 17..32 bytes. + MCP(16); + MCP_END(16); + } } else { diff --git a/libs/libmemcpy/CMakeLists.txt b/libs/libmemcpy/CMakeLists.txt index ae91ddf7bb6..eb98411e358 100644 --- a/libs/libmemcpy/CMakeLists.txt +++ b/libs/libmemcpy/CMakeLists.txt @@ -39,7 +39,11 @@ else () list (APPEND memcpy_sources memcpy.cpp) endif() -add_sources_compile_flag_avx2 (${memcpy_sources}) +check_then_add_sources_compile_flag ( + TIFLASH_COMPILER_ARCH_HASWELL_SUPPORT + "${TIFLASH_COMPILER_ARCH_HASWELL_FLAG}" + ${memcpy_sources} +) add_library (memcpy STATIC ${memcpy_sources}) target_include_directories(memcpy PUBLIC ${TiFlash_SOURCE_DIR}/libs/libcommon/include) diff --git a/libs/libsnappy/CMakeLists.txt b/libs/libsnappy/CMakeLists.txt deleted file mode 100644 index c7ebbbe3059..00000000000 --- a/libs/libsnappy/CMakeLists.txt +++ /dev/null @@ -1,27 +0,0 @@ -# Copyright 2022 PingCAP, Ltd. -# -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -if (TIFLASH_ENABLE_AVX_SUPPORT) - set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -mavx2") - set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} mavx2") - set (SNAPPY_REQUIRE_AVX2 ON) -endif () - -#set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -fomit-frame-pointer") -#set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -fomit-frame-pointer") - -set (SNAPPY_BUILD_TESTS OFF CACHE BOOL "Build Snappy's own tests.") -set (SNAPPY_BUILD_BENCHMARKS OFF CACHE BOOL "Build Snappy for fuzzing.") - -add_subdirectory (snappy) diff --git a/tests/fullstack-test2/mpp/mpp-version.test b/tests/fullstack-test2/mpp/mpp-version.test new file mode 100644 index 00000000000..19711e9cdde --- /dev/null +++ b/tests/fullstack-test2/mpp/mpp-version.test @@ -0,0 +1,48 @@ +# Copyright 2022 PingCAP, Ltd. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +# Preparation. +=> DBGInvoke __init_fail_point() + +mysql> drop table if exists test.t; +mysql> create table if not exists test.t(a int, b int); + +mysql> insert into test.t values(1,5), (2,5), (3,5), (4,5); +mysql> alter table test.t set tiflash replica 1; +func> wait_table test t + +mysql> use test; set tidb_enforce_mpp=on; set tidb_isolation_read_engines='tiflash'; select count(1) as c from t group by b; + ++---+ +| c | ++---+ +| 4 | ++---+ + +=> DBGInvoke __enable_fail_point(invalid_mpp_version) + +mysql> use test; set tidb_enforce_mpp=on; set tidb_isolation_read_engines='tiflash'; select count(1) as c from t group by b; + +ERROR 1105 (HY000): rpc error: code = Internal desc = Failed to handling mpp dispatch request, reason=`Invalid mpp version -1, TiFlash expects version: min 0, max 1, should upgrade TiDB/planner` +=> DBGInvoke __disable_fail_point(invalid_mpp_version) + +mysql> use test; set tidb_enforce_mpp=on; set tidb_isolation_read_engines='tiflash'; select count(1) as c from t group by b; + ++---+ +| c | ++---+ +| 4 | ++---+ + +mysql> drop table if exists test.t diff --git a/tiflash-tools/.gitignore b/tiflash-tools/.gitignore deleted file mode 100644 index 6b1b5d54dad..00000000000 --- a/tiflash-tools/.gitignore +++ /dev/null @@ -1,5 +0,0 @@ -/dist/ -/build/ -*.spec -version.py -/tikv_util/.eggs/ diff --git a/tiflash-tools/define.py b/tiflash-tools/define.py deleted file mode 100644 index 2e01aca50a1..00000000000 --- a/tiflash-tools/define.py +++ /dev/null @@ -1,18 +0,0 @@ -#!/usr/bin/python3 - -TIFLASH = 'tiflash' -TIFLASH_LABEL = {'key': 'engine', 'value': TIFLASH} -REGION_COUNT = 'region_count' -TIFLASH_REGION_COUNT = 'flash_region_count' -LOCATION_LABELS = 'location_labels' -REPLICA_COUNT = 'replica_count' -LEARNER = 'learner' -AVAILABLE = 'available' -PRIORITY = 'high_priority' -TIFLASH_CLUSTER_MUTEX_KEY = '/{}/cluster/leader'.format(TIFLASH) -TIFLASH_CLUSTER_HTTP_PORT = '/{}/cluster/http_port/'.format(TIFLASH) -LABEL_CONSTRAINTS = 'label_constraints' -DDL_GLOBAL_SCHEMA_VERSION = '/tidb/ddl/global_schema_version' -TIFLASH_LAST_HANDLED_SCHEMA_VERSION = '/{}/cluster/last_handled_schema_version'.format(TIFLASH) -TIFLASH_LAST_HANDLED_SCHEMA_VERSION_TSO_SPLIT = b'_tso_' -TIFLASH_LAST_HANDLED_SCHEMA_TIME_OUT = 300 diff --git a/tiflash-tools/flash_cluster_manager.py b/tiflash-tools/flash_cluster_manager.py deleted file mode 100644 index 320b92e3223..00000000000 --- a/tiflash-tools/flash_cluster_manager.py +++ /dev/null @@ -1,138 +0,0 @@ -#!/usr/bin/python3 -import argparse -import logging - -import define -import flash_http_client -from pd_client import PDClient - - -class Store: - def __eq__(self, other): - return self.inner == other - - def __str__(self): - return str(self.inner) - - def __init__(self, pd_store): - self.inner = pd_store - address = self.inner['address'] - host, port = address.split(':') - self.address = '{}:{}'.format(host, port) - _, status_port = self.inner['status_address'].split(':') - self.tiflash_status_address = '{}:{}'.format(host, status_port) - - @property - def id(self): - return self.inner['id'] - - -class Runner: - def __init__(self): - parser = argparse.ArgumentParser(description="check compress statistic", - formatter_class=argparse.ArgumentDefaultsHelpFormatter) - parser.add_argument( - '--pd-address', default="172.16.4.39:2174") - parser.add_argument( - '--type', help='run type', required=True, choices=('show', 'clean', 'set', )) - parser.add_argument( - '--method', choices=('SNAPPY', 'LZ4', 'ZSTD', 'NONE')) - self.args = parser.parse_args() - try: - assert self.args.pd_address - pd_address = self.args.pd_address.split(",") - pd_client = PDClient(pd_address) - self.tiflash_stores = {store_id: Store(store) for store_id, store in pd_client.get_store_by_labels( - define.TIFLASH_LABEL).items()} - except Exception as e: - logging.exception(e) - - def run(self): - if self.args.type == 'show': - self.run_show() - elif self.args.type == 'clean': - self.run_clean() - elif self.args.type == 'set': - self.run_set_method() - elif self.args.type == 'get-cnt': - self.run_show_cnt() - elif self.args.type == 'get-encode': - self.run_get_encode() - - def run_clean(self): - res = {store_id: flash_http_client.clean_compress_info( - store.tiflash_status_address) for store_id, store in self.tiflash_stores.items()} - print(res) - - def run_set_method(self): - assert self.args.method - res = {store_id: flash_http_client.set_compress_method( - store.tiflash_status_address, self.args.method) for store_id, store in self.tiflash_stores.items()} - print(res) - - def run_show(self): - compress_info = {store_id: flash_http_client.get_compress_info( - store.tiflash_status_address) for store_id, store in self.tiflash_stores.items()} - tol_compressed_size = 0 - tol_uncompressed_size = 0 - method = None - for store_id, info in compress_info.items(): - print('store {}, addr {}, {}'.format( - store_id, self.tiflash_stores[store_id].tiflash_status_address, info)) - tol_compressed_size += int(info['compressed_size']) - tol_uncompressed_size += int(info['uncompressed_size']) - if method: - assert method == info['method'] - else: - method = info['method'] - compress_rate = 0.0 if not tol_uncompressed_size else tol_compressed_size / \ - tol_uncompressed_size - MB = 1024**2 - msg = 'method: {}, compress_rate: {:.3f}, compressed_size: {:.3f}MB, uncompressed_size: {:.3f}MB'.format( - method, compress_rate, - tol_compressed_size/MB, tol_uncompressed_size/MB, - ) - print(msg) - - def run_show_cnt(self): - compress_info = {store_id: flash_http_client.get_stream_info( - store.tiflash_status_address) for store_id, store in self.tiflash_stores.items()} - for store_id, info in compress_info.items(): - print('store {}: {}'.format(store_id, info)) - - def run_get_encode(self): - compress_info = {store_id: flash_http_client.get_codec_info( - store.tiflash_status_address) for store_id, store in self.tiflash_stores.items()} - all_bytes = {} - all_time = {} - all_hash_row = {} - all_hash_time = {} - for store_id, info in compress_info.items(): - all_bytes[store_id] = 0 - all_time[store_id] = 0 - all_hash_row[store_id] = 0 - all_hash_time[store_id] = 0 - x = [x for x in info.split(",")] - for o in x: - a, b = o.split(':') - a, b = a.strip(), int(b.strip()) - if a == 'uncompress-bytes': - all_bytes[store_id] += b - elif a == 'time': - all_time[store_id] += b - elif a == 'hash-part-write': - all_hash_row[store_id] += b - elif a == 'hash-part-time': - all_hash_time[store_id] += b - - all_time[store_id] = all_time[store_id]/(10**9) - all_hash_time[store_id] = all_hash_time[store_id]/(10**9) - all_bytes[store_id] /= 1024 ** 2 - y = all_bytes[store_id] / \ - all_time[store_id] if all_time[store_id] else 0.0 - print("store {}: uncompress-bytes: {:.4f}MB, time: {:.4f}s, MBPS/core: {:.4f}, hash-part-write: {}rows, hash-part-time: {:.4f}s".format( - store_id, all_bytes[store_id], all_time[store_id], y, all_hash_row[store_id], all_hash_time[store_id])) - - -if __name__ == '__main__': - Runner().run() diff --git a/tiflash-tools/flash_http_client.py b/tiflash-tools/flash_http_client.py deleted file mode 100644 index 0870ddf0459..00000000000 --- a/tiflash-tools/flash_http_client.py +++ /dev/null @@ -1,51 +0,0 @@ -#!/usr/bin/python3 - -import util - - -def curl_flash(address, params): - if type(params) != dict: - params = {'query': params} - r = util.curl_http(address, params) - return r - - -def get_compress_info(tiflash_status_address): - res = util.curl_http( - '{}/tiflash/tzg-compress'.format(tiflash_status_address)) - util.check_status_code(res) - data = res.text - data = [e.strip() for e in data.split(',')] - res = {} - for e in data: - a, b = e.split(":") - res[a.strip()] = b.strip() - return res - - -def get_stream_info(tiflash_status_address): - res = util.curl_http( - '{}/tiflash/get-tzg-compress-stream-cnt'.format(tiflash_status_address)) - util.check_status_code(res) - return res.text - - -def get_codec_info(tiflash_status_address): - res = util.curl_http( - '{}/tiflash/get-tzg-encode-info'.format(tiflash_status_address)) - util.check_status_code(res) - return res.text - - -def set_compress_method(tiflash_status_address, method): - res = util.curl_http( - '{}/tiflash/set-tzg-compress-method/{}'.format(tiflash_status_address, method)) - util.check_status_code(res) - return res.text - - -def clean_compress_info(tiflash_status_address, ): - res = util.curl_http( - '{}/tiflash/tzg-clean-compress'.format(tiflash_status_address, )) - util.check_status_code(res) - return res.text diff --git a/tiflash-tools/pd_client.py b/tiflash-tools/pd_client.py deleted file mode 100644 index 4f25bc800e5..00000000000 --- a/tiflash-tools/pd_client.py +++ /dev/null @@ -1,126 +0,0 @@ -#!/usr/bin/python3 -import logging -from typing import Optional - -import uri -import util - - -class PDClient: - PD_API_PREFIX = 'pd/api' - PD_API_VERSION = 'v1' - - def get_all_regions_json(self): - r = util.curl_http('{}/{}/{}/regions'.format(self.leader, - PDClient.PD_API_PREFIX, PDClient.PD_API_VERSION)) - return util.try_get_json(r) - - def get_regions_by_key_json(self, key: str, limit=16): - r = util.curl_http( - '{}/{}/{}/regions/key'.format(self.leader, - PDClient.PD_API_PREFIX, PDClient.PD_API_VERSION), - {'key': key, 'limit': limit}) - return util.try_get_json(r) - - def get_region_by_id_json(self, region_id: int): - r = util.curl_http( - '{}/{}/{}/region/id/{}'.format(self.leader, PDClient.PD_API_PREFIX, PDClient.PD_API_VERSION, region_id)) - return util.try_get_json(r) - - def get_all_stores_json(self): - r = util.curl_http( - '{}/{}/{}/stores'.format(self.leader, PDClient.PD_API_PREFIX, PDClient.PD_API_VERSION)) - return util.try_get_json(r) - - def get_members_json(self, *args): - url = args[0] if args else self.leader - r = util.curl_http( - '{}/{}/{}/members'.format(url, PDClient.PD_API_PREFIX, PDClient.PD_API_VERSION)) - return util.try_get_json(r) - - def get_stats_region_by_range_json(self, start_key, end_key): - r = util.curl_http( - '{}/{}/{}/stats/region'.format(self.leader, - PDClient.PD_API_PREFIX, PDClient.PD_API_VERSION), - {'start_key': start_key, 'end_key': end_key}, - ) - return util.try_get_json(r) - - def get_group_rules(self, group): - r = util.curl_http( - '{}/{}/{}/config/rules/group/{}'.format(self.leader, PDClient.PD_API_PREFIX, PDClient.PD_API_VERSION, - group)) - res = util.try_get_json(r) - res = res if res is not None else {} - for e in res: - if not isinstance(e, dict): - raise Exception('Got placement rules fail: {}'.format(r.text)) - from placement_rule import PlacementRule - return {e['id']: PlacementRule(**e) for e in res} - - def get_all_rules(self): - r = util.curl_http( - '{}/{}/{}/config/rules'.format(self.leader, PDClient.PD_API_PREFIX, PDClient.PD_API_VERSION)) - res = util.try_get_json(r) - return res if res is not None else {} - - def get_rule(self, group, rule_id): - r = util.curl_http( - '{}/{}/{}/config/rule/{}/{}'.format(self.leader, PDClient.PD_API_PREFIX, PDClient.PD_API_VERSION, group, - rule_id)) - return util.try_get_json(r) - - def set_rule(self, rule): - r = util.post_http( - '{}/{}/{}/config/rule'.format(self.leader, PDClient.PD_API_PREFIX, PDClient.PD_API_VERSION), rule) - return r.status_code - - def set_accelerate_schedule(self, start_key, end_key): - r = util.post_http( - '{}/{}/{}/regions/accelerate-schedule'.format( - self.leader, PDClient.PD_API_PREFIX, PDClient.PD_API_VERSION), - {'start_key': start_key, 'end_key': end_key}, ) - if r.status_code != 200: - raise Exception( - "fail to accelerate schedule range [{},{}), error msg: {}".format(start_key, end_key, r.text)) - - def remove_rule(self, group, rule_id): - r = util.delete_http( - '{}/{}/{}/config/rule/{}/{}'.format(self.leader, PDClient.PD_API_PREFIX, PDClient.PD_API_VERSION, group, - rule_id)) - return r.status_code - - def _try_update_leader_etcd(self, url): - resp = self.get_members_json(url) - leader = resp.get('leader', {}) - client_urls = leader.get('client_urls', []) - if client_urls: - _client_urls = [] - for member in resp.get('members', {}): - _client_urls.extend(member.get('client_urls', [])) - self.urls = _client_urls - self.leader = uri.URI(client_urls[0]).authority - - def _update_leader_etcd(self): - errors = [] - for url in self.urls: - try: - return self._try_update_leader_etcd(url) - except Exception as e: - errors.append(e) - raise Exception("can not find pd leader: {}".format(errors)) - - def get_store_by_labels(self, flash_label): - res = {} - all_stores = self.get_all_stores_json() - for store in all_stores['stores']: - store = store['store'] - for label in store.get('labels', []): - if label == flash_label: - res[store['id']] = store - return res - - def __init__(self, urls): - self.urls = urls - self.leader = "" - self._update_leader_etcd() diff --git a/tiflash-tools/util.py b/tiflash-tools/util.py deleted file mode 100644 index 5b01da3dbeb..00000000000 --- a/tiflash-tools/util.py +++ /dev/null @@ -1,138 +0,0 @@ -#!/usr/bin/python3 -import errno -import fcntl -import logging -import os -import socket -import time - -import requests - - -def wrap_run_time(func): - def wrap_func(*args, **kwargs): - bg = time.time() - r = func(*args, **kwargs) - print('time cost {}'.format(time.time() - bg)) - return r - - return wrap_func - - -class FLOCK(object): - def __init__(self, name): - self.obj = open(name, 'w') - self.fd = self.obj.fileno() - - def lock(self): - try: - fcntl.lockf(self.fd, fcntl.LOCK_EX | fcntl.LOCK_NB) - return True - except OSError: - logging.error( - 'Cannot lock file {}. Another instance in same directory is already running'.format(self.obj.name)) - return False - - -def gen_http_kwargs(): - kwargs = {} - http_name = 'http' - return http_name, kwargs - - -def curl_http(uri, params=None): - if params is None: - params = {} - http_name, kwargs = gen_http_kwargs() - r = requests.get('{}://{}'.format(http_name, uri), params, **kwargs) - return r - - -def check_status_code(r): - if r.status_code != 200: - raise Exception( - 'unexpected status code `{}` from `{}` error-msg `{}`'.format(r.status_code, r.url, r.text)) - - -def try_get_json(r): - check_status_code(r) - return r.json() - - -def post_http(uri, params): - http_name, kwargs = gen_http_kwargs() - r = requests.post('{}://{}'.format(http_name, uri), json=params, **kwargs) - return r - - -def delete_http(uri): - http_name, kwargs = gen_http_kwargs() - r = requests.delete('{}://{}'.format(http_name, uri), **kwargs) - return r - - -def obj_2_dict(obj): - pr = {} - for name in dir(obj): - value = getattr(obj, name) - if not name.startswith('_') and not callable(value): - pr[name] = value - return pr - - -def make_compare_pd_key(key): - return (1, key) if key else (0, '') - - -def net_is_used(ip, port): - s = socket.socket(socket.AF_INET, socket.SOCK_STREAM) - result = s.connect_ex((ip, port)) - s.close() - return result == 0 - - -def pid_exists(pid): - """Check whether pid exists in the current process table. - UNIX only. - """ - if pid < 0: - return False - if pid == 0: - # According to "man 2 kill" PID 0 refers to every process - # in the process group of the calling process. - # On certain systems 0 is a valid PID but we have no way - # to know that in a portable fashion. - raise ValueError('invalid PID 0') - try: - os.kill(pid, 0) - except OSError as err: - if err.errno == errno.ESRCH: - # ESRCH == No such process - return False - elif err.errno == errno.EPERM: - # EPERM clearly means there's a process to deny access to - return True - else: - # According to "man 2 kill" possible error values are - # (EINVAL, EPERM, ESRCH) - raise - else: - return True - - -def pid_exists2(pid): - if pid == 0: - return True - return pid_exists(pid) - - -def compute_addr_list(addrs): - return [e.strip() for e in addrs.split(',') if e] - - -def main(): - pass - - -if __name__ == '__main__': - main() From c9608032bde687779a3005420f8211c167dda0f5 Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Tue, 10 Jan 2023 23:41:09 +0800 Subject: [PATCH 56/93] 50 --- dbms/src/Flash/Mpp/MPPTask.cpp | 23 +++++++++-------------- dbms/src/Flash/Mpp/MPPTask.h | 2 +- 2 files changed, 10 insertions(+), 15 deletions(-) diff --git a/dbms/src/Flash/Mpp/MPPTask.cpp b/dbms/src/Flash/Mpp/MPPTask.cpp index f6e5d942823..2b73629df7d 100644 --- a/dbms/src/Flash/Mpp/MPPTask.cpp +++ b/dbms/src/Flash/Mpp/MPPTask.cpp @@ -26,7 +26,6 @@ #include #include #include -#include #include #include #include @@ -34,6 +33,7 @@ #include #include #include +#include #include #include @@ -52,8 +52,9 @@ extern const char exception_during_mpp_register_tunnel_for_non_root_mpp_task[]; extern const char force_no_local_region_for_mpp_task[]; } // namespace FailPoints -MPPTask::MPPTask(const mpp::TaskMeta & meta, const ContextPtr & context_) - : id(meta) +MPPTask::MPPTask(const mpp::TaskMeta & meta_, const ContextPtr & context_) + : meta(meta_) + , id(meta) , context(context_) , manager(context_->getTMTContext().getMPPTaskManager().get()) , schedule_entry(manager, id) @@ -120,8 +121,6 @@ void MPPTask::run() void MPPTask::registerTunnels(const mpp::DispatchTaskRequest & task_request) { - const mpp::TaskMeta & sender_meta = task_request.meta(); - auto tunnel_set_local = std::make_shared(log->identifier()); std::chrono::seconds timeout(task_request.timeout()); const auto & exchange_sender = dag_req.root_executor().exchange_sender(); @@ -129,19 +128,16 @@ void MPPTask::registerTunnels(const mpp::DispatchTaskRequest & task_request) for (int i = 0; i < exchange_sender.encoded_task_meta_size(); ++i) { // exchange sender will register the tunnels and wait receiver to found a connection. - mpp::TaskMeta receiver_meta; - if (unlikely(!receiver_meta.ParseFromString(exchange_sender.encoded_task_meta(i)))) + mpp::TaskMeta task_meta; + if (unlikely(!task_meta.ParseFromString(exchange_sender.encoded_task_meta(i)))) throw TiFlashException("Failed to decode task meta info in ExchangeSender", Errors::Coprocessor::BadRequest); - - bool is_local = context->getSettingsRef().enable_local_tunnel && sender_meta.address() == receiver_meta.address(); - + bool is_local = context->getSettingsRef().enable_local_tunnel && meta.address() == task_meta.address(); bool is_async = !is_local && context->getSettingsRef().enable_async_server; - - MPPTunnelPtr tunnel = std::make_shared(receiver_meta, sender_meta, timeout, context->getSettingsRef().max_threads, is_local, is_async, log->identifier()); + MPPTunnelPtr tunnel = std::make_shared(task_meta, task_request.meta(), timeout, context->getSettingsRef().max_threads, is_local, is_async, log->identifier()); LOG_DEBUG(log, "begin to register the tunnel {}, is_local: {}, is_async: {}", tunnel->id(), is_local, is_async); if (status != INITIALIZING) throw Exception(fmt::format("The tunnel {} can not be registered, because the task is not in initializing state", tunnel->id())); - tunnel_set_local->registerTunnel(MPPTaskId(receiver_meta), tunnel); + tunnel_set_local->registerTunnel(MPPTaskId(task_meta), tunnel); if (!dag_context->isRootMPPTask()) { FAIL_POINT_TRIGGER_EXCEPTION(FailPoints::exception_during_mpp_register_tunnel_for_non_root_mpp_task); @@ -279,7 +275,6 @@ void MPPTask::prepare(const mpp::DispatchTaskRequest & task_request) } is_root_mpp_task = task_meta.task_id() == -1; } - dag_context = std::make_unique(dag_req, task_request.meta(), is_root_mpp_task); dag_context->log = log; dag_context->tables_regions_info = std::move(tables_regions_info); diff --git a/dbms/src/Flash/Mpp/MPPTask.h b/dbms/src/Flash/Mpp/MPPTask.h index 81a3ae35585..b2c203bf9f5 100644 --- a/dbms/src/Flash/Mpp/MPPTask.h +++ b/dbms/src/Flash/Mpp/MPPTask.h @@ -109,7 +109,7 @@ class MPPTask : public std::enable_shared_from_this void initExchangeReceivers(); tipb::DAGRequest dag_req; - // mpp::TaskMeta meta; + mpp::TaskMeta meta; MPPTaskId id; ContextPtr context; From 74111b842fd28b9980a37203e1a1ade2083aa637 Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Tue, 10 Jan 2023 23:45:13 +0800 Subject: [PATCH 57/93] 51 --- dbms/src/Flash/Mpp/ExchangeReceiver.cpp | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/dbms/src/Flash/Mpp/ExchangeReceiver.cpp b/dbms/src/Flash/Mpp/ExchangeReceiver.cpp index 9dd4f37ed4b..021ee2410e5 100644 --- a/dbms/src/Flash/Mpp/ExchangeReceiver.cpp +++ b/dbms/src/Flash/Mpp/ExchangeReceiver.cpp @@ -26,11 +26,8 @@ #include #include -#include #include -#include "IO/CompressedStream.h" - namespace DB { namespace FailPoints @@ -64,7 +61,7 @@ bool pushPacket(size_t source_index, bool push_succeed = true; const mpp::Error * error_ptr = nullptr; - auto & packet = tracked_packet->getPacket(); + auto & packet = tracked_packet->packet; if (packet.has_error()) error_ptr = &packet.error(); const String * resp_ptr = nullptr; @@ -103,7 +100,7 @@ bool pushPacket(size_t source_index, if (resp_ptr == nullptr && error_ptr == nullptr && chunks[i].empty()) continue; - auto recv_msg = std::make_shared( + std::shared_ptr recv_msg = std::make_shared( source_index, req_info, tracked_packet, From 441ad5694147b4c32f192b097168a5c48d6dbe11 Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Wed, 11 Jan 2023 12:36:22 +0800 Subject: [PATCH 58/93] 52 Signed-off-by: Zhigao Tong --- .../Coprocessor/tests/gtest_ti_remote_block_inputstream.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/dbms/src/Flash/Coprocessor/tests/gtest_ti_remote_block_inputstream.cpp b/dbms/src/Flash/Coprocessor/tests/gtest_ti_remote_block_inputstream.cpp index 0162b940ce4..2c1ed933ac3 100644 --- a/dbms/src/Flash/Coprocessor/tests/gtest_ti_remote_block_inputstream.cpp +++ b/dbms/src/Flash/Coprocessor/tests/gtest_ti_remote_block_inputstream.cpp @@ -116,6 +116,10 @@ struct MockWriter write(tmp); } uint16_t getPartitionNum() const { return 1; } + bool isLocal(size_t index) const + { + return index == 0; + } PacketQueuePtr queue; bool add_summary = false; From 218d2728230cbe02c6d5e5a845325a7e16b370b2 Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Wed, 11 Jan 2023 15:54:46 +0800 Subject: [PATCH 59/93] 53 Signed-off-by: Zhigao Tong --- .../Flash/Coprocessor/CHBlockChunkCodecV1.cpp | 18 ++++++++++++++++-- .../Flash/Coprocessor/CHBlockChunkCodecV1.h | 2 +- dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp | 2 +- 3 files changed, 18 insertions(+), 4 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.cpp b/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.cpp index 55725992b53..463bb7def46 100644 --- a/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.cpp +++ b/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.cpp @@ -1,5 +1,19 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + #include #include @@ -11,7 +25,7 @@ namespace DB { extern void WriteColumnData(const IDataType & type, const ColumnPtr & column, WriteBuffer & ostr, size_t offset, size_t limit); -size_t GetExtraInfoSize(const Block & block) +size_t ApproxBlockHeaderBytes(const Block & block) { size_t size = 8 + 8; /// to hold some length of structures, such as column number, row number... size_t columns = block.columns(); @@ -32,7 +46,7 @@ size_t GetExtraInfoSize(const Block & block) size_t ApproxBlockBytes(const Block & block) { - return block.bytes() + GetExtraInfoSize(block); + return block.bytes() + ApproxBlockHeaderBytes(block); } void EncodeHeader(WriteBuffer & ostr, const Block & header, size_t rows) diff --git a/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.h b/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.h index d5f6966097e..66cf32141a1 100644 --- a/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.h +++ b/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.h @@ -22,7 +22,7 @@ namespace DB { size_t ApproxBlockBytes(const Block & block); -size_t GetExtraInfoSize(const Block & block); +size_t ApproxBlockHeaderBytes(const Block & block); using CompressedCHBlockChunkReadBuffer = CompressedReadBuffer; using CompressedCHBlockChunkWriteBuffer = CompressedWriteBuffer; void EncodeHeader(WriteBuffer & ostr, const Block & header, size_t rows); diff --git a/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp b/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp index a801908aa1e..e5f44df185a 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp +++ b/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp @@ -138,7 +138,7 @@ void HashPartitionWriterV1::partitionAndEncodeThenWriteBlocks } } - size_t header_size = GetExtraInfoSize(dest_block_header); + size_t header_size = ApproxBlockHeaderBytes(dest_block_header); for (size_t part_id = 0; part_id < partition_num; ++part_id) { From 63b19e94772aeb1423adc195cc1977ac94fdb7ba Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Wed, 11 Jan 2023 15:55:34 +0800 Subject: [PATCH 60/93] 54 Signed-off-by: Zhigao Tong --- dbms/src/Flash/Mpp/MppVersion.h | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/dbms/src/Flash/Mpp/MppVersion.h b/dbms/src/Flash/Mpp/MppVersion.h index aeeb17b889d..2a4880fb923 100644 --- a/dbms/src/Flash/Mpp/MppVersion.h +++ b/dbms/src/Flash/Mpp/MppVersion.h @@ -1,3 +1,17 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + #pragma once #include From 545540ca5bb884ddba1af6a76a38c484fbabd59b Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Wed, 11 Jan 2023 16:15:10 +0800 Subject: [PATCH 61/93] 55 Signed-off-by: Zhigao Tong --- dbms/src/Flash/Coprocessor/DAGContext.h | 6 ------ dbms/src/Flash/Mpp/FineGrainedShuffleWriter.cpp | 8 ++++++++ dbms/src/Flash/Mpp/HashPartitionWriter.cpp | 6 ++++-- dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp | 17 ++++++++++++----- .../Mpp/tests/gtest_mpp_exchange_writer.cpp | 7 +------ 5 files changed, 25 insertions(+), 19 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/DAGContext.h b/dbms/src/Flash/Coprocessor/DAGContext.h index 5c37300c99d..ce8b93e92d2 100644 --- a/dbms/src/Flash/Coprocessor/DAGContext.h +++ b/dbms/src/Flash/Coprocessor/DAGContext.h @@ -37,11 +37,6 @@ namespace DB { -namespace tests -{ -class TestMPPExchangeWriter; -} - class Context; class MPPTunnelSet; class ExchangeReceiver; @@ -387,7 +382,6 @@ class DAGContext private: void initExecutorIdToJoinIdMap(); void initOutputInfo(); - friend class tests::TestMPPExchangeWriter; private: std::shared_ptr process_list_entry; diff --git a/dbms/src/Flash/Mpp/FineGrainedShuffleWriter.cpp b/dbms/src/Flash/Mpp/FineGrainedShuffleWriter.cpp index c51d4ae8654..edd31161f84 100644 --- a/dbms/src/Flash/Mpp/FineGrainedShuffleWriter.cpp +++ b/dbms/src/Flash/Mpp/FineGrainedShuffleWriter.cpp @@ -19,6 +19,11 @@ #include #include +namespace DB +{ +extern size_t ApproxBlockBytes(const Block & block); +} + namespace DB { template @@ -110,6 +115,7 @@ void FineGrainedShuffleWriter::initScatterColumns() template void FineGrainedShuffleWriter::batchWriteFineGrainedShuffle() { + size_t ori_block_mem_size = 0; auto tracked_packets = HashBaseWriterHelper::createPackets(partition_num); if (likely(!blocks.empty())) { @@ -120,6 +126,7 @@ void FineGrainedShuffleWriter::batchWriteFineGrainedShuffle() while (!blocks.empty()) { const auto & block = blocks.back(); + ori_block_mem_size += ApproxBlockBytes(block); HashBaseWriterHelper::scatterColumnsForFineGrainedShuffle(block, partition_col_ids, collators, partition_key_containers_for_reuse, partition_num, fine_grained_shuffle_stream_count, hash, selector, scattered); blocks.pop_back(); } @@ -156,6 +163,7 @@ void FineGrainedShuffleWriter::batchWriteFineGrainedShuffle() } writePackets(tracked_packets); + GET_METRIC(tiflash_exchange_data_bytes, type_hash_original_all).Increment(ori_block_mem_size); } template diff --git a/dbms/src/Flash/Mpp/HashPartitionWriter.cpp b/dbms/src/Flash/Mpp/HashPartitionWriter.cpp index 565a0e1b148..0861f767325 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriter.cpp +++ b/dbms/src/Flash/Mpp/HashPartitionWriter.cpp @@ -19,6 +19,10 @@ #include #include +namespace DB +{ +extern size_t ApproxBlockBytes(const Block & block); +} namespace DB { template @@ -65,8 +69,6 @@ void HashPartitionWriter::write(const Block & block) partitionAndEncodeThenWriteBlocks(); } -extern size_t ApproxBlockBytes(const Block & block); - template void HashPartitionWriter::partitionAndEncodeThenWriteBlocks() { diff --git a/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp b/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp index e5f44df185a..d0d972d3ff8 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp +++ b/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp @@ -154,11 +154,18 @@ void HashPartitionWriterV1::partitionAndEncodeThenWriteBlocks return res + 8 /*partition rows*/; }); - // Each partition encode format: - // header meta(include all row count of the partition); - // repeated: - // row count; - // columns data; + // compression method flag; NONE = 0x02, LZ4 = 0x82, ZSTD = 0x90 + // ... + // header meta: + // columns count; + // total row count (multi parts); + // for each column: + // column name; + // column type; + // for each part: + // row count; + // columns data; + size_t init_size = part_column_bytes + header_size + 1 /*compression method*/; // Reserve enough memory buffer size diff --git a/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp b/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp index d0f921f56d3..2b2bad17b77 100644 --- a/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp +++ b/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp @@ -17,6 +17,7 @@ #include #include #include +#include #include #include #include @@ -30,11 +31,6 @@ #include #include -#include "Flash/Mpp/MppVersion.h" -#include "IO/CompressedStream.h" -#include "ext/scope_guard.h" -#include "mpp.pb.h" - namespace DB { namespace tests @@ -49,7 +45,6 @@ class TestMPPExchangeWriter : public testing::Test dag_context_ptr->is_mpp_task = true; dag_context_ptr->is_root_mpp_task = false; dag_context_ptr->result_field_types = makeFields(); - dag_context_ptr->mpp_task_meta.set_mpp_version(DB::GetMppVersion()); context.setDAGContext(dag_context_ptr.get()); } From 5f697d2b2b706d8c33893a0b62bf1c61acc126f9 Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Wed, 11 Jan 2023 17:08:49 +0800 Subject: [PATCH 62/93] 56 Signed-off-by: Zhigao Tong --- dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.cpp | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.cpp b/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.cpp index 776ba293c6f..5e1c514e0ac 100644 --- a/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.cpp +++ b/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.cpp @@ -14,7 +14,6 @@ #include #include -#include #include namespace DB @@ -85,13 +84,12 @@ std::optional CHBlockChunkDecodeAndSquash::decodeAndSquash(const String & res.swap(accumulated_block); return res; } - ReadBuffer * istr_ptr = &istr; if (!accumulated_block) { /// hard-code 1.5 here, since final column size will be more than rows_limit in most situations, /// so it should be larger than 1.0, just use 1.5 here, no special meaning - Block block = codec.decodeImpl(*istr_ptr, static_cast(rows_limit * 1.5)); + Block block = codec.decodeImpl(istr, static_cast(rows_limit * 1.5)); if (block) accumulated_block.emplace(std::move(block)); } @@ -100,7 +98,7 @@ std::optional CHBlockChunkDecodeAndSquash::decodeAndSquash(const String & /// Dimensions size_t columns = 0; size_t rows = 0; - codec.readBlockMeta(*istr_ptr, columns, rows); + codec.readBlockMeta(istr, columns, rows); if (rows) { @@ -108,8 +106,8 @@ std::optional CHBlockChunkDecodeAndSquash::decodeAndSquash(const String & for (size_t i = 0; i < columns; ++i) { ColumnWithTypeAndName column; - codec.readColumnMeta(i, *istr_ptr, column); - CHBlockChunkCodec::readData(*column.type, *(mutable_columns[i]), *istr_ptr, rows); + codec.readColumnMeta(i, istr, column); + CHBlockChunkCodec::readData(*column.type, *(mutable_columns[i]), istr, rows); } accumulated_block->setColumns(std::move(mutable_columns)); } From 0e6fb4b94f028d4b2fdaccd7a1d002cae18ad408 Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Wed, 11 Jan 2023 17:12:47 +0800 Subject: [PATCH 63/93] 57 Signed-off-by: Zhigao Tong --- dbms/src/Flash/Coprocessor/DAGContext.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/dbms/src/Flash/Coprocessor/DAGContext.cpp b/dbms/src/Flash/Coprocessor/DAGContext.cpp index c7f8b3ad516..3919343e509 100644 --- a/dbms/src/Flash/Coprocessor/DAGContext.cpp +++ b/dbms/src/Flash/Coprocessor/DAGContext.cpp @@ -17,7 +17,6 @@ #include #include #include -#include #include #include From 2be58a40b3e6e4a62dcfa9c399996f021401d42b Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Thu, 12 Jan 2023 11:55:21 +0800 Subject: [PATCH 64/93] 58 Signed-off-by: Zhigao Tong --- cmake/cpu_features.cmake | 8 +++++++- contrib/lz4-cmake/CMakeLists.txt | 2 +- contrib/zstd-cmake/CMakeLists.txt | 2 +- dbms/CMakeLists.txt | 2 +- dbms/src/Functions/CMakeLists.txt | 2 +- libs/libcommon/CMakeLists.txt | 2 +- libs/libmemcpy/CMakeLists.txt | 2 +- 7 files changed, 13 insertions(+), 7 deletions(-) diff --git a/cmake/cpu_features.cmake b/cmake/cpu_features.cmake index 3f6aad2ed05..5eb56288ebe 100644 --- a/cmake/cpu_features.cmake +++ b/cmake/cpu_features.cmake @@ -95,11 +95,15 @@ elseif (ARCH_AMD64) # so we do not set the flags to avoid core dump in old machines option (TIFLASH_ENABLE_AVX_SUPPORT "Use AVX/AVX2 instructions on x86_64" ON) option (TIFLASH_ENABLE_AVX512_SUPPORT "Use AVX512 instructions on x86_64" ON) + + # `haswell` was released since 2013 with cpu feature avx2, bmi2. It's a practical arch for optimizer + option (TIFLASH_ENABLE_ARCH_HASWELL_SUPPORT "Use instructions based on architecture `haswell` on x86_64" ON) option (NO_SSE42_OR_HIGHER "Disable SSE42 or higher on x86_64 for maximum compatibility with older/embedded hardware." OFF) if (NO_SSE42_OR_HIGHER) SET(TIFLASH_ENABLE_AVX_SUPPORT OFF) SET(TIFLASH_ENABLE_AVX512_SUPPORT OFF) + SET (TIFLASH_ENABLE_ARCH_HASWELL_SUPPORT OFF) endif() set (TEST_FLAG "-mssse3") @@ -214,9 +218,11 @@ elseif (ARCH_AMD64) set (TIFLASH_COMPILER_BMI2_FLAG "-mbmi2") check_cxx_compiler_flag("${TIFLASH_COMPILER_BMI2_FLAG}" TIFLASH_COMPILER_BMI2_SUPPORT) - # `haswell` was released since 2013 with cpu feature avx2, bmi2. It's a practical arch for optimizer set (TIFLASH_COMPILER_ARCH_HASWELL_FLAG "-march=haswell") check_cxx_compiler_flag("${TIFLASH_COMPILER_ARCH_HASWELL_FLAG}" TIFLASH_COMPILER_ARCH_HASWELL_SUPPORT) + if (NOT TIFLASH_COMPILER_ARCH_HASWELL_SUPPORT) + set (TIFLASH_ENABLE_ARCH_HASWELL_SUPPORT OFF) + endif () else () # ignore all other platforms endif () diff --git a/contrib/lz4-cmake/CMakeLists.txt b/contrib/lz4-cmake/CMakeLists.txt index eb757e79668..d46fe8c5905 100644 --- a/contrib/lz4-cmake/CMakeLists.txt +++ b/contrib/lz4-cmake/CMakeLists.txt @@ -11,7 +11,7 @@ target_compile_definitions(lz4 PUBLIC LZ4_DISABLE_DEPRECATE_WARNINGS=1) target_include_directories(lz4 PUBLIC ${LIBRARY_DIR}) -if (TIFLASH_COMPILER_ARCH_HASWELL_SUPPORT) +if (TIFLASH_ENABLE_ARCH_HASWELL_SUPPORT) set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${TIFLASH_COMPILER_ARCH_HASWELL_FLAG}") set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${TIFLASH_COMPILER_ARCH_HASWELL_FLAG}") endif () diff --git a/contrib/zstd-cmake/CMakeLists.txt b/contrib/zstd-cmake/CMakeLists.txt index 83b4e5693f6..7ab9edd8469 100644 --- a/contrib/zstd-cmake/CMakeLists.txt +++ b/contrib/zstd-cmake/CMakeLists.txt @@ -156,7 +156,7 @@ ADD_LIBRARY(zstd ${Sources} ${Headers}) target_include_directories (zstd PUBLIC ${LIBRARY_DIR}) -if (TIFLASH_COMPILER_ARCH_HASWELL_SUPPORT) +if (TIFLASH_ENABLE_ARCH_HASWELL_SUPPORT) set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${TIFLASH_COMPILER_ARCH_HASWELL_FLAG}") set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${TIFLASH_COMPILER_ARCH_HASWELL_FLAG}") endif () diff --git a/dbms/CMakeLists.txt b/dbms/CMakeLists.txt index c30eb79b09b..a9c82b2ac96 100644 --- a/dbms/CMakeLists.txt +++ b/dbms/CMakeLists.txt @@ -106,7 +106,7 @@ add_headers_only(dbms src/Flash/Coprocessor) add_headers_only(dbms src/Server) check_then_add_sources_compile_flag ( - TIFLASH_COMPILER_ARCH_HASWELL_SUPPORT + TIFLASH_ENABLE_ARCH_HASWELL_SUPPORT "${TIFLASH_COMPILER_ARCH_HASWELL_FLAG}" src/Columns/ColumnString.cpp src/Columns/ColumnsCommon.cpp diff --git a/dbms/src/Functions/CMakeLists.txt b/dbms/src/Functions/CMakeLists.txt index 71769cdd96f..562af2bcf40 100644 --- a/dbms/src/Functions/CMakeLists.txt +++ b/dbms/src/Functions/CMakeLists.txt @@ -20,7 +20,7 @@ add_headers_and_sources(clickhouse_functions ./Conditional) add_headers_and_sources(clickhouse_functions ${TiFlash_BINARY_DIR}/dbms/src/Functions) check_then_add_sources_compile_flag ( - TIFLASH_COMPILER_ARCH_HASWELL_SUPPORT + TIFLASH_ENABLE_ARCH_HASWELL_SUPPORT "${TIFLASH_COMPILER_ARCH_HASWELL_FLAG}" CollationStringOptimized.cpp ) diff --git a/libs/libcommon/CMakeLists.txt b/libs/libcommon/CMakeLists.txt index 7632250b512..a72e228eb2a 100644 --- a/libs/libcommon/CMakeLists.txt +++ b/libs/libcommon/CMakeLists.txt @@ -166,7 +166,7 @@ check_then_add_sources_compile_flag ( ) check_then_add_sources_compile_flag ( - TIFLASH_COMPILER_ARCH_HASWELL_SUPPORT + TIFLASH_ENABLE_ARCH_HASWELL_SUPPORT "${TIFLASH_COMPILER_ARCH_HASWELL_FLAG}" src/mem_utils_avx2.cpp src/crc64_avx2.cpp diff --git a/libs/libmemcpy/CMakeLists.txt b/libs/libmemcpy/CMakeLists.txt index eb98411e358..a9ba22a9ced 100644 --- a/libs/libmemcpy/CMakeLists.txt +++ b/libs/libmemcpy/CMakeLists.txt @@ -40,7 +40,7 @@ else () endif() check_then_add_sources_compile_flag ( - TIFLASH_COMPILER_ARCH_HASWELL_SUPPORT + TIFLASH_ENABLE_ARCH_HASWELL_SUPPORT "${TIFLASH_COMPILER_ARCH_HASWELL_FLAG}" ${memcpy_sources} ) From 1a95d51f74588f91013f38d4d0d94dcf01e793be Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Thu, 12 Jan 2023 12:22:34 +0800 Subject: [PATCH 65/93] 59 Signed-off-by: Zhigao Tong --- dbms/src/Flash/Coprocessor/CHBlockChunkCodec.cpp | 11 +++++++++-- dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.cpp | 9 --------- dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.h | 1 - dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp | 6 +++++- 4 files changed, 14 insertions(+), 13 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/CHBlockChunkCodec.cpp b/dbms/src/Flash/Coprocessor/CHBlockChunkCodec.cpp index a44da75c506..f5cc6ea66cc 100644 --- a/dbms/src/Flash/Coprocessor/CHBlockChunkCodec.cpp +++ b/dbms/src/Flash/Coprocessor/CHBlockChunkCodec.cpp @@ -65,13 +65,15 @@ CHBlockChunkCodec::CHBlockChunkCodec(const DAGSchema & schema) size_t getExtraInfoSize(const Block & block) { - size_t size = 64; /// to hold some length of structures, such as column number, row number... + size_t size = 8 + 8; /// to hold some length of structures, such as column number, row number... size_t columns = block.columns(); for (size_t i = 0; i < columns; ++i) { const ColumnWithTypeAndName & column = block.safeGetByPosition(i); size += column.name.size(); + size += 8; size += column.type->getName().size(); + size += 8; if (column.column->isColumnConst()) { size += column.column->byteSize() * column.column->size(); @@ -106,6 +108,11 @@ void CHBlockChunkCodec::readData(const IDataType & type, IColumn & column, ReadB type.deserializeBinaryBulkWithMultipleStreams(column, input_stream_getter, rows, 0, false, {}); } +size_t ApproxBlockBytes(const Block & block) +{ + return block.bytes() + getExtraInfoSize(block); +} + void CHBlockChunkCodecStream::encode(const Block & block, size_t start, size_t end) { /// only check block schema in CHBlock codec because for both @@ -117,7 +124,7 @@ void CHBlockChunkCodecStream::encode(const Block & block, size_t start, size_t e throw TiFlashException("CHBlock encode only support encode whole block", Errors::Coprocessor::Internal); assert(output == nullptr); - output = std::make_unique(block.bytes() + getExtraInfoSize(block)); + output = std::make_unique(ApproxBlockBytes(block)); block.checkNumberOfRows(); size_t columns = block.columns(); diff --git a/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.cpp b/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.cpp index 463bb7def46..01ca1a44a23 100644 --- a/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.cpp +++ b/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.cpp @@ -36,19 +36,10 @@ size_t ApproxBlockHeaderBytes(const Block & block) size += 8; size += column.type->getName().size(); size += 8; - if (column.column->isColumnConst()) - { - size += column.column->byteSize() * column.column->size(); - } } return size; } -size_t ApproxBlockBytes(const Block & block) -{ - return block.bytes() + ApproxBlockHeaderBytes(block); -} - void EncodeHeader(WriteBuffer & ostr, const Block & header, size_t rows) { size_t columns = header.columns(); diff --git a/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.h b/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.h index 66cf32141a1..f3759417b77 100644 --- a/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.h +++ b/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.h @@ -21,7 +21,6 @@ namespace DB { -size_t ApproxBlockBytes(const Block & block); size_t ApproxBlockHeaderBytes(const Block & block); using CompressedCHBlockChunkReadBuffer = CompressedReadBuffer; using CompressedCHBlockChunkWriteBuffer = CompressedWriteBuffer; diff --git a/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp b/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp index d0d972d3ff8..79a86ebd243 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp +++ b/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp @@ -21,6 +21,10 @@ #include #include +namespace DB +{ +extern size_t ApproxBlockBytes(const Block & block); +} namespace DB { @@ -155,7 +159,7 @@ void HashPartitionWriterV1::partitionAndEncodeThenWriteBlocks }); // compression method flag; NONE = 0x02, LZ4 = 0x82, ZSTD = 0x90 - // ... + // ... // header meta: // columns count; // total row count (multi parts); From 5970ea6d9a4e47ed68ca3fe00d315c12044e0d69 Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Thu, 12 Jan 2023 12:33:09 +0800 Subject: [PATCH 66/93] 60 Signed-off-by: Zhigao Tong --- dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.cpp | 10 ++++------ dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.cpp | 1 + dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp | 2 +- 3 files changed, 6 insertions(+), 7 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.cpp b/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.cpp index 01ca1a44a23..f5f042fe125 100644 --- a/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.cpp +++ b/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.cpp @@ -17,9 +17,6 @@ #include #include -#include -#include -#include namespace DB { @@ -94,7 +91,7 @@ Block DecodeHeader(ReadBuffer & istr, const Block & header, size_t & total_rows) return res; } -[[maybe_unused]] static inline void DecodeColumns_by_block(ReadBuffer & istr, Block & res, size_t rows_to_read, size_t reserve_size) +static inline void decodeColumnsByBlock(ReadBuffer & istr, Block & res, size_t rows_to_read, size_t reserve_size) { if (!rows_to_read) return; @@ -137,7 +134,8 @@ Block DecodeHeader(ReadBuffer & istr, const Block & header, size_t & total_rows) res.setColumns(std::move(mutable_columns)); } -[[maybe_unused]] static inline void DecodeColumns_by_col(ReadBuffer & istr, Block & res, size_t rows_to_read, size_t reserve_size) +// Deprecated +[[maybe_unused]] static inline void decodeColumnsByCol(ReadBuffer & istr, Block & res, size_t rows_to_read, size_t reserve_size) { if (!rows_to_read) return; @@ -187,7 +185,7 @@ Block DecodeHeader(ReadBuffer & istr, const Block & header, size_t & total_rows) void DecodeColumns(ReadBuffer & istr, Block & res, size_t rows_to_read, size_t reserve_size) { - return DecodeColumns_by_block(istr, res, rows_to_read, reserve_size); + return decodeColumnsByBlock(istr, res, rows_to_read, reserve_size); } } // namespace DB \ No newline at end of file diff --git a/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.cpp b/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.cpp index 5e1c514e0ac..df9325eef0d 100644 --- a/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.cpp +++ b/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.cpp @@ -28,6 +28,7 @@ CHBlockChunkDecodeAndSquash::CHBlockChunkDecodeAndSquash( std::optional CHBlockChunkDecodeAndSquash::decodeAndSquashWithCompression(std::string_view sv) { + // read first byte of compression method flag which defined in `CompressionMethodByte` if (static_cast(sv[0]) == CompressionMethodByte::NONE) { ReadBufferFromString istr(sv.substr(1, sv.size() - 1)); diff --git a/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp b/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp index 79a86ebd243..a5915df936f 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp +++ b/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp @@ -158,7 +158,7 @@ void HashPartitionWriterV1::partitionAndEncodeThenWriteBlocks return res + 8 /*partition rows*/; }); - // compression method flag; NONE = 0x02, LZ4 = 0x82, ZSTD = 0x90 + // compression method flag; NONE, LZ4, ZSTD, defined in `CompressionMethodByte` // ... // header meta: // columns count; From 68beb5fb519082f653badb506a1070fa14046a7e Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Thu, 12 Jan 2023 12:47:28 +0800 Subject: [PATCH 67/93] 61 Signed-off-by: Zhigao Tong --- dbms/src/Flash/FlashService.cpp | 2 +- dbms/src/Flash/Mpp/ReceiverChannelWriter.cpp | 2 +- tests/fullstack-test2/mpp/mpp-version.test | 3 ++- 3 files changed, 4 insertions(+), 3 deletions(-) diff --git a/dbms/src/Flash/FlashService.cpp b/dbms/src/Flash/FlashService.cpp index 98547d1351d..82671392ddc 100644 --- a/dbms/src/Flash/FlashService.cpp +++ b/dbms/src/Flash/FlashService.cpp @@ -234,7 +234,7 @@ grpc::Status FlashService::DispatchMPPTask( { auto && err_msg = fmt::format("Failed to handling mpp dispatch request, reason=`{}`", DB::GenMppVersionErrorMessage(mpp_version)); LOG_WARNING(log, err_msg); - return grpc::Status(grpc::StatusCode::INTERNAL, std::move(err_msg)); + return grpc::Status(grpc::StatusCode::CANCELLED, std::move(err_msg)); } GET_METRIC(tiflash_coprocessor_request_count, type_dispatch_mpp_task).Increment(); diff --git a/dbms/src/Flash/Mpp/ReceiverChannelWriter.cpp b/dbms/src/Flash/Mpp/ReceiverChannelWriter.cpp index 4d86fc339fe..0982fc3b461 100644 --- a/dbms/src/Flash/Mpp/ReceiverChannelWriter.cpp +++ b/dbms/src/Flash/Mpp/ReceiverChannelWriter.cpp @@ -18,7 +18,7 @@ namespace DB { namespace { -inline void injectFailPointReceiverPushFail(bool & push_succeed, ReceiverMode mode) +inline void injectFailPointReceiverPushFail([[maybe_unused]] bool & push_succeed, ReceiverMode mode) { switch (mode) { diff --git a/tests/fullstack-test2/mpp/mpp-version.test b/tests/fullstack-test2/mpp/mpp-version.test index 19711e9cdde..0f63d2a81f1 100644 --- a/tests/fullstack-test2/mpp/mpp-version.test +++ b/tests/fullstack-test2/mpp/mpp-version.test @@ -34,7 +34,8 @@ mysql> use test; set tidb_enforce_mpp=on; set tidb_isolation_read_engines='tifla mysql> use test; set tidb_enforce_mpp=on; set tidb_isolation_read_engines='tiflash'; select count(1) as c from t group by b; -ERROR 1105 (HY000): rpc error: code = Internal desc = Failed to handling mpp dispatch request, reason=`Invalid mpp version -1, TiFlash expects version: min 0, max 1, should upgrade TiDB/planner` +ERROR 1105 (HY000): rpc error: code = Canceled desc = Failed to handling mpp dispatch request, reason=`Invalid mpp version -1, TiFlash expects version: min 0, max 1, should upgrade TiDB/planner` + => DBGInvoke __disable_fail_point(invalid_mpp_version) mysql> use test; set tidb_enforce_mpp=on; set tidb_isolation_read_engines='tiflash'; select count(1) as c from t group by b; From dbdb0cecfa02aab5801557891dcc6391c34ffa76 Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Thu, 12 Jan 2023 14:18:14 +0800 Subject: [PATCH 68/93] update tipb Signed-off-by: Zhigao Tong --- contrib/tipb | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/tipb b/contrib/tipb index 566ee9ba807..110fba31903 160000 --- a/contrib/tipb +++ b/contrib/tipb @@ -1 +1 @@ -Subproject commit 566ee9ba807d80382a2cdf2b0c0ef267ef5b85e8 +Subproject commit 110fba3190357c4e31b6e47799058d3b686bd13c From 0dc605f2b553cae2e7b9e5f572509b9ccd3ea875 Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Thu, 12 Jan 2023 15:16:30 +0800 Subject: [PATCH 69/93] fix gtests Signed-off-by: Zhigao Tong --- dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp | 2 -- dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp | 2 +- tests/fullstack-test2/mpp/mpp-version.test | 2 +- 3 files changed, 2 insertions(+), 4 deletions(-) diff --git a/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp b/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp index a5915df936f..4b30bc63ba2 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp +++ b/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp @@ -59,8 +59,6 @@ HashPartitionWriterV1::HashPartitionWriterV1( , collators(std::move(collators_)) , compression_method(ToInternalCompressionMethod(compression_mode_)) { - assert(dag_context.getMPPTaskMeta().mpp_version() > 0); - if (partition_batch_limit < 0) { partition_batch_limit = 8192LL * partition_num; diff --git a/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp b/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp index 2b2bad17b77..c624e09a35f 100644 --- a/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp +++ b/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp @@ -388,7 +388,7 @@ static CompressionMethodByte ToCompressionMethodByte(CompressionMethod m) return CompressionMethodByte::NONE; } -TEST_F(TestMPPExchangeWriter, testHashPartitionWriterV1) +TEST_F(TestMPPExchangeWriter, TestHashPartitionWriterV1) try { const size_t block_rows = 64; diff --git a/tests/fullstack-test2/mpp/mpp-version.test b/tests/fullstack-test2/mpp/mpp-version.test index 0f63d2a81f1..bb11ab9b291 100644 --- a/tests/fullstack-test2/mpp/mpp-version.test +++ b/tests/fullstack-test2/mpp/mpp-version.test @@ -34,7 +34,7 @@ mysql> use test; set tidb_enforce_mpp=on; set tidb_isolation_read_engines='tifla mysql> use test; set tidb_enforce_mpp=on; set tidb_isolation_read_engines='tiflash'; select count(1) as c from t group by b; -ERROR 1105 (HY000): rpc error: code = Canceled desc = Failed to handling mpp dispatch request, reason=`Invalid mpp version -1, TiFlash expects version: min 0, max 1, should upgrade TiDB/planner` +ERROR 1105 (HY000) at line 1: rpc error: code = Canceled desc = Failed to handling mpp dispatch request, reason=`Invalid mpp version -1, TiFlash expects version: min 0, max 1, should upgrade TiDB/planner` => DBGInvoke __disable_fail_point(invalid_mpp_version) From 7c3a70d5ff663ad252a5d484908b8e7341831239 Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Thu, 12 Jan 2023 16:52:00 +0800 Subject: [PATCH 70/93] 62 Signed-off-by: Zhigao Tong --- dbms/src/Flash/FlashService.cpp | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/dbms/src/Flash/FlashService.cpp b/dbms/src/Flash/FlashService.cpp index 82671392ddc..7f9443b7a7f 100644 --- a/dbms/src/Flash/FlashService.cpp +++ b/dbms/src/Flash/FlashService.cpp @@ -292,19 +292,16 @@ static grpc::Status CheckMppVersionForEstablishMPPConnection(const mpp::Establis if (!DB::CheckMppVersion(sender_mpp_version)) { - err_reason += fmt::format("sender failed: {};", DB::GenMppVersionErrorMessage(sender_mpp_version)); + err_reason += fmt::format("sender failed: {}; ", DB::GenMppVersionErrorMessage(sender_mpp_version)); } if (!DB::CheckMppVersion(receiver_mpp_version)) { - err_reason += fmt::format("receiver failed: {};", DB::GenMppVersionErrorMessage(receiver_mpp_version)); + err_reason += fmt::format("receiver failed: {}; ", DB::GenMppVersionErrorMessage(receiver_mpp_version)); } if (!err_reason.empty()) { - auto && err_msg = fmt::format("Failed to establish MPP connection, sender-meta=`{}`, receiver-meta=`{}`, reason=`{}`", - request->sender_meta().DebugString(), - request->receiver_meta().DebugString(), - err_reason); + auto && err_msg = fmt::format("Failed to establish MPP connection, reason=`{}`", err_reason); return grpc::Status(grpc::StatusCode::INTERNAL, std::move(err_msg)); } return grpc::Status::OK; From de28cc9156251b68ac884e1468e88ffebc98d4a4 Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Fri, 13 Jan 2023 11:57:16 +0800 Subject: [PATCH 71/93] 62 asan Signed-off-by: Zhigao Tong --- dbms/src/Columns/ColumnsCommon.cpp | 9 ++------ .../include/common/avx2_byte_count.h | 8 +++++-- .../libcommon/include/common/avx2_mem_utils.h | 4 ++++ libs/libcommon/include/common/avx2_strstr.h | 23 +++++++++++-------- .../src/tests/gtest_mem_utils_opt.cpp | 1 - 5 files changed, 25 insertions(+), 20 deletions(-) diff --git a/dbms/src/Columns/ColumnsCommon.cpp b/dbms/src/Columns/ColumnsCommon.cpp index 4fb521b510f..e2d2a130313 100644 --- a/dbms/src/Columns/ColumnsCommon.cpp +++ b/dbms/src/Columns/ColumnsCommon.cpp @@ -44,13 +44,8 @@ inline UInt64 ToBits64(const Int8 * bytes64) } #endif -#if defined(ADDRESS_SANITIZER) || defined(THREAD_SANITIZER) -NO_INLINE NO_SANITIZE_ADDRESS NO_SANITIZE_THREAD -#else -ALWAYS_INLINE inline -#endif - static size_t - CountBytesInFilter(const UInt8 * filt, size_t start, size_t end) +ALWAYS_INLINE inline static size_t +CountBytesInFilter(const UInt8 * filt, size_t start, size_t end) { #if defined(__AVX2__) size_t size = end - start; diff --git a/libs/libcommon/include/common/avx2_byte_count.h b/libs/libcommon/include/common/avx2_byte_count.h index 71c0541b7bb..1c59cc8a9a8 100644 --- a/libs/libcommon/include/common/avx2_byte_count.h +++ b/libs/libcommon/include/common/avx2_byte_count.h @@ -18,8 +18,12 @@ namespace mem_utils::details { - -ALWAYS_INLINE static inline uint64_t avx2_byte_count(const char * src, size_t size, char target) +#if defined(MEM_UTILS_FUNC_NO_SANITIZE) +MEM_UTILS_FUNC_NO_SANITIZE +#else +ALWAYS_INLINE static inline +#endif +uint64_t avx2_byte_count(const char * src, size_t size, char target) { uint64_t zero_bytes_cnt = 0; const auto check_block32 = _mm256_set1_epi8(target); diff --git a/libs/libcommon/include/common/avx2_mem_utils.h b/libs/libcommon/include/common/avx2_mem_utils.h index e2b1ac81b0d..e56087c893c 100644 --- a/libs/libcommon/include/common/avx2_mem_utils.h +++ b/libs/libcommon/include/common/avx2_mem_utils.h @@ -24,6 +24,10 @@ #include #include +#if defined(ADDRESS_SANITIZER) || defined(THREAD_SANITIZER) +#define MEM_UTILS_FUNC_NO_SANITIZE [[maybe_unused]] static NO_INLINE NO_SANITIZE_ADDRESS NO_SANITIZE_THREAD +#endif + namespace mem_utils::details { diff --git a/libs/libcommon/include/common/avx2_strstr.h b/libs/libcommon/include/common/avx2_strstr.h index d9722dffbb1..706719ab2f8 100644 --- a/libs/libcommon/include/common/avx2_strstr.h +++ b/libs/libcommon/include/common/avx2_strstr.h @@ -235,12 +235,13 @@ ALWAYS_INLINE static inline const char * avx2_strstr_impl(const char * src, size #undef M } -ALWAYS_INLINE static inline size_t avx2_strstr(const char * src, size_t n, const char * needle, size_t k) -{ -#if defined(ADDRESS_SANITIZER) - return std::string_view{src, n}.find({needle, k}); // memchr@plt -> bcmp@plt +#if defined(MEM_UTILS_FUNC_NO_SANITIZE) +MEM_UTILS_FUNC_NO_SANITIZE +#else +ALWAYS_INLINE static inline #endif - +size_t avx2_strstr(const char * src, size_t n, const char * needle, size_t k) +{ const auto * p = avx2_strstr_impl(src, n, needle, k); return p ? p - src : std::string_view::npos; } @@ -248,12 +249,14 @@ ALWAYS_INLINE static inline size_t avx2_strstr(std::string_view src, std::string { return avx2_strstr(src.data(), src.size(), needle.data(), needle.size()); } -ALWAYS_INLINE static inline const char * avx2_memchr(const char * src, size_t n, char target) -{ -#if defined(ADDRESS_SANITIZER) - return static_cast(std::memchr(src, target, n)); // memchr@plt -#endif +#if defined(MEM_UTILS_FUNC_NO_SANITIZE) +MEM_UTILS_FUNC_NO_SANITIZE +#else +ALWAYS_INLINE static inline +#endif +const char * avx2_memchr(const char * src, size_t n, char target) +{ if (unlikely(n < 1)) { return nullptr; diff --git a/libs/libcommon/src/tests/gtest_mem_utils_opt.cpp b/libs/libcommon/src/tests/gtest_mem_utils_opt.cpp index 689a6e94004..6fd9fd26d1f 100644 --- a/libs/libcommon/src/tests/gtest_mem_utils_opt.cpp +++ b/libs/libcommon/src/tests/gtest_mem_utils_opt.cpp @@ -235,7 +235,6 @@ void TestMemByteCount(size_t size) char target = 8; std::string oa(size + 100, target); char * start = oa.data(); - start = reinterpret_cast(ALIGNED_ADDR(size_t(start), 32)) + 32; for (auto * pos = start; pos < start + 32; ++pos) { ASSERT_EQ(mem_utils::avx2_byte_count(pos, size, target), size); From 1a8cc8f7773599c54b2cfe06334a6187140a0826 Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Fri, 13 Jan 2023 16:34:04 +0800 Subject: [PATCH 72/93] support data compression in FineGrainedShuffle --- .../Flash/Coprocessor/CHBlockChunkCodecV1.cpp | 15 ++ .../Flash/Coprocessor/CHBlockChunkCodecV1.h | 1 + dbms/src/Flash/Mpp/ExchangeReceiver.cpp | 12 +- .../Flash/Mpp/FineGrainedShuffleWriterV1.cpp | 228 ++++++++++++++++++ .../Flash/Mpp/FineGrainedShuffleWriterV1.h | 69 ++++++ dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp | 23 +- dbms/src/Flash/Mpp/newMPPExchangeWriter.h | 28 ++- .../Mpp/tests/gtest_mpp_exchange_writer.cpp | 94 +++++++- 8 files changed, 429 insertions(+), 41 deletions(-) create mode 100644 dbms/src/Flash/Mpp/FineGrainedShuffleWriterV1.cpp create mode 100644 dbms/src/Flash/Mpp/FineGrainedShuffleWriterV1.h diff --git a/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.cpp b/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.cpp index f5f042fe125..adf26c6f86a 100644 --- a/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.cpp +++ b/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.cpp @@ -188,4 +188,19 @@ void DecodeColumns(ReadBuffer & istr, Block & res, size_t rows_to_read, size_t r return decodeColumnsByBlock(istr, res, rows_to_read, reserve_size); } +CompressionMethod ToInternalCompressionMethod(tipb::CompressionMode compression_mode) +{ + switch (compression_mode) + { + case tipb::CompressionMode::NONE: + return CompressionMethod::NONE; + case tipb::CompressionMode::FAST: + return CompressionMethod::LZ4; // use LZ4 method as fast mode + case tipb::CompressionMode::HIGH_COMPRESSION: + return CompressionMethod::ZSTD; // use ZSTD method as HC mode + default: + return CompressionMethod::NONE; + } +} + } // namespace DB \ No newline at end of file diff --git a/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.h b/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.h index f3759417b77..12156bd8a35 100644 --- a/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.h +++ b/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.h @@ -27,5 +27,6 @@ using CompressedCHBlockChunkWriteBuffer = CompressedWriteBuffer; void EncodeHeader(WriteBuffer & ostr, const Block & header, size_t rows); void DecodeColumns(ReadBuffer & istr, Block & res, size_t rows_to_read, size_t reserve_size = 0); Block DecodeHeader(ReadBuffer & istr, const Block & header, size_t & rows); +CompressionMethod ToInternalCompressionMethod(tipb::CompressionMode compression_mode); } // namespace DB diff --git a/dbms/src/Flash/Mpp/ExchangeReceiver.cpp b/dbms/src/Flash/Mpp/ExchangeReceiver.cpp index be4710bd8f1..53410abc9ba 100644 --- a/dbms/src/Flash/Mpp/ExchangeReceiver.cpp +++ b/dbms/src/Flash/Mpp/ExchangeReceiver.cpp @@ -599,7 +599,7 @@ DecodeDetail ExchangeReceiverBase::decodeChunks( { case DB::MPPDataPacketV0: { - for (const String * chunk : recv_msg->chunks) + for (const auto * chunk : recv_msg->chunks) { auto result = decoder_ptr->decodeAndSquash(*chunk); if (!result) @@ -614,14 +614,10 @@ DecodeDetail ExchangeReceiverBase::decodeChunks( } case DB::MPPDataPacketV1: { - RUNTIME_CHECK(packet.chunks().size() == int(recv_msg->chunks.size()), - packet.chunks().size(), - recv_msg->chunks.size()); - - for (auto && chunk : packet.chunks()) + for (const auto * chunk : recv_msg->chunks) { - assert(!chunk.empty()); - auto && result = decoder_ptr->decodeAndSquashWithCompression(chunk); + assert(!chunk->empty()); + auto && result = decoder_ptr->decodeAndSquashWithCompression(*chunk); if (!result || !result->rows()) continue; detail.rows += result->rows(); diff --git a/dbms/src/Flash/Mpp/FineGrainedShuffleWriterV1.cpp b/dbms/src/Flash/Mpp/FineGrainedShuffleWriterV1.cpp new file mode 100644 index 00000000000..e34c2a7894e --- /dev/null +++ b/dbms/src/Flash/Mpp/FineGrainedShuffleWriterV1.cpp @@ -0,0 +1,228 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ +extern size_t ApproxBlockBytes(const Block & block); +extern void WriteColumnData(const IDataType & type, const ColumnPtr & column, WriteBuffer & ostr, size_t offset, size_t limit); +} // namespace DB + +namespace DB +{ +template +FineGrainedShuffleWriterV1::FineGrainedShuffleWriterV1( + ExchangeWriterPtr writer_, + std::vector partition_col_ids_, + TiDB::TiDBCollators collators_, + DAGContext & dag_context_, + uint64_t fine_grained_shuffle_stream_count_, + UInt64 fine_grained_shuffle_batch_size_, + tipb::CompressionMode compression_mode_) + : DAGResponseWriter(/*records_per_chunk=*/-1, dag_context_) + , writer(writer_) + , partition_col_ids(std::move(partition_col_ids_)) + , collators(std::move(collators_)) + , fine_grained_shuffle_stream_count(fine_grained_shuffle_stream_count_) + , fine_grained_shuffle_batch_size(fine_grained_shuffle_batch_size_) + , batch_send_row_limit(fine_grained_shuffle_batch_size * fine_grained_shuffle_stream_count) + , hash(0) + , compression_method(ToInternalCompressionMethod(compression_mode_)) + +{ + rows_in_blocks = 0; + partition_num = writer_->getPartitionNum(); + RUNTIME_CHECK(partition_num > 0); + RUNTIME_CHECK(dag_context.encode_type == tipb::EncodeType::TypeCHBlock); +} + +template +void FineGrainedShuffleWriterV1::prepare(const Block & sample_block) +{ + /// Initialize header block, use column type to create new empty column to handle potential null column cases + const auto & column_with_type_and_names = sample_block.getColumnsWithTypeAndName(); + for (const auto & column : column_with_type_and_names) + { + MutableColumnPtr empty_column = column.type->createColumn(); + ColumnWithTypeAndName new_column(std::move(empty_column), column.type, column.name); + header.insert(new_column); + } + num_columns = header.columns(); + // fine_grained_shuffle_stream_count is in (0, 1024], and partition_num is uint16_t, so will not overflow. + num_bucket = partition_num * fine_grained_shuffle_stream_count; + partition_key_containers_for_reuse.resize(collators.size()); + initScatterColumns(); + prepared = true; +} + +template +void FineGrainedShuffleWriterV1::flush() +{ + if (rows_in_blocks > 0) + batchWriteFineGrainedShuffle(); +} + +template +void FineGrainedShuffleWriterV1::write(const Block & block) +{ + RUNTIME_CHECK_MSG(prepared, "FineGrainedShuffleWriterV1 should be prepared before writing."); + RUNTIME_CHECK_MSG( + block.columns() == dag_context.result_field_types.size(), + "Output column size mismatch with field type size"); + + size_t rows = block.rows(); + if (rows > 0) + { + rows_in_blocks += rows; + blocks.push_back(block); + } + + if (blocks.size() == fine_grained_shuffle_stream_count || static_cast(rows_in_blocks) >= batch_send_row_limit) + batchWriteFineGrainedShuffle(); +} + +template +void FineGrainedShuffleWriterV1::initScatterColumns() +{ + scattered.resize(num_columns); + for (size_t col_id = 0; col_id < num_columns; ++col_id) + { + auto & column = header.getByPosition(col_id).column; + + scattered[col_id].reserve(num_bucket); + for (size_t chunk_id = 0; chunk_id < num_bucket; ++chunk_id) + { + scattered[col_id].emplace_back(column->cloneEmpty()); + scattered[col_id][chunk_id]->reserve(1024); + } + } +} + +template +void FineGrainedShuffleWriterV1::batchWriteFineGrainedShuffle() +{ + size_t ori_block_mem_size = 0; + auto tracked_packets = HashBaseWriterHelper::createPackets(partition_num, DB::MPPDataPacketV1); + if (likely(!blocks.empty())) + { + assert(rows_in_blocks > 0); + assert(fine_grained_shuffle_stream_count <= 1024); + + size_t header_size = ApproxBlockHeaderBytes(header); + HashBaseWriterHelper::materializeBlocks(blocks); + while (!blocks.empty()) + { + const auto & block = blocks.back(); + ori_block_mem_size += ApproxBlockBytes(block); + HashBaseWriterHelper::scatterColumnsForFineGrainedShuffle(block, partition_col_ids, collators, partition_key_containers_for_reuse, partition_num, fine_grained_shuffle_stream_count, hash, selector, scattered); + blocks.pop_back(); + } + + // serialize each partitioned block and write it to its destination + size_t part_id = 0; + for (size_t bucket_idx = 0; bucket_idx < num_bucket; bucket_idx += fine_grained_shuffle_stream_count, ++part_id) + { + for (uint64_t stream_idx = 0; stream_idx < fine_grained_shuffle_stream_count; ++stream_idx) + { + // assemble scatter columns into a block + MutableColumns columns; + columns.reserve(num_columns); + for (size_t col_id = 0; col_id < num_columns; ++col_id) + columns.emplace_back(std::move(scattered[col_id][bucket_idx + stream_idx])); + + size_t part_column_bytes = std::accumulate(columns.begin(), columns.end(), 0, [](auto res, const auto & column) { + return res += column->byteSize(); + }) + + 8 /*partition rows*/; + + { + size_t init_size = part_column_bytes + header_size + 1 /*compression method*/; + + // Reserve enough memory buffer size + auto output_buffer = std::make_unique(init_size); + std::unique_ptr compress_codec{}; + WriteBuffer * ostr_ptr = output_buffer.get(); + + // Init compression writer + if (!writer->isLocal(part_id) && compression_method != CompressionMethod::NONE) + { + // CompressedWriteBuffer will encode compression method flag as first byte + compress_codec = std::make_unique( + *output_buffer, + CompressionSettings(compression_method), + init_size); + ostr_ptr = compress_codec.get(); + } + else + { + // Write compression method flag + output_buffer->write(static_cast(CompressionMethodByte::NONE)); + } + + size_t rows = columns.front()->size(); + // Encode header + EncodeHeader(*ostr_ptr, header, rows); + if (rows) + { + // Encode row count for next columns + writeVarUInt(rows, *ostr_ptr); + + // Encode columns data + for (size_t col_id = 0; col_id < num_columns; ++col_id) + { + auto && col_type_name = header.getByPosition(col_id); + WriteColumnData(*col_type_name.type, columns[col_id]->getPtr(), *ostr_ptr, 0, 0); + } + } + + // Flush rest buffer + if (compress_codec) + compress_codec->next(); + tracked_packets[part_id]->getPacket().add_chunks(output_buffer->releaseStr()); + tracked_packets[part_id]->getPacket().add_stream_ids(stream_idx); + } + + for (size_t col_id = 0; col_id < num_columns; ++col_id) + { + columns[col_id]->popBack(columns[col_id]->size()); // clear column + scattered[col_id][bucket_idx + stream_idx] = std::move(columns[col_id]); + } + } + } + rows_in_blocks = 0; + } + + writePackets(tracked_packets); + GET_METRIC(tiflash_exchange_data_bytes, type_hash_original_all).Increment(ori_block_mem_size); +} + +template +extern void WritePackets(TrackedMppDataPacketPtrs & packets, ExchangeWriterPtr & writer); + +template +void FineGrainedShuffleWriterV1::writePackets(TrackedMppDataPacketPtrs & packets) +{ + WritePackets(packets, writer); +} + +template class FineGrainedShuffleWriterV1; + +} // namespace DB diff --git a/dbms/src/Flash/Mpp/FineGrainedShuffleWriterV1.h b/dbms/src/Flash/Mpp/FineGrainedShuffleWriterV1.h new file mode 100644 index 00000000000..47601ad57d0 --- /dev/null +++ b/dbms/src/Flash/Mpp/FineGrainedShuffleWriterV1.h @@ -0,0 +1,69 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#pragma once + +#include +#include +#include +#include + +namespace DB +{ +class DAGContext; +enum class CompressionMethod; +template +class FineGrainedShuffleWriterV1 : public DAGResponseWriter +{ +public: + FineGrainedShuffleWriterV1( + ExchangeWriterPtr writer_, + std::vector partition_col_ids_, + TiDB::TiDBCollators collators_, + DAGContext & dag_context_, + UInt64 fine_grained_shuffle_stream_count_, + UInt64 fine_grained_shuffle_batch_size, + tipb::CompressionMode compression_mode_); + void prepare(const Block & sample_block) override; + void write(const Block & block) override; + void flush() override; + +private: + void batchWriteFineGrainedShuffle(); + + void writePackets(TrackedMppDataPacketPtrs & packets); + + void initScatterColumns(); + +private: + ExchangeWriterPtr writer; + std::vector blocks; + std::vector partition_col_ids; + TiDB::TiDBCollators collators; + size_t rows_in_blocks = 0; + uint16_t partition_num; + UInt64 fine_grained_shuffle_stream_count; + UInt64 fine_grained_shuffle_batch_size; + + Block header; + bool prepared = false; + size_t num_columns = 0, num_bucket = 0, batch_send_row_limit = 0; // Assign they initial values to pass clang-tidy check, they will be initialized in prepare method + std::vector partition_key_containers_for_reuse; + WeakHash32 hash; + IColumn::Selector selector; + std::vector scattered; // size = num_columns + CompressionMethod compression_method{}; +}; + +} // namespace DB diff --git a/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp b/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp index 4b30bc63ba2..bd65867117a 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp +++ b/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp @@ -14,7 +14,6 @@ #include #include -#include #include #include #include @@ -24,25 +23,11 @@ namespace DB { extern size_t ApproxBlockBytes(const Block & block); -} -namespace DB -{ +extern void WriteColumnData(const IDataType & type, const ColumnPtr & column, WriteBuffer & ostr, size_t offset, size_t limit); +} // namespace DB -static inline CompressionMethod ToInternalCompressionMethod(tipb::CompressionMode compression_mode) +namespace DB { - switch (compression_mode) - { - case tipb::CompressionMode::NONE: - return CompressionMethod::NONE; - case tipb::CompressionMode::FAST: - return CompressionMethod::LZ4; // use LZ4 method as fast mode - case tipb::CompressionMode::HIGH_COMPRESSION: - return CompressionMethod::ZSTD; // use ZSTD method as HC mode - default: - return CompressionMethod::NONE; - } -} - template HashPartitionWriterV1::HashPartitionWriterV1( ExchangeWriterPtr writer_, @@ -93,8 +78,6 @@ void HashPartitionWriterV1::write(const Block & block) partitionAndEncodeThenWriteBlocks(); } -extern void WriteColumnData(const IDataType & type, const ColumnPtr & column, WriteBuffer & ostr, size_t offset, size_t limit); - template void HashPartitionWriterV1::partitionAndEncodeThenWriteBlocks() { diff --git a/dbms/src/Flash/Mpp/newMPPExchangeWriter.h b/dbms/src/Flash/Mpp/newMPPExchangeWriter.h index 80e1bf0e320..841e80cef90 100644 --- a/dbms/src/Flash/Mpp/newMPPExchangeWriter.h +++ b/dbms/src/Flash/Mpp/newMPPExchangeWriter.h @@ -17,6 +17,7 @@ #include #include #include +#include #include #include #include @@ -58,16 +59,23 @@ std::unique_ptr NewMPPExchangeWriter( { if (enable_fine_grained_shuffle) { - // TODO: support data compression if necessary - RUNTIME_CHECK(compression_mode == tipb::CompressionMode::NONE); - - return std::make_unique>( - writer, - partition_col_ids, - partition_col_collators, - dag_context, - fine_grained_shuffle_stream_count, - fine_grained_shuffle_batch_size); + if (DB::MppVersion::MppVersionV0 == dag_context.getMPPTaskMeta().mpp_version()) + return std::make_unique>( + writer, + partition_col_ids, + partition_col_collators, + dag_context, + fine_grained_shuffle_stream_count, + fine_grained_shuffle_batch_size); + else + return std::make_unique>( + writer, + partition_col_ids, + partition_col_collators, + dag_context, + fine_grained_shuffle_stream_count, + fine_grained_shuffle_batch_size, + compression_mode); } else { diff --git a/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp b/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp index c624e09a35f..5ac4bec922f 100644 --- a/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp +++ b/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp @@ -26,15 +26,17 @@ #include #include +#include #include #include -#include -#include +#include namespace DB { namespace tests { +static CompressionMethodByte ToCompressionMethodByte(CompressionMethod m); + class TestMPPExchangeWriter : public testing::Test { protected: @@ -128,7 +130,7 @@ struct MockExchangeWriter void broadcastOrPassThroughWrite(TrackedMppDataPacketPtr && packet) { checker(packet, 0); } void partitionWrite(TrackedMppDataPacketPtr && packet, uint16_t part_id) { checker(packet, part_id); } - void write(tipb::SelectResponse &) { FAIL() << "cannot reach here, only consider CH Block format"; } + static void write(tipb::SelectResponse &) { FAIL() << "cannot reach here, only consider CH Block format"; } void sendExecutionSummary(const tipb::SelectResponse & response) { auto tracked_packet = std::make_shared(); @@ -207,6 +209,92 @@ try } CATCH +TEST_F(TestMPPExchangeWriter, TestFineGrainedShuffleWriterV1) +try +{ + const size_t block_rows = 64; + const size_t block_num = 64; + const uint16_t part_num = 4; + const uint32_t fine_grained_shuffle_stream_count = 8; + const Int64 fine_grained_shuffle_batch_size = 108; + + // 1. Build Block. + std::vector blocks; + for (size_t i = 0; i < block_num; ++i) + { + blocks.emplace_back(prepareUniformBlock(block_rows)); + blocks.emplace_back(prepareUniformBlock(0)); + } + Block header = blocks.back(); + + // 2. Build MockExchangeWriter. + std::unordered_map write_report; + auto checker = [&write_report](const TrackedMppDataPacketPtr & packet, uint16_t part_id) { + write_report[part_id].emplace_back(packet); + }; + auto mock_writer = std::make_shared(checker, part_num); + + // 3. Start to write. + auto dag_writer = std::make_shared>>( + mock_writer, + part_col_ids, + part_col_collators, + *dag_context_ptr, + fine_grained_shuffle_stream_count, + fine_grained_shuffle_batch_size, + tipb::CompressionMode::FAST); + dag_writer->prepare(blocks[0].cloneEmpty()); + for (const auto & block : blocks) + dag_writer->write(block); + dag_writer->flush(); + + // 4. Start to check write_report. + size_t per_part_rows = block_rows * block_num / part_num; + ASSERT_EQ(write_report.size(), part_num); + std::vector rows_of_stream_ids(fine_grained_shuffle_stream_count, 0); + + CHBlockChunkDecodeAndSquash decoder(header, 512); + + for (size_t part_index = 0; part_index < part_num; ++part_index) + { + size_t part_decoded_block_rows = 0; + + for (const auto & packet : write_report[part_index]) + { + ASSERT_EQ(packet->getPacket().chunks_size(), packet->getPacket().stream_ids_size()); + for (int i = 0; i < packet->getPacket().chunks_size(); ++i) + { + const auto & chunk = packet->getPacket().chunks(i); + + if (part_index == 0) + { + ASSERT_EQ(CompressionMethodByte(chunk[0]), CompressionMethodByte::NONE); + } + else + { + ASSERT_EQ(CompressionMethodByte(chunk[0]), ToCompressionMethodByte(ToInternalCompressionMethod(tipb::CompressionMode::FAST))); + } + + auto && result = decoder.decodeAndSquashWithCompression(chunk); + if (!result) + { + result = decoder.flush(); + } + assert(result); + auto decoded_block = std::move(*result); + part_decoded_block_rows += decoded_block.rows(); + rows_of_stream_ids[packet->getPacket().stream_ids(i)] += decoded_block.rows(); + } + } + ASSERT_EQ(part_decoded_block_rows, per_part_rows); + } + + size_t per_stream_id_rows = block_rows * block_num / fine_grained_shuffle_stream_count; + for (size_t rows : rows_of_stream_ids) + ASSERT_EQ(rows, per_stream_id_rows); +} +CATCH + TEST_F(TestMPPExchangeWriter, testFineGrainedShuffleWriter) try { From efcb1be5d6f046923e4dffe38e008a443a3c22c7 Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Fri, 13 Jan 2023 21:05:44 +0800 Subject: [PATCH 73/93] 63 --- dbms/src/Flash/Mpp/FineGrainedShuffleWriterV1.cpp | 4 ++-- dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp | 9 +++++++-- 2 files changed, 9 insertions(+), 4 deletions(-) diff --git a/dbms/src/Flash/Mpp/FineGrainedShuffleWriterV1.cpp b/dbms/src/Flash/Mpp/FineGrainedShuffleWriterV1.cpp index e34c2a7894e..faedf71a7a1 100644 --- a/dbms/src/Flash/Mpp/FineGrainedShuffleWriterV1.cpp +++ b/dbms/src/Flash/Mpp/FineGrainedShuffleWriterV1.cpp @@ -215,12 +215,12 @@ void FineGrainedShuffleWriterV1::batchWriteFineGrainedShuffle } template -extern void WritePackets(TrackedMppDataPacketPtrs & packets, ExchangeWriterPtr & writer); +extern void WritePackets(CompressionMethod compression_method, TrackedMppDataPacketPtrs && packets, ExchangeWriterPtr & writer); template void FineGrainedShuffleWriterV1::writePackets(TrackedMppDataPacketPtrs & packets) { - WritePackets(packets, writer); + WritePackets(compression_method, std::move(packets), writer); } template class FineGrainedShuffleWriterV1; diff --git a/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp b/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp index bd65867117a..e8e9e947142 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp +++ b/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp @@ -250,7 +250,7 @@ static void updateHashPartitionWriterMetrics(CompressionMethod method, size_t sz } template -void HashPartitionWriterV1::writePackets(TrackedMppDataPacketPtrs && packets) +void WritePackets(CompressionMethod compression_method, TrackedMppDataPacketPtrs && packets, ExchangeWriterPtr & writer) { for (size_t part_id = 0; part_id < packets.size(); ++part_id) { @@ -258,7 +258,6 @@ void HashPartitionWriterV1::writePackets(TrackedMppDataPacket assert(packet); auto & inner_packet = packet->getPacket(); - inner_packet.chunks(); if (auto sz = inner_packet.ByteSizeLong(); likely(inner_packet.chunks_size() > 0)) { @@ -268,6 +267,12 @@ void HashPartitionWriterV1::writePackets(TrackedMppDataPacket } } +template +void HashPartitionWriterV1::writePackets(TrackedMppDataPacketPtrs && packets) +{ + WritePackets(compression_method, std::move(packets), writer); +} + template class HashPartitionWriterV1; } // namespace DB From a15977501bb7f8f064e07d9a364120d48b969fb6 Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Sun, 15 Jan 2023 21:04:50 +0800 Subject: [PATCH 74/93] 64 --- dbms/src/Common/MPMCQueue.h | 4 ++-- dbms/src/Debug/ReadIndexStressTest.cpp | 4 ++-- dbms/src/Storages/Transaction/LearnerRead.cpp | 7 +++---- .../Storages/Transaction/tests/gtest_read_index_worker.cpp | 4 ++-- 4 files changed, 9 insertions(+), 10 deletions(-) diff --git a/dbms/src/Common/MPMCQueue.h b/dbms/src/Common/MPMCQueue.h index 709ce336516..1cdf4f0eb29 100644 --- a/dbms/src/Common/MPMCQueue.h +++ b/dbms/src/Common/MPMCQueue.h @@ -73,7 +73,6 @@ class MPMCQueue public: using Status = MPMCQueueStatus; using Result = MPMCQueueResult; - using SteadyClock = std::chrono::steady_clock; explicit MPMCQueue(size_t capacity_) : capacity(capacity_) @@ -220,7 +219,8 @@ class MPMCQueue } private: - using TimePoint = std::chrono::time_point; + using SteadyClock = std::chrono::steady_clock; + using TimePoint = SteadyClock::time_point; using WaitingNode = MPMCQueueDetail::WaitingNode; void notifyAll() diff --git a/dbms/src/Debug/ReadIndexStressTest.cpp b/dbms/src/Debug/ReadIndexStressTest.cpp index a4bd22f8cd5..7ab02671586 100644 --- a/dbms/src/Debug/ReadIndexStressTest.cpp +++ b/dbms/src/Debug/ReadIndexStressTest.cpp @@ -164,9 +164,9 @@ ReadIndexStressTest::TimeCost ReadIndexStressTest::run( const auto & kvstore = *tmt.getKVStore(); size_t timeout_ms = 10 * 1000; const auto wrap_time_cost = [&](std::function && f) { - auto start_time = Clock::now(); + auto start_time = std::chrono::steady_clock::now(); f(); - auto end_time = Clock ::now(); + auto end_time = std::chrono::steady_clock ::now(); auto time_cost = std::chrono::duration_cast(end_time - start_time); LOG_INFO(logger, "time cost {}", time_cost); return time_cost; diff --git a/dbms/src/Storages/Transaction/LearnerRead.cpp b/dbms/src/Storages/Transaction/LearnerRead.cpp index 8a4cdefa04b..c7fba4fabb7 100644 --- a/dbms/src/Storages/Transaction/LearnerRead.cpp +++ b/dbms/src/Storages/Transaction/LearnerRead.cpp @@ -30,9 +30,8 @@ #include #include #include -#include +#include -#include namespace DB { @@ -442,8 +441,8 @@ LearnerReadSnapshot doLearnerRead( auto end_time = Clock::now(); LOG_DEBUG( log, - "[Learner Read] batch read index | wait index cost {} ms totally, regions_num={}, concurrency={}", - std::chrono::duration_cast(end_time - start_time).count(), + "[Learner Read] batch read index | wait index cost {} totally, regions_num={}, concurrency={}", + std::chrono::duration_cast(end_time - start_time), num_regions, concurrent_num); diff --git a/dbms/src/Storages/Transaction/tests/gtest_read_index_worker.cpp b/dbms/src/Storages/Transaction/tests/gtest_read_index_worker.cpp index 48ee974a6e2..5c5ac03dd8e 100644 --- a/dbms/src/Storages/Transaction/tests/gtest_read_index_worker.cpp +++ b/dbms/src/Storages/Transaction/tests/gtest_read_index_worker.cpp @@ -381,9 +381,9 @@ void ReadIndexTest::testNormal() std::vector reqs; reqs = {make_read_index_reqs(5, 12), make_read_index_reqs(1, 12), make_read_index_reqs(2, 12)}; - Timepoint start = Clock::now(); + auto start = std::chrono::steady_clock::now(); auto resps = manager->batchReadIndex(reqs, 20); - auto time_cost = Clock::now() - start; + auto time_cost = std::chrono::steady_clock::now() - start; ASSERT_GE(time_cost, std::chrono::milliseconds{20}); // meet timeout ASSERT_EQ(resps[0].first.read_index(), 669); ASSERT_EQ(resps[1].first.region_error().has_region_not_found(), true); // timeout to region error not found From 77278d84ac77ce790a784f1e72b1fcef6cbb2ee0 Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Mon, 16 Jan 2023 10:56:13 +0800 Subject: [PATCH 75/93] 65 Signed-off-by: Zhigao Tong --- libs/libcommon/include/common/fixed_mem_eq.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/libs/libcommon/include/common/fixed_mem_eq.h b/libs/libcommon/include/common/fixed_mem_eq.h index 85691008c6d..39980660418 100644 --- a/libs/libcommon/include/common/fixed_mem_eq.h +++ b/libs/libcommon/include/common/fixed_mem_eq.h @@ -123,7 +123,7 @@ ALWAYS_INLINE inline bool memcmp_eq_fixed_size(const char * a, const char * b) ret */ - return std::memcmp(a, b, k) == 0; + return __builtin_memcmp(a, b, k) == 0; } else if constexpr (k > 8) { From 18e3ccdbb965453ee94e171133a73c967289fced Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Mon, 16 Jan 2023 16:37:21 +0800 Subject: [PATCH 76/93] 66 Signed-off-by: Zhigao Tong --- .../Flash/Coprocessor/CHBlockChunkCodecV1.cpp | 1 - .../Flash/Coprocessor/CHBlockChunkCodecV1.h | 238 ++++++++++++++++++ .../Coprocessor/ChunkDecodeAndSquash.cpp | 21 +- .../tests/gtest_chunk_decode_and_squash.cpp | 41 ++- .../Flash/Mpp/FineGrainedShuffleWriterV1.cpp | 71 ++---- dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp | 82 +----- .../Mpp/tests/gtest_mpp_exchange_writer.cpp | 115 ++++----- 7 files changed, 368 insertions(+), 201 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.cpp b/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.cpp index adf26c6f86a..e4b6f146aec 100644 --- a/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.cpp +++ b/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.cpp @@ -20,7 +20,6 @@ namespace DB { -extern void WriteColumnData(const IDataType & type, const ColumnPtr & column, WriteBuffer & ostr, size_t offset, size_t limit); size_t ApproxBlockHeaderBytes(const Block & block) { diff --git a/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.h b/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.h index 12156bd8a35..56f0ad21792 100644 --- a/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.h +++ b/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.h @@ -28,5 +28,243 @@ void EncodeHeader(WriteBuffer & ostr, const Block & header, size_t rows); void DecodeColumns(ReadBuffer & istr, Block & res, size_t rows_to_read, size_t reserve_size = 0); Block DecodeHeader(ReadBuffer & istr, const Block & header, size_t & rows); CompressionMethod ToInternalCompressionMethod(tipb::CompressionMode compression_mode); +extern void WriteColumnData(const IDataType & type, const ColumnPtr & column, WriteBuffer & ostr, size_t offset, size_t limit); + +struct CHBlockChunkCodecV1 +{ + using Self = CHBlockChunkCodecV1; + + const Block & header; + size_t header_size; + size_t encoded_rows{}; + + explicit CHBlockChunkCodecV1(const Block & header_) + : header(header_) + , header_size(ApproxBlockHeaderBytes(header)) + { + } + static std::string encode(const Block & block, CompressionMethod compression_method, bool always_keep_header) + { + return Self{block}.encode(compression_method, always_keep_header); + } + void clear() + { + encoded_rows = 0; + } + + std::string encode(CompressionMethod compression_method, bool always_keep_header) + { + return encodeImpl(header, compression_method, always_keep_header); + } + + std::string encode(const MutableColumns & columns, CompressionMethod compression_method, bool always_keep_header) + { + return encodeImpl(columns, compression_method, always_keep_header); + } + std::string encode(const Columns & columns, CompressionMethod compression_method, bool always_keep_header) + { + return encodeImpl(columns, compression_method, always_keep_header); + } + std::string encode(const std::vector & columns, CompressionMethod compression_method, bool always_keep_header) + { + return encodeImpl(columns, compression_method, always_keep_header); + } + std::string encode(std::vector && columns, CompressionMethod compression_method, bool always_keep_header) + { + return encodeImpl(std::move(columns), compression_method, always_keep_header); + } + + template + static void getColumnEncodeInfoImpl(ColumnsHolder && columns_holder, size_t & bytes, size_t & rows) + { + bytes += 8 /*rows*/; + + if constexpr (isBlockType()) + { + rows += columns_holder.rows(); + for (size_t col_index = 0; col_index < columns_holder.columns(); ++col_index) + { + auto && col_type_name = columns_holder.getByPosition(col_index); + bytes += col_type_name.column->byteSize(); + } + } + else + { + rows += columns_holder.front()->size(); + for (const auto & elem : columns_holder) + bytes += elem->byteSize(); + } + } + static const ColumnPtr & toColumnPtr(const Columns & c, size_t index) + { + return c[index]; + } + static ColumnPtr toColumnPtr(MutableColumns && c, size_t index) + { + return std::move(c[index]); + } + static ColumnPtr toColumnPtr(const MutableColumns & c, size_t index) + { + return c[index]->getPtr(); + } + static const ColumnPtr & toColumnPtr(const Block & block, size_t index) + { + return block.getByPosition(index).column; + } + + template + size_t getRowsByColumns(ColumnsHolder && columns_holder) + { + size_t rows = columns_holder.front()->size(); + return rows; + } + + template + constexpr static bool isBlockType() + { + return std::is_same_v>, Block>; + } + + template + size_t getRows(ColumnsHolder && columns_holder) + { + if constexpr (isBlockType()) + { + size_t rows = columns_holder.rows(); + return rows; + } + else + { + size_t rows = columns_holder.front()->size(); + return rows; + } + } + + template + void encodeColumnImpl(ColumnsHolder && columns_holder, WriteBuffer * ostr_ptr) + { + size_t rows = getRows(std::forward(columns_holder)); + if (!rows) + return; + + // Encode row count for next columns + writeVarUInt(rows, *ostr_ptr); + + // Encode columns data + for (size_t col_index = 0; col_index < header.columns(); ++col_index) + { + auto && col_type_name = header.getByPosition(col_index); + auto && column_ptr = toColumnPtr(std::forward(columns_holder), col_index); + WriteColumnData(*col_type_name.type, column_ptr, *ostr_ptr, 0, 0); + } + + encoded_rows += rows; + } + void encodeColumn(const MutableColumns & columns, WriteBuffer * ostr_ptr) + { + return encodeColumnImpl(columns, ostr_ptr); + } + void encodeColumn(const Columns & columns, WriteBuffer * ostr_ptr) + { + return encodeColumnImpl(columns, ostr_ptr); + } + void encodeColumn(const std::vector & batch_columns, WriteBuffer * ostr_ptr) + { + for (auto && batch : batch_columns) + { + encodeColumnImpl(batch, ostr_ptr); + } + } + void encodeColumn(std::vector && batch_columns, WriteBuffer * ostr_ptr) + { + for (auto && batch : batch_columns) + { + encodeColumnImpl(std::move(batch), ostr_ptr); + } + } + void encodeColumn(const Block & block, WriteBuffer * ostr_ptr) + { + assert(&block == &header); + return encodeColumnImpl(block, ostr_ptr); + } + + static void getColumnEncodeInfo(const std::vector & batch_columns, size_t & bytes, size_t & rows) + { + for (auto && columns : batch_columns) + { + getColumnEncodeInfoImpl(columns, bytes, rows); + } + } + static void getColumnEncodeInfo(const MutableColumns & columns, size_t & bytes, size_t & rows) + { + getColumnEncodeInfoImpl(columns, bytes, rows); + } + static void getColumnEncodeInfo(const Columns & columns, size_t & bytes, size_t & rows) + { + getColumnEncodeInfoImpl(columns, bytes, rows); + } + static void getColumnEncodeInfo(const Block & block, size_t & bytes, size_t & rows) + { + getColumnEncodeInfoImpl(block, bytes, rows); + } + + template + std::string encodeImpl(VecColumns && batch_columns, CompressionMethod compression_method, bool always_keep_header) + { + size_t column_encode_bytes = 0; + size_t rows = 0; + + getColumnEncodeInfo(batch_columns, column_encode_bytes, rows); + + if unlikely (rows <= 0 && !always_keep_header) + { + return ""; + } + + // compression method flag; NONE, LZ4, ZSTD, defined in `CompressionMethodByte` + // ... + // header meta: + // columns count; + // total row count (multi parts); + // for each column: + // column name; + // column type; + // for each part: + // row count; + // columns data; + + size_t init_size = column_encode_bytes + header_size + 1 /*compression method*/; + auto output_buffer = std::make_unique(init_size); + std::unique_ptr compress_codec{}; + WriteBuffer * ostr_ptr = output_buffer.get(); + + // Init compression writer + if (compression_method != CompressionMethod::NONE) + { + // CompressedWriteBuffer will encode compression method flag as first byte + compress_codec = std::make_unique( + *output_buffer, + CompressionSettings(compression_method), + init_size); + ostr_ptr = compress_codec.get(); + } + else + { + // Write compression method flag + output_buffer->write(static_cast(CompressionMethodByte::NONE)); + } + + // Encode header + EncodeHeader(*ostr_ptr, header, rows); + if (rows > 0) + encodeColumn(std::forward(batch_columns), ostr_ptr); + + // Flush rest buffer + if (compress_codec) + compress_codec->next(); + + return output_buffer->releaseStr(); + } +}; } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.cpp b/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.cpp index df9325eef0d..a540b64202f 100644 --- a/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.cpp +++ b/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.cpp @@ -28,6 +28,14 @@ CHBlockChunkDecodeAndSquash::CHBlockChunkDecodeAndSquash( std::optional CHBlockChunkDecodeAndSquash::decodeAndSquashWithCompression(std::string_view sv) { + if unlikely (sv.empty()) + { + std::optional res; + if (accumulated_block) + res.swap(accumulated_block); + return res; + } + // read first byte of compression method flag which defined in `CompressionMethodByte` if (static_cast(sv[0]) == CompressionMethodByte::NONE) { @@ -44,20 +52,15 @@ std::optional CHBlockChunkDecodeAndSquash::decodeAndSquashWithCompression { std::optional res; - if (istr.eof()) - { - if (accumulated_block) - res.swap(accumulated_block); - return res; - } - if (!accumulated_block) { size_t rows{}; Block block = DecodeHeader(istr, codec.header, rows); - DecodeColumns(istr, block, rows, static_cast(rows_limit * 1.5)); - if (block) + if (rows) + { + DecodeColumns(istr, block, rows, static_cast(rows_limit * 1.5)); accumulated_block.emplace(std::move(block)); + } } else { diff --git a/dbms/src/Flash/Coprocessor/tests/gtest_chunk_decode_and_squash.cpp b/dbms/src/Flash/Coprocessor/tests/gtest_chunk_decode_and_squash.cpp index 69b6abc5569..ae0e0f02a47 100644 --- a/dbms/src/Flash/Coprocessor/tests/gtest_chunk_decode_and_squash.cpp +++ b/dbms/src/Flash/Coprocessor/tests/gtest_chunk_decode_and_squash.cpp @@ -15,6 +15,8 @@ #include #include #include +#include +#include #include #include #include @@ -23,7 +25,6 @@ #include #include -#include namespace DB { namespace tests @@ -93,8 +94,8 @@ class TestChunkDecodeAndSquash : public testing::Test void doTestWork(bool flush_something) { - const size_t block_rows = 1024; - const size_t block_num = 256; + const size_t block_rows = 256; + const size_t block_num = 64; std::mt19937_64 rand_gen; // 1. Build Blocks. std::vector blocks; @@ -109,12 +110,28 @@ class TestChunkDecodeAndSquash : public testing::Test // 2. encode all blocks std::unique_ptr codec_stream = std::make_unique()->newCodecStream(makeFields()); std::vector encode_str_vec(block_num); + std::vector encode_str_use_compression(block_num, true); + size_t round_index = 0; for (const auto & block : blocks) { - codec_stream->encode(block, 0, block.rows()); - encode_str_vec.push_back(codec_stream->getString()); - codec_stream->clear(); + if (round_index % 3 == 0) + { + codec_stream->encode(block, 0, block.rows()); + encode_str_vec.push_back(codec_stream->getString()); + codec_stream->clear(); + encode_str_use_compression.emplace_back(false); + } + else + { + auto && str = CHBlockChunkCodecV1::encode(block, CompressionMethod::LZ4, true); + assert(!str.empty()); + assert(static_cast(str[0]) == CompressionMethodByte::LZ4); + encode_str_vec.push_back(std::move(str)); + encode_str_use_compression.emplace_back(true); + } + round_index++; } + round_index = 0; // 3. DecodeAndSquash all these blocks Block header = blocks.back(); @@ -122,7 +139,17 @@ class TestChunkDecodeAndSquash : public testing::Test CHBlockChunkDecodeAndSquash decoder(header, block_rows * 4); for (const auto & str : encode_str_vec) { - auto result = decoder.decodeAndSquash(str); + std::optional result{}; + if (!encode_str_use_compression[round_index]) + { + result = decoder.decodeAndSquash(str); + } + else + { + result = decoder.decodeAndSquashWithCompression(str); + } + round_index++; + if (result) decoded_blocks.push_back(std::move(result.value())); } diff --git a/dbms/src/Flash/Mpp/FineGrainedShuffleWriterV1.cpp b/dbms/src/Flash/Mpp/FineGrainedShuffleWriterV1.cpp index faedf71a7a1..fd5080a31d8 100644 --- a/dbms/src/Flash/Mpp/FineGrainedShuffleWriterV1.cpp +++ b/dbms/src/Flash/Mpp/FineGrainedShuffleWriterV1.cpp @@ -20,10 +20,11 @@ #include #include +#include + namespace DB { extern size_t ApproxBlockBytes(const Block & block); -extern void WriteColumnData(const IDataType & type, const ColumnPtr & column, WriteBuffer & ostr, size_t offset, size_t limit); } // namespace DB namespace DB @@ -126,16 +127,22 @@ void FineGrainedShuffleWriterV1::batchWriteFineGrainedShuffle assert(rows_in_blocks > 0); assert(fine_grained_shuffle_stream_count <= 1024); - size_t header_size = ApproxBlockHeaderBytes(header); HashBaseWriterHelper::materializeBlocks(blocks); + [[maybe_unused]] size_t total_rows = 0, encoded_rows = 0; + while (!blocks.empty()) { const auto & block = blocks.back(); + ori_block_mem_size += ApproxBlockBytes(block); + total_rows += block.rows(); + HashBaseWriterHelper::scatterColumnsForFineGrainedShuffle(block, partition_col_ids, collators, partition_key_containers_for_reuse, partition_num, fine_grained_shuffle_stream_count, hash, selector, scattered); blocks.pop_back(); } + auto && codec = CHBlockChunkCodecV1{header}; + // serialize each partitioned block and write it to its destination size_t part_id = 0; for (size_t bucket_idx = 0; bucket_idx < num_bucket; bucket_idx += fine_grained_shuffle_stream_count, ++part_id) @@ -148,57 +155,13 @@ void FineGrainedShuffleWriterV1::batchWriteFineGrainedShuffle for (size_t col_id = 0; col_id < num_columns; ++col_id) columns.emplace_back(std::move(scattered[col_id][bucket_idx + stream_idx])); - size_t part_column_bytes = std::accumulate(columns.begin(), columns.end(), 0, [](auto res, const auto & column) { - return res += column->byteSize(); - }) - + 8 /*partition rows*/; - - { - size_t init_size = part_column_bytes + header_size + 1 /*compression method*/; - - // Reserve enough memory buffer size - auto output_buffer = std::make_unique(init_size); - std::unique_ptr compress_codec{}; - WriteBuffer * ostr_ptr = output_buffer.get(); - - // Init compression writer - if (!writer->isLocal(part_id) && compression_method != CompressionMethod::NONE) - { - // CompressedWriteBuffer will encode compression method flag as first byte - compress_codec = std::make_unique( - *output_buffer, - CompressionSettings(compression_method), - init_size); - ostr_ptr = compress_codec.get(); - } - else - { - // Write compression method flag - output_buffer->write(static_cast(CompressionMethodByte::NONE)); - } - - size_t rows = columns.front()->size(); - // Encode header - EncodeHeader(*ostr_ptr, header, rows); - if (rows) - { - // Encode row count for next columns - writeVarUInt(rows, *ostr_ptr); - - // Encode columns data - for (size_t col_id = 0; col_id < num_columns; ++col_id) - { - auto && col_type_name = header.getByPosition(col_id); - WriteColumnData(*col_type_name.type, columns[col_id]->getPtr(), *ostr_ptr, 0, 0); - } - } - - // Flush rest buffer - if (compress_codec) - compress_codec->next(); - tracked_packets[part_id]->getPacket().add_chunks(output_buffer->releaseStr()); - tracked_packets[part_id]->getPacket().add_stream_ids(stream_idx); - } + auto method = writer->isLocal(part_id) ? CompressionMethod::NONE : compression_method; + auto && res = codec.encode(columns, method, true /*keep header info*/); + assert(!res.empty()); + tracked_packets[part_id]->getPacket().add_chunks(std::move(res)); + tracked_packets[part_id]->getPacket().add_stream_ids(stream_idx); + encoded_rows += codec.encoded_rows; + codec.clear(); for (size_t col_id = 0; col_id < num_columns; ++col_id) { @@ -208,6 +171,8 @@ void FineGrainedShuffleWriterV1::batchWriteFineGrainedShuffle } } rows_in_blocks = 0; + + assert(encoded_rows == total_rows); } writePackets(tracked_packets); diff --git a/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp b/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp index e8e9e947142..27a38e902b3 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp +++ b/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp @@ -23,7 +23,6 @@ namespace DB { extern size_t ApproxBlockBytes(const Block & block); -extern void WriteColumnData(const IDataType & type, const ColumnPtr & column, WriteBuffer & ostr, size_t offset, size_t limit); } // namespace DB namespace DB @@ -123,85 +122,18 @@ void HashPartitionWriterV1::partitionAndEncodeThenWriteBlocks } } - size_t header_size = ApproxBlockHeaderBytes(dest_block_header); + auto && codec = CHBlockChunkCodecV1{dest_block_header}; for (size_t part_id = 0; part_id < partition_num; ++part_id) { auto & part_columns = dest_columns[part_id]; - size_t part_rows = std::accumulate(part_columns.begin(), part_columns.end(), 0, [](const auto & r, const auto & columns) { return r + columns.front()->size(); }); - - if (!part_rows) + auto method = writer->isLocal(part_id) ? CompressionMethod::NONE : compression_method; + auto && res = codec.encode(std::move(part_columns), method, false /*scape empty part*/); + if (res.empty()) continue; - - size_t part_column_bytes = std::accumulate(part_columns.begin(), part_columns.end(), 0, [](auto res, const auto & columns) { - for (const auto & elem : columns) - res += elem->byteSize(); - return res + 8 /*partition rows*/; - }); - - // compression method flag; NONE, LZ4, ZSTD, defined in `CompressionMethodByte` - // ... - // header meta: - // columns count; - // total row count (multi parts); - // for each column: - // column name; - // column type; - // for each part: - // row count; - // columns data; - - size_t init_size = part_column_bytes + header_size + 1 /*compression method*/; - - // Reserve enough memory buffer size - auto output_buffer = std::make_unique(init_size); - std::unique_ptr compress_codec{}; - WriteBuffer * ostr_ptr = output_buffer.get(); - - // Init compression writer - if (!writer->isLocal(part_id) && compression_method != CompressionMethod::NONE) - { - // CompressedWriteBuffer will encode compression method flag as first byte - compress_codec = std::make_unique( - *output_buffer, - CompressionSettings(compression_method), - init_size); - ostr_ptr = compress_codec.get(); - } - else - { - // Write compression method flag - output_buffer->write(static_cast(CompressionMethodByte::NONE)); - } - - // Encode header - EncodeHeader(*ostr_ptr, dest_block_header, part_rows); - - for (auto && columns : part_columns) - { - size_t rows = columns.front()->size(); - if (!rows) - continue; - - // Encode row count for next columns - writeVarUInt(rows, *ostr_ptr); - encoded_rows += rows; - - // Encode columns data - for (size_t col_index = 0; col_index < dest_block_header.columns(); ++col_index) - { - auto && col_type_name = dest_block_header.getByPosition(col_index); - WriteColumnData(*col_type_name.type, std::move(columns[col_index]), *ostr_ptr, 0, 0); - } - - columns.clear(); - } - - // Flush rest buffer - if (compress_codec) - compress_codec->next(); - - tracked_packets[part_id]->getPacket().add_chunks(output_buffer->releaseStr()); + encoded_rows += codec.encoded_rows; + tracked_packets[part_id]->getPacket().add_chunks(std::move(res)); + codec.clear(); } assert(encoded_rows == total_rows); assert(blocks.empty()); diff --git a/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp b/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp index 5ac4bec922f..bd220ce36ec 100644 --- a/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp +++ b/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp @@ -35,7 +35,7 @@ namespace DB { namespace tests { -static CompressionMethodByte ToCompressionMethodByte(CompressionMethod m); +static CompressionMethodByte GetCompressionMethodByte(CompressionMethod m); class TestMPPExchangeWriter : public testing::Test { @@ -227,71 +227,74 @@ try } Block header = blocks.back(); - // 2. Build MockExchangeWriter. - std::unordered_map write_report; - auto checker = [&write_report](const TrackedMppDataPacketPtr & packet, uint16_t part_id) { - write_report[part_id].emplace_back(packet); - }; - auto mock_writer = std::make_shared(checker, part_num); - - // 3. Start to write. - auto dag_writer = std::make_shared>>( - mock_writer, - part_col_ids, - part_col_collators, - *dag_context_ptr, - fine_grained_shuffle_stream_count, - fine_grained_shuffle_batch_size, - tipb::CompressionMode::FAST); - dag_writer->prepare(blocks[0].cloneEmpty()); - for (const auto & block : blocks) - dag_writer->write(block); - dag_writer->flush(); + for (auto mode : {tipb::CompressionMode::NONE, tipb::CompressionMode::FAST, tipb::CompressionMode::HIGH_COMPRESSION}) + { + // 2. Build MockExchangeWriter. + std::unordered_map write_report; + auto checker = [&write_report](const TrackedMppDataPacketPtr & packet, uint16_t part_id) { + write_report[part_id].emplace_back(packet); + }; + auto mock_writer = std::make_shared(checker, part_num); - // 4. Start to check write_report. - size_t per_part_rows = block_rows * block_num / part_num; - ASSERT_EQ(write_report.size(), part_num); - std::vector rows_of_stream_ids(fine_grained_shuffle_stream_count, 0); + // 3. Start to write. + auto dag_writer = std::make_shared>>( + mock_writer, + part_col_ids, + part_col_collators, + *dag_context_ptr, + fine_grained_shuffle_stream_count, + fine_grained_shuffle_batch_size, + mode); + dag_writer->prepare(blocks[0].cloneEmpty()); + for (const auto & block : blocks) + dag_writer->write(block); + dag_writer->flush(); - CHBlockChunkDecodeAndSquash decoder(header, 512); + // 4. Start to check write_report. + size_t per_part_rows = block_rows * block_num / part_num; + ASSERT_EQ(write_report.size(), part_num); + std::vector rows_of_stream_ids(fine_grained_shuffle_stream_count, 0); - for (size_t part_index = 0; part_index < part_num; ++part_index) - { - size_t part_decoded_block_rows = 0; + CHBlockChunkDecodeAndSquash decoder(header, 512); - for (const auto & packet : write_report[part_index]) + for (size_t part_index = 0; part_index < part_num; ++part_index) { - ASSERT_EQ(packet->getPacket().chunks_size(), packet->getPacket().stream_ids_size()); - for (int i = 0; i < packet->getPacket().chunks_size(); ++i) - { - const auto & chunk = packet->getPacket().chunks(i); + size_t part_decoded_block_rows = 0; - if (part_index == 0) - { - ASSERT_EQ(CompressionMethodByte(chunk[0]), CompressionMethodByte::NONE); - } - else + for (const auto & packet : write_report[part_index]) + { + ASSERT_EQ(packet->getPacket().chunks_size(), packet->getPacket().stream_ids_size()); + for (int i = 0; i < packet->getPacket().chunks_size(); ++i) { - ASSERT_EQ(CompressionMethodByte(chunk[0]), ToCompressionMethodByte(ToInternalCompressionMethod(tipb::CompressionMode::FAST))); - } + const auto & chunk = packet->getPacket().chunks(i); - auto && result = decoder.decodeAndSquashWithCompression(chunk); - if (!result) - { - result = decoder.flush(); + if (part_index == 0) + { + ASSERT_EQ(CompressionMethodByte(chunk[0]), CompressionMethodByte::NONE); + } + else + { + ASSERT_EQ(CompressionMethodByte(chunk[0]), GetCompressionMethodByte(ToInternalCompressionMethod(mode))); + } + + auto && result = decoder.decodeAndSquashWithCompression(chunk); + if (!result) + { + result = decoder.flush(); + } + assert(result); + auto decoded_block = std::move(*result); + part_decoded_block_rows += decoded_block.rows(); + rows_of_stream_ids[packet->getPacket().stream_ids(i)] += decoded_block.rows(); } - assert(result); - auto decoded_block = std::move(*result); - part_decoded_block_rows += decoded_block.rows(); - rows_of_stream_ids[packet->getPacket().stream_ids(i)] += decoded_block.rows(); } + ASSERT_EQ(part_decoded_block_rows, per_part_rows); } - ASSERT_EQ(part_decoded_block_rows, per_part_rows); - } - size_t per_stream_id_rows = block_rows * block_num / fine_grained_shuffle_stream_count; - for (size_t rows : rows_of_stream_ids) - ASSERT_EQ(rows, per_stream_id_rows); + size_t per_stream_id_rows = block_rows * block_num / fine_grained_shuffle_stream_count; + for (size_t rows : rows_of_stream_ids) + ASSERT_EQ(rows, per_stream_id_rows); + } } CATCH @@ -460,7 +463,7 @@ try } CATCH -static CompressionMethodByte ToCompressionMethodByte(CompressionMethod m) +static CompressionMethodByte GetCompressionMethodByte(CompressionMethod m) { switch (m) { @@ -537,7 +540,7 @@ try } else { - ASSERT_EQ(CompressionMethodByte(chunk[0]), ToCompressionMethodByte(ToInternalCompressionMethod(mode))); + ASSERT_EQ(CompressionMethodByte(chunk[0]), GetCompressionMethodByte(ToInternalCompressionMethod(mode))); } auto && result = decoder.decodeAndSquashWithCompression(chunk); From 95f9d9af2e100886a58310af87f0cb75ed62b89e Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Mon, 16 Jan 2023 18:17:27 +0800 Subject: [PATCH 77/93] 67 Signed-off-by: Zhigao Tong --- dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.cpp | 2 +- dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.h | 2 +- .../Flash/Coprocessor/tests/gtest_chunk_decode_and_squash.cpp | 2 +- dbms/src/Flash/Mpp/ExchangeReceiver.cpp | 2 +- dbms/src/Flash/Mpp/FineGrainedShuffleWriter.cpp | 3 ++- dbms/src/Flash/Mpp/HashBaseWriterHelper.h | 2 +- dbms/src/Flash/Mpp/HashPartitionWriter.cpp | 3 ++- dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp | 4 ++-- 8 files changed, 11 insertions(+), 9 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.cpp b/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.cpp index a540b64202f..165b6a1f73c 100644 --- a/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.cpp +++ b/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.cpp @@ -26,7 +26,7 @@ CHBlockChunkDecodeAndSquash::CHBlockChunkDecodeAndSquash( { } -std::optional CHBlockChunkDecodeAndSquash::decodeAndSquashWithCompression(std::string_view sv) +std::optional CHBlockChunkDecodeAndSquash::decodeAndSquashV1(std::string_view sv) { if unlikely (sv.empty()) { diff --git a/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.h b/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.h index 2c10da769e0..86b29023050 100644 --- a/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.h +++ b/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.h @@ -27,7 +27,7 @@ class CHBlockChunkDecodeAndSquash CHBlockChunkDecodeAndSquash(const Block & header, size_t rows_limit_); ~CHBlockChunkDecodeAndSquash() = default; std::optional decodeAndSquash(const String &); - std::optional decodeAndSquashWithCompression(std::string_view); + std::optional decodeAndSquashV1(std::string_view); std::optional flush(); private: diff --git a/dbms/src/Flash/Coprocessor/tests/gtest_chunk_decode_and_squash.cpp b/dbms/src/Flash/Coprocessor/tests/gtest_chunk_decode_and_squash.cpp index ae0e0f02a47..9bb419a4578 100644 --- a/dbms/src/Flash/Coprocessor/tests/gtest_chunk_decode_and_squash.cpp +++ b/dbms/src/Flash/Coprocessor/tests/gtest_chunk_decode_and_squash.cpp @@ -146,7 +146,7 @@ class TestChunkDecodeAndSquash : public testing::Test } else { - result = decoder.decodeAndSquashWithCompression(str); + result = decoder.decodeAndSquashV1(str); } round_index++; diff --git a/dbms/src/Flash/Mpp/ExchangeReceiver.cpp b/dbms/src/Flash/Mpp/ExchangeReceiver.cpp index 53410abc9ba..d219fa6effc 100644 --- a/dbms/src/Flash/Mpp/ExchangeReceiver.cpp +++ b/dbms/src/Flash/Mpp/ExchangeReceiver.cpp @@ -617,7 +617,7 @@ DecodeDetail ExchangeReceiverBase::decodeChunks( for (const auto * chunk : recv_msg->chunks) { assert(!chunk->empty()); - auto && result = decoder_ptr->decodeAndSquashWithCompression(*chunk); + auto && result = decoder_ptr->decodeAndSquashV1(*chunk); if (!result || !result->rows()) continue; detail.rows += result->rows(); diff --git a/dbms/src/Flash/Mpp/FineGrainedShuffleWriter.cpp b/dbms/src/Flash/Mpp/FineGrainedShuffleWriter.cpp index edd31161f84..bbbc7dfdf6a 100644 --- a/dbms/src/Flash/Mpp/FineGrainedShuffleWriter.cpp +++ b/dbms/src/Flash/Mpp/FineGrainedShuffleWriter.cpp @@ -18,6 +18,7 @@ #include #include #include +#include namespace DB { @@ -116,7 +117,7 @@ template void FineGrainedShuffleWriter::batchWriteFineGrainedShuffle() { size_t ori_block_mem_size = 0; - auto tracked_packets = HashBaseWriterHelper::createPackets(partition_num); + auto tracked_packets = HashBaseWriterHelper::createPackets(partition_num, DB::MPPDataPacketV0); if (likely(!blocks.empty())) { assert(rows_in_blocks > 0); diff --git a/dbms/src/Flash/Mpp/HashBaseWriterHelper.h b/dbms/src/Flash/Mpp/HashBaseWriterHelper.h index 03bbb247048..645e20a2035 100644 --- a/dbms/src/Flash/Mpp/HashBaseWriterHelper.h +++ b/dbms/src/Flash/Mpp/HashBaseWriterHelper.h @@ -39,7 +39,7 @@ void computeHash(size_t rows, std::vector & partition_key_containers, WeakHash32 & hash); -DB::TrackedMppDataPacketPtrs createPackets(size_t partition_num, uint64_t version = 0); +DB::TrackedMppDataPacketPtrs createPackets(size_t partition_num, uint64_t version); void scatterColumns(const Block & input_block, const std::vector & partition_col_ids, diff --git a/dbms/src/Flash/Mpp/HashPartitionWriter.cpp b/dbms/src/Flash/Mpp/HashPartitionWriter.cpp index 0861f767325..7d2fcdb2af9 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriter.cpp +++ b/dbms/src/Flash/Mpp/HashPartitionWriter.cpp @@ -18,6 +18,7 @@ #include #include #include +#include namespace DB { @@ -72,7 +73,7 @@ void HashPartitionWriter::write(const Block & block) template void HashPartitionWriter::partitionAndEncodeThenWriteBlocks() { - auto tracked_packets = HashBaseWriterHelper::createPackets(partition_num); + auto tracked_packets = HashBaseWriterHelper::createPackets(partition_num, DB::MPPDataPacketV0); size_t ori_block_mem_size = 0; diff --git a/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp b/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp index bd220ce36ec..53b1d0c682a 100644 --- a/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp +++ b/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp @@ -277,7 +277,7 @@ try ASSERT_EQ(CompressionMethodByte(chunk[0]), GetCompressionMethodByte(ToInternalCompressionMethod(mode))); } - auto && result = decoder.decodeAndSquashWithCompression(chunk); + auto && result = decoder.decodeAndSquashV1(chunk); if (!result) { result = decoder.flush(); @@ -543,7 +543,7 @@ try ASSERT_EQ(CompressionMethodByte(chunk[0]), GetCompressionMethodByte(ToInternalCompressionMethod(mode))); } - auto && result = decoder.decodeAndSquashWithCompression(chunk); + auto && result = decoder.decodeAndSquashV1(chunk); if (!result) continue; decoded_block_rows += result->rows(); From 35457d568531fdd0c2726af132efc55d47a278bd Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Mon, 16 Jan 2023 22:13:30 +0800 Subject: [PATCH 78/93] 68: statistic --- .../Flash/Coprocessor/CHBlockChunkCodecV1.h | 44 +++++++++++++------ .../Flash/Mpp/FineGrainedShuffleWriter.cpp | 3 -- .../Flash/Mpp/FineGrainedShuffleWriterV1.cpp | 26 +++++------ dbms/src/Flash/Mpp/HashPartitionWriter.cpp | 27 +++--------- dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp | 27 ++++++------ 5 files changed, 63 insertions(+), 64 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.h b/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.h index 56f0ad21792..285e56e2291 100644 --- a/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.h +++ b/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.h @@ -35,43 +35,51 @@ struct CHBlockChunkCodecV1 using Self = CHBlockChunkCodecV1; const Block & header; - size_t header_size; + const size_t header_size; size_t encoded_rows{}; + size_t original_size{}; + size_t compressed_size{}; + bool always_keep_header{}; - explicit CHBlockChunkCodecV1(const Block & header_) + explicit CHBlockChunkCodecV1(const Block & header_, bool always_keep_header_) : header(header_) , header_size(ApproxBlockHeaderBytes(header)) + , always_keep_header(always_keep_header_) { } + static std::string encode(const Block & block, CompressionMethod compression_method, bool always_keep_header) { - return Self{block}.encode(compression_method, always_keep_header); + return Self{block, always_keep_header}.encode(compression_method); } + void clear() { encoded_rows = 0; + original_size = 0; + compressed_size = 0; } - std::string encode(CompressionMethod compression_method, bool always_keep_header) + std::string encode(CompressionMethod compression_method) { - return encodeImpl(header, compression_method, always_keep_header); + return encodeImpl(header, compression_method); } - std::string encode(const MutableColumns & columns, CompressionMethod compression_method, bool always_keep_header) + std::string encode(const MutableColumns & columns, CompressionMethod compression_method) { - return encodeImpl(columns, compression_method, always_keep_header); + return encodeImpl(columns, compression_method); } - std::string encode(const Columns & columns, CompressionMethod compression_method, bool always_keep_header) + std::string encode(const Columns & columns, CompressionMethod compression_method) { - return encodeImpl(columns, compression_method, always_keep_header); + return encodeImpl(columns, compression_method); } - std::string encode(const std::vector & columns, CompressionMethod compression_method, bool always_keep_header) + std::string encode(const std::vector & columns, CompressionMethod compression_method) { - return encodeImpl(columns, compression_method, always_keep_header); + return encodeImpl(columns, compression_method); } - std::string encode(std::vector && columns, CompressionMethod compression_method, bool always_keep_header) + std::string encode(std::vector && columns, CompressionMethod compression_method) { - return encodeImpl(std::move(columns), compression_method, always_keep_header); + return encodeImpl(std::move(columns), compression_method); } template @@ -209,7 +217,7 @@ struct CHBlockChunkCodecV1 } template - std::string encodeImpl(VecColumns && batch_columns, CompressionMethod compression_method, bool always_keep_header) + std::string encodeImpl(VecColumns && batch_columns, CompressionMethod compression_method) { size_t column_encode_bytes = 0; size_t rows = 0; @@ -261,7 +269,15 @@ struct CHBlockChunkCodecV1 // Flush rest buffer if (compress_codec) + { compress_codec->next(); + original_size += compress_codec->getUncompressedBytes(); + compressed_size += compress_codec->getCompressedBytes(); + } + else + { + original_size += output_buffer->count(); + } return output_buffer->releaseStr(); } diff --git a/dbms/src/Flash/Mpp/FineGrainedShuffleWriter.cpp b/dbms/src/Flash/Mpp/FineGrainedShuffleWriter.cpp index bbbc7dfdf6a..ab8a1dd503d 100644 --- a/dbms/src/Flash/Mpp/FineGrainedShuffleWriter.cpp +++ b/dbms/src/Flash/Mpp/FineGrainedShuffleWriter.cpp @@ -116,7 +116,6 @@ void FineGrainedShuffleWriter::initScatterColumns() template void FineGrainedShuffleWriter::batchWriteFineGrainedShuffle() { - size_t ori_block_mem_size = 0; auto tracked_packets = HashBaseWriterHelper::createPackets(partition_num, DB::MPPDataPacketV0); if (likely(!blocks.empty())) { @@ -127,7 +126,6 @@ void FineGrainedShuffleWriter::batchWriteFineGrainedShuffle() while (!blocks.empty()) { const auto & block = blocks.back(); - ori_block_mem_size += ApproxBlockBytes(block); HashBaseWriterHelper::scatterColumnsForFineGrainedShuffle(block, partition_col_ids, collators, partition_key_containers_for_reuse, partition_num, fine_grained_shuffle_stream_count, hash, selector, scattered); blocks.pop_back(); } @@ -164,7 +162,6 @@ void FineGrainedShuffleWriter::batchWriteFineGrainedShuffle() } writePackets(tracked_packets); - GET_METRIC(tiflash_exchange_data_bytes, type_hash_original_all).Increment(ori_block_mem_size); } template diff --git a/dbms/src/Flash/Mpp/FineGrainedShuffleWriterV1.cpp b/dbms/src/Flash/Mpp/FineGrainedShuffleWriterV1.cpp index fd5080a31d8..db41ba73ab7 100644 --- a/dbms/src/Flash/Mpp/FineGrainedShuffleWriterV1.cpp +++ b/dbms/src/Flash/Mpp/FineGrainedShuffleWriterV1.cpp @@ -120,28 +120,30 @@ void FineGrainedShuffleWriterV1::initScatterColumns() template void FineGrainedShuffleWriterV1::batchWriteFineGrainedShuffle() { - size_t ori_block_mem_size = 0; + if (blocks.empty()) + return; + size_t ori_all_packets_size = 0; auto tracked_packets = HashBaseWriterHelper::createPackets(partition_num, DB::MPPDataPacketV1); - if (likely(!blocks.empty())) + { assert(rows_in_blocks > 0); assert(fine_grained_shuffle_stream_count <= 1024); HashBaseWriterHelper::materializeBlocks(blocks); - [[maybe_unused]] size_t total_rows = 0, encoded_rows = 0; + [[maybe_unused]] size_t total_rows = 0; while (!blocks.empty()) { const auto & block = blocks.back(); - - ori_block_mem_size += ApproxBlockBytes(block); total_rows += block.rows(); - HashBaseWriterHelper::scatterColumnsForFineGrainedShuffle(block, partition_col_ids, collators, partition_key_containers_for_reuse, partition_num, fine_grained_shuffle_stream_count, hash, selector, scattered); blocks.pop_back(); } - auto && codec = CHBlockChunkCodecV1{header}; + auto && codec = CHBlockChunkCodecV1{ + header, + true /*keep header info*/, + }; // serialize each partitioned block and write it to its destination size_t part_id = 0; @@ -156,12 +158,10 @@ void FineGrainedShuffleWriterV1::batchWriteFineGrainedShuffle columns.emplace_back(std::move(scattered[col_id][bucket_idx + stream_idx])); auto method = writer->isLocal(part_id) ? CompressionMethod::NONE : compression_method; - auto && res = codec.encode(columns, method, true /*keep header info*/); + auto && res = codec.encode(columns, method); assert(!res.empty()); tracked_packets[part_id]->getPacket().add_chunks(std::move(res)); tracked_packets[part_id]->getPacket().add_stream_ids(stream_idx); - encoded_rows += codec.encoded_rows; - codec.clear(); for (size_t col_id = 0; col_id < num_columns; ++col_id) { @@ -170,13 +170,13 @@ void FineGrainedShuffleWriterV1::batchWriteFineGrainedShuffle } } } + assert(codec.encoded_rows == total_rows); + ori_all_packets_size += codec.original_size; rows_in_blocks = 0; - - assert(encoded_rows == total_rows); } writePackets(tracked_packets); - GET_METRIC(tiflash_exchange_data_bytes, type_hash_original_all).Increment(ori_block_mem_size); + GET_METRIC(tiflash_exchange_data_bytes, type_hash_original_all).Increment(ori_all_packets_size); } template diff --git a/dbms/src/Flash/Mpp/HashPartitionWriter.cpp b/dbms/src/Flash/Mpp/HashPartitionWriter.cpp index 7d2fcdb2af9..da509ffe741 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriter.cpp +++ b/dbms/src/Flash/Mpp/HashPartitionWriter.cpp @@ -75,8 +75,6 @@ void HashPartitionWriter::partitionAndEncodeThenWriteBlocks() { auto tracked_packets = HashBaseWriterHelper::createPackets(partition_num, DB::MPPDataPacketV0); - size_t ori_block_mem_size = 0; - if (!blocks.empty()) { assert(rows_in_blocks > 0); @@ -88,8 +86,6 @@ void HashPartitionWriter::partitionAndEncodeThenWriteBlocks() while (!blocks.empty()) { const auto & block = blocks.back(); - ori_block_mem_size += ApproxBlockBytes(block); - auto dest_tbl_cols = HashBaseWriterHelper::createDestColumns(block, partition_num); HashBaseWriterHelper::scatterColumns(block, partition_col_ids, collators, partition_key_containers, partition_num, dest_tbl_cols); blocks.pop_back(); @@ -111,20 +107,6 @@ void HashPartitionWriter::partitionAndEncodeThenWriteBlocks() } writePackets(tracked_packets); - - GET_METRIC(tiflash_exchange_data_bytes, type_hash_original_all).Increment(ori_block_mem_size); -} - -static void updateHashPartitionWriterMetrics(size_t sz, bool is_local) -{ - if (is_local) - { - GET_METRIC(tiflash_exchange_data_bytes, type_hash_none_local).Increment(sz); - } - else - { - GET_METRIC(tiflash_exchange_data_bytes, type_hash_none_remote).Increment(sz); - } } template @@ -137,10 +119,15 @@ void WritePackets(TrackedMppDataPacketPtrs & packets, ExchangeWriterPtr & writer auto & inner_packet = packet->getPacket(); - if (auto sz = inner_packet.ByteSizeLong(); likely(inner_packet.chunks_size() > 0)) + if (const auto sz = inner_packet.ByteSizeLong(); likely(inner_packet.chunks_size() > 0)) { writer->partitionWrite(std::move(packet), part_id); - updateHashPartitionWriterMetrics(sz, writer->isLocal(part_id)); + + GET_METRIC(tiflash_exchange_data_bytes, type_hash_original_all).Increment(sz); + if (writer->isLocal(part_id)) + GET_METRIC(tiflash_exchange_data_bytes, type_hash_none_local).Increment(sz); + else + GET_METRIC(tiflash_exchange_data_bytes, type_hash_none_remote).Increment(sz); } } } diff --git a/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp b/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp index 27a38e902b3..0f969b33dcc 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp +++ b/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp @@ -80,17 +80,13 @@ void HashPartitionWriterV1::write(const Block & block) template void HashPartitionWriterV1::partitionAndEncodeThenWriteBlocks() { - // return partitionAndEncodeThenWriteBlocksTest(); - if (blocks.empty()) return; // Set mpp packet data version to `1` auto tracked_packets = HashBaseWriterHelper::createPackets(partition_num, DB::MPPDataPacketV1); - // Sum of all approximate block data memory size - size_t ori_block_mem_size = 0; - + size_t ori_all_packets_size = 0; { assert(rows_in_blocks > 0); @@ -101,14 +97,13 @@ void HashPartitionWriterV1::partitionAndEncodeThenWriteBlocks std::vector partition_key_containers(collators.size()); std::vector> dest_columns(partition_num); - [[maybe_unused]] size_t total_rows = 0, encoded_rows = 0; + [[maybe_unused]] size_t total_rows = 0; while (!blocks.empty()) { const auto & block = blocks.back(); block.checkNumberOfRows(); - ori_block_mem_size += ApproxBlockBytes(block); total_rows += block.rows(); auto && dest_tbl_cols = HashBaseWriterHelper::createDestColumns(block, partition_num); @@ -122,27 +117,31 @@ void HashPartitionWriterV1::partitionAndEncodeThenWriteBlocks } } - auto && codec = CHBlockChunkCodecV1{dest_block_header}; + auto && codec = CHBlockChunkCodecV1{ + dest_block_header, + false /*scape empty part*/, + }; for (size_t part_id = 0; part_id < partition_num; ++part_id) { auto & part_columns = dest_columns[part_id]; auto method = writer->isLocal(part_id) ? CompressionMethod::NONE : compression_method; - auto && res = codec.encode(std::move(part_columns), method, false /*scape empty part*/); - if (res.empty()) + auto && res = codec.encode(std::move(part_columns), method); + if unlikely (res.empty()) continue; - encoded_rows += codec.encoded_rows; + tracked_packets[part_id]->getPacket().add_chunks(std::move(res)); - codec.clear(); } - assert(encoded_rows == total_rows); + assert(codec.encoded_rows == total_rows); assert(blocks.empty()); + + ori_all_packets_size += codec.original_size; rows_in_blocks = 0; } writePackets(std::move(tracked_packets)); - GET_METRIC(tiflash_exchange_data_bytes, type_hash_original_all).Increment(ori_block_mem_size); + GET_METRIC(tiflash_exchange_data_bytes, type_hash_original_all).Increment(ori_all_packets_size); } static void updateHashPartitionWriterMetrics(CompressionMethod method, size_t sz, bool is_local) From e145ab7194492ae074cab8fba619d1bbc4536730 Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Tue, 17 Jan 2023 13:10:37 +0800 Subject: [PATCH 79/93] combine FineGrainedShuffleWriter & HashPartitionWriterV1 Signed-off-by: Zhigao Tong --- .../Flash/Mpp/FineGrainedShuffleWriter.cpp | 121 ++++++++-- dbms/src/Flash/Mpp/FineGrainedShuffleWriter.h | 18 +- .../Flash/Mpp/FineGrainedShuffleWriterV1.cpp | 193 ---------------- .../Flash/Mpp/FineGrainedShuffleWriterV1.h | 69 ------ dbms/src/Flash/Mpp/HashBaseWriterHelper.cpp | 15 ++ dbms/src/Flash/Mpp/HashBaseWriterHelper.h | 8 + dbms/src/Flash/Mpp/HashPartitionWriter.cpp | 157 ++++++++++++- dbms/src/Flash/Mpp/HashPartitionWriter.h | 18 +- dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp | 209 ------------------ dbms/src/Flash/Mpp/HashPartitionWriterV1.h | 63 ------ dbms/src/Flash/Mpp/MppVersion.h | 1 + dbms/src/Flash/Mpp/newMPPExchangeWriter.h | 60 +++-- .../Mpp/tests/gtest_mpp_exchange_writer.cpp | 22 +- 13 files changed, 356 insertions(+), 598 deletions(-) delete mode 100644 dbms/src/Flash/Mpp/FineGrainedShuffleWriterV1.cpp delete mode 100644 dbms/src/Flash/Mpp/FineGrainedShuffleWriterV1.h delete mode 100644 dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp delete mode 100644 dbms/src/Flash/Mpp/HashPartitionWriterV1.h diff --git a/dbms/src/Flash/Mpp/FineGrainedShuffleWriter.cpp b/dbms/src/Flash/Mpp/FineGrainedShuffleWriter.cpp index ab8a1dd503d..af42461748b 100644 --- a/dbms/src/Flash/Mpp/FineGrainedShuffleWriter.cpp +++ b/dbms/src/Flash/Mpp/FineGrainedShuffleWriter.cpp @@ -14,6 +14,7 @@ #include #include +#include #include #include #include @@ -22,11 +23,60 @@ namespace DB { -extern size_t ApproxBlockBytes(const Block & block); -} +template +extern void WritePackets(CompressionMethod compression_method, TrackedMppDataPacketPtrs && packets, ExchangeWriterPtr & writer); +template +extern void WritePackets(TrackedMppDataPacketPtrs & packets, ExchangeWriterPtr & writer); +} // namespace DB namespace DB { + +template +FineGrainedShuffleWriter::~FineGrainedShuffleWriter() = default; + +template +void FineGrainedShuffleWriter::batchWriteFineGrainedShuffleImplV1(TrackedMppDataPacketPtrs && tracked_packets) +{ + assert(rows_in_blocks > 0); + + auto && codec = CHBlockChunkCodecV1{ + header, + true /*keep header info*/, + }; + + // serialize each partitioned block and write it to its destination + size_t part_id = 0; + for (size_t bucket_idx = 0; bucket_idx < num_bucket; bucket_idx += fine_grained_shuffle_stream_count, ++part_id) + { + for (uint64_t stream_idx = 0; stream_idx < fine_grained_shuffle_stream_count; ++stream_idx) + { + // assemble scatter columns into a block + MutableColumns columns; + columns.reserve(num_columns); + for (size_t col_id = 0; col_id < num_columns; ++col_id) + columns.emplace_back(std::move(scattered[col_id][bucket_idx + stream_idx])); + + auto method = writer->isLocal(part_id) ? CompressionMethod::NONE : compression_method; + auto && res = codec.encode(columns, method); + assert(!res.empty()); + tracked_packets[part_id]->getPacket().add_chunks(std::move(res)); + tracked_packets[part_id]->getPacket().add_stream_ids(stream_idx); + + for (size_t col_id = 0; col_id < num_columns; ++col_id) + { + columns[col_id]->popBack(columns[col_id]->size()); // clear column + scattered[col_id][bucket_idx + stream_idx] = std::move(columns[col_id]); + } + } + } + assert(codec.encoded_rows == rows_in_blocks); + rows_in_blocks = 0; + + WritePackets(compression_method, std::move(tracked_packets), writer); + GET_METRIC(tiflash_exchange_data_bytes, type_hash_original_all).Increment(codec.original_size); +} + template FineGrainedShuffleWriter::FineGrainedShuffleWriter( ExchangeWriterPtr writer_, @@ -34,7 +84,9 @@ FineGrainedShuffleWriter::FineGrainedShuffleWriter( TiDB::TiDBCollators collators_, DAGContext & dag_context_, uint64_t fine_grained_shuffle_stream_count_, - UInt64 fine_grained_shuffle_batch_size_) + UInt64 fine_grained_shuffle_batch_size_, + MPPDataPacketVersion data_codec_version_, + tipb::CompressionMode compression_mode_) : DAGResponseWriter(/*records_per_chunk=*/-1, dag_context_) , writer(writer_) , partition_col_ids(std::move(partition_col_ids_)) @@ -43,12 +95,27 @@ FineGrainedShuffleWriter::FineGrainedShuffleWriter( , fine_grained_shuffle_batch_size(fine_grained_shuffle_batch_size_) , batch_send_row_limit(fine_grained_shuffle_batch_size * fine_grained_shuffle_stream_count) , hash(0) + , data_codec_version(data_codec_version_) + , compression_method(ToInternalCompressionMethod(compression_mode_)) { rows_in_blocks = 0; partition_num = writer_->getPartitionNum(); RUNTIME_CHECK(partition_num > 0); RUNTIME_CHECK(dag_context.encode_type == tipb::EncodeType::TypeCHBlock); - chunk_codec_stream = std::make_unique()->newCodecStream(dag_context.result_field_types); + + switch (data_codec_version) + { + case MPPDataPacketV0: + { + chunk_codec_stream = std::make_unique()->newCodecStream(dag_context.result_field_types); + break; + } + default: + { + codec_helper_v1 = std::make_unique(dag_context.result_field_types); + break; + } + } } template @@ -116,20 +183,52 @@ void FineGrainedShuffleWriter::initScatterColumns() template void FineGrainedShuffleWriter::batchWriteFineGrainedShuffle() { - auto tracked_packets = HashBaseWriterHelper::createPackets(partition_num, DB::MPPDataPacketV0); - if (likely(!blocks.empty())) + if unlikely (blocks.empty()) + return; + + auto tracked_packets = HashBaseWriterHelper::createPackets(partition_num, data_codec_version); { assert(rows_in_blocks > 0); assert(fine_grained_shuffle_stream_count <= 1024); HashBaseWriterHelper::materializeBlocks(blocks); + + if (codec_helper_v1) + { + for (auto && block : blocks) + { + codec_helper_v1->checkBlock(block); + } + } + while (!blocks.empty()) { const auto & block = blocks.back(); HashBaseWriterHelper::scatterColumnsForFineGrainedShuffle(block, partition_col_ids, collators, partition_key_containers_for_reuse, partition_num, fine_grained_shuffle_stream_count, hash, selector, scattered); blocks.pop_back(); } + } + + switch (data_codec_version) + { + case MPPDataPacketV0: + { + batchWriteFineGrainedShuffleImpl(std::move(tracked_packets)); + break; + } + default: + { + batchWriteFineGrainedShuffleImplV1(std::move(tracked_packets)); + break; + } + } +} +template +void FineGrainedShuffleWriter::batchWriteFineGrainedShuffleImpl(TrackedMppDataPacketPtrs && tracked_packets) +{ + assert(rows_in_blocks > 0); + { // serialize each partitioned block and write it to its destination size_t part_id = 0; for (size_t bucket_idx = 0; bucket_idx < num_bucket; bucket_idx += fine_grained_shuffle_stream_count, ++part_id) @@ -161,17 +260,9 @@ void FineGrainedShuffleWriter::batchWriteFineGrainedShuffle() rows_in_blocks = 0; } - writePackets(tracked_packets); + WritePackets(tracked_packets, writer); } -template -extern void WritePackets(TrackedMppDataPacketPtrs & packets, ExchangeWriterPtr & writer); - -template -void FineGrainedShuffleWriter::writePackets(TrackedMppDataPacketPtrs & packets) -{ - WritePackets(packets, writer); -} template class FineGrainedShuffleWriter; diff --git a/dbms/src/Flash/Mpp/FineGrainedShuffleWriter.h b/dbms/src/Flash/Mpp/FineGrainedShuffleWriter.h index 6b2db46770c..c98131ee812 100644 --- a/dbms/src/Flash/Mpp/FineGrainedShuffleWriter.h +++ b/dbms/src/Flash/Mpp/FineGrainedShuffleWriter.h @@ -19,9 +19,16 @@ #include #include +namespace DB::HashBaseWriterHelper +{ +struct HashPartitionWriterHelperV1; +} + namespace DB { class DAGContext; +enum class CompressionMethod; +enum MPPDataPacketVersion : int64_t; template class FineGrainedShuffleWriter : public DAGResponseWriter @@ -33,13 +40,18 @@ class FineGrainedShuffleWriter : public DAGResponseWriter TiDB::TiDBCollators collators_, DAGContext & dag_context_, UInt64 fine_grained_shuffle_stream_count_, - UInt64 fine_grained_shuffle_batch_size); + UInt64 fine_grained_shuffle_batch_size, + MPPDataPacketVersion data_codec_version_, + tipb::CompressionMode compression_mode_); void prepare(const Block & sample_block) override; void write(const Block & block) override; void flush() override; + ~FineGrainedShuffleWriter() override; private: void batchWriteFineGrainedShuffle(); + void batchWriteFineGrainedShuffleImpl(TrackedMppDataPacketPtrs &&); + void batchWriteFineGrainedShuffleImplV1(TrackedMppDataPacketPtrs &&); void writePackets(TrackedMppDataPacketPtrs & packets); @@ -63,6 +75,10 @@ class FineGrainedShuffleWriter : public DAGResponseWriter WeakHash32 hash; IColumn::Selector selector; std::vector scattered; // size = num_columns + // support data compression + MPPDataPacketVersion data_codec_version; + CompressionMethod compression_method{}; + std::unique_ptr codec_helper_v1{}; }; } // namespace DB diff --git a/dbms/src/Flash/Mpp/FineGrainedShuffleWriterV1.cpp b/dbms/src/Flash/Mpp/FineGrainedShuffleWriterV1.cpp deleted file mode 100644 index db41ba73ab7..00000000000 --- a/dbms/src/Flash/Mpp/FineGrainedShuffleWriterV1.cpp +++ /dev/null @@ -1,193 +0,0 @@ -// Copyright 2022 PingCAP, Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -#include -#include -#include -#include -#include -#include -#include - -#include - -namespace DB -{ -extern size_t ApproxBlockBytes(const Block & block); -} // namespace DB - -namespace DB -{ -template -FineGrainedShuffleWriterV1::FineGrainedShuffleWriterV1( - ExchangeWriterPtr writer_, - std::vector partition_col_ids_, - TiDB::TiDBCollators collators_, - DAGContext & dag_context_, - uint64_t fine_grained_shuffle_stream_count_, - UInt64 fine_grained_shuffle_batch_size_, - tipb::CompressionMode compression_mode_) - : DAGResponseWriter(/*records_per_chunk=*/-1, dag_context_) - , writer(writer_) - , partition_col_ids(std::move(partition_col_ids_)) - , collators(std::move(collators_)) - , fine_grained_shuffle_stream_count(fine_grained_shuffle_stream_count_) - , fine_grained_shuffle_batch_size(fine_grained_shuffle_batch_size_) - , batch_send_row_limit(fine_grained_shuffle_batch_size * fine_grained_shuffle_stream_count) - , hash(0) - , compression_method(ToInternalCompressionMethod(compression_mode_)) - -{ - rows_in_blocks = 0; - partition_num = writer_->getPartitionNum(); - RUNTIME_CHECK(partition_num > 0); - RUNTIME_CHECK(dag_context.encode_type == tipb::EncodeType::TypeCHBlock); -} - -template -void FineGrainedShuffleWriterV1::prepare(const Block & sample_block) -{ - /// Initialize header block, use column type to create new empty column to handle potential null column cases - const auto & column_with_type_and_names = sample_block.getColumnsWithTypeAndName(); - for (const auto & column : column_with_type_and_names) - { - MutableColumnPtr empty_column = column.type->createColumn(); - ColumnWithTypeAndName new_column(std::move(empty_column), column.type, column.name); - header.insert(new_column); - } - num_columns = header.columns(); - // fine_grained_shuffle_stream_count is in (0, 1024], and partition_num is uint16_t, so will not overflow. - num_bucket = partition_num * fine_grained_shuffle_stream_count; - partition_key_containers_for_reuse.resize(collators.size()); - initScatterColumns(); - prepared = true; -} - -template -void FineGrainedShuffleWriterV1::flush() -{ - if (rows_in_blocks > 0) - batchWriteFineGrainedShuffle(); -} - -template -void FineGrainedShuffleWriterV1::write(const Block & block) -{ - RUNTIME_CHECK_MSG(prepared, "FineGrainedShuffleWriterV1 should be prepared before writing."); - RUNTIME_CHECK_MSG( - block.columns() == dag_context.result_field_types.size(), - "Output column size mismatch with field type size"); - - size_t rows = block.rows(); - if (rows > 0) - { - rows_in_blocks += rows; - blocks.push_back(block); - } - - if (blocks.size() == fine_grained_shuffle_stream_count || static_cast(rows_in_blocks) >= batch_send_row_limit) - batchWriteFineGrainedShuffle(); -} - -template -void FineGrainedShuffleWriterV1::initScatterColumns() -{ - scattered.resize(num_columns); - for (size_t col_id = 0; col_id < num_columns; ++col_id) - { - auto & column = header.getByPosition(col_id).column; - - scattered[col_id].reserve(num_bucket); - for (size_t chunk_id = 0; chunk_id < num_bucket; ++chunk_id) - { - scattered[col_id].emplace_back(column->cloneEmpty()); - scattered[col_id][chunk_id]->reserve(1024); - } - } -} - -template -void FineGrainedShuffleWriterV1::batchWriteFineGrainedShuffle() -{ - if (blocks.empty()) - return; - size_t ori_all_packets_size = 0; - auto tracked_packets = HashBaseWriterHelper::createPackets(partition_num, DB::MPPDataPacketV1); - - { - assert(rows_in_blocks > 0); - assert(fine_grained_shuffle_stream_count <= 1024); - - HashBaseWriterHelper::materializeBlocks(blocks); - [[maybe_unused]] size_t total_rows = 0; - - while (!blocks.empty()) - { - const auto & block = blocks.back(); - total_rows += block.rows(); - HashBaseWriterHelper::scatterColumnsForFineGrainedShuffle(block, partition_col_ids, collators, partition_key_containers_for_reuse, partition_num, fine_grained_shuffle_stream_count, hash, selector, scattered); - blocks.pop_back(); - } - - auto && codec = CHBlockChunkCodecV1{ - header, - true /*keep header info*/, - }; - - // serialize each partitioned block and write it to its destination - size_t part_id = 0; - for (size_t bucket_idx = 0; bucket_idx < num_bucket; bucket_idx += fine_grained_shuffle_stream_count, ++part_id) - { - for (uint64_t stream_idx = 0; stream_idx < fine_grained_shuffle_stream_count; ++stream_idx) - { - // assemble scatter columns into a block - MutableColumns columns; - columns.reserve(num_columns); - for (size_t col_id = 0; col_id < num_columns; ++col_id) - columns.emplace_back(std::move(scattered[col_id][bucket_idx + stream_idx])); - - auto method = writer->isLocal(part_id) ? CompressionMethod::NONE : compression_method; - auto && res = codec.encode(columns, method); - assert(!res.empty()); - tracked_packets[part_id]->getPacket().add_chunks(std::move(res)); - tracked_packets[part_id]->getPacket().add_stream_ids(stream_idx); - - for (size_t col_id = 0; col_id < num_columns; ++col_id) - { - columns[col_id]->popBack(columns[col_id]->size()); // clear column - scattered[col_id][bucket_idx + stream_idx] = std::move(columns[col_id]); - } - } - } - assert(codec.encoded_rows == total_rows); - ori_all_packets_size += codec.original_size; - rows_in_blocks = 0; - } - - writePackets(tracked_packets); - GET_METRIC(tiflash_exchange_data_bytes, type_hash_original_all).Increment(ori_all_packets_size); -} - -template -extern void WritePackets(CompressionMethod compression_method, TrackedMppDataPacketPtrs && packets, ExchangeWriterPtr & writer); - -template -void FineGrainedShuffleWriterV1::writePackets(TrackedMppDataPacketPtrs & packets) -{ - WritePackets(compression_method, std::move(packets), writer); -} - -template class FineGrainedShuffleWriterV1; - -} // namespace DB diff --git a/dbms/src/Flash/Mpp/FineGrainedShuffleWriterV1.h b/dbms/src/Flash/Mpp/FineGrainedShuffleWriterV1.h deleted file mode 100644 index 47601ad57d0..00000000000 --- a/dbms/src/Flash/Mpp/FineGrainedShuffleWriterV1.h +++ /dev/null @@ -1,69 +0,0 @@ -// Copyright 2022 PingCAP, Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -#pragma once - -#include -#include -#include -#include - -namespace DB -{ -class DAGContext; -enum class CompressionMethod; -template -class FineGrainedShuffleWriterV1 : public DAGResponseWriter -{ -public: - FineGrainedShuffleWriterV1( - ExchangeWriterPtr writer_, - std::vector partition_col_ids_, - TiDB::TiDBCollators collators_, - DAGContext & dag_context_, - UInt64 fine_grained_shuffle_stream_count_, - UInt64 fine_grained_shuffle_batch_size, - tipb::CompressionMode compression_mode_); - void prepare(const Block & sample_block) override; - void write(const Block & block) override; - void flush() override; - -private: - void batchWriteFineGrainedShuffle(); - - void writePackets(TrackedMppDataPacketPtrs & packets); - - void initScatterColumns(); - -private: - ExchangeWriterPtr writer; - std::vector blocks; - std::vector partition_col_ids; - TiDB::TiDBCollators collators; - size_t rows_in_blocks = 0; - uint16_t partition_num; - UInt64 fine_grained_shuffle_stream_count; - UInt64 fine_grained_shuffle_batch_size; - - Block header; - bool prepared = false; - size_t num_columns = 0, num_bucket = 0, batch_send_row_limit = 0; // Assign they initial values to pass clang-tidy check, they will be initialized in prepare method - std::vector partition_key_containers_for_reuse; - WeakHash32 hash; - IColumn::Selector selector; - std::vector scattered; // size = num_columns - CompressionMethod compression_method{}; -}; - -} // namespace DB diff --git a/dbms/src/Flash/Mpp/HashBaseWriterHelper.cpp b/dbms/src/Flash/Mpp/HashBaseWriterHelper.cpp index cdd7bcdb1c5..751966711e0 100644 --- a/dbms/src/Flash/Mpp/HashBaseWriterHelper.cpp +++ b/dbms/src/Flash/Mpp/HashBaseWriterHelper.cpp @@ -12,7 +12,9 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include #include +#include namespace DB::HashBaseWriterHelper { @@ -181,4 +183,17 @@ void scatterColumnsForFineGrainedShuffle(const Block & block, } } +HashPartitionWriterHelperV1::HashPartitionWriterHelperV1(const std::vector & field_types) +{ + for (const auto & field_type : field_types) + { + expected_types.emplace_back(getDataTypeByFieldTypeForComputingLayer(field_type)); + } +} +void HashPartitionWriterHelperV1::checkBlock(const Block & block) const +{ + DB::assertBlockSchema(expected_types, block, "HashPartitionWriterHelper"); + block.checkNumberOfRows(); +} + } // namespace DB::HashBaseWriterHelper diff --git a/dbms/src/Flash/Mpp/HashBaseWriterHelper.h b/dbms/src/Flash/Mpp/HashBaseWriterHelper.h index 645e20a2035..d42a12eceee 100644 --- a/dbms/src/Flash/Mpp/HashBaseWriterHelper.h +++ b/dbms/src/Flash/Mpp/HashBaseWriterHelper.h @@ -57,4 +57,12 @@ void scatterColumnsForFineGrainedShuffle(const Block & block, WeakHash32 & hash, IColumn::Selector & selector, std::vector & scattered); + +// Used to hold expected types for codec +struct HashPartitionWriterHelperV1 +{ + DataTypes expected_types; + explicit HashPartitionWriterHelperV1(const std::vector & field_types); + void checkBlock(const Block & block) const; +}; } // namespace DB::HashBaseWriterHelper diff --git a/dbms/src/Flash/Mpp/HashPartitionWriter.cpp b/dbms/src/Flash/Mpp/HashPartitionWriter.cpp index da509ffe741..3dcd53add7a 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriter.cpp +++ b/dbms/src/Flash/Mpp/HashPartitionWriter.cpp @@ -14,6 +14,7 @@ #include #include +#include #include #include #include @@ -22,28 +23,154 @@ namespace DB { -extern size_t ApproxBlockBytes(const Block & block); +static void updateHashPartitionWriterMetrics(CompressionMethod method, size_t sz, bool is_local) +{ + if (is_local) + { + method = CompressionMethod::NONE; + } + + switch (method) + { + case CompressionMethod::NONE: + { + if (is_local) + { + GET_METRIC(tiflash_exchange_data_bytes, type_hash_none_local).Increment(sz); + } + else + { + GET_METRIC(tiflash_exchange_data_bytes, type_hash_none_remote).Increment(sz); + } + break; + } + case CompressionMethod::LZ4: + { + GET_METRIC(tiflash_exchange_data_bytes, type_hash_lz4).Increment(sz); + break; + } + case CompressionMethod::ZSTD: + { + GET_METRIC(tiflash_exchange_data_bytes, type_hash_zstd).Increment(sz); + break; + } + default: + break; + } } -namespace DB + +template +void WritePackets(CompressionMethod compression_method, TrackedMppDataPacketPtrs && packets, ExchangeWriterPtr & writer) { + for (size_t part_id = 0; part_id < packets.size(); ++part_id) + { + auto & packet = packets[part_id]; + assert(packet); + + auto & inner_packet = packet->getPacket(); + + if (auto sz = inner_packet.ByteSizeLong(); likely(inner_packet.chunks_size() > 0)) + { + writer->partitionWrite(std::move(packet), part_id); + updateHashPartitionWriterMetrics(compression_method, sz, writer->isLocal(part_id)); + } + } +} + +template +void HashPartitionWriter::partitionAndEncodeThenWriteBlocksImplV1() +{ + assert(rows_in_blocks > 0); + assert(codec_helper_v1); + + auto tracked_packets = HashBaseWriterHelper::createPackets(partition_num, data_codec_version); + HashBaseWriterHelper::materializeBlocks(blocks); + // All blocks are same, use one block's meta info as header + Block dest_block_header = blocks.back().cloneEmpty(); + std::vector partition_key_containers(collators.size()); + std::vector> dest_columns(partition_num); + + while (!blocks.empty()) + { + const auto & block = blocks.back(); + codec_helper_v1->checkBlock(block); + auto && dest_tbl_cols = HashBaseWriterHelper::createDestColumns(block, partition_num); + HashBaseWriterHelper::scatterColumns(block, partition_col_ids, collators, partition_key_containers, partition_num, dest_tbl_cols); + blocks.pop_back(); + + for (size_t part_id = 0; part_id < partition_num; ++part_id) + { + auto & columns = dest_tbl_cols[part_id]; + dest_columns[part_id].emplace_back(std::move(columns)); + } + } + + auto && codec = CHBlockChunkCodecV1{ + dest_block_header, + false /*scape empty part*/, + }; + + for (size_t part_id = 0; part_id < partition_num; ++part_id) + { + auto & part_columns = dest_columns[part_id]; + auto method = writer->isLocal(part_id) ? CompressionMethod::NONE : compression_method; + auto && res = codec.encode(std::move(part_columns), method); + if unlikely (res.empty()) + continue; + + tracked_packets[part_id]->getPacket().add_chunks(std::move(res)); + } + assert(codec.encoded_rows == rows_in_blocks); + assert(blocks.empty()); + + rows_in_blocks = 0; + + WritePackets(compression_method, std::move(tracked_packets), writer); + GET_METRIC(tiflash_exchange_data_bytes, type_hash_original_all).Increment(codec.original_size); +} + +template +HashPartitionWriter::~HashPartitionWriter() = default; + template HashPartitionWriter::HashPartitionWriter( ExchangeWriterPtr writer_, std::vector partition_col_ids_, TiDB::TiDBCollators collators_, Int64 batch_send_min_limit_, - DAGContext & dag_context_) + DAGContext & dag_context_, + MPPDataPacketVersion data_codec_version_, + tipb::CompressionMode compression_mode_) : DAGResponseWriter(/*records_per_chunk=*/-1, dag_context_) , batch_send_min_limit(batch_send_min_limit_) , writer(writer_) , partition_col_ids(std::move(partition_col_ids_)) , collators(std::move(collators_)) + , data_codec_version(data_codec_version_) + , compression_method(ToInternalCompressionMethod(compression_mode_)) { rows_in_blocks = 0; partition_num = writer_->getPartitionNum(); RUNTIME_CHECK(partition_num > 0); RUNTIME_CHECK(dag_context.encode_type == tipb::EncodeType::TypeCHBlock); - chunk_codec_stream = std::make_unique()->newCodecStream(dag_context.result_field_types); + + switch (data_codec_version) + { + case MPPDataPacketV0: + { + chunk_codec_stream = std::make_unique()->newCodecStream(dag_context.result_field_types); + break; + } + default: + { + if (batch_send_min_limit < 0) + { + batch_send_min_limit = 8192LL * partition_num; + } + codec_helper_v1 = std::make_unique(dag_context.result_field_types); + break; + } + } } template @@ -73,9 +200,27 @@ void HashPartitionWriter::write(const Block & block) template void HashPartitionWriter::partitionAndEncodeThenWriteBlocks() { - auto tracked_packets = HashBaseWriterHelper::createPackets(partition_num, DB::MPPDataPacketV0); + if unlikely (blocks.empty()) + return; + switch (data_codec_version) + { + case MPPDataPacketV0: + { + partitionAndEncodeThenWriteBlocksImpl(); + break; + } + default: + { + partitionAndEncodeThenWriteBlocksImplV1(); + break; + } + } +} - if (!blocks.empty()) +template +void HashPartitionWriter::partitionAndEncodeThenWriteBlocksImpl() +{ + auto tracked_packets = HashBaseWriterHelper::createPackets(partition_num, data_codec_version); { assert(rows_in_blocks > 0); diff --git a/dbms/src/Flash/Mpp/HashPartitionWriter.h b/dbms/src/Flash/Mpp/HashPartitionWriter.h index deebcd3dce7..c97c4aa84f6 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriter.h +++ b/dbms/src/Flash/Mpp/HashPartitionWriter.h @@ -19,25 +19,37 @@ #include #include +namespace DB::HashBaseWriterHelper +{ +struct HashPartitionWriterHelperV1; +} namespace DB { class DAGContext; +enum class CompressionMethod; +enum MPPDataPacketVersion : int64_t; template class HashPartitionWriter : public DAGResponseWriter { public: + // If `batch_send_min_limit_` is LT 0, `batch_send_min_limit` will be set to `8192 * partition_num` HashPartitionWriter( ExchangeWriterPtr writer_, std::vector partition_col_ids_, TiDB::TiDBCollators collators_, Int64 batch_send_min_limit_, - DAGContext & dag_context_); + DAGContext & dag_context_, + MPPDataPacketVersion data_codec_version, + tipb::CompressionMode compression_mode_); void write(const Block & block) override; void flush() override; + ~HashPartitionWriter() override; private: void partitionAndEncodeThenWriteBlocks(); + void partitionAndEncodeThenWriteBlocksImpl(); + void partitionAndEncodeThenWriteBlocksImplV1(); void writePackets(TrackedMppDataPacketPtrs & packets); @@ -50,6 +62,10 @@ class HashPartitionWriter : public DAGResponseWriter size_t rows_in_blocks; uint16_t partition_num; std::unique_ptr chunk_codec_stream; + // support data compression + MPPDataPacketVersion data_codec_version; + CompressionMethod compression_method{}; + std::unique_ptr codec_helper_v1{}; }; } // namespace DB diff --git a/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp b/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp deleted file mode 100644 index 0f969b33dcc..00000000000 --- a/dbms/src/Flash/Mpp/HashPartitionWriterV1.cpp +++ /dev/null @@ -1,209 +0,0 @@ -// Copyright 2022 PingCAP, Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ -extern size_t ApproxBlockBytes(const Block & block); -} // namespace DB - -namespace DB -{ -template -HashPartitionWriterV1::HashPartitionWriterV1( - ExchangeWriterPtr writer_, - std::vector partition_col_ids_, - TiDB::TiDBCollators collators_, - Int64 partition_batch_limit_, - DAGContext & dag_context_, - tipb::CompressionMode compression_mode_) - : DAGResponseWriter(/*records_per_chunk=*/-1, dag_context_) - , partition_num(writer_->getPartitionNum()) - , partition_batch_limit(partition_batch_limit_) - , writer(writer_) - , partition_col_ids(std::move(partition_col_ids_)) - , collators(std::move(collators_)) - , compression_method(ToInternalCompressionMethod(compression_mode_)) -{ - if (partition_batch_limit < 0) - { - partition_batch_limit = 8192LL * partition_num; - } - - rows_in_blocks = 0; - RUNTIME_CHECK(partition_num > 0); - RUNTIME_CHECK(dag_context.encode_type == tipb::EncodeType::TypeCHBlock); -} - -template -void HashPartitionWriterV1::flush() -{ - if (rows_in_blocks > 0) - partitionAndEncodeThenWriteBlocks(); -} - -template -void HashPartitionWriterV1::write(const Block & block) -{ - RUNTIME_CHECK_MSG( - block.columns() == dag_context.result_field_types.size(), - "Output column size mismatch with field type size"); - size_t rows = block.rows(); - rows_in_blocks += rows; - if (rows > 0) - { - blocks.push_back(block); - } - - if (static_cast(rows_in_blocks) > partition_batch_limit) - partitionAndEncodeThenWriteBlocks(); -} - -template -void HashPartitionWriterV1::partitionAndEncodeThenWriteBlocks() -{ - if (blocks.empty()) - return; - - // Set mpp packet data version to `1` - auto tracked_packets = HashBaseWriterHelper::createPackets(partition_num, DB::MPPDataPacketV1); - - size_t ori_all_packets_size = 0; - { - assert(rows_in_blocks > 0); - - HashBaseWriterHelper::materializeBlocks(blocks); - - // All blocks are same, use one block's meta info as header - Block dest_block_header = blocks.back().cloneEmpty(); - - std::vector partition_key_containers(collators.size()); - std::vector> dest_columns(partition_num); - [[maybe_unused]] size_t total_rows = 0; - - while (!blocks.empty()) - { - const auto & block = blocks.back(); - block.checkNumberOfRows(); - - total_rows += block.rows(); - - auto && dest_tbl_cols = HashBaseWriterHelper::createDestColumns(block, partition_num); - HashBaseWriterHelper::scatterColumns(block, partition_col_ids, collators, partition_key_containers, partition_num, dest_tbl_cols); - blocks.pop_back(); - - for (size_t part_id = 0; part_id < partition_num; ++part_id) - { - auto & columns = dest_tbl_cols[part_id]; - dest_columns[part_id].emplace_back(std::move(columns)); - } - } - - auto && codec = CHBlockChunkCodecV1{ - dest_block_header, - false /*scape empty part*/, - }; - - for (size_t part_id = 0; part_id < partition_num; ++part_id) - { - auto & part_columns = dest_columns[part_id]; - auto method = writer->isLocal(part_id) ? CompressionMethod::NONE : compression_method; - auto && res = codec.encode(std::move(part_columns), method); - if unlikely (res.empty()) - continue; - - tracked_packets[part_id]->getPacket().add_chunks(std::move(res)); - } - assert(codec.encoded_rows == total_rows); - assert(blocks.empty()); - - ori_all_packets_size += codec.original_size; - rows_in_blocks = 0; - } - - writePackets(std::move(tracked_packets)); - - GET_METRIC(tiflash_exchange_data_bytes, type_hash_original_all).Increment(ori_all_packets_size); -} - -static void updateHashPartitionWriterMetrics(CompressionMethod method, size_t sz, bool is_local) -{ - if (is_local) - { - method = CompressionMethod::NONE; - } - - switch (method) - { - case CompressionMethod::NONE: - { - if (is_local) - { - GET_METRIC(tiflash_exchange_data_bytes, type_hash_none_local).Increment(sz); - } - else - { - GET_METRIC(tiflash_exchange_data_bytes, type_hash_none_remote).Increment(sz); - } - break; - } - case CompressionMethod::LZ4: - { - GET_METRIC(tiflash_exchange_data_bytes, type_hash_lz4).Increment(sz); - break; - } - case CompressionMethod::ZSTD: - { - GET_METRIC(tiflash_exchange_data_bytes, type_hash_zstd).Increment(sz); - break; - } - default: - break; - } -} - -template -void WritePackets(CompressionMethod compression_method, TrackedMppDataPacketPtrs && packets, ExchangeWriterPtr & writer) -{ - for (size_t part_id = 0; part_id < packets.size(); ++part_id) - { - auto & packet = packets[part_id]; - assert(packet); - - auto & inner_packet = packet->getPacket(); - - if (auto sz = inner_packet.ByteSizeLong(); likely(inner_packet.chunks_size() > 0)) - { - writer->partitionWrite(std::move(packet), part_id); - updateHashPartitionWriterMetrics(compression_method, sz, writer->isLocal(part_id)); - } - } -} - -template -void HashPartitionWriterV1::writePackets(TrackedMppDataPacketPtrs && packets) -{ - WritePackets(compression_method, std::move(packets), writer); -} - -template class HashPartitionWriterV1; - -} // namespace DB diff --git a/dbms/src/Flash/Mpp/HashPartitionWriterV1.h b/dbms/src/Flash/Mpp/HashPartitionWriterV1.h deleted file mode 100644 index 028e909ba08..00000000000 --- a/dbms/src/Flash/Mpp/HashPartitionWriterV1.h +++ /dev/null @@ -1,63 +0,0 @@ -// Copyright 2022 PingCAP, Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -#pragma once - -#include -#include -#include -#include -#include - -namespace tipb -{ -enum CompressionMode : int; -} - -namespace DB -{ -enum class CompressionMethod; -template -class HashPartitionWriterV1 : public DAGResponseWriter -{ -public: - // If `partition_batch_limit_` is LT 0, `partition_batch_limit` will be set to `8192 * partition_num` - HashPartitionWriterV1( - ExchangeWriterPtr writer_, - std::vector partition_col_ids_, - TiDB::TiDBCollators collators_, - Int64 partition_batch_limit_, - DAGContext & dag_context_, - tipb::CompressionMode compression_mode_); - void write(const Block & block) override; - void flush() override; - -private: - void partitionAndEncodeThenWriteBlocks(); - void partitionAndEncodeThenWriteBlocksTest(); - - void writePackets(TrackedMppDataPacketPtrs && packets); - -private: - uint16_t partition_num; - Int64 partition_batch_limit; - ExchangeWriterPtr writer; - std::vector blocks; - std::vector partition_col_ids; - TiDB::TiDBCollators collators; - size_t rows_in_blocks; - CompressionMethod compression_method{}; -}; - -} // namespace DB diff --git a/dbms/src/Flash/Mpp/MppVersion.h b/dbms/src/Flash/Mpp/MppVersion.h index 2a4880fb923..b3ca0546fb5 100644 --- a/dbms/src/Flash/Mpp/MppVersion.h +++ b/dbms/src/Flash/Mpp/MppVersion.h @@ -30,6 +30,7 @@ enum MPPDataPacketVersion : int64_t { MPPDataPacketV0 = 0, MPPDataPacketV1, + // MPPDataPacketMAX, }; diff --git a/dbms/src/Flash/Mpp/newMPPExchangeWriter.h b/dbms/src/Flash/Mpp/newMPPExchangeWriter.h index 841e80cef90..162ccede84a 100644 --- a/dbms/src/Flash/Mpp/newMPPExchangeWriter.h +++ b/dbms/src/Flash/Mpp/newMPPExchangeWriter.h @@ -17,9 +17,7 @@ #include #include #include -#include #include -#include #include namespace DB @@ -57,43 +55,37 @@ std::unique_ptr NewMPPExchangeWriter( { if (exchange_type == tipb::ExchangeType::Hash) { + auto mpp_version = dag_context.getMPPTaskMeta().mpp_version(); + auto data_codec_version = mpp_version == MppVersionV0 + ? MPPDataPacketV0 + : MPPDataPacketV1; + if (enable_fine_grained_shuffle) { - if (DB::MppVersion::MppVersionV0 == dag_context.getMPPTaskMeta().mpp_version()) - return std::make_unique>( - writer, - partition_col_ids, - partition_col_collators, - dag_context, - fine_grained_shuffle_stream_count, - fine_grained_shuffle_batch_size); - else - return std::make_unique>( - writer, - partition_col_ids, - partition_col_collators, - dag_context, - fine_grained_shuffle_stream_count, - fine_grained_shuffle_batch_size, - compression_mode); + return std::make_unique>( + writer, + partition_col_ids, + partition_col_collators, + dag_context, + fine_grained_shuffle_stream_count, + fine_grained_shuffle_batch_size, + data_codec_version, + compression_mode); } else { - if (DB::MppVersion::MppVersionV0 == dag_context.getMPPTaskMeta().mpp_version()) - return std::make_unique>( - writer, - partition_col_ids, - partition_col_collators, - batch_send_min_limit, - dag_context); - else - return std::make_unique>( - writer, - partition_col_ids, - partition_col_collators, - batch_send_min_limit_compression, - dag_context, - compression_mode); + auto chosen_batch_send_min_limit = mpp_version == MppVersionV0 + ? batch_send_min_limit + : batch_send_min_limit_compression; + + return std::make_unique>( + writer, + partition_col_ids, + partition_col_collators, + chosen_batch_send_min_limit, + dag_context, + data_codec_version, + compression_mode); } } else diff --git a/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp b/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp index 53b1d0c682a..14c0b3dee70 100644 --- a/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp +++ b/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp @@ -26,9 +26,7 @@ #include #include -#include #include -#include #include namespace DB @@ -182,7 +180,9 @@ try part_col_collators, *dag_context_ptr, fine_grained_shuffle_stream_count, - fine_grained_shuffle_batch_size); + fine_grained_shuffle_batch_size, + DB::MPPDataPacketV0, + tipb::CompressionMode::NONE); dag_writer->prepare(block.cloneEmpty()); dag_writer->write(block); dag_writer->flush(); @@ -237,13 +237,14 @@ try auto mock_writer = std::make_shared(checker, part_num); // 3. Start to write. - auto dag_writer = std::make_shared>>( + auto dag_writer = std::make_shared>>( mock_writer, part_col_ids, part_col_collators, *dag_context_ptr, fine_grained_shuffle_stream_count, fine_grained_shuffle_batch_size, + DB::MPPDataPacketV1, mode); dag_writer->prepare(blocks[0].cloneEmpty()); for (const auto & block : blocks) @@ -264,6 +265,8 @@ try for (const auto & packet : write_report[part_index]) { ASSERT_EQ(packet->getPacket().chunks_size(), packet->getPacket().stream_ids_size()); + ASSERT_EQ(DB::MPPDataPacketV1, packet->getPacket().version()); + for (int i = 0; i < packet->getPacket().chunks_size(); ++i) { const auto & chunk = packet->getPacket().chunks(i); @@ -330,7 +333,9 @@ try part_col_collators, *dag_context_ptr, fine_grained_shuffle_stream_count, - fine_grained_shuffle_batch_size); + fine_grained_shuffle_batch_size, + DB::MPPDataPacketV0, + tipb::CompressionMode::NONE); dag_writer->prepare(blocks[0].cloneEmpty()); for (const auto & block : blocks) dag_writer->write(block); @@ -391,7 +396,9 @@ try part_col_ids, part_col_collators, batch_send_min_limit, - *dag_context_ptr); + *dag_context_ptr, + DB::MPPDataPacketV0, + tipb::CompressionMode::NONE); for (const auto & block : blocks) dag_writer->write(block); dag_writer->flush(); @@ -506,12 +513,13 @@ try auto mock_writer = std::make_shared(checker, part_num); // 3. Start to write. - auto dag_writer = std::make_shared>>( + auto dag_writer = std::make_shared>>( mock_writer, part_col_ids, part_col_collators, batch_send_min_limit, *dag_context_ptr, + DB::MPPDataPacketV1, mode); for (const auto & block : blocks) dag_writer->write(block); From 22f242f2ffe7e9de9bd2d6605af80b95d1b18748 Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Tue, 17 Jan 2023 16:17:28 +0800 Subject: [PATCH 80/93] 69 Signed-off-by: Zhigao Tong --- dbms/src/Storages/Transaction/ReadIndexWorker.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/Transaction/ReadIndexWorker.cpp b/dbms/src/Storages/Transaction/ReadIndexWorker.cpp index 27ad9e14d81..b3e342daf29 100644 --- a/dbms/src/Storages/Transaction/ReadIndexWorker.cpp +++ b/dbms/src/Storages/Transaction/ReadIndexWorker.cpp @@ -42,7 +42,7 @@ static std::mutex global_logger_mutex; auto _ = std::lock_guard(global_logger_mutex); \ std::cout << fmt::format( \ "[{}][{}:{}][{}]", \ - std::chrono::steady_clock::now(), \ + Clock::now(), \ &__FILE__[LogFmtDetails::getFileNameOffset(__FILE__)], \ __LINE__, \ formatted_message) \ From e3724f126f8e36c18bf7f70667c7e961e48d3c8e Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Tue, 17 Jan 2023 18:35:19 +0800 Subject: [PATCH 81/93] rebase to pingcap/master Signed-off-by: Zhigao Tong --- .../Flash/Coprocessor/CHBlockChunkCodecV1.h | 31 +++- .../gtest_ti_remote_block_inputstream.cpp | 5 +- dbms/src/Flash/Mpp/ExchangeReceiver.cpp | 6 +- .../Flash/Mpp/FineGrainedShuffleWriter.cpp | 62 +++++++- dbms/src/Flash/Mpp/FineGrainedShuffleWriter.h | 8 +- dbms/src/Flash/Mpp/HashPartitionWriter.cpp | 148 ++++++++++++++++-- dbms/src/Flash/Mpp/HashPartitionWriter.h | 15 +- dbms/src/Flash/Mpp/MPPTunnelSet.cpp | 142 +++++++++++++++-- dbms/src/Flash/Mpp/MPPTunnelSet.h | 25 +-- dbms/src/Flash/Mpp/MPPTunnelSetHelper.cpp | 82 +++++++++- dbms/src/Flash/Mpp/MPPTunnelSetHelper.h | 30 +++- dbms/src/Flash/Mpp/TrackedMppDataPacket.h | 12 +- .../Mpp/tests/gtest_mpp_exchange_writer.cpp | 74 ++++++--- dbms/src/Flash/Mpp/tests/gtest_mpptunnel.cpp | 2 +- 14 files changed, 547 insertions(+), 95 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.h b/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.h index 285e56e2291..929267e3fc1 100644 --- a/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.h +++ b/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.h @@ -83,24 +83,41 @@ struct CHBlockChunkCodecV1 } template - static void getColumnEncodeInfoImpl(ColumnsHolder && columns_holder, size_t & bytes, size_t & rows) + static void getColumnEncodeInfoImpl(ColumnsHolder && columns_holder, size_t & bytes, size_t & total_rows) { bytes += 8 /*rows*/; if constexpr (isBlockType()) { - rows += columns_holder.rows(); - for (size_t col_index = 0; col_index < columns_holder.columns(); ++col_index) + const Block & block = columns_holder; + const auto rows = block.rows(); + total_rows += rows; + for (size_t col_index = 0; col_index < block.columns(); ++col_index) { - auto && col_type_name = columns_holder.getByPosition(col_index); + auto && col_type_name = block.getByPosition(col_index); bytes += col_type_name.column->byteSize(); + RUNTIME_ASSERT(rows == col_type_name.column->size()); } } else { - rows += columns_holder.front()->size(); - for (const auto & elem : columns_holder) - bytes += elem->byteSize(); + if (columns_holder.front()) + { + const auto rows = columns_holder.front()->size(); + total_rows += rows; + for (const auto & elem : columns_holder) + { + bytes += elem->byteSize(); + RUNTIME_ASSERT(rows == elem->size()); + } + } + else + { + for (const auto & elem : columns_holder) + { + RUNTIME_ASSERT(!elem); + } + } } } static const ColumnPtr & toColumnPtr(const Columns & c, size_t index) diff --git a/dbms/src/Flash/Coprocessor/tests/gtest_ti_remote_block_inputstream.cpp b/dbms/src/Flash/Coprocessor/tests/gtest_ti_remote_block_inputstream.cpp index 8f91b5e6ffc..4daeb64e3ed 100644 --- a/dbms/src/Flash/Coprocessor/tests/gtest_ti_remote_block_inputstream.cpp +++ b/dbms/src/Flash/Coprocessor/tests/gtest_ti_remote_block_inputstream.cpp @@ -28,11 +28,10 @@ #include #include -#include #include #include #include -#include + namespace DB { @@ -85,7 +84,7 @@ struct MockWriter void broadcastOrPassThroughWrite(Blocks & blocks) { - auto packet = MPPTunnelSetHelper::toPacket(blocks, result_field_types); + auto packet = MPPTunnelSetHelper::toPacket(blocks, result_field_types, MPPDataPacketV0); ++total_packets; if (!packet->packet.chunks().empty()) total_bytes += packet->packet.ByteSizeLong(); diff --git a/dbms/src/Flash/Mpp/ExchangeReceiver.cpp b/dbms/src/Flash/Mpp/ExchangeReceiver.cpp index d219fa6effc..322a0e11c5c 100644 --- a/dbms/src/Flash/Mpp/ExchangeReceiver.cpp +++ b/dbms/src/Flash/Mpp/ExchangeReceiver.cpp @@ -85,7 +85,7 @@ class AsyncRequestHandler : public UnaryCallback { packets.resize(batch_packet_count); for (auto & packet : packets) - packet = std::make_shared(); + packet = std::make_shared(MPPDataPacketV0); start(); } @@ -264,7 +264,7 @@ class AsyncRequestHandler : public UnaryCallback return false; // can't reuse packet since it is sent to readers. - packet = std::make_shared(); + packet = std::make_shared(MPPDataPacketV0); } return true; } @@ -517,7 +517,7 @@ void ExchangeReceiverBase::readLoop(const Request & req) for (;;) { LOG_TRACE(log, "begin next "); - TrackedMppDataPacketPtr packet = std::make_shared(); + TrackedMppDataPacketPtr packet = std::make_shared(MPPDataPacketV0); bool success = reader->read(packet); if (!success) break; diff --git a/dbms/src/Flash/Mpp/FineGrainedShuffleWriter.cpp b/dbms/src/Flash/Mpp/FineGrainedShuffleWriter.cpp index ad6ac0d6faf..b04851f8e30 100644 --- a/dbms/src/Flash/Mpp/FineGrainedShuffleWriter.cpp +++ b/dbms/src/Flash/Mpp/FineGrainedShuffleWriter.cpp @@ -14,6 +14,7 @@ #include #include +#include #include #include #include @@ -21,6 +22,9 @@ namespace DB { + +const char * FineGrainedShuffleWriterLabels[] = {"FineGrainedShuffleWriter", "FineGrainedShuffleWriter-V1"}; + template FineGrainedShuffleWriter::FineGrainedShuffleWriter( ExchangeWriterPtr writer_, @@ -28,7 +32,9 @@ FineGrainedShuffleWriter::FineGrainedShuffleWriter( TiDB::TiDBCollators collators_, DAGContext & dag_context_, uint64_t fine_grained_shuffle_stream_count_, - UInt64 fine_grained_shuffle_batch_size_) + UInt64 fine_grained_shuffle_batch_size_, + MPPDataPacketVersion data_codec_version_, + tipb::CompressionMode compression_mode_) : DAGResponseWriter(/*records_per_chunk=*/-1, dag_context_) , writer(writer_) , partition_col_ids(std::move(partition_col_ids_)) @@ -37,6 +43,8 @@ FineGrainedShuffleWriter::FineGrainedShuffleWriter( , fine_grained_shuffle_batch_size(fine_grained_shuffle_batch_size_) , batch_send_row_limit(fine_grained_shuffle_batch_size * fine_grained_shuffle_stream_count) , hash(0) + , data_codec_version(data_codec_version_) + , compression_method(ToInternalCompressionMethod(compression_mode_)) { rows_in_blocks = 0; partition_num = writer_->getPartitionNum(); @@ -60,6 +68,23 @@ void FineGrainedShuffleWriter::prepare(const Block & sample_b num_bucket = partition_num * fine_grained_shuffle_stream_count; partition_key_containers_for_reuse.resize(collators.size()); initScatterColumns(); + + switch (data_codec_version) + { + case MPPDataPacketV0: + break; + case MPPDataPacketV1: + default: + { + for (const auto & field_type : dag_context.result_field_types) + { + expected_types.emplace_back(getDataTypeByFieldTypeForComputingLayer(field_type)); + } + assertBlockSchema(expected_types, header, FineGrainedShuffleWriterLabels[MPPDataPacketV1]); + break; + } + } + prepared = true; } @@ -107,9 +132,12 @@ void FineGrainedShuffleWriter::initScatterColumns() } template -void FineGrainedShuffleWriter::batchWriteFineGrainedShuffle() +template +void FineGrainedShuffleWriter::batchWriteFineGrainedShuffleImpl() { - if (likely(!blocks.empty())) + if (blocks.empty()) + return; + { assert(rows_in_blocks > 0); assert(fine_grained_shuffle_stream_count <= 1024); @@ -118,6 +146,11 @@ void FineGrainedShuffleWriter::batchWriteFineGrainedShuffle() while (!blocks.empty()) { const auto & block = blocks.back(); + if constexpr (version != MPPDataPacketV0) + { + // check schema + assertBlockSchema(expected_types, block, FineGrainedShuffleWriterLabels[MPPDataPacketV1]); + } HashBaseWriterHelper::scatterColumnsForFineGrainedShuffle(block, partition_col_ids, collators, partition_key_containers_for_reuse, partition_num, fine_grained_shuffle_stream_count, hash, selector, scattered); blocks.pop_back(); } @@ -132,12 +165,33 @@ void FineGrainedShuffleWriter::batchWriteFineGrainedShuffle() bucket_idx, fine_grained_shuffle_stream_count, num_columns, - part_id); + part_id, + data_codec_version, + compression_method); } rows_in_blocks = 0; } } +template +void FineGrainedShuffleWriter::batchWriteFineGrainedShuffle() +{ + switch (data_codec_version) + { + case MPPDataPacketV0: + { + batchWriteFineGrainedShuffleImpl(); + break; + } + case MPPDataPacketV1: + default: + { + batchWriteFineGrainedShuffleImpl(); + break; + } + } +} + template class FineGrainedShuffleWriter; } // namespace DB diff --git a/dbms/src/Flash/Mpp/FineGrainedShuffleWriter.h b/dbms/src/Flash/Mpp/FineGrainedShuffleWriter.h index 8730a8fb6bc..44b26dfc2ae 100644 --- a/dbms/src/Flash/Mpp/FineGrainedShuffleWriter.h +++ b/dbms/src/Flash/Mpp/FineGrainedShuffleWriter.h @@ -46,14 +46,12 @@ class FineGrainedShuffleWriter : public DAGResponseWriter void prepare(const Block & sample_block) override; void write(const Block & block) override; void flush() override; - ~FineGrainedShuffleWriter() override; private: void batchWriteFineGrainedShuffle(); - void batchWriteFineGrainedShuffleImpl(TrackedMppDataPacketPtrs &&); - void batchWriteFineGrainedShuffleImplV1(TrackedMppDataPacketPtrs &&); - void initScatterColumns(); + template + void batchWriteFineGrainedShuffleImpl(); private: ExchangeWriterPtr writer; @@ -73,9 +71,9 @@ class FineGrainedShuffleWriter : public DAGResponseWriter IColumn::Selector selector; std::vector scattered; // size = num_columns // support data compression + DataTypes expected_types; MPPDataPacketVersion data_codec_version; CompressionMethod compression_method{}; - std::unique_ptr codec_helper_v1{}; }; } // namespace DB diff --git a/dbms/src/Flash/Mpp/HashPartitionWriter.cpp b/dbms/src/Flash/Mpp/HashPartitionWriter.cpp index 7ae30a1b4e7..d24737d18a8 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriter.cpp +++ b/dbms/src/Flash/Mpp/HashPartitionWriter.cpp @@ -14,6 +14,7 @@ #include #include +#include #include #include #include @@ -21,59 +22,186 @@ namespace DB { +constexpr ssize_t MAX_BATCH_SEND_MIN_LIMIT_MEM_SIZE = 1024 * 1024 * 128; // 128MB + template HashPartitionWriter::HashPartitionWriter( ExchangeWriterPtr writer_, std::vector partition_col_ids_, TiDB::TiDBCollators collators_, Int64 batch_send_min_limit_, - DAGContext & dag_context_) + DAGContext & dag_context_, + MPPDataPacketVersion data_codec_version_, + tipb::CompressionMode compression_mode_) : DAGResponseWriter(/*records_per_chunk=*/-1, dag_context_) , batch_send_min_limit(batch_send_min_limit_) , writer(writer_) , partition_col_ids(std::move(partition_col_ids_)) , collators(std::move(collators_)) + , batch_send_min_limit_mem_size(batch_send_min_limit_) + , data_codec_version(data_codec_version_) + , compression_method(ToInternalCompressionMethod(compression_mode_)) { rows_in_blocks = 0; partition_num = writer_->getPartitionNum(); RUNTIME_CHECK(partition_num > 0); RUNTIME_CHECK(dag_context.encode_type == tipb::EncodeType::TypeCHBlock); + + switch (data_codec_version) + { + case MPPDataPacketV0: + break; + case MPPDataPacketV1: + default: + { + if (batch_send_min_limit_mem_size < 0) + { + // set upper limit if not specified + batch_send_min_limit_mem_size = std::min(MAX_BATCH_SEND_MIN_LIMIT_MEM_SIZE, 1024 * 64 * partition_num /* 64KB * partition-num */); + } + for (const auto & field_type : dag_context.result_field_types) + { + expected_types.emplace_back(getDataTypeByFieldTypeForComputingLayer(field_type)); + } + break; + } + } } template void HashPartitionWriter::flush() { - if (rows_in_blocks > 0) + if (0 == rows_in_blocks) + return; + + switch (data_codec_version) + { + case MPPDataPacketV0: + { partitionAndWriteBlocks(); + break; + } + case MPPDataPacketV1: + default: + { + partitionAndWriteBlocksV1(); + break; + } + } } template -void HashPartitionWriter::write(const Block & block) +void HashPartitionWriter::writeImplV1(const Block & block) { - RUNTIME_CHECK_MSG( - block.columns() == dag_context.result_field_types.size(), - "Output column size mismatch with field type size"); size_t rows = block.rows(); if (rows > 0) { rows_in_blocks += rows; + mem_size_in_blocks += block.bytes(); blocks.push_back(block); } + if (mem_size_in_blocks > batch_send_min_limit_mem_size) + partitionAndWriteBlocksV1(); +} +template +void HashPartitionWriter::writeImpl(const Block & block) +{ + size_t rows = block.rows(); + if (rows > 0) + { + rows_in_blocks += rows; + blocks.push_back(block); + } if (static_cast(rows_in_blocks) > batch_send_min_limit) partitionAndWriteBlocks(); } +template +void HashPartitionWriter::write(const Block & block) +{ + RUNTIME_CHECK_MSG( + block.columns() == dag_context.result_field_types.size(), + "Output column size mismatch with field type size"); + + switch (data_codec_version) + { + case MPPDataPacketV0: + { + return writeImpl(block); + } + case MPPDataPacketV1: + default: + { + return writeImplV1(block); + } + } +} + +template +void HashPartitionWriter::partitionAndWriteBlocksV1() +{ + assert(rows_in_blocks > 0); + assert(mem_size_in_blocks > 0); + + if unlikely (blocks.empty()) + return; + + HashBaseWriterHelper::materializeBlocks(blocks); + // All blocks are same, use one block's meta info as header + Block dest_block_header = blocks.back().cloneEmpty(); + std::vector partition_key_containers(collators.size()); + std::vector> dest_columns(partition_num); + size_t total_rows = 0; + + while (!blocks.empty()) + { + const auto & block = blocks.back(); + { + // check schema + assertBlockSchema(expected_types, block, "HashPartitionWriter-V1"); + } + auto && dest_tbl_cols = HashBaseWriterHelper::createDestColumns(block, partition_num); + HashBaseWriterHelper::scatterColumns(block, partition_col_ids, collators, partition_key_containers, partition_num, dest_tbl_cols); + blocks.pop_back(); + + for (size_t part_id = 0; part_id < partition_num; ++part_id) + { + auto & columns = dest_tbl_cols[part_id]; + // check column size + size_t expect_size = columns.front()->size(); + for (auto && col : columns) + { + auto size = col->size(); + RUNTIME_CHECK(size == expect_size, size, expect_size); + } + total_rows += expect_size; + dest_columns[part_id].emplace_back(std::move(columns)); + } + } + RUNTIME_CHECK(rows_in_blocks, total_rows); + + for (size_t part_id = 0; part_id < partition_num; ++part_id) + { + writer->partitionWrite(dest_block_header, std::move(dest_columns[part_id]), part_id, data_codec_version, compression_method); + } + + assert(blocks.empty()); + rows_in_blocks = 0; + mem_size_in_blocks = 0; +} + template void HashPartitionWriter::partitionAndWriteBlocks() { + assert(rows_in_blocks > 0); + + if unlikely (blocks.empty()) + return; + std::vector partition_blocks; partition_blocks.resize(partition_num); - - if (!blocks.empty()) { - assert(rows_in_blocks > 0); - HashBaseWriterHelper::materializeBlocks(blocks); std::vector partition_key_containers(collators.size()); diff --git a/dbms/src/Flash/Mpp/HashPartitionWriter.h b/dbms/src/Flash/Mpp/HashPartitionWriter.h index f90dc4ddb7f..43b78e1b8db 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriter.h +++ b/dbms/src/Flash/Mpp/HashPartitionWriter.h @@ -22,6 +22,8 @@ namespace DB { class DAGContext; +enum class CompressionMethod; +enum MPPDataPacketVersion : int64_t; template class HashPartitionWriter : public DAGResponseWriter @@ -32,12 +34,17 @@ class HashPartitionWriter : public DAGResponseWriter std::vector partition_col_ids_, TiDB::TiDBCollators collators_, Int64 batch_send_min_limit_, - DAGContext & dag_context_); + DAGContext & dag_context_, + MPPDataPacketVersion data_codec_version_, + tipb::CompressionMode compression_mode_); void write(const Block & block) override; void flush() override; private: + void writeImpl(const Block & block); + void writeImplV1(const Block & block); void partitionAndWriteBlocks(); + void partitionAndWriteBlocksV1(); void writePartitionBlocks(std::vector & partition_blocks); @@ -49,6 +56,12 @@ class HashPartitionWriter : public DAGResponseWriter TiDB::TiDBCollators collators; size_t rows_in_blocks; uint16_t partition_num; + // support data compression + int64_t mem_size_in_blocks{}; + int64_t batch_send_min_limit_mem_size; + DataTypes expected_types; + MPPDataPacketVersion data_codec_version; + CompressionMethod compression_method{}; }; } // namespace DB diff --git a/dbms/src/Flash/Mpp/MPPTunnelSet.cpp b/dbms/src/Flash/Mpp/MPPTunnelSet.cpp index 7f49749bbba..b240718811e 100644 --- a/dbms/src/Flash/Mpp/MPPTunnelSet.cpp +++ b/dbms/src/Flash/Mpp/MPPTunnelSet.cpp @@ -14,6 +14,7 @@ #include #include +#include #include #include #include @@ -33,7 +34,7 @@ void checkPacketSize(size_t size) TrackedMppDataPacketPtr serializePacket(const tipb::SelectResponse & response) { - auto tracked_packet = std::make_shared(); + auto tracked_packet = std::make_shared(MPPDataPacketV0); tracked_packet->serializeByResponse(response); checkPacketSize(tracked_packet->getPacket().ByteSizeLong()); return tracked_packet; @@ -62,11 +63,56 @@ void MPPTunnelSetBase::write(tipb::SelectResponse & response) tunnels.back()->write(serializePacket(response)); } +static inline void updatePartitionWriterMetrics(size_t packet_bytes, bool is_local) +{ + // statistic + GET_METRIC(tiflash_exchange_data_bytes, type_hash_original_all).Increment(packet_bytes); + // compression method is always NONE + if (is_local) + GET_METRIC(tiflash_exchange_data_bytes, type_hash_none_local).Increment(packet_bytes); + else + GET_METRIC(tiflash_exchange_data_bytes, type_hash_none_remote).Increment(packet_bytes); +} + +static inline void updatePartitionWriterMetrics(CompressionMethod method, size_t original_size, size_t sz, bool is_local) +{ + // statistic + GET_METRIC(tiflash_exchange_data_bytes, type_hash_original_all).Increment(original_size); + + switch (method) + { + case CompressionMethod::NONE: + { + if (is_local) + { + GET_METRIC(tiflash_exchange_data_bytes, type_hash_none_local).Increment(sz); + } + else + { + GET_METRIC(tiflash_exchange_data_bytes, type_hash_none_remote).Increment(sz); + } + break; + } + case CompressionMethod::LZ4: + { + GET_METRIC(tiflash_exchange_data_bytes, type_hash_lz4).Increment(sz); + break; + } + case CompressionMethod::ZSTD: + { + GET_METRIC(tiflash_exchange_data_bytes, type_hash_zstd).Increment(sz); + break; + } + default: + break; + } +} + template void MPPTunnelSetBase::broadcastOrPassThroughWrite(Blocks & blocks) { RUNTIME_CHECK(!tunnels.empty()); - auto tracked_packet = MPPTunnelSetHelper::toPacket(blocks, result_field_types); + auto tracked_packet = MPPTunnelSetHelper::toPacket(blocks, result_field_types, MPPDataPacketV0); auto packet_bytes = tracked_packet->getPacket().ByteSizeLong(); checkPacketSize(packet_bytes); // TODO avoid copy packet for broadcast. @@ -89,12 +135,76 @@ void MPPTunnelSetBase::broadcastOrPassThroughWrite(Blocks & blocks) template void MPPTunnelSetBase::partitionWrite(Blocks & blocks, int16_t partition_id) { - auto tracked_packet = MPPTunnelSetHelper::toPacket(blocks, result_field_types); - if (likely(tracked_packet->getPacket().chunks_size() > 0)) - { - checkPacketSize(tracked_packet->getPacket().ByteSizeLong()); - tunnels[partition_id]->write(std::move(tracked_packet)); - } + auto tracked_packet = MPPTunnelSetHelper::toPacket(blocks, result_field_types, MPPDataPacketV0); + + if unlikely (tracked_packet->getPacket().chunks_size() <= 0) + return; + + auto packet_bytes = tracked_packet->getPacket().ByteSizeLong(); + checkPacketSize(packet_bytes); + tunnels[partition_id]->write(std::move(tracked_packet)); + updatePartitionWriterMetrics(packet_bytes, isLocal(partition_id)); +} + +template +void MPPTunnelSetBase::partitionWrite( + const Block & header, + std::vector && part_columns, + int16_t partition_id, + MPPDataPacketVersion version, + CompressionMethod compression_method) +{ + assert(version > MPPDataPacketV0); + + bool is_local = isLocal(partition_id); + compression_method = is_local ? CompressionMethod::NONE : compression_method; + + size_t original_size = 0; + auto tracked_packet = MPPTunnelSetHelper::ToPacket(header, std::move(part_columns), version, compression_method, original_size); + if (!tracked_packet) + return; + + auto packet_bytes = tracked_packet->getPacket().ByteSizeLong(); + checkPacketSize(packet_bytes); + tunnels[partition_id]->write(std::move(tracked_packet)); + updatePartitionWriterMetrics(compression_method, original_size, packet_bytes, is_local); +} + +template +void MPPTunnelSetBase::fineGrainedShuffleWrite( + const Block & header, + std::vector & scattered, + size_t bucket_idx, + UInt64 fine_grained_shuffle_stream_count, + size_t num_columns, + int16_t partition_id, + MPPDataPacketVersion version, + CompressionMethod compression_method) +{ + if (version == MPPDataPacketV0) + return fineGrainedShuffleWrite(header, scattered, bucket_idx, fine_grained_shuffle_stream_count, num_columns, partition_id); + + bool is_local = isLocal(partition_id); + compression_method = is_local ? CompressionMethod::NONE : compression_method; + + size_t original_size = 0; + auto tracked_packet = MPPTunnelSetHelper::ToFineGrainedPacket( + header, + scattered, + bucket_idx, + fine_grained_shuffle_stream_count, + num_columns, + version, + compression_method, + original_size); + + if unlikely (tracked_packet->getPacket().chunks_size() <= 0) + return; + + auto packet_bytes = tracked_packet->getPacket().ByteSizeLong(); + checkPacketSize(packet_bytes); + tunnels[partition_id]->write(std::move(tracked_packet)); + updatePartitionWriterMetrics(compression_method, original_size, packet_bytes, is_local); } template @@ -112,12 +222,16 @@ void MPPTunnelSetBase::fineGrainedShuffleWrite( bucket_idx, fine_grained_shuffle_stream_count, num_columns, - result_field_types); - if (likely(tracked_packet->getPacket().chunks_size() > 0)) - { - checkPacketSize(tracked_packet->getPacket().ByteSizeLong()); - tunnels[partition_id]->write(std::move(tracked_packet)); - } + result_field_types, + MPPDataPacketV0); + + if unlikely (tracked_packet->getPacket().chunks_size() <= 0) + return; + + auto packet_bytes = tracked_packet->getPacket().ByteSizeLong(); + checkPacketSize(packet_bytes); + tunnels[partition_id]->write(std::move(tracked_packet)); + updatePartitionWriterMetrics(packet_bytes, isLocal(partition_id)); } template diff --git a/dbms/src/Flash/Mpp/MPPTunnelSet.h b/dbms/src/Flash/Mpp/MPPTunnelSet.h index 07986e4397a..9d8b6285242 100644 --- a/dbms/src/Flash/Mpp/MPPTunnelSet.h +++ b/dbms/src/Flash/Mpp/MPPTunnelSet.h @@ -16,16 +16,7 @@ #include #include -#ifdef __clang__ -#pragma clang diagnostic push -#pragma clang diagnostic ignored "-Wdeprecated-declarations" -#endif -#include -#ifdef __clang__ -#pragma clang diagnostic pop -#endif - -#include +#include namespace DB { @@ -41,10 +32,15 @@ class MPPTunnelSetBase : private boost::noncopyable // this is a root mpp writing. void write(tipb::SelectResponse & response); // this is a broadcast or pass through writing. + // data codec version V0 void broadcastOrPassThroughWrite(Blocks & blocks); // this is a partition writing. + // data codec version V0 void partitionWrite(Blocks & blocks, int16_t partition_id); + // data codec version > V0 + void partitionWrite(const Block & header, std::vector && part_columns, int16_t partition_id, MPPDataPacketVersion version, CompressionMethod compression_method); // this is a fine grained shuffle writing. + // data codec version V0 void fineGrainedShuffleWrite( const Block & header, std::vector & scattered, @@ -52,6 +48,15 @@ class MPPTunnelSetBase : private boost::noncopyable UInt64 fine_grained_shuffle_stream_count, size_t num_columns, int16_t partition_id); + void fineGrainedShuffleWrite( + const Block & header, + std::vector & scattered, + size_t bucket_idx, + UInt64 fine_grained_shuffle_stream_count, + size_t num_columns, + int16_t partition_id, + MPPDataPacketVersion version, + CompressionMethod compression_method); /// this is a execution summary writing. /// for both broadcast writing and partition/fine grained shuffle writing, only /// return meaningful execution summary for the first tunnel, diff --git a/dbms/src/Flash/Mpp/MPPTunnelSetHelper.cpp b/dbms/src/Flash/Mpp/MPPTunnelSetHelper.cpp index d17f52b8284..47b3ee76c19 100644 --- a/dbms/src/Flash/Mpp/MPPTunnelSetHelper.cpp +++ b/dbms/src/Flash/Mpp/MPPTunnelSetHelper.cpp @@ -13,15 +13,43 @@ // limitations under the License. #include +#include #include namespace DB::MPPTunnelSetHelper { -TrackedMppDataPacketPtr toPacket(Blocks & blocks, const std::vector & field_types) + +TrackedMppDataPacketPtr ToPacket( + const Block & header, + std::vector && part_columns, + MPPDataPacketVersion version, + CompressionMethod method, + size_t & original_size) +{ + assert(version > MPPDataPacketV0); + + auto && codec = CHBlockChunkCodecV1{ + header, + false /*scape empty part*/, + }; + + auto && res = codec.encode(std::move(part_columns), method); + if unlikely (res.empty()) + return nullptr; + + auto tracked_packet = std::make_shared(version); + tracked_packet->addChunk(std::move(res)); + original_size += codec.original_size; + return tracked_packet; +} + +TrackedMppDataPacketPtr toPacket(Blocks & blocks, const std::vector & field_types, MPPDataPacketVersion version) { + assert(version == MPPDataPacketV0); + CHBlockChunkCodec codec; auto codec_stream = codec.newCodecStream(field_types); - auto tracked_packet = std::make_shared(); + auto tracked_packet = std::make_shared(version); while (!blocks.empty()) { const auto & block = blocks.back(); @@ -33,17 +61,63 @@ TrackedMppDataPacketPtr toPacket(Blocks & blocks, const std::vector & scattered, + size_t bucket_idx, + UInt64 fine_grained_shuffle_stream_count, + size_t num_columns, + MPPDataPacketVersion version, + CompressionMethod method, + size_t & original_size) +{ + assert(version > MPPDataPacketV0); + + auto && codec = CHBlockChunkCodecV1{ + header, + true /*keep header even if no rows*/, + }; + auto tracked_packet = std::make_shared(version); + + for (uint64_t stream_idx = 0; stream_idx < fine_grained_shuffle_stream_count; ++stream_idx) + { + // assemble scatter columns into a block + MutableColumns columns; + columns.reserve(num_columns); + for (size_t col_id = 0; col_id < num_columns; ++col_id) + columns.emplace_back(std::move(scattered[col_id][bucket_idx + stream_idx])); + + auto && res = codec.encode(columns, method); + assert(!res.empty()); + + tracked_packet->addChunk(std::move(res)); + tracked_packet->getPacket().add_stream_ids(stream_idx); + + for (size_t col_id = 0; col_id < num_columns; ++col_id) + { + columns[col_id]->popBack(columns[col_id]->size()); // clear column + scattered[col_id][bucket_idx + stream_idx] = std::move(columns[col_id]); + } + } + + original_size += codec.original_size; + return tracked_packet; +} + TrackedMppDataPacketPtr toFineGrainedPacket( const Block & header, std::vector & scattered, size_t bucket_idx, UInt64 fine_grained_shuffle_stream_count, size_t num_columns, - const std::vector & field_types) + const std::vector & field_types, + MPPDataPacketVersion version) { + assert(version == MPPDataPacketV0); + CHBlockChunkCodec codec; auto codec_stream = codec.newCodecStream(field_types); - auto tracked_packet = std::make_shared(); + auto tracked_packet = std::make_shared(version); for (uint64_t stream_idx = 0; stream_idx < fine_grained_shuffle_stream_count; ++stream_idx) { // assemble scatter columns into a block diff --git a/dbms/src/Flash/Mpp/MPPTunnelSetHelper.h b/dbms/src/Flash/Mpp/MPPTunnelSetHelper.h index 38bddcef962..030e74a06bb 100644 --- a/dbms/src/Flash/Mpp/MPPTunnelSetHelper.h +++ b/dbms/src/Flash/Mpp/MPPTunnelSetHelper.h @@ -15,12 +15,24 @@ #pragma once #include +#include #include -#include + +namespace DB +{ +enum class CompressionMethod; +} namespace DB::MPPTunnelSetHelper { -TrackedMppDataPacketPtr toPacket(Blocks & blocks, const std::vector & field_types); +TrackedMppDataPacketPtr toPacket(Blocks & blocks, const std::vector & field_types, MPPDataPacketVersion version); + +TrackedMppDataPacketPtr ToPacket( + const Block & header, + std::vector && part_columns, + MPPDataPacketVersion version, + CompressionMethod compression_method, + size_t & original_size); TrackedMppDataPacketPtr toFineGrainedPacket( const Block & header, @@ -28,5 +40,17 @@ TrackedMppDataPacketPtr toFineGrainedPacket( size_t bucket_idx, UInt64 fine_grained_shuffle_stream_count, size_t num_columns, - const std::vector & field_types); + const std::vector & field_types, + MPPDataPacketVersion version); + +TrackedMppDataPacketPtr ToFineGrainedPacket( + const Block & header, + std::vector & scattered, + size_t bucket_idx, + UInt64 fine_grained_shuffle_stream_count, + size_t num_columns, + MPPDataPacketVersion version, + CompressionMethod compression_method, + size_t & original_size); + } // namespace DB::MPPTunnelSetHelper diff --git a/dbms/src/Flash/Mpp/TrackedMppDataPacket.h b/dbms/src/Flash/Mpp/TrackedMppDataPacket.h index ad7e2a3cdd2..f9530cb689b 100644 --- a/dbms/src/Flash/Mpp/TrackedMppDataPacket.h +++ b/dbms/src/Flash/Mpp/TrackedMppDataPacket.h @@ -116,13 +116,17 @@ struct TrackedMppDataPacket packet = data; } - explicit TrackedMppDataPacket() + explicit TrackedMppDataPacket(int64_t version) : mem_tracker_wrapper(current_memory_tracker) - {} + { + packet.set_version(version); + } - explicit TrackedMppDataPacket(MemoryTracker * memory_tracker) + explicit TrackedMppDataPacket(MemoryTracker * memory_tracker, int64_t version) : mem_tracker_wrapper(memory_tracker) - {} + { + packet.set_version(version); + } TrackedMppDataPacket(const mpp::MPPDataPacket & data, size_t size, MemoryTracker * memory_tracker) : mem_tracker_wrapper(size, memory_tracker) diff --git a/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp b/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp index 5036a905bd3..bea50b6ef53 100644 --- a/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp +++ b/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp @@ -17,7 +17,6 @@ #include #include #include -#include #include #include #include @@ -28,7 +27,6 @@ #include #include #include -#include namespace DB { @@ -130,14 +128,50 @@ struct MockExchangeWriter , part_num(part_num_) , result_field_types(dag_context.result_field_types) {} - + void partitionWrite( + const Block & header, + std::vector && part_columns, + int16_t part_id, + MPPDataPacketVersion version, + CompressionMethod method) + { + method = isLocal(part_id) ? CompressionMethod::NONE : method; + size_t original_size = 0; + auto tracked_packet = MPPTunnelSetHelper::ToPacket(header, std::move(part_columns), version, method, original_size); + checker(tracked_packet, part_id); + } + void fineGrainedShuffleWrite( + const Block & header, + std::vector & scattered, + size_t bucket_idx, + UInt64 fine_grained_shuffle_stream_count, + size_t num_columns, + int16_t part_id, + MPPDataPacketVersion version, + CompressionMethod method) + { + if (version == MPPDataPacketV0) + return fineGrainedShuffleWrite(header, scattered, bucket_idx, fine_grained_shuffle_stream_count, num_columns, part_id); + method = isLocal(part_id) ? CompressionMethod::NONE : method; + size_t original_size = 0; + auto tracked_packet = MPPTunnelSetHelper::ToFineGrainedPacket( + header, + scattered, + bucket_idx, + fine_grained_shuffle_stream_count, + num_columns, + version, + method, + original_size); + checker(tracked_packet, part_id); + } void broadcastOrPassThroughWrite(Blocks & blocks) { - checker(MPPTunnelSetHelper::toPacket(blocks, result_field_types), 0); + checker(MPPTunnelSetHelper::toPacket(blocks, result_field_types, MPPDataPacketV0), 0); } void partitionWrite(Blocks & blocks, uint16_t part_id) { - checker(MPPTunnelSetHelper::toPacket(blocks, result_field_types), part_id); + checker(MPPTunnelSetHelper::toPacket(blocks, result_field_types, MPPDataPacketV0), part_id); } void fineGrainedShuffleWrite( const Block & header, @@ -153,14 +187,15 @@ struct MockExchangeWriter bucket_idx, fine_grained_shuffle_stream_count, num_columns, - result_field_types); + result_field_types, + MPPDataPacketV0); checker(tracked_packet, part_id); } static void write(tipb::SelectResponse &) { FAIL() << "cannot reach here, only consider CH Block format"; } void sendExecutionSummary(const tipb::SelectResponse & response) { - auto tracked_packet = std::make_shared(); + auto tracked_packet = std::make_shared(MPPDataPacketV0); tracked_packet->serializeByResponse(response); checker(tracked_packet, 0); } @@ -264,7 +299,7 @@ try auto checker = [&write_report](const TrackedMppDataPacketPtr & packet, uint16_t part_id) { write_report[part_id].emplace_back(packet); }; - auto mock_writer = std::make_shared(checker, part_num); + auto mock_writer = std::make_shared(checker, part_num, *dag_context_ptr); // 3. Start to write. auto dag_writer = std::make_shared>>( @@ -301,15 +336,9 @@ try { const auto & chunk = packet->getPacket().chunks(i); - if (part_index == 0) - { - ASSERT_EQ(CompressionMethodByte(chunk[0]), CompressionMethodByte::NONE); - } - else - { - ASSERT_EQ(CompressionMethodByte(chunk[0]), GetCompressionMethodByte(ToInternalCompressionMethod(mode))); - } + auto tar_method_byte = mock_writer->isLocal(part_index) ? CompressionMethodByte::NONE : GetCompressionMethodByte(ToInternalCompressionMethod(mode)); + ASSERT_EQ(CompressionMethodByte(chunk[0]), tar_method_byte); auto && result = decoder.decodeAndSquashV1(chunk); if (!result) { @@ -540,7 +569,7 @@ try auto checker = [&write_report](const TrackedMppDataPacketPtr & packet, uint16_t part_id) { write_report[part_id].emplace_back(packet); }; - auto mock_writer = std::make_shared(checker, part_num); + auto mock_writer = std::make_shared(checker, part_num, *dag_context_ptr); // 3. Start to write. auto dag_writer = std::make_shared>>( @@ -572,15 +601,8 @@ try for (auto && chunk : packet.chunks()) { - if (part_index == 0) - { - ASSERT_EQ(CompressionMethodByte(chunk[0]), CompressionMethodByte::NONE); - } - else - { - ASSERT_EQ(CompressionMethodByte(chunk[0]), GetCompressionMethodByte(ToInternalCompressionMethod(mode))); - } - + auto tar_method_byte = mock_writer->isLocal(part_index) ? CompressionMethodByte::NONE : GetCompressionMethodByte(ToInternalCompressionMethod(mode)); + ASSERT_EQ(CompressionMethodByte(chunk[0]), tar_method_byte); auto && result = decoder.decodeAndSquashV1(chunk); if (!result) continue; diff --git a/dbms/src/Flash/Mpp/tests/gtest_mpptunnel.cpp b/dbms/src/Flash/Mpp/tests/gtest_mpptunnel.cpp index 758b741c07c..3b0f3b44f0d 100644 --- a/dbms/src/Flash/Mpp/tests/gtest_mpptunnel.cpp +++ b/dbms/src/Flash/Mpp/tests/gtest_mpptunnel.cpp @@ -39,7 +39,7 @@ namespace { TrackedMppDataPacketPtr newDataPacket(const String & data) { - auto data_packet_ptr = std::make_shared(); + auto data_packet_ptr = std::make_shared(MPPDataPacketV0); data_packet_ptr->getPacket().set_data(data); return data_packet_ptr; } From 8848edec0b970a9dd927125e04e98895f6d7cc6e Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Wed, 18 Jan 2023 14:49:52 +0800 Subject: [PATCH 82/93] remove useless code Signed-off-by: Zhigao Tong --- .../Flash/Coprocessor/CHBlockChunkCodecV1.cpp | 49 ------------------- 1 file changed, 49 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.cpp b/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.cpp index e4b6f146aec..6230208eb67 100644 --- a/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.cpp +++ b/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.cpp @@ -133,55 +133,6 @@ static inline void decodeColumnsByBlock(ReadBuffer & istr, Block & res, size_t r res.setColumns(std::move(mutable_columns)); } -// Deprecated -[[maybe_unused]] static inline void decodeColumnsByCol(ReadBuffer & istr, Block & res, size_t rows_to_read, size_t reserve_size) -{ - if (!rows_to_read) - return; - - auto && mutable_columns = res.mutateColumns(); - for (auto && column : mutable_columns) - { - if (reserve_size > 0) - column->reserve(std::max(rows_to_read, reserve_size)); - else - column->reserve(rows_to_read + column->size()); - } - - std::vector column_batch; - { - size_t sz{}; - readVarUInt(sz, istr); - column_batch.resize(sz); - for (size_t i = 0; i < sz; ++i) - { - readVarUInt(column_batch[i], istr); - } - assert(std::accumulate(column_batch.begin(), column_batch.end(), 0, [](auto c, auto & e) { return c + e; }) == int(rows_to_read)); - } - - for (size_t i = 0; i < res.columns(); ++i) - { - for (const auto & sz : column_batch) - { - if (!sz) - continue; - /// Data - res.getByPosition(i).type->deserializeBinaryBulkWithMultipleStreams( - *mutable_columns[i], - [&](const IDataType::SubstreamPath &) { - return &istr; - }, - sz, - 0, - {}, - {}); - } - } - - res.setColumns(std::move(mutable_columns)); -} - void DecodeColumns(ReadBuffer & istr, Block & res, size_t rows_to_read, size_t reserve_size) { return decodeColumnsByBlock(istr, res, rows_to_read, reserve_size); From 89dd1955eaa44fd045bf97a036d4f127a1608b12 Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Wed, 18 Jan 2023 16:13:55 +0800 Subject: [PATCH 83/93] 70 Signed-off-by: Zhigao Tong --- .../Flash/Coprocessor/CHBlockChunkCodecV1.h | 39 +++++++------------ dbms/src/Flash/Mpp/HashPartitionWriter.cpp | 11 ++---- .../Mpp/tests/gtest_mpp_exchange_writer.cpp | 23 +++++------ 3 files changed, 30 insertions(+), 43 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.h b/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.h index 929267e3fc1..a3886d5a299 100644 --- a/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.h +++ b/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.h @@ -90,32 +90,25 @@ struct CHBlockChunkCodecV1 if constexpr (isBlockType()) { const Block & block = columns_holder; - const auto rows = block.rows(); - total_rows += rows; - for (size_t col_index = 0; col_index < block.columns(); ++col_index) + if (const auto rows = block.rows(); rows) { - auto && col_type_name = block.getByPosition(col_index); - bytes += col_type_name.column->byteSize(); - RUNTIME_ASSERT(rows == col_type_name.column->size()); + block.checkNumberOfRows(); + total_rows += rows; + bytes += block.bytes(); } } else { - if (columns_holder.front()) + // check each column + if likely (columns_holder.front()) { const auto rows = columns_holder.front()->size(); total_rows += rows; for (const auto & elem : columns_holder) { - bytes += elem->byteSize(); + RUNTIME_ASSERT(elem); RUNTIME_ASSERT(rows == elem->size()); - } - } - else - { - for (const auto & elem : columns_holder) - { - RUNTIME_ASSERT(!elem); + bytes += elem->byteSize(); } } } @@ -137,13 +130,6 @@ struct CHBlockChunkCodecV1 return block.getByPosition(index).column; } - template - size_t getRowsByColumns(ColumnsHolder && columns_holder) - { - size_t rows = columns_holder.front()->size(); - return rows; - } - template constexpr static bool isBlockType() { @@ -155,11 +141,14 @@ struct CHBlockChunkCodecV1 { if constexpr (isBlockType()) { - size_t rows = columns_holder.rows(); + const Block & block = columns_holder; + size_t rows = block.rows(); return rows; } else { + if unlikely (!columns_holder.front()) + return 0; size_t rows = columns_holder.front()->size(); return rows; } @@ -241,9 +230,9 @@ struct CHBlockChunkCodecV1 getColumnEncodeInfo(batch_columns, column_encode_bytes, rows); - if unlikely (rows <= 0 && !always_keep_header) + if unlikely (0 == rows && !always_keep_header) { - return ""; + return {}; } // compression method flag; NONE, LZ4, ZSTD, defined in `CompressionMethodByte` diff --git a/dbms/src/Flash/Mpp/HashPartitionWriter.cpp b/dbms/src/Flash/Mpp/HashPartitionWriter.cpp index d24737d18a8..aa05f9048ea 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriter.cpp +++ b/dbms/src/Flash/Mpp/HashPartitionWriter.cpp @@ -23,6 +23,7 @@ namespace DB { constexpr ssize_t MAX_BATCH_SEND_MIN_LIMIT_MEM_SIZE = 1024 * 1024 * 128; // 128MB +const char * HashPartitionWriterLabels[] = {"HashPartitionWriter", "HashPartitionWriter-V1"}; template HashPartitionWriter::HashPartitionWriter( @@ -159,7 +160,7 @@ void HashPartitionWriter::partitionAndWriteBlocksV1() const auto & block = blocks.back(); { // check schema - assertBlockSchema(expected_types, block, "HashPartitionWriter-V1"); + assertBlockSchema(expected_types, block, HashPartitionWriterLabels[MPPDataPacketV1]); } auto && dest_tbl_cols = HashBaseWriterHelper::createDestColumns(block, partition_num); HashBaseWriterHelper::scatterColumns(block, partition_col_ids, collators, partition_key_containers, partition_num, dest_tbl_cols); @@ -168,13 +169,9 @@ void HashPartitionWriter::partitionAndWriteBlocksV1() for (size_t part_id = 0; part_id < partition_num; ++part_id) { auto & columns = dest_tbl_cols[part_id]; - // check column size + if unlikely (!columns.front()) + continue; size_t expect_size = columns.front()->size(); - for (auto && col : columns) - { - auto size = col->size(); - RUNTIME_CHECK(size == expect_size, size, expect_size); - } total_rows += expect_size; dest_columns[part_id].emplace_back(std::move(columns)); } diff --git a/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp b/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp index bea50b6ef53..f09a362351f 100644 --- a/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp +++ b/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp @@ -290,7 +290,7 @@ try blocks.emplace_back(prepareUniformBlock(block_rows)); blocks.emplace_back(prepareUniformBlock(0)); } - Block header = blocks.back(); + const auto & header = blocks.back().cloneEmpty(); for (auto mode : {tipb::CompressionMode::NONE, tipb::CompressionMode::FAST, tipb::CompressionMode::HIGH_COMPRESSION}) { @@ -550,20 +550,20 @@ try { const size_t block_rows = 64; const size_t block_num = 64; - const size_t batch_send_min_limit = 16; + const size_t batch_send_min_limit = 1024 * 1024 * 1024; const uint16_t part_num = 4; - for (auto mode : {tipb::CompressionMode::NONE, tipb::CompressionMode::FAST, tipb::CompressionMode::HIGH_COMPRESSION}) + // 1. Build Blocks. + std::vector blocks; + for (size_t i = 0; i < block_num; ++i) { - // 1. Build Blocks. - std::vector blocks; - for (size_t i = 0; i < block_num; ++i) - { - blocks.emplace_back(prepareUniformBlock(block_rows)); - blocks.emplace_back(prepareUniformBlock(0)); - } - Block header = blocks.back(); + blocks.emplace_back(prepareUniformBlock(block_rows)); + blocks.emplace_back(prepareUniformBlock(0)); + } + const auto & header = blocks.back().cloneEmpty(); + for (auto mode : {tipb::CompressionMode::NONE, tipb::CompressionMode::FAST, tipb::CompressionMode::HIGH_COMPRESSION}) + { // 2. Build MockExchangeWriter. std::unordered_map write_report; auto checker = [&write_report](const TrackedMppDataPacketPtr & packet, uint16_t part_id) { @@ -582,6 +582,7 @@ try mode); for (const auto & block : blocks) dag_writer->write(block); + dag_writer->write(header); // write empty dag_writer->flush(); // 4. Start to check write_report. From d371444ae13a7cbcf4e74c6ab0aa65e3fdcedea3 Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Wed, 18 Jan 2023 18:12:37 +0800 Subject: [PATCH 84/93] 71 Signed-off-by: Zhigao Tong --- dbms/src/Flash/Mpp/ExchangeReceiver.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/dbms/src/Flash/Mpp/ExchangeReceiver.cpp b/dbms/src/Flash/Mpp/ExchangeReceiver.cpp index 322a0e11c5c..06ab2f88af9 100644 --- a/dbms/src/Flash/Mpp/ExchangeReceiver.cpp +++ b/dbms/src/Flash/Mpp/ExchangeReceiver.cpp @@ -22,7 +22,6 @@ #include #include #include -#include #include #include #include From 19ff254bc11f10023659025c42dc4198fd73c8f7 Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Wed, 18 Jan 2023 21:11:38 +0800 Subject: [PATCH 85/93] 72 --- dbms/src/Flash/Mpp/HashPartitionWriter.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/dbms/src/Flash/Mpp/HashPartitionWriter.cpp b/dbms/src/Flash/Mpp/HashPartitionWriter.cpp index aa05f9048ea..8a68a94b5d8 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriter.cpp +++ b/dbms/src/Flash/Mpp/HashPartitionWriter.cpp @@ -191,8 +191,6 @@ void HashPartitionWriter::partitionAndWriteBlocksV1() template void HashPartitionWriter::partitionAndWriteBlocks() { - assert(rows_in_blocks > 0); - if unlikely (blocks.empty()) return; From cd91a7c9cc16f475144c4dceb0367e3a196b9e42 Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Thu, 19 Jan 2023 09:11:37 +0800 Subject: [PATCH 86/93] 73 Signed-off-by: Zhigao Tong --- .../Flash/Coprocessor/CHBlockChunkCodecV1.cpp | 361 +++++++++++++++++- .../Flash/Coprocessor/CHBlockChunkCodecV1.h | 262 +------------ .../tests/gtest_block_chunk_codec.cpp | 171 +++++++++ .../tests/gtest_chunk_decode_and_squash.cpp | 3 +- 4 files changed, 548 insertions(+), 249 deletions(-) create mode 100644 dbms/src/Flash/Coprocessor/tests/gtest_block_chunk_codec.cpp diff --git a/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.cpp b/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.cpp index 6230208eb67..139665e3e8e 100644 --- a/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.cpp +++ b/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.cpp @@ -16,7 +16,7 @@ #include #include - +#include namespace DB { @@ -153,4 +153,363 @@ CompressionMethod ToInternalCompressionMethod(tipb::CompressionMode compression_ } } +template +constexpr static bool isBlockType() +{ + return std::is_same_v>, Block>; +} + +template +static void calcColumnEncodeInfoImpl(ColumnsHolder && columns_holder, size_t & bytes, size_t & total_rows) +{ + bytes += 8 /*rows*/; + + if constexpr (isBlockType()) + { + const Block & block = columns_holder; + if (const auto rows = block.rows(); rows) + { + block.checkNumberOfRows(); + total_rows += rows; + bytes += block.bytes(); + } + } + else + { + // check each column + if likely (columns_holder.front()) + { + const auto rows = columns_holder.front()->size(); + total_rows += rows; + for (const auto & column : columns_holder) + { + RUNTIME_ASSERT(column); + RUNTIME_ASSERT(rows == column->size()); + bytes += column->byteSize(); + } + } + else + { + for (const auto & column : columns_holder) + { + RUNTIME_ASSERT(!column); + } + } + } +} + +static void calcColumnEncodeInfo(const std::vector & batch_columns, size_t & bytes, size_t & rows) +{ + for (auto && columns : batch_columns) + { + calcColumnEncodeInfoImpl(columns, bytes, rows); + } +} +static void calcColumnEncodeInfo(const std::vector & batch_columns, size_t & bytes, size_t & rows) +{ + for (auto && columns : batch_columns) + { + calcColumnEncodeInfoImpl(columns, bytes, rows); + } +} +static void calcColumnEncodeInfo(const std::vector & blocks, size_t & bytes, size_t & rows) +{ + for (auto && block : blocks) + { + calcColumnEncodeInfoImpl(block, bytes, rows); + } +} +static void calcColumnEncodeInfo(const MutableColumns & columns, size_t & bytes, size_t & rows) +{ + calcColumnEncodeInfoImpl(columns, bytes, rows); +} +static void calcColumnEncodeInfo(const Columns & columns, size_t & bytes, size_t & rows) +{ + calcColumnEncodeInfoImpl(columns, bytes, rows); +} +static void calcColumnEncodeInfo(const Block & block, size_t & bytes, size_t & rows) +{ + calcColumnEncodeInfoImpl(block, bytes, rows); +} + +struct CHBlockChunkCodecV1Impl +{ + CHBlockChunkCodecV1 & inner; + + explicit CHBlockChunkCodecV1Impl(CHBlockChunkCodecV1 & inner_) + : inner(inner_) + {} + + std::string encode(CompressionMethod compression_method) + { + return encodeImpl(inner.header, compression_method); + } + std::string encode(const std::vector & blocks, CompressionMethod compression_method) + { + return encodeImpl(blocks, compression_method); + } + + static const ColumnPtr & toColumnPtr(const Columns & c, size_t index) + { + return c[index]; + } + static ColumnPtr toColumnPtr(Columns && c, size_t index) + { + return std::move(c[index]); + } + static ColumnPtr toColumnPtr(MutableColumns && c, size_t index) + { + return std::move(c[index]); + } + static ColumnPtr toColumnPtr(const MutableColumns & c, size_t index) + { + return c[index]->getPtr(); + } + static const ColumnPtr & toColumnPtr(const Block & block, size_t index) + { + return block.getByPosition(index).column; + } + + template + static size_t getRows(ColumnsHolder && columns_holder) + { + if constexpr (isBlockType()) + { + const Block & block = columns_holder; + size_t rows = block.rows(); + return rows; + } + else + { + if unlikely (!columns_holder.front()) + return 0; + size_t rows = columns_holder.front()->size(); + return rows; + } + } + + template + void encodeColumnImpl(ColumnsHolder && columns_holder, WriteBuffer * ostr_ptr) + { + size_t rows = getRows(std::forward(columns_holder)); + if (!rows) + return; + + // Encode row count for next columns + writeVarUInt(rows, *ostr_ptr); + + // Encode columns data + for (size_t col_index = 0; col_index < inner.header.columns(); ++col_index) + { + auto && col_type_name = inner.header.getByPosition(col_index); + auto && column_ptr = toColumnPtr(std::forward(columns_holder), col_index); + WriteColumnData(*col_type_name.type, column_ptr, *ostr_ptr, 0, 0); + } + + inner.encoded_rows += rows; + } + void encodeColumn(const MutableColumns & columns, WriteBuffer * ostr_ptr) + { + return encodeColumnImpl(columns, ostr_ptr); + } + void encodeColumn(const Columns & columns, WriteBuffer * ostr_ptr) + { + return encodeColumnImpl(columns, ostr_ptr); + } + void encodeColumn(const std::vector & batch_columns, WriteBuffer * ostr_ptr) + { + for (auto && batch : batch_columns) + { + encodeColumnImpl(batch, ostr_ptr); + } + } + void encodeColumn(std::vector && batch_columns, WriteBuffer * ostr_ptr) + { + for (auto && batch : batch_columns) + { + encodeColumnImpl(std::move(batch), ostr_ptr); + } + } + void encodeColumn(const std::vector & batch_columns, WriteBuffer * ostr_ptr) + { + for (auto && batch : batch_columns) + { + encodeColumnImpl(batch, ostr_ptr); + } + } + void encodeColumn(std::vector && batch_columns, WriteBuffer * ostr_ptr) + { + for (auto && batch : batch_columns) + { + encodeColumnImpl(std::move(batch), ostr_ptr); + } + } + void encodeColumn(const Block & block, WriteBuffer * ostr_ptr) + { + assert(&block == &inner.header); + return encodeColumnImpl(block, ostr_ptr); + } + void encodeColumn(const std::vector & blocks, WriteBuffer * ostr_ptr) + { + for (auto && block : blocks) + { + encodeColumnImpl(std::move(block), ostr_ptr); + } + } + template + std::string encodeImpl(VecColumns && batch_columns, CompressionMethod compression_method) + { + size_t column_encode_bytes = 0; + size_t rows = 0; + + // Calculate total rows and check data valid + calcColumnEncodeInfo(batch_columns, column_encode_bytes, rows); + + if unlikely (0 == rows && !inner.always_keep_header) + { + // no rows and no need to encode header + return {}; + } + + // compression method flag; NONE, LZ4, ZSTD, defined in `CompressionMethodByte` + // ... encoded by compression pattern ... + // header meta: + // columns count; + // total row count (multi parts); + // for each column: + // column name; + // column type; + // for each part: + // row count; + // columns data; + + size_t init_size = column_encode_bytes + inner.header_size + 1 /*compression method*/; + auto output_buffer = std::make_unique(init_size); + std::unique_ptr compress_codec{}; + WriteBuffer * ostr_ptr = output_buffer.get(); + + // Init compression writer + if (compression_method != CompressionMethod::NONE) + { + // CompressedWriteBuffer will encode compression method flag as first byte + compress_codec = std::make_unique( + *output_buffer, + CompressionSettings(compression_method), + init_size); + ostr_ptr = compress_codec.get(); + } + else + { + // Write compression method flag + output_buffer->write(static_cast(CompressionMethodByte::NONE)); + } + + // Encode header + EncodeHeader(*ostr_ptr, inner.header, rows); + if (rows > 0) + { + encodeColumn(std::forward(batch_columns), ostr_ptr); + } + // Flush rest buffer + if (compress_codec) + { + compress_codec->next(); + inner.original_size += compress_codec->getUncompressedBytes(); + inner.compressed_size += compress_codec->getCompressedBytes(); + } + else + { + inner.original_size += output_buffer->count(); + } + + return output_buffer->releaseStr(); + } +}; + +CHBlockChunkCodecV1::CHBlockChunkCodecV1(const Block & header_, bool always_keep_header_) + : header(header_) + , header_size(ApproxBlockHeaderBytes(header)) + , always_keep_header(always_keep_header_) +{ +} + +std::string CHBlockChunkCodecV1::encode(CompressionMethod compression_method) +{ + return CHBlockChunkCodecV1Impl{*this}.encode(compression_method); +} + +void CHBlockChunkCodecV1::clear() +{ + encoded_rows = 0; + original_size = 0; + compressed_size = 0; +} + +std::string CHBlockChunkCodecV1::encode(const MutableColumns & columns, CompressionMethod compression_method) +{ + return CHBlockChunkCodecV1Impl{*this}.encodeImpl(columns, compression_method); +} +std::string CHBlockChunkCodecV1::encode(const Columns & columns, CompressionMethod compression_method) +{ + return CHBlockChunkCodecV1Impl{*this}.encodeImpl(columns, compression_method); +} +std::string CHBlockChunkCodecV1::encode(const std::vector & columns, CompressionMethod compression_method) +{ + return CHBlockChunkCodecV1Impl{*this}.encodeImpl(columns, compression_method); +} +std::string CHBlockChunkCodecV1::encode(std::vector && columns, CompressionMethod compression_method) +{ + return CHBlockChunkCodecV1Impl{*this}.encodeImpl(std::move(columns), compression_method); +} +std::string CHBlockChunkCodecV1::encode(const std::vector & columns, CompressionMethod compression_method) +{ + return CHBlockChunkCodecV1Impl{*this}.encodeImpl(std::move(columns), compression_method); +} +std::string CHBlockChunkCodecV1::encode(std::vector && columns, CompressionMethod compression_method) +{ + return CHBlockChunkCodecV1Impl{*this}.encodeImpl(std::move(columns), compression_method); +} +std::string CHBlockChunkCodecV1::encode(const std::vector & blocks, CompressionMethod compression_method, bool check_schema) +{ + if (check_schema) + { + for (auto && block : blocks) + { + CodecUtils::checkColumnSize(header.columns(), block.columns()); + for (size_t column_index = 0; column_index < header.columns(); ++column_index) + { + auto && type_name = block.getByPosition(column_index).type->getName(); + CodecUtils::checkDataTypeName(column_index, header.getByPosition(column_index).type->getName(), type_name); + } + } + } + + return CHBlockChunkCodecV1Impl{*this}.encode(blocks, compression_method); +} + +static Block decodeCompression(const Block & header, ReadBuffer & istr) +{ + size_t decoded_rows{}; + auto decoded_block = DecodeHeader(istr, header, decoded_rows); + DecodeColumns(istr, decoded_block, decoded_rows, 0); + assert(decoded_rows == decoded_block.rows()); + return decoded_block; +} + +Block CHBlockChunkCodecV1::decode(const Block & header, std::string_view str) +{ + assert(!str.empty()); + + // read first byte of compression method flag which defined in `CompressionMethodByte` + if (static_cast(str[0]) == CompressionMethodByte::NONE) + { + str = str.substr(1, str.size() - 1); + ReadBufferFromString buff_str(str); + return decodeCompression(header, buff_str); + } + ReadBufferFromString buff_str(str); + auto && istr = CompressedCHBlockChunkReadBuffer(buff_str); + return decodeCompression(header, istr); +} + } // namespace DB \ No newline at end of file diff --git a/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.h b/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.h index a3886d5a299..1171d6c4e42 100644 --- a/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.h +++ b/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.h @@ -36,257 +36,25 @@ struct CHBlockChunkCodecV1 const Block & header; const size_t header_size; + const bool always_keep_header; + size_t encoded_rows{}; size_t original_size{}; size_t compressed_size{}; - bool always_keep_header{}; - - explicit CHBlockChunkCodecV1(const Block & header_, bool always_keep_header_) - : header(header_) - , header_size(ApproxBlockHeaderBytes(header)) - , always_keep_header(always_keep_header_) - { - } - - static std::string encode(const Block & block, CompressionMethod compression_method, bool always_keep_header) - { - return Self{block, always_keep_header}.encode(compression_method); - } - - void clear() - { - encoded_rows = 0; - original_size = 0; - compressed_size = 0; - } - - std::string encode(CompressionMethod compression_method) - { - return encodeImpl(header, compression_method); - } - - std::string encode(const MutableColumns & columns, CompressionMethod compression_method) - { - return encodeImpl(columns, compression_method); - } - std::string encode(const Columns & columns, CompressionMethod compression_method) - { - return encodeImpl(columns, compression_method); - } - std::string encode(const std::vector & columns, CompressionMethod compression_method) - { - return encodeImpl(columns, compression_method); - } - std::string encode(std::vector && columns, CompressionMethod compression_method) - { - return encodeImpl(std::move(columns), compression_method); - } - - template - static void getColumnEncodeInfoImpl(ColumnsHolder && columns_holder, size_t & bytes, size_t & total_rows) - { - bytes += 8 /*rows*/; - - if constexpr (isBlockType()) - { - const Block & block = columns_holder; - if (const auto rows = block.rows(); rows) - { - block.checkNumberOfRows(); - total_rows += rows; - bytes += block.bytes(); - } - } - else - { - // check each column - if likely (columns_holder.front()) - { - const auto rows = columns_holder.front()->size(); - total_rows += rows; - for (const auto & elem : columns_holder) - { - RUNTIME_ASSERT(elem); - RUNTIME_ASSERT(rows == elem->size()); - bytes += elem->byteSize(); - } - } - } - } - static const ColumnPtr & toColumnPtr(const Columns & c, size_t index) - { - return c[index]; - } - static ColumnPtr toColumnPtr(MutableColumns && c, size_t index) - { - return std::move(c[index]); - } - static ColumnPtr toColumnPtr(const MutableColumns & c, size_t index) - { - return c[index]->getPtr(); - } - static const ColumnPtr & toColumnPtr(const Block & block, size_t index) - { - return block.getByPosition(index).column; - } - - template - constexpr static bool isBlockType() - { - return std::is_same_v>, Block>; - } - - template - size_t getRows(ColumnsHolder && columns_holder) - { - if constexpr (isBlockType()) - { - const Block & block = columns_holder; - size_t rows = block.rows(); - return rows; - } - else - { - if unlikely (!columns_holder.front()) - return 0; - size_t rows = columns_holder.front()->size(); - return rows; - } - } - - template - void encodeColumnImpl(ColumnsHolder && columns_holder, WriteBuffer * ostr_ptr) - { - size_t rows = getRows(std::forward(columns_holder)); - if (!rows) - return; - - // Encode row count for next columns - writeVarUInt(rows, *ostr_ptr); - - // Encode columns data - for (size_t col_index = 0; col_index < header.columns(); ++col_index) - { - auto && col_type_name = header.getByPosition(col_index); - auto && column_ptr = toColumnPtr(std::forward(columns_holder), col_index); - WriteColumnData(*col_type_name.type, column_ptr, *ostr_ptr, 0, 0); - } - - encoded_rows += rows; - } - void encodeColumn(const MutableColumns & columns, WriteBuffer * ostr_ptr) - { - return encodeColumnImpl(columns, ostr_ptr); - } - void encodeColumn(const Columns & columns, WriteBuffer * ostr_ptr) - { - return encodeColumnImpl(columns, ostr_ptr); - } - void encodeColumn(const std::vector & batch_columns, WriteBuffer * ostr_ptr) - { - for (auto && batch : batch_columns) - { - encodeColumnImpl(batch, ostr_ptr); - } - } - void encodeColumn(std::vector && batch_columns, WriteBuffer * ostr_ptr) - { - for (auto && batch : batch_columns) - { - encodeColumnImpl(std::move(batch), ostr_ptr); - } - } - void encodeColumn(const Block & block, WriteBuffer * ostr_ptr) - { - assert(&block == &header); - return encodeColumnImpl(block, ostr_ptr); - } - - static void getColumnEncodeInfo(const std::vector & batch_columns, size_t & bytes, size_t & rows) - { - for (auto && columns : batch_columns) - { - getColumnEncodeInfoImpl(columns, bytes, rows); - } - } - static void getColumnEncodeInfo(const MutableColumns & columns, size_t & bytes, size_t & rows) - { - getColumnEncodeInfoImpl(columns, bytes, rows); - } - static void getColumnEncodeInfo(const Columns & columns, size_t & bytes, size_t & rows) - { - getColumnEncodeInfoImpl(columns, bytes, rows); - } - static void getColumnEncodeInfo(const Block & block, size_t & bytes, size_t & rows) - { - getColumnEncodeInfoImpl(block, bytes, rows); - } - - template - std::string encodeImpl(VecColumns && batch_columns, CompressionMethod compression_method) - { - size_t column_encode_bytes = 0; - size_t rows = 0; - - getColumnEncodeInfo(batch_columns, column_encode_bytes, rows); - - if unlikely (0 == rows && !always_keep_header) - { - return {}; - } - - // compression method flag; NONE, LZ4, ZSTD, defined in `CompressionMethodByte` - // ... - // header meta: - // columns count; - // total row count (multi parts); - // for each column: - // column name; - // column type; - // for each part: - // row count; - // columns data; - - size_t init_size = column_encode_bytes + header_size + 1 /*compression method*/; - auto output_buffer = std::make_unique(init_size); - std::unique_ptr compress_codec{}; - WriteBuffer * ostr_ptr = output_buffer.get(); - - // Init compression writer - if (compression_method != CompressionMethod::NONE) - { - // CompressedWriteBuffer will encode compression method flag as first byte - compress_codec = std::make_unique( - *output_buffer, - CompressionSettings(compression_method), - init_size); - ostr_ptr = compress_codec.get(); - } - else - { - // Write compression method flag - output_buffer->write(static_cast(CompressionMethodByte::NONE)); - } - - // Encode header - EncodeHeader(*ostr_ptr, header, rows); - if (rows > 0) - encodeColumn(std::forward(batch_columns), ostr_ptr); - - // Flush rest buffer - if (compress_codec) - { - compress_codec->next(); - original_size += compress_codec->getUncompressedBytes(); - compressed_size += compress_codec->getCompressedBytes(); - } - else - { - original_size += output_buffer->count(); - } - return output_buffer->releaseStr(); - } + void clear(); + explicit CHBlockChunkCodecV1(const Block & header_, bool always_keep_header_); + // + std::string encode(const MutableColumns & columns, CompressionMethod compression_method); + std::string encode(std::vector && columns, CompressionMethod compression_method); + std::string encode(const std::vector & columns, CompressionMethod compression_method); + std::string encode(const Columns & columns, CompressionMethod compression_method); + std::string encode(const std::vector & columns, CompressionMethod compression_method); + std::string encode(std::vector && columns, CompressionMethod compression_method); + std::string encode(CompressionMethod compression_method); + std::string encode(const std::vector & blocks, CompressionMethod compression_method, bool check_schema = true); + // + static Block decode(const Block & header, std::string_view str); }; } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/tests/gtest_block_chunk_codec.cpp b/dbms/src/Flash/Coprocessor/tests/gtest_block_chunk_codec.cpp new file mode 100644 index 00000000000..f6e82e3c16b --- /dev/null +++ b/dbms/src/Flash/Coprocessor/tests/gtest_block_chunk_codec.cpp @@ -0,0 +1,171 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include +#include +#include +#include +#include + +#include + +#include "Core/Block.h" + +namespace DB::tests +{ + +// Return a block with **rows** and 5 Int64 column. +static Block prepareBlock(size_t rows) +{ + Block block; + for (size_t i = 0; i < 5; ++i) + { + DataTypePtr int64_data_type = std::make_shared(); + auto int64_column = ColumnGenerator::instance().generate({rows, "Int64", RANDOM}).column; + block.insert(ColumnWithTypeAndName{ + std::move(int64_column), + int64_data_type, + String("col") + std::to_string(i)}); + } + return block; +} + +template +void test_enocde_release_data(VecCol && batch_columns, const Block & header, const size_t total_rows) +{ + // encode and release columns + const auto mode = CompressionMethod::LZ4; + + for (auto && columns : batch_columns) + { + for (auto && col : columns) + ASSERT_TRUE(col); + } + auto codec = CHBlockChunkCodecV1{header, false}; + auto str = codec.encode(std::forward(batch_columns), mode); + ASSERT_FALSE(str.empty()); + ASSERT_EQ(codec.encoded_rows, total_rows); + ASSERT_NE(codec.compressed_size, 0); + ASSERT_NE(codec.original_size, 0); + auto decoded_block = CHBlockChunkCodecV1::decode(header, str); + ASSERT_EQ(total_rows, decoded_block.rows()); + for (auto && columns : batch_columns) + { + for (auto && col : columns) + { + if (col) + ASSERT_EQ(col->size(), 0); + } + } + { + // test no rows + auto & empty_batch_columns = batch_columns; + auto str = codec.encode(empty_batch_columns, mode); + ASSERT_TRUE(str.empty()); + } +} + +TEST(CHBlockChunkCodec, ChunkCodecV1) +{ + size_t block_num = 10; + size_t rows = 10; + std::vector blocks; + auto header = prepareBlock(0); + for (size_t i = 0; i < block_num; ++i) + { + auto block = prepareBlock(rows); + blocks.emplace_back(std::move(block)); + } + blocks.emplace_back(prepareBlock(0)); + + auto total_rows = rows * block_num; + for (auto mode : {CompressionMethod::NONE, CompressionMethod::LZ4, CompressionMethod::ZSTD}) + { + { + // encode nothing if no rows + auto codec = CHBlockChunkCodecV1{header, false}; + auto str = codec.encode(mode); + ASSERT_TRUE(str.empty()); + ASSERT_EQ(codec.encoded_rows, 0); + ASSERT_EQ(codec.compressed_size, 0); + ASSERT_EQ(codec.original_size, 0); + } + { + // encode header schema if no rows + auto codec = CHBlockChunkCodecV1{header, true}; + auto str = codec.encode(mode); + ASSERT_FALSE(str.empty()); + ASSERT_EQ(codec.encoded_rows, 0); + + if (mode == CompressionMethod::NONE) + ASSERT_EQ(codec.compressed_size, 0); + else + ASSERT_NE(codec.compressed_size, 0); + + ASSERT_NE(codec.original_size, 0); + + auto decoded_block = CHBlockChunkCodecV1::decode(header, str); + ASSERT_EQ(0, decoded_block.rows()); + } + { + auto codec = CHBlockChunkCodecV1{header, false}; + auto str = codec.encode(blocks, mode); + ASSERT_FALSE(str.empty()); + ASSERT_EQ(codec.encoded_rows, total_rows); + + if (mode == CompressionMethod::NONE) + ASSERT_EQ(codec.compressed_size, 0); + else + ASSERT_NE(codec.compressed_size, 0); + + ASSERT_NE(codec.original_size, 0); + + auto decoded_block = CHBlockChunkCodecV1::decode(header, str); + ASSERT_EQ(total_rows, decoded_block.rows()); + } + { + auto columns = prepareBlock(rows).getColumns(); + auto codec = CHBlockChunkCodecV1{header, false}; + auto str = codec.encode(columns, mode); + ASSERT_FALSE(str.empty()); + ASSERT_EQ(codec.encoded_rows, rows); + auto decoded_block = CHBlockChunkCodecV1::decode(header, str); + ASSERT_EQ(decoded_block.rows(), rows); + } + { + auto columns = prepareBlock(rows).mutateColumns(); + auto codec = CHBlockChunkCodecV1{header, false}; + auto str = codec.encode(columns, mode); + ASSERT_FALSE(str.empty()); + ASSERT_EQ(codec.encoded_rows, rows); + auto decoded_block = CHBlockChunkCodecV1::decode(header, str); + ASSERT_EQ(decoded_block.rows(), rows); + } + } + { + std::vector batch_columns; + for (size_t i = 0; i < block_num; ++i) + batch_columns.emplace_back(prepareBlock(rows).mutateColumns()); + batch_columns.emplace_back(prepareBlock(0).mutateColumns()); + test_enocde_release_data(std::move(batch_columns), header, total_rows); + } + { + std::vector batch_columns; + for (size_t i = 0; i < block_num; ++i) + batch_columns.emplace_back(prepareBlock(rows).getColumns()); + batch_columns.emplace_back(prepareBlock(0).getColumns()); + test_enocde_release_data(std::move(batch_columns), header, total_rows); + } +} +} // namespace DB::tests diff --git a/dbms/src/Flash/Coprocessor/tests/gtest_chunk_decode_and_squash.cpp b/dbms/src/Flash/Coprocessor/tests/gtest_chunk_decode_and_squash.cpp index 9bb419a4578..c322f71c7a7 100644 --- a/dbms/src/Flash/Coprocessor/tests/gtest_chunk_decode_and_squash.cpp +++ b/dbms/src/Flash/Coprocessor/tests/gtest_chunk_decode_and_squash.cpp @@ -123,7 +123,8 @@ class TestChunkDecodeAndSquash : public testing::Test } else { - auto && str = CHBlockChunkCodecV1::encode(block, CompressionMethod::LZ4, true); + auto codec = CHBlockChunkCodecV1{block, true}; + auto && str = codec.encode(CompressionMethod::LZ4); assert(!str.empty()); assert(static_cast(str[0]) == CompressionMethodByte::LZ4); encode_str_vec.push_back(std::move(str)); From b7922d2e9a47bc672ab745998dbc95658744fb81 Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Thu, 19 Jan 2023 12:51:21 +0800 Subject: [PATCH 87/93] 74 Signed-off-by: Zhigao Tong --- dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.cpp | 6 +++--- dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.h | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.cpp b/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.cpp index 165b6a1f73c..d609b167f01 100644 --- a/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.cpp +++ b/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.cpp @@ -40,15 +40,15 @@ std::optional CHBlockChunkDecodeAndSquash::decodeAndSquashV1(std::string_ if (static_cast(sv[0]) == CompressionMethodByte::NONE) { ReadBufferFromString istr(sv.substr(1, sv.size() - 1)); - return decodeAndSquashWithCompressionImpl(istr); + return decodeAndSquashV1Impl(istr); } ReadBufferFromString istr(sv); auto && compress_buffer = CompressedCHBlockChunkReadBuffer(istr); - return decodeAndSquashWithCompressionImpl(compress_buffer); + return decodeAndSquashV1Impl(compress_buffer); } -std::optional CHBlockChunkDecodeAndSquash::decodeAndSquashWithCompressionImpl(ReadBuffer & istr) +std::optional CHBlockChunkDecodeAndSquash::decodeAndSquashV1Impl(ReadBuffer & istr) { std::optional res; diff --git a/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.h b/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.h index 86b29023050..df5586d01b1 100644 --- a/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.h +++ b/dbms/src/Flash/Coprocessor/ChunkDecodeAndSquash.h @@ -31,7 +31,7 @@ class CHBlockChunkDecodeAndSquash std::optional flush(); private: - std::optional decodeAndSquashWithCompressionImpl(ReadBuffer & istr); + std::optional decodeAndSquashV1Impl(ReadBuffer & istr); private: CHBlockChunkCodec codec; From 19facbf3572cb5066a1d5ad0d0f93446dcd9afba Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Thu, 19 Jan 2023 13:00:41 +0800 Subject: [PATCH 88/93] 75 Signed-off-by: Zhigao Tong --- .../Flash/Coprocessor/CHBlockChunkCodecV1.cpp | 29 ++++++++++++------- .../Flash/Coprocessor/CHBlockChunkCodecV1.h | 4 +-- .../tests/gtest_block_chunk_codec.cpp | 4 +-- .../tests/gtest_chunk_decode_and_squash.cpp | 2 +- 4 files changed, 24 insertions(+), 15 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.cpp b/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.cpp index 139665e3e8e..c8a2f0eb04d 100644 --- a/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.cpp +++ b/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.cpp @@ -240,9 +240,9 @@ struct CHBlockChunkCodecV1Impl : inner(inner_) {} - std::string encode(CompressionMethod compression_method) + std::string encode(const Block & block, CompressionMethod compression_method) { - return encodeImpl(inner.header, compression_method); + return encodeImpl(block, compression_method); } std::string encode(const std::vector & blocks, CompressionMethod compression_method) { @@ -433,9 +433,23 @@ CHBlockChunkCodecV1::CHBlockChunkCodecV1(const Block & header_, bool always_keep { } -std::string CHBlockChunkCodecV1::encode(CompressionMethod compression_method) +static void checkSchema(const Block & header, const Block & block) { - return CHBlockChunkCodecV1Impl{*this}.encode(compression_method); + CodecUtils::checkColumnSize(header.columns(), block.columns()); + for (size_t column_index = 0; column_index < header.columns(); ++column_index) + { + auto && type_name = block.getByPosition(column_index).type->getName(); + CodecUtils::checkDataTypeName(column_index, header.getByPosition(column_index).type->getName(), type_name); + } +} + +std::string CHBlockChunkCodecV1::encode(const Block & block, CompressionMethod compression_method, bool check_schema) +{ + if (check_schema) + { + checkSchema(header, block); + } + return CHBlockChunkCodecV1Impl{*this}.encode(block, compression_method); } void CHBlockChunkCodecV1::clear() @@ -475,12 +489,7 @@ std::string CHBlockChunkCodecV1::encode(const std::vector & blocks, Compr { for (auto && block : blocks) { - CodecUtils::checkColumnSize(header.columns(), block.columns()); - for (size_t column_index = 0; column_index < header.columns(); ++column_index) - { - auto && type_name = block.getByPosition(column_index).type->getName(); - CodecUtils::checkDataTypeName(column_index, header.getByPosition(column_index).type->getName(), type_name); - } + checkSchema(header, block); } } diff --git a/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.h b/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.h index 1171d6c4e42..e7aa5def633 100644 --- a/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.h +++ b/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.h @@ -30,7 +30,7 @@ Block DecodeHeader(ReadBuffer & istr, const Block & header, size_t & rows); CompressionMethod ToInternalCompressionMethod(tipb::CompressionMode compression_mode); extern void WriteColumnData(const IDataType & type, const ColumnPtr & column, WriteBuffer & ostr, size_t offset, size_t limit); -struct CHBlockChunkCodecV1 +struct CHBlockChunkCodecV1 : boost::noncopyable { using Self = CHBlockChunkCodecV1; @@ -51,7 +51,7 @@ struct CHBlockChunkCodecV1 std::string encode(const Columns & columns, CompressionMethod compression_method); std::string encode(const std::vector & columns, CompressionMethod compression_method); std::string encode(std::vector && columns, CompressionMethod compression_method); - std::string encode(CompressionMethod compression_method); + std::string encode(const Block & block, CompressionMethod compression_method, bool check_schema = true); std::string encode(const std::vector & blocks, CompressionMethod compression_method, bool check_schema = true); // static Block decode(const Block & header, std::string_view str); diff --git a/dbms/src/Flash/Coprocessor/tests/gtest_block_chunk_codec.cpp b/dbms/src/Flash/Coprocessor/tests/gtest_block_chunk_codec.cpp index f6e82e3c16b..f40d4abb197 100644 --- a/dbms/src/Flash/Coprocessor/tests/gtest_block_chunk_codec.cpp +++ b/dbms/src/Flash/Coprocessor/tests/gtest_block_chunk_codec.cpp @@ -95,7 +95,7 @@ TEST(CHBlockChunkCodec, ChunkCodecV1) { // encode nothing if no rows auto codec = CHBlockChunkCodecV1{header, false}; - auto str = codec.encode(mode); + auto str = codec.encode(header, mode); ASSERT_TRUE(str.empty()); ASSERT_EQ(codec.encoded_rows, 0); ASSERT_EQ(codec.compressed_size, 0); @@ -104,7 +104,7 @@ TEST(CHBlockChunkCodec, ChunkCodecV1) { // encode header schema if no rows auto codec = CHBlockChunkCodecV1{header, true}; - auto str = codec.encode(mode); + auto str = codec.encode(header, mode); ASSERT_FALSE(str.empty()); ASSERT_EQ(codec.encoded_rows, 0); diff --git a/dbms/src/Flash/Coprocessor/tests/gtest_chunk_decode_and_squash.cpp b/dbms/src/Flash/Coprocessor/tests/gtest_chunk_decode_and_squash.cpp index c322f71c7a7..17f5044a400 100644 --- a/dbms/src/Flash/Coprocessor/tests/gtest_chunk_decode_and_squash.cpp +++ b/dbms/src/Flash/Coprocessor/tests/gtest_chunk_decode_and_squash.cpp @@ -124,7 +124,7 @@ class TestChunkDecodeAndSquash : public testing::Test else { auto codec = CHBlockChunkCodecV1{block, true}; - auto && str = codec.encode(CompressionMethod::LZ4); + auto && str = codec.encode(block, CompressionMethod::LZ4); assert(!str.empty()); assert(static_cast(str[0]) == CompressionMethodByte::LZ4); encode_str_vec.push_back(std::move(str)); From 2fa197b73c215a40860c294350f7e0b8e2fd2c39 Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Thu, 19 Jan 2023 14:03:14 +0800 Subject: [PATCH 89/93] 76 Signed-off-by: Zhigao Tong --- .../Flash/Coprocessor/CHBlockChunkCodecV1.cpp | 1 - .../tests/gtest_block_chunk_codec.cpp | 33 ++++++++++++++----- 2 files changed, 25 insertions(+), 9 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.cpp b/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.cpp index c8a2f0eb04d..ff60ef86c5b 100644 --- a/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.cpp +++ b/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.cpp @@ -346,7 +346,6 @@ struct CHBlockChunkCodecV1Impl } void encodeColumn(const Block & block, WriteBuffer * ostr_ptr) { - assert(&block == &inner.header); return encodeColumnImpl(block, ostr_ptr); } void encodeColumn(const std::vector & blocks, WriteBuffer * ostr_ptr) diff --git a/dbms/src/Flash/Coprocessor/tests/gtest_block_chunk_codec.cpp b/dbms/src/Flash/Coprocessor/tests/gtest_block_chunk_codec.cpp index f40d4abb197..67dae7cf2bf 100644 --- a/dbms/src/Flash/Coprocessor/tests/gtest_block_chunk_codec.cpp +++ b/dbms/src/Flash/Coprocessor/tests/gtest_block_chunk_codec.cpp @@ -18,9 +18,6 @@ #include #include -#include - -#include "Core/Block.h" namespace DB::tests { @@ -47,11 +44,6 @@ void test_enocde_release_data(VecCol && batch_columns, const Block & header, con // encode and release columns const auto mode = CompressionMethod::LZ4; - for (auto && columns : batch_columns) - { - for (auto && col : columns) - ASSERT_TRUE(col); - } auto codec = CHBlockChunkCodecV1{header, false}; auto str = codec.encode(std::forward(batch_columns), mode); ASSERT_FALSE(str.empty()); @@ -119,6 +111,15 @@ TEST(CHBlockChunkCodec, ChunkCodecV1) ASSERT_EQ(0, decoded_block.rows()); } { + auto codec = CHBlockChunkCodecV1{header, false}; + auto str = codec.encode(blocks.front(), mode); + ASSERT_FALSE(str.empty()); + ASSERT_EQ(codec.encoded_rows, blocks.front().rows()); + auto decoded_block = CHBlockChunkCodecV1::decode(header, str); + ASSERT_EQ(blocks.front().rows(), decoded_block.rows()); + } + { + // test encode blocks auto codec = CHBlockChunkCodecV1{header, false}; auto str = codec.encode(blocks, mode); ASSERT_FALSE(str.empty()); @@ -158,6 +159,14 @@ TEST(CHBlockChunkCodec, ChunkCodecV1) for (size_t i = 0; i < block_num; ++i) batch_columns.emplace_back(prepareBlock(rows).mutateColumns()); batch_columns.emplace_back(prepareBlock(0).mutateColumns()); + { + auto tmp = prepareBlock(0).mutateColumns(); + for (auto && col : tmp) + { + col.reset(); + } + batch_columns.emplace_back(std::move(tmp)); + } test_enocde_release_data(std::move(batch_columns), header, total_rows); } { @@ -165,6 +174,14 @@ TEST(CHBlockChunkCodec, ChunkCodecV1) for (size_t i = 0; i < block_num; ++i) batch_columns.emplace_back(prepareBlock(rows).getColumns()); batch_columns.emplace_back(prepareBlock(0).getColumns()); + { + auto tmp = prepareBlock(0).getColumns(); + for (auto && col : tmp) + { + col.reset(); + } + batch_columns.emplace_back(std::move(tmp)); + } test_enocde_release_data(std::move(batch_columns), header, total_rows); } } From 6060432a90f5915b2ce5d0e79602acb908969a1c Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Mon, 23 Jan 2023 21:06:53 +0800 Subject: [PATCH 90/93] 77 Signed-off-by: Zhigao Tong --- dbms/src/Common/TiFlashMetrics.h | 19 +++++---- .../gtest_ti_remote_block_inputstream.cpp | 2 +- dbms/src/Flash/Mpp/HashPartitionWriter.cpp | 13 +++--- dbms/src/Flash/Mpp/HashPartitionWriter.h | 1 - dbms/src/Flash/Mpp/MPPTunnelSet.cpp | 41 +++++++++++-------- dbms/src/Flash/Mpp/MPPTunnelSetHelper.cpp | 6 +-- dbms/src/Flash/Mpp/MPPTunnelSetHelper.h | 4 +- .../Mpp/tests/gtest_mpp_exchange_writer.cpp | 7 ++-- 8 files changed, 50 insertions(+), 43 deletions(-) diff --git a/dbms/src/Common/TiFlashMetrics.h b/dbms/src/Common/TiFlashMetrics.h index f497b2960cb..4273e0c137c 100644 --- a/dbms/src/Common/TiFlashMetrics.h +++ b/dbms/src/Common/TiFlashMetrics.h @@ -86,15 +86,16 @@ namespace DB F(type_mpp_establish_conn, {{"type", "mpp_tunnel"}}), \ F(type_mpp_establish_conn_local, {{"type", "mpp_tunnel_local"}}), \ F(type_cancel_mpp_task, {{"type", "cancel_mpp_task"}})) \ - M(tiflash_exchange_data_bytes, "Total bytes of exchange operator", Counter, \ - F(type_hash_original_all, {"type", "hash_original_all"}), \ - F(type_hash_none_remote, {"type", "hash_none_remote"}), \ - F(type_hash_none_local, {"type", "hash_none_local"}), \ - F(type_hash_lz4, {"type", "hash_lz4"}), \ - F(type_hash_zstd, {"type", "hash_zstd"}), \ - F(type_broadcast_passthrough_original_all, {"type", "broadcast_passthrough_original_all"}), \ - F(type_broadcast_passthrough_none_local, {"type", "broadcast_passthrough_none_local"}), \ - ) \ + M(tiflash_exchange_data_bytes, "Total bytes sent by exchange operators", Counter, \ + F(type_hash_original, {"type", "hash_original"}), /*the original data size by hash exchange*/ \ + F(type_hash_none_compression_remote, {"type", "hash_none_compression_remote"}), /*the remote exchange data size by hash partition with no compression*/\ + F(type_hash_none_compression_local, {"type", "hash_none_compression_local"}), /*the local exchange data size by hash partition with no compression*/ \ + F(type_hash_lz4_compression, {"type", "hash_lz4_compression"}), /*the exchange data size by hash partition with lz4 compression*/ \ + F(type_hash_zstd_compression, {"type", "hash_zstd_compression"}), /*the exchange data size by hash partition with zstd compression*/ \ + F(type_broadcast_passthrough_original, {"type", "broadcast_passthrough_original"}), /*the original exchange data size by broadcast/passthough*/ \ + F(type_broadcast_passthrough_none_compression_local, {"type", "broadcast_passthrough_none_compression_local"}), /*the local exchange data size by broadcast/passthough with no compression*/ \ + F(type_broadcast_passthrough_none_compression_remote, {"type", "broadcast_passthrough_none_compression_remote"}), /*the remote exchange data size by broadcast/passthough with no compression*/ \ + ) \ M(tiflash_schema_version, "Current version of tiflash cached schema", Gauge) \ M(tiflash_schema_applying, "Whether the schema is applying or not (holding lock)", Gauge) \ M(tiflash_schema_apply_count, "Total number of each kinds of apply", Counter, F(type_diff, {"type", "diff"}), \ diff --git a/dbms/src/Flash/Coprocessor/tests/gtest_ti_remote_block_inputstream.cpp b/dbms/src/Flash/Coprocessor/tests/gtest_ti_remote_block_inputstream.cpp index aabf7318315..7794d9c37c7 100644 --- a/dbms/src/Flash/Coprocessor/tests/gtest_ti_remote_block_inputstream.cpp +++ b/dbms/src/Flash/Coprocessor/tests/gtest_ti_remote_block_inputstream.cpp @@ -86,7 +86,7 @@ struct MockWriter void broadcastOrPassThroughWrite(Blocks & blocks) { - auto packet = MPPTunnelSetHelper::toPacket(blocks, result_field_types, MPPDataPacketV0); + auto packet = MPPTunnelSetHelper::ToPacket(blocks, result_field_types, MPPDataPacketV0); ++total_packets; if (!packet->packet.chunks().empty()) total_bytes += packet->packet.ByteSizeLong(); diff --git a/dbms/src/Flash/Mpp/HashPartitionWriter.cpp b/dbms/src/Flash/Mpp/HashPartitionWriter.cpp index 8a68a94b5d8..04f9e2a40a2 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriter.cpp +++ b/dbms/src/Flash/Mpp/HashPartitionWriter.cpp @@ -39,7 +39,6 @@ HashPartitionWriter::HashPartitionWriter( , writer(writer_) , partition_col_ids(std::move(partition_col_ids_)) , collators(std::move(collators_)) - , batch_send_min_limit_mem_size(batch_send_min_limit_) , data_codec_version(data_codec_version_) , compression_method(ToInternalCompressionMethod(compression_mode_)) { @@ -55,10 +54,11 @@ HashPartitionWriter::HashPartitionWriter( case MPPDataPacketV1: default: { - if (batch_send_min_limit_mem_size < 0) + // make `batch_send_min_limit` always GT 0 + if (batch_send_min_limit <= 0) { // set upper limit if not specified - batch_send_min_limit_mem_size = std::min(MAX_BATCH_SEND_MIN_LIMIT_MEM_SIZE, 1024 * 64 * partition_num /* 64KB * partition-num */); + batch_send_min_limit = 8 * 1024 * partition_num /* 8K * partition-num */; } for (const auto & field_type : dag_context.result_field_types) { @@ -101,7 +101,8 @@ void HashPartitionWriter::writeImplV1(const Block & block) mem_size_in_blocks += block.bytes(); blocks.push_back(block); } - if (mem_size_in_blocks > batch_send_min_limit_mem_size) + if (static_cast(rows_in_blocks) >= batch_send_min_limit + || mem_size_in_blocks >= MAX_BATCH_SEND_MIN_LIMIT_MEM_SIZE) partitionAndWriteBlocksV1(); } @@ -144,9 +145,7 @@ void HashPartitionWriter::partitionAndWriteBlocksV1() { assert(rows_in_blocks > 0); assert(mem_size_in_blocks > 0); - - if unlikely (blocks.empty()) - return; + assert(!blocks.empty()); HashBaseWriterHelper::materializeBlocks(blocks); // All blocks are same, use one block's meta info as header diff --git a/dbms/src/Flash/Mpp/HashPartitionWriter.h b/dbms/src/Flash/Mpp/HashPartitionWriter.h index 43b78e1b8db..096e6df465c 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriter.h +++ b/dbms/src/Flash/Mpp/HashPartitionWriter.h @@ -58,7 +58,6 @@ class HashPartitionWriter : public DAGResponseWriter uint16_t partition_num; // support data compression int64_t mem_size_in_blocks{}; - int64_t batch_send_min_limit_mem_size; DataTypes expected_types; MPPDataPacketVersion data_codec_version; CompressionMethod compression_method{}; diff --git a/dbms/src/Flash/Mpp/MPPTunnelSet.cpp b/dbms/src/Flash/Mpp/MPPTunnelSet.cpp index b240718811e..90c44ae79b5 100644 --- a/dbms/src/Flash/Mpp/MPPTunnelSet.cpp +++ b/dbms/src/Flash/Mpp/MPPTunnelSet.cpp @@ -66,18 +66,18 @@ void MPPTunnelSetBase::write(tipb::SelectResponse & response) static inline void updatePartitionWriterMetrics(size_t packet_bytes, bool is_local) { // statistic - GET_METRIC(tiflash_exchange_data_bytes, type_hash_original_all).Increment(packet_bytes); + GET_METRIC(tiflash_exchange_data_bytes, type_hash_original).Increment(packet_bytes); // compression method is always NONE if (is_local) - GET_METRIC(tiflash_exchange_data_bytes, type_hash_none_local).Increment(packet_bytes); + GET_METRIC(tiflash_exchange_data_bytes, type_hash_none_compression_local).Increment(packet_bytes); else - GET_METRIC(tiflash_exchange_data_bytes, type_hash_none_remote).Increment(packet_bytes); + GET_METRIC(tiflash_exchange_data_bytes, type_hash_none_compression_remote).Increment(packet_bytes); } static inline void updatePartitionWriterMetrics(CompressionMethod method, size_t original_size, size_t sz, bool is_local) { // statistic - GET_METRIC(tiflash_exchange_data_bytes, type_hash_original_all).Increment(original_size); + GET_METRIC(tiflash_exchange_data_bytes, type_hash_original).Increment(original_size); switch (method) { @@ -85,22 +85,22 @@ static inline void updatePartitionWriterMetrics(CompressionMethod method, size_t { if (is_local) { - GET_METRIC(tiflash_exchange_data_bytes, type_hash_none_local).Increment(sz); + GET_METRIC(tiflash_exchange_data_bytes, type_hash_none_compression_local).Increment(sz); } else { - GET_METRIC(tiflash_exchange_data_bytes, type_hash_none_remote).Increment(sz); + GET_METRIC(tiflash_exchange_data_bytes, type_hash_none_compression_remote).Increment(sz); } break; } case CompressionMethod::LZ4: { - GET_METRIC(tiflash_exchange_data_bytes, type_hash_lz4).Increment(sz); + GET_METRIC(tiflash_exchange_data_bytes, type_hash_lz4_compression).Increment(sz); break; } case CompressionMethod::ZSTD: { - GET_METRIC(tiflash_exchange_data_bytes, type_hash_zstd).Increment(sz); + GET_METRIC(tiflash_exchange_data_bytes, type_hash_zstd_compression).Increment(sz); break; } default: @@ -112,7 +112,7 @@ template void MPPTunnelSetBase::broadcastOrPassThroughWrite(Blocks & blocks) { RUNTIME_CHECK(!tunnels.empty()); - auto tracked_packet = MPPTunnelSetHelper::toPacket(blocks, result_field_types, MPPDataPacketV0); + auto tracked_packet = MPPTunnelSetHelper::ToPacket(blocks, result_field_types, MPPDataPacketV0); auto packet_bytes = tracked_packet->getPacket().ByteSizeLong(); checkPacketSize(packet_bytes); // TODO avoid copy packet for broadcast. @@ -121,21 +121,28 @@ void MPPTunnelSetBase::broadcastOrPassThroughWrite(Blocks & blocks) tunnels[0]->write(std::move(tracked_packet)); { // statistic - auto tunnel_cnt = getPartitionNum(); - size_t local_tunnel_cnt = 0; - for (size_t i = 0; i < tunnel_cnt; ++i) + size_t data_bytes = 0; + size_t local_data_bytes = 0; { - local_tunnel_cnt += isLocal(i); + auto tunnel_cnt = getPartitionNum(); + size_t local_tunnel_cnt = 0; + for (size_t i = 0; i < tunnel_cnt; ++i) + { + local_tunnel_cnt += isLocal(i); + } + data_bytes = packet_bytes * tunnel_cnt; + local_data_bytes = packet_bytes * local_tunnel_cnt; } - GET_METRIC(tiflash_exchange_data_bytes, type_broadcast_passthrough_original_all).Increment(packet_bytes * tunnel_cnt); - GET_METRIC(tiflash_exchange_data_bytes, type_broadcast_passthrough_none_local).Increment(packet_bytes * local_tunnel_cnt); + GET_METRIC(tiflash_exchange_data_bytes, type_broadcast_passthrough_original).Increment(data_bytes); + GET_METRIC(tiflash_exchange_data_bytes, type_broadcast_passthrough_none_compression_local).Increment(local_data_bytes); + GET_METRIC(tiflash_exchange_data_bytes, type_broadcast_passthrough_none_compression_remote).Increment(data_bytes - local_data_bytes); } } template void MPPTunnelSetBase::partitionWrite(Blocks & blocks, int16_t partition_id) { - auto tracked_packet = MPPTunnelSetHelper::toPacket(blocks, result_field_types, MPPDataPacketV0); + auto tracked_packet = MPPTunnelSetHelper::ToPacket(blocks, result_field_types, MPPDataPacketV0); if unlikely (tracked_packet->getPacket().chunks_size() <= 0) return; @@ -216,7 +223,7 @@ void MPPTunnelSetBase::fineGrainedShuffleWrite( size_t num_columns, int16_t partition_id) { - auto tracked_packet = MPPTunnelSetHelper::toFineGrainedPacket( + auto tracked_packet = MPPTunnelSetHelper::ToFineGrainedPacket( header, scattered, bucket_idx, diff --git a/dbms/src/Flash/Mpp/MPPTunnelSetHelper.cpp b/dbms/src/Flash/Mpp/MPPTunnelSetHelper.cpp index 47b3ee76c19..8de5fa5f22a 100644 --- a/dbms/src/Flash/Mpp/MPPTunnelSetHelper.cpp +++ b/dbms/src/Flash/Mpp/MPPTunnelSetHelper.cpp @@ -30,7 +30,7 @@ TrackedMppDataPacketPtr ToPacket( auto && codec = CHBlockChunkCodecV1{ header, - false /*scape empty part*/, + false /*escape empty part*/, }; auto && res = codec.encode(std::move(part_columns), method); @@ -43,7 +43,7 @@ TrackedMppDataPacketPtr ToPacket( return tracked_packet; } -TrackedMppDataPacketPtr toPacket(Blocks & blocks, const std::vector & field_types, MPPDataPacketVersion version) +TrackedMppDataPacketPtr ToPacket(Blocks & blocks, const std::vector & field_types, MPPDataPacketVersion version) { assert(version == MPPDataPacketV0); @@ -104,7 +104,7 @@ TrackedMppDataPacketPtr ToFineGrainedPacket( return tracked_packet; } -TrackedMppDataPacketPtr toFineGrainedPacket( +TrackedMppDataPacketPtr ToFineGrainedPacket( const Block & header, std::vector & scattered, size_t bucket_idx, diff --git a/dbms/src/Flash/Mpp/MPPTunnelSetHelper.h b/dbms/src/Flash/Mpp/MPPTunnelSetHelper.h index 030e74a06bb..c04e531d2e9 100644 --- a/dbms/src/Flash/Mpp/MPPTunnelSetHelper.h +++ b/dbms/src/Flash/Mpp/MPPTunnelSetHelper.h @@ -25,7 +25,7 @@ enum class CompressionMethod; namespace DB::MPPTunnelSetHelper { -TrackedMppDataPacketPtr toPacket(Blocks & blocks, const std::vector & field_types, MPPDataPacketVersion version); +TrackedMppDataPacketPtr ToPacket(Blocks & blocks, const std::vector & field_types, MPPDataPacketVersion version); TrackedMppDataPacketPtr ToPacket( const Block & header, @@ -34,7 +34,7 @@ TrackedMppDataPacketPtr ToPacket( CompressionMethod compression_method, size_t & original_size); -TrackedMppDataPacketPtr toFineGrainedPacket( +TrackedMppDataPacketPtr ToFineGrainedPacket( const Block & header, std::vector & scattered, size_t bucket_idx, diff --git a/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp b/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp index f09a362351f..6f310a4bf20 100644 --- a/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp +++ b/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp @@ -135,6 +135,7 @@ struct MockExchangeWriter MPPDataPacketVersion version, CompressionMethod method) { + assert(version > MPPDataPacketV0); method = isLocal(part_id) ? CompressionMethod::NONE : method; size_t original_size = 0; auto tracked_packet = MPPTunnelSetHelper::ToPacket(header, std::move(part_columns), version, method, original_size); @@ -167,11 +168,11 @@ struct MockExchangeWriter } void broadcastOrPassThroughWrite(Blocks & blocks) { - checker(MPPTunnelSetHelper::toPacket(blocks, result_field_types, MPPDataPacketV0), 0); + checker(MPPTunnelSetHelper::ToPacket(blocks, result_field_types, MPPDataPacketV0), 0); } void partitionWrite(Blocks & blocks, uint16_t part_id) { - checker(MPPTunnelSetHelper::toPacket(blocks, result_field_types, MPPDataPacketV0), part_id); + checker(MPPTunnelSetHelper::ToPacket(blocks, result_field_types, MPPDataPacketV0), part_id); } void fineGrainedShuffleWrite( const Block & header, @@ -181,7 +182,7 @@ struct MockExchangeWriter size_t num_columns, int16_t part_id) { - auto tracked_packet = MPPTunnelSetHelper::toFineGrainedPacket( + auto tracked_packet = MPPTunnelSetHelper::ToFineGrainedPacket( header, scattered, bucket_idx, From 57032d1105c69d535cfe6d625769cb99a6c7ca1e Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Sat, 28 Jan 2023 13:36:26 +0800 Subject: [PATCH 91/93] 78 Signed-off-by: Zhigao Tong --- dbms/src/Flash/Mpp/HashPartitionWriter.cpp | 2 +- metrics/grafana/tiflash_summary.json | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Flash/Mpp/HashPartitionWriter.cpp b/dbms/src/Flash/Mpp/HashPartitionWriter.cpp index 04f9e2a40a2..18e87bdc1c2 100644 --- a/dbms/src/Flash/Mpp/HashPartitionWriter.cpp +++ b/dbms/src/Flash/Mpp/HashPartitionWriter.cpp @@ -22,7 +22,7 @@ namespace DB { -constexpr ssize_t MAX_BATCH_SEND_MIN_LIMIT_MEM_SIZE = 1024 * 1024 * 128; // 128MB +constexpr ssize_t MAX_BATCH_SEND_MIN_LIMIT_MEM_SIZE = 1024 * 1024 * 64; // 64MB: 8192 Rows * 256 Byte/row * 32 partitions const char * HashPartitionWriterLabels[] = {"HashPartitionWriter", "HashPartitionWriter-V1"}; template diff --git a/metrics/grafana/tiflash_summary.json b/metrics/grafana/tiflash_summary.json index d0604a07852..eb8d77937a2 100644 --- a/metrics/grafana/tiflash_summary.json +++ b/metrics/grafana/tiflash_summary.json @@ -3386,7 +3386,7 @@ "h": 7, "w": 12, "x": 12, - "y": 54 + "y": 24 }, "hiddenSeries": false, "id": 165, From 3c8155f3ec598e03b0ba20f07fcc76875dd19ee1 Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Sun, 29 Jan 2023 09:46:10 +0800 Subject: [PATCH 92/93] 79 Signed-off-by: Zhigao Tong --- .../Flash/Coprocessor/CHBlockChunkCodecV1.cpp | 46 +++++++++---------- .../Flash/Coprocessor/CHBlockChunkCodecV1.h | 20 ++++---- .../tests/gtest_block_chunk_codec.cpp | 41 ++++++++--------- .../tests/gtest_chunk_decode_and_squash.cpp | 6 +-- dbms/src/Flash/Mpp/ExchangeReceiver.cpp | 5 +- dbms/src/Flash/Mpp/MPPTunnelSetHelper.cpp | 25 +++++----- 6 files changed, 69 insertions(+), 74 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.cpp b/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.cpp index ff60ef86c5b..5840b0e8e57 100644 --- a/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.cpp +++ b/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.cpp @@ -240,11 +240,11 @@ struct CHBlockChunkCodecV1Impl : inner(inner_) {} - std::string encode(const Block & block, CompressionMethod compression_method) + CHBlockChunkCodecV1::EncodeRes encode(const Block & block, CompressionMethod compression_method) { return encodeImpl(block, compression_method); } - std::string encode(const std::vector & blocks, CompressionMethod compression_method) + CHBlockChunkCodecV1::EncodeRes encode(const std::vector & blocks, CompressionMethod compression_method) { return encodeImpl(blocks, compression_method); } @@ -356,15 +356,17 @@ struct CHBlockChunkCodecV1Impl } } template - std::string encodeImpl(VecColumns && batch_columns, CompressionMethod compression_method) + CHBlockChunkCodecV1::EncodeRes encodeImpl(VecColumns && batch_columns, CompressionMethod compression_method) { size_t column_encode_bytes = 0; - size_t rows = 0; - - // Calculate total rows and check data valid - calcColumnEncodeInfo(batch_columns, column_encode_bytes, rows); - - if unlikely (0 == rows && !inner.always_keep_header) + const size_t rows = ({ + size_t rows = 0; + // Calculate total rows and check data valid + calcColumnEncodeInfo(batch_columns, column_encode_bytes, rows); + rows; + }); + + if unlikely (0 == rows) { // no rows and no need to encode header return {}; @@ -405,10 +407,9 @@ struct CHBlockChunkCodecV1Impl // Encode header EncodeHeader(*ostr_ptr, inner.header, rows); - if (rows > 0) - { - encodeColumn(std::forward(batch_columns), ostr_ptr); - } + // Encode column data + encodeColumn(std::forward(batch_columns), ostr_ptr); + // Flush rest buffer if (compress_codec) { @@ -425,10 +426,9 @@ struct CHBlockChunkCodecV1Impl } }; -CHBlockChunkCodecV1::CHBlockChunkCodecV1(const Block & header_, bool always_keep_header_) +CHBlockChunkCodecV1::CHBlockChunkCodecV1(const Block & header_) : header(header_) , header_size(ApproxBlockHeaderBytes(header)) - , always_keep_header(always_keep_header_) { } @@ -442,7 +442,7 @@ static void checkSchema(const Block & header, const Block & block) } } -std::string CHBlockChunkCodecV1::encode(const Block & block, CompressionMethod compression_method, bool check_schema) +CHBlockChunkCodecV1::EncodeRes CHBlockChunkCodecV1::encode(const Block & block, CompressionMethod compression_method, bool check_schema) { if (check_schema) { @@ -458,31 +458,31 @@ void CHBlockChunkCodecV1::clear() compressed_size = 0; } -std::string CHBlockChunkCodecV1::encode(const MutableColumns & columns, CompressionMethod compression_method) +CHBlockChunkCodecV1::EncodeRes CHBlockChunkCodecV1::encode(const MutableColumns & columns, CompressionMethod compression_method) { return CHBlockChunkCodecV1Impl{*this}.encodeImpl(columns, compression_method); } -std::string CHBlockChunkCodecV1::encode(const Columns & columns, CompressionMethod compression_method) +CHBlockChunkCodecV1::EncodeRes CHBlockChunkCodecV1::encode(const Columns & columns, CompressionMethod compression_method) { return CHBlockChunkCodecV1Impl{*this}.encodeImpl(columns, compression_method); } -std::string CHBlockChunkCodecV1::encode(const std::vector & columns, CompressionMethod compression_method) +CHBlockChunkCodecV1::EncodeRes CHBlockChunkCodecV1::encode(const std::vector & columns, CompressionMethod compression_method) { return CHBlockChunkCodecV1Impl{*this}.encodeImpl(columns, compression_method); } -std::string CHBlockChunkCodecV1::encode(std::vector && columns, CompressionMethod compression_method) +CHBlockChunkCodecV1::EncodeRes CHBlockChunkCodecV1::encode(std::vector && columns, CompressionMethod compression_method) { return CHBlockChunkCodecV1Impl{*this}.encodeImpl(std::move(columns), compression_method); } -std::string CHBlockChunkCodecV1::encode(const std::vector & columns, CompressionMethod compression_method) +CHBlockChunkCodecV1::EncodeRes CHBlockChunkCodecV1::encode(const std::vector & columns, CompressionMethod compression_method) { return CHBlockChunkCodecV1Impl{*this}.encodeImpl(std::move(columns), compression_method); } -std::string CHBlockChunkCodecV1::encode(std::vector && columns, CompressionMethod compression_method) +CHBlockChunkCodecV1::EncodeRes CHBlockChunkCodecV1::encode(std::vector && columns, CompressionMethod compression_method) { return CHBlockChunkCodecV1Impl{*this}.encodeImpl(std::move(columns), compression_method); } -std::string CHBlockChunkCodecV1::encode(const std::vector & blocks, CompressionMethod compression_method, bool check_schema) +CHBlockChunkCodecV1::EncodeRes CHBlockChunkCodecV1::encode(const std::vector & blocks, CompressionMethod compression_method, bool check_schema) { if (check_schema) { diff --git a/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.h b/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.h index e7aa5def633..76331ce8314 100644 --- a/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.h +++ b/dbms/src/Flash/Coprocessor/CHBlockChunkCodecV1.h @@ -33,26 +33,26 @@ extern void WriteColumnData(const IDataType & type, const ColumnPtr & column, Wr struct CHBlockChunkCodecV1 : boost::noncopyable { using Self = CHBlockChunkCodecV1; + using EncodeRes = std::string; const Block & header; const size_t header_size; - const bool always_keep_header; size_t encoded_rows{}; size_t original_size{}; size_t compressed_size{}; void clear(); - explicit CHBlockChunkCodecV1(const Block & header_, bool always_keep_header_); + explicit CHBlockChunkCodecV1(const Block & header_); // - std::string encode(const MutableColumns & columns, CompressionMethod compression_method); - std::string encode(std::vector && columns, CompressionMethod compression_method); - std::string encode(const std::vector & columns, CompressionMethod compression_method); - std::string encode(const Columns & columns, CompressionMethod compression_method); - std::string encode(const std::vector & columns, CompressionMethod compression_method); - std::string encode(std::vector && columns, CompressionMethod compression_method); - std::string encode(const Block & block, CompressionMethod compression_method, bool check_schema = true); - std::string encode(const std::vector & blocks, CompressionMethod compression_method, bool check_schema = true); + EncodeRes encode(const MutableColumns & columns, CompressionMethod compression_method); + EncodeRes encode(std::vector && columns, CompressionMethod compression_method); + EncodeRes encode(const std::vector & columns, CompressionMethod compression_method); + EncodeRes encode(const Columns & columns, CompressionMethod compression_method); + EncodeRes encode(const std::vector & columns, CompressionMethod compression_method); + EncodeRes encode(std::vector && columns, CompressionMethod compression_method); + EncodeRes encode(const Block & block, CompressionMethod compression_method, bool check_schema = true); + EncodeRes encode(const std::vector & blocks, CompressionMethod compression_method, bool check_schema = true); // static Block decode(const Block & header, std::string_view str); }; diff --git a/dbms/src/Flash/Coprocessor/tests/gtest_block_chunk_codec.cpp b/dbms/src/Flash/Coprocessor/tests/gtest_block_chunk_codec.cpp index 67dae7cf2bf..d9f7a5f5c78 100644 --- a/dbms/src/Flash/Coprocessor/tests/gtest_block_chunk_codec.cpp +++ b/dbms/src/Flash/Coprocessor/tests/gtest_block_chunk_codec.cpp @@ -44,7 +44,9 @@ void test_enocde_release_data(VecCol && batch_columns, const Block & header, con // encode and release columns const auto mode = CompressionMethod::LZ4; - auto codec = CHBlockChunkCodecV1{header, false}; + auto codec = CHBlockChunkCodecV1{ + header, + }; auto str = codec.encode(std::forward(batch_columns), mode); ASSERT_FALSE(str.empty()); ASSERT_EQ(codec.encoded_rows, total_rows); @@ -86,7 +88,9 @@ TEST(CHBlockChunkCodec, ChunkCodecV1) { { // encode nothing if no rows - auto codec = CHBlockChunkCodecV1{header, false}; + auto codec = CHBlockChunkCodecV1{ + header, + }; auto str = codec.encode(header, mode); ASSERT_TRUE(str.empty()); ASSERT_EQ(codec.encoded_rows, 0); @@ -94,24 +98,9 @@ TEST(CHBlockChunkCodec, ChunkCodecV1) ASSERT_EQ(codec.original_size, 0); } { - // encode header schema if no rows - auto codec = CHBlockChunkCodecV1{header, true}; - auto str = codec.encode(header, mode); - ASSERT_FALSE(str.empty()); - ASSERT_EQ(codec.encoded_rows, 0); - - if (mode == CompressionMethod::NONE) - ASSERT_EQ(codec.compressed_size, 0); - else - ASSERT_NE(codec.compressed_size, 0); - - ASSERT_NE(codec.original_size, 0); - - auto decoded_block = CHBlockChunkCodecV1::decode(header, str); - ASSERT_EQ(0, decoded_block.rows()); - } - { - auto codec = CHBlockChunkCodecV1{header, false}; + auto codec = CHBlockChunkCodecV1{ + header, + }; auto str = codec.encode(blocks.front(), mode); ASSERT_FALSE(str.empty()); ASSERT_EQ(codec.encoded_rows, blocks.front().rows()); @@ -120,7 +109,9 @@ TEST(CHBlockChunkCodec, ChunkCodecV1) } { // test encode blocks - auto codec = CHBlockChunkCodecV1{header, false}; + auto codec = CHBlockChunkCodecV1{ + header, + }; auto str = codec.encode(blocks, mode); ASSERT_FALSE(str.empty()); ASSERT_EQ(codec.encoded_rows, total_rows); @@ -137,7 +128,9 @@ TEST(CHBlockChunkCodec, ChunkCodecV1) } { auto columns = prepareBlock(rows).getColumns(); - auto codec = CHBlockChunkCodecV1{header, false}; + auto codec = CHBlockChunkCodecV1{ + header, + }; auto str = codec.encode(columns, mode); ASSERT_FALSE(str.empty()); ASSERT_EQ(codec.encoded_rows, rows); @@ -146,7 +139,9 @@ TEST(CHBlockChunkCodec, ChunkCodecV1) } { auto columns = prepareBlock(rows).mutateColumns(); - auto codec = CHBlockChunkCodecV1{header, false}; + auto codec = CHBlockChunkCodecV1{ + header, + }; auto str = codec.encode(columns, mode); ASSERT_FALSE(str.empty()); ASSERT_EQ(codec.encoded_rows, rows); diff --git a/dbms/src/Flash/Coprocessor/tests/gtest_chunk_decode_and_squash.cpp b/dbms/src/Flash/Coprocessor/tests/gtest_chunk_decode_and_squash.cpp index 17f5044a400..da81bae040f 100644 --- a/dbms/src/Flash/Coprocessor/tests/gtest_chunk_decode_and_squash.cpp +++ b/dbms/src/Flash/Coprocessor/tests/gtest_chunk_decode_and_squash.cpp @@ -123,10 +123,10 @@ class TestChunkDecodeAndSquash : public testing::Test } else { - auto codec = CHBlockChunkCodecV1{block, true}; + auto codec = CHBlockChunkCodecV1{block}; auto && str = codec.encode(block, CompressionMethod::LZ4); - assert(!str.empty()); - assert(static_cast(str[0]) == CompressionMethodByte::LZ4); + if (!str.empty()) + assert(static_cast(str[0]) == CompressionMethodByte::LZ4); encode_str_vec.push_back(std::move(str)); encode_str_use_compression.emplace_back(true); } diff --git a/dbms/src/Flash/Mpp/ExchangeReceiver.cpp b/dbms/src/Flash/Mpp/ExchangeReceiver.cpp index 7be072b08cb..88f29305dbe 100644 --- a/dbms/src/Flash/Mpp/ExchangeReceiver.cpp +++ b/dbms/src/Flash/Mpp/ExchangeReceiver.cpp @@ -653,7 +653,7 @@ DecodeDetail ExchangeReceiverBase::decodeChunks( // Record total packet size even if fine grained shuffle is enabled. detail.packet_bytes = packet.ByteSizeLong(); - switch (packet.version()) + switch (auto version = packet.version(); version) { case DB::MPPDataPacketV0: { @@ -674,7 +674,6 @@ DecodeDetail ExchangeReceiverBase::decodeChunks( { for (const auto * chunk : recv_msg->chunks) { - assert(!chunk->empty()); auto && result = decoder_ptr->decodeAndSquashV1(*chunk); if (!result || !result->rows()) continue; @@ -685,7 +684,7 @@ DecodeDetail ExchangeReceiverBase::decodeChunks( } default: { - RUNTIME_CHECK_MSG(false, "Unknown mpp packet version {}, please update TiFlash instance", packet.version()); + RUNTIME_CHECK_MSG(false, "Unknown mpp packet version {}, please update TiFlash instance", version); break; } } diff --git a/dbms/src/Flash/Mpp/MPPTunnelSetHelper.cpp b/dbms/src/Flash/Mpp/MPPTunnelSetHelper.cpp index 8de5fa5f22a..2d074b4f51a 100644 --- a/dbms/src/Flash/Mpp/MPPTunnelSetHelper.cpp +++ b/dbms/src/Flash/Mpp/MPPTunnelSetHelper.cpp @@ -30,7 +30,6 @@ TrackedMppDataPacketPtr ToPacket( auto && codec = CHBlockChunkCodecV1{ header, - false /*escape empty part*/, }; auto && res = codec.encode(std::move(part_columns), method); @@ -75,7 +74,6 @@ TrackedMppDataPacketPtr ToFineGrainedPacket( auto && codec = CHBlockChunkCodecV1{ header, - true /*keep header even if no rows*/, }; auto tracked_packet = std::make_shared(version); @@ -88,10 +86,11 @@ TrackedMppDataPacketPtr ToFineGrainedPacket( columns.emplace_back(std::move(scattered[col_id][bucket_idx + stream_idx])); auto && res = codec.encode(columns, method); - assert(!res.empty()); - - tracked_packet->addChunk(std::move(res)); - tracked_packet->getPacket().add_stream_ids(stream_idx); + if (!res.empty()) + { + tracked_packet->addChunk(std::move(res)); + tracked_packet->getPacket().add_stream_ids(stream_idx); + } for (size_t col_id = 0; col_id < num_columns; ++col_id) { @@ -126,12 +125,14 @@ TrackedMppDataPacketPtr ToFineGrainedPacket( for (size_t col_id = 0; col_id < num_columns; ++col_id) columns.emplace_back(std::move(scattered[col_id][bucket_idx + stream_idx])); auto block = header.cloneWithColumns(std::move(columns)); - - // encode into packet - codec_stream->encode(block, 0, block.rows()); - tracked_packet->addChunk(codec_stream->getString()); - tracked_packet->getPacket().add_stream_ids(stream_idx); - codec_stream->clear(); + if (block.rows()) + { + // encode into packet + codec_stream->encode(block, 0, block.rows()); + tracked_packet->addChunk(codec_stream->getString()); + tracked_packet->getPacket().add_stream_ids(stream_idx); + codec_stream->clear(); + } // disassemble the block back to scatter columns columns = block.mutateColumns(); From 68921eefb0c24d95699b908c3b0d68d7345b86b5 Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Mon, 30 Jan 2023 10:12:56 +0800 Subject: [PATCH 93/93] 80 Signed-off-by: Zhigao Tong --- .../Coprocessor/DAGQueryBlockInterpreter.cpp | 2 +- .../tests/gtest_ti_remote_block_inputstream.cpp | 5 ++++- dbms/src/Flash/Mpp/MPPTunnelSet.cpp | 16 ++++++++-------- dbms/src/Flash/Mpp/MPPTunnelSet.h | 2 +- dbms/src/Flash/Mpp/MPPTunnelSetHelper.cpp | 16 +++++++--------- dbms/src/Flash/Mpp/MPPTunnelSetHelper.h | 7 +++---- dbms/src/Flash/Mpp/newMPPExchangeWriter.h | 2 +- .../Mpp/tests/gtest_mpp_exchange_writer.cpp | 9 ++++----- .../Planner/plans/PhysicalExchangeSender.cpp | 2 +- 9 files changed, 30 insertions(+), 31 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp index df3f3e4c8e5..0544b7cce03 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp @@ -740,7 +740,7 @@ void DAGQueryBlockInterpreter::handleExchangeSender(DAGPipeline & pipeline) } pipeline.transform([&](auto & stream) { // construct writer - std::unique_ptr response_writer = NewMPPExchangeWriter( + std::unique_ptr response_writer = newMPPExchangeWriter( dagContext().tunnel_set, partition_col_ids, partition_col_collators, diff --git a/dbms/src/Flash/Coprocessor/tests/gtest_ti_remote_block_inputstream.cpp b/dbms/src/Flash/Coprocessor/tests/gtest_ti_remote_block_inputstream.cpp index 7794d9c37c7..5fe4a7d8c80 100644 --- a/dbms/src/Flash/Coprocessor/tests/gtest_ti_remote_block_inputstream.cpp +++ b/dbms/src/Flash/Coprocessor/tests/gtest_ti_remote_block_inputstream.cpp @@ -86,8 +86,11 @@ struct MockWriter void broadcastOrPassThroughWrite(Blocks & blocks) { - auto packet = MPPTunnelSetHelper::ToPacket(blocks, result_field_types, MPPDataPacketV0); + auto && packet = MPPTunnelSetHelper::ToPacketV0(blocks, result_field_types); ++total_packets; + if (!packet) + return; + if (!packet->packet.chunks().empty()) total_bytes += packet->packet.ByteSizeLong(); queue->push(std::move(packet)); diff --git a/dbms/src/Flash/Mpp/MPPTunnelSet.cpp b/dbms/src/Flash/Mpp/MPPTunnelSet.cpp index 90c44ae79b5..a308a9717a3 100644 --- a/dbms/src/Flash/Mpp/MPPTunnelSet.cpp +++ b/dbms/src/Flash/Mpp/MPPTunnelSet.cpp @@ -112,7 +112,10 @@ template void MPPTunnelSetBase::broadcastOrPassThroughWrite(Blocks & blocks) { RUNTIME_CHECK(!tunnels.empty()); - auto tracked_packet = MPPTunnelSetHelper::ToPacket(blocks, result_field_types, MPPDataPacketV0); + auto && tracked_packet = MPPTunnelSetHelper::ToPacketV0(blocks, result_field_types); + if (!tracked_packet) + return; + auto packet_bytes = tracked_packet->getPacket().ByteSizeLong(); checkPacketSize(packet_bytes); // TODO avoid copy packet for broadcast. @@ -142,11 +145,9 @@ void MPPTunnelSetBase::broadcastOrPassThroughWrite(Blocks & blocks) template void MPPTunnelSetBase::partitionWrite(Blocks & blocks, int16_t partition_id) { - auto tracked_packet = MPPTunnelSetHelper::ToPacket(blocks, result_field_types, MPPDataPacketV0); - - if unlikely (tracked_packet->getPacket().chunks_size() <= 0) + auto && tracked_packet = MPPTunnelSetHelper::ToPacketV0(blocks, result_field_types); + if (!tracked_packet) return; - auto packet_bytes = tracked_packet->getPacket().ByteSizeLong(); checkPacketSize(packet_bytes); tunnels[partition_id]->write(std::move(tracked_packet)); @@ -223,14 +224,13 @@ void MPPTunnelSetBase::fineGrainedShuffleWrite( size_t num_columns, int16_t partition_id) { - auto tracked_packet = MPPTunnelSetHelper::ToFineGrainedPacket( + auto tracked_packet = MPPTunnelSetHelper::ToFineGrainedPacketV0( header, scattered, bucket_idx, fine_grained_shuffle_stream_count, num_columns, - result_field_types, - MPPDataPacketV0); + result_field_types); if unlikely (tracked_packet->getPacket().chunks_size() <= 0) return; diff --git a/dbms/src/Flash/Mpp/MPPTunnelSet.h b/dbms/src/Flash/Mpp/MPPTunnelSet.h index 9d8b6285242..e0a8d4115d2 100644 --- a/dbms/src/Flash/Mpp/MPPTunnelSet.h +++ b/dbms/src/Flash/Mpp/MPPTunnelSet.h @@ -83,9 +83,9 @@ class MPPTunnelSetBase : private boost::noncopyable const std::vector & getTunnels() const { return tunnels; } +private: bool isLocal(size_t index) const; - private: std::vector tunnels; std::unordered_map receiver_task_id_to_index_map; diff --git a/dbms/src/Flash/Mpp/MPPTunnelSetHelper.cpp b/dbms/src/Flash/Mpp/MPPTunnelSetHelper.cpp index 2d074b4f51a..eb65327a7f8 100644 --- a/dbms/src/Flash/Mpp/MPPTunnelSetHelper.cpp +++ b/dbms/src/Flash/Mpp/MPPTunnelSetHelper.cpp @@ -42,13 +42,14 @@ TrackedMppDataPacketPtr ToPacket( return tracked_packet; } -TrackedMppDataPacketPtr ToPacket(Blocks & blocks, const std::vector & field_types, MPPDataPacketVersion version) +TrackedMppDataPacketPtr ToPacketV0(Blocks & blocks, const std::vector & field_types) { - assert(version == MPPDataPacketV0); + if (blocks.empty()) + return nullptr; CHBlockChunkCodec codec; auto codec_stream = codec.newCodecStream(field_types); - auto tracked_packet = std::make_shared(version); + auto tracked_packet = std::make_shared(MPPDataPacketV0); while (!blocks.empty()) { const auto & block = blocks.back(); @@ -103,20 +104,17 @@ TrackedMppDataPacketPtr ToFineGrainedPacket( return tracked_packet; } -TrackedMppDataPacketPtr ToFineGrainedPacket( +TrackedMppDataPacketPtr ToFineGrainedPacketV0( const Block & header, std::vector & scattered, size_t bucket_idx, UInt64 fine_grained_shuffle_stream_count, size_t num_columns, - const std::vector & field_types, - MPPDataPacketVersion version) + const std::vector & field_types) { - assert(version == MPPDataPacketV0); - CHBlockChunkCodec codec; auto codec_stream = codec.newCodecStream(field_types); - auto tracked_packet = std::make_shared(version); + auto tracked_packet = std::make_shared(MPPDataPacketV0); for (uint64_t stream_idx = 0; stream_idx < fine_grained_shuffle_stream_count; ++stream_idx) { // assemble scatter columns into a block diff --git a/dbms/src/Flash/Mpp/MPPTunnelSetHelper.h b/dbms/src/Flash/Mpp/MPPTunnelSetHelper.h index c04e531d2e9..050547dd330 100644 --- a/dbms/src/Flash/Mpp/MPPTunnelSetHelper.h +++ b/dbms/src/Flash/Mpp/MPPTunnelSetHelper.h @@ -25,7 +25,7 @@ enum class CompressionMethod; namespace DB::MPPTunnelSetHelper { -TrackedMppDataPacketPtr ToPacket(Blocks & blocks, const std::vector & field_types, MPPDataPacketVersion version); +TrackedMppDataPacketPtr ToPacketV0(Blocks & blocks, const std::vector & field_types); TrackedMppDataPacketPtr ToPacket( const Block & header, @@ -34,14 +34,13 @@ TrackedMppDataPacketPtr ToPacket( CompressionMethod compression_method, size_t & original_size); -TrackedMppDataPacketPtr ToFineGrainedPacket( +TrackedMppDataPacketPtr ToFineGrainedPacketV0( const Block & header, std::vector & scattered, size_t bucket_idx, UInt64 fine_grained_shuffle_stream_count, size_t num_columns, - const std::vector & field_types, - MPPDataPacketVersion version); + const std::vector & field_types); TrackedMppDataPacketPtr ToFineGrainedPacket( const Block & header, diff --git a/dbms/src/Flash/Mpp/newMPPExchangeWriter.h b/dbms/src/Flash/Mpp/newMPPExchangeWriter.h index 162ccede84a..9cb23d11734 100644 --- a/dbms/src/Flash/Mpp/newMPPExchangeWriter.h +++ b/dbms/src/Flash/Mpp/newMPPExchangeWriter.h @@ -23,7 +23,7 @@ namespace DB { template -std::unique_ptr NewMPPExchangeWriter( +std::unique_ptr newMPPExchangeWriter( const ExchangeWriterPtr & writer, const std::vector & partition_col_ids, const TiDB::TiDBCollators & partition_col_collators, diff --git a/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp b/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp index 6f310a4bf20..0ce4fda70dd 100644 --- a/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp +++ b/dbms/src/Flash/Mpp/tests/gtest_mpp_exchange_writer.cpp @@ -168,11 +168,11 @@ struct MockExchangeWriter } void broadcastOrPassThroughWrite(Blocks & blocks) { - checker(MPPTunnelSetHelper::ToPacket(blocks, result_field_types, MPPDataPacketV0), 0); + checker(MPPTunnelSetHelper::ToPacketV0(blocks, result_field_types), 0); } void partitionWrite(Blocks & blocks, uint16_t part_id) { - checker(MPPTunnelSetHelper::ToPacket(blocks, result_field_types, MPPDataPacketV0), part_id); + checker(MPPTunnelSetHelper::ToPacketV0(blocks, result_field_types), part_id); } void fineGrainedShuffleWrite( const Block & header, @@ -182,14 +182,13 @@ struct MockExchangeWriter size_t num_columns, int16_t part_id) { - auto tracked_packet = MPPTunnelSetHelper::ToFineGrainedPacket( + auto tracked_packet = MPPTunnelSetHelper::ToFineGrainedPacketV0( header, scattered, bucket_idx, fine_grained_shuffle_stream_count, num_columns, - result_field_types, - MPPDataPacketV0); + result_field_types); checker(tracked_packet, part_id); } diff --git a/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp b/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp index e6e6a1c4ac7..9db3d6f4a70 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp +++ b/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp @@ -70,7 +70,7 @@ void PhysicalExchangeSender::transformImpl(DAGPipeline & pipeline, Context & con } pipeline.transform([&](auto & stream) { // construct writer - std::unique_ptr response_writer = NewMPPExchangeWriter( + std::unique_ptr response_writer = newMPPExchangeWriter( dag_context.tunnel_set, partition_col_ids, partition_col_collators,