diff --git a/contrib/tiflash-proxy b/contrib/tiflash-proxy index 8d13ac0a371..5bd6c2b2b5f 160000 --- a/contrib/tiflash-proxy +++ b/contrib/tiflash-proxy @@ -1 +1 @@ -Subproject commit 8d13ac0a37114d6d4417be48d3da5e9ca7906a11 +Subproject commit 5bd6c2b2b5ff6a9f90de900c8e8c1eb4cfecc129 diff --git a/dbms/src/Columns/ColumnUtils.cpp b/dbms/src/Columns/ColumnUtils.cpp index f8b90a9978c..df7e0b69d0e 100644 --- a/dbms/src/Columns/ColumnUtils.cpp +++ b/dbms/src/Columns/ColumnUtils.cpp @@ -12,7 +12,9 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include #include +#include namespace DB { @@ -30,4 +32,10 @@ bool columnEqual(const ColumnPtr & expected, const ColumnPtr & actual, String & } return true; } +void convertColumnToNullable(ColumnWithTypeAndName & column) +{ + column.type = makeNullable(column.type); + if (column.column) + column.column = makeNullable(column.column); +} } // namespace DB diff --git a/dbms/src/Columns/ColumnUtils.h b/dbms/src/Columns/ColumnUtils.h index 2ef52cf429d..549449683d3 100644 --- a/dbms/src/Columns/ColumnUtils.h +++ b/dbms/src/Columns/ColumnUtils.h @@ -15,8 +15,10 @@ #pragma once #include +#include namespace DB { bool columnEqual(const ColumnPtr & expected, const ColumnPtr & actual, String & unequal_msg); +void convertColumnToNullable(ColumnWithTypeAndName & column); } // namespace DB diff --git a/dbms/src/Common/ComputeLabelHolder.cpp b/dbms/src/Common/ComputeLabelHolder.cpp index cc3d973c966..af8b068e32b 100644 --- a/dbms/src/Common/ComputeLabelHolder.cpp +++ b/dbms/src/Common/ComputeLabelHolder.cpp @@ -33,7 +33,7 @@ void ComputeLabelHolder::init(const Poco::Util::LayeredConfiguration & conf) std::lock_guard lock(mu); RUNTIME_ASSERT(!label_got, log, "Can't init after label got"); cluster_id = conf.getString(cluster_id_key, "unknown"); - auto service_addr = conf.getString("flash.service_addr", "unknown"); + auto service_addr = conf.getString("flash.service_addr", "0.0.0.0:3930"); std::replace(service_addr.begin(), service_addr.end(), ':', '_'); std::replace(service_addr.begin(), service_addr.end(), '.', '_'); process_id = fmt::format("compute_{}_{}", service_addr, microsecondsUTC()); diff --git a/dbms/src/Common/Config/ConfigProcessor.cpp b/dbms/src/Common/Config/ConfigProcessor.cpp index c9a0d25afcb..40bd0b4c096 100644 --- a/dbms/src/Common/Config/ConfigProcessor.cpp +++ b/dbms/src/Common/Config/ConfigProcessor.cpp @@ -84,11 +84,3 @@ ConfigProcessor::LoadedConfig ConfigProcessor::loadConfig() return LoadedConfig{configuration, false, config_doc}; } - -void ConfigProcessor::savePreprocessedConfig(const LoadedConfig & loaded_config) -{ - std::ofstream out(preprocessed_path); - cpptoml::toml_writer writer(out); - loaded_config.preprocessed_conf->accept(std::move(writer)); - out.close(); -} diff --git a/dbms/src/Common/Config/ConfigProcessor.h b/dbms/src/Common/Config/ConfigProcessor.h index 7618f58e915..e315459f780 100644 --- a/dbms/src/Common/Config/ConfigProcessor.h +++ b/dbms/src/Common/Config/ConfigProcessor.h @@ -63,8 +63,6 @@ class ConfigProcessor LoadedConfig loadConfig(); - void savePreprocessedConfig(const LoadedConfig & loaded_config); - public: /// Is the file named as result of config preprocessing, not as original files. static bool isPreprocessedFile(const std::string & config_path); diff --git a/dbms/src/Common/Config/ConfigReloader.cpp b/dbms/src/Common/Config/ConfigReloader.cpp index baf716b44cd..11f4d9ab3c9 100644 --- a/dbms/src/Common/Config/ConfigReloader.cpp +++ b/dbms/src/Common/Config/ConfigReloader.cpp @@ -85,7 +85,9 @@ void ConfigReloader::reloadIfNewer(bool force, bool throw_on_error) } } - if (force || new_files.isDifferOrNewerThan(files) || config_object_updated) + if (force + || (new_files.valid() && new_files.isDifferOrNewerThan(files)) + || config_object_updated) { ConfigProcessor config_processor(path); ConfigProcessor::LoadedConfig loaded_config; @@ -103,8 +105,6 @@ void ConfigReloader::reloadIfNewer(bool force, bool throw_on_error) return; } - config_processor.savePreprocessedConfig(loaded_config); - /** We should remember last modification time if and only if config was sucessfully loaded * Otherwise a race condition could occur during config files update: * File is contain raw (and non-valid) data, therefore config is not applied. diff --git a/dbms/src/Common/FileChangesTracker.h b/dbms/src/Common/FileChangesTracker.h index 10af6c6f6be..edd2fdd0802 100644 --- a/dbms/src/Common/FileChangesTracker.h +++ b/dbms/src/Common/FileChangesTracker.h @@ -43,6 +43,11 @@ struct FilesChangesTracker { std::set files; + bool valid() const + { + return !files.empty(); + } + void addIfExists(const std::string & path) { if (!path.empty() && Poco::File(path).exists()) diff --git a/dbms/src/Common/TiFlashMetrics.h b/dbms/src/Common/TiFlashMetrics.h index bfab1cd2428..c3ba79b7e7c 100644 --- a/dbms/src/Common/TiFlashMetrics.h +++ b/dbms/src/Common/TiFlashMetrics.h @@ -218,8 +218,8 @@ namespace DB F(type_lock_conflict, {"type", "lock_conflict"}), F(type_delete_conflict, {"type", "delete_conflict"}), \ F(type_delete_risk, {"type", "delete_risk"})) \ M(tiflash_disaggregated_object_lock_request_duration_seconds, "Bucketed histogram of S3 object lock/delete request duration", Histogram, \ - F(type_lock, {{"type", "cop"}}, ExpBuckets{0.001, 2, 20}), \ - F(type_delete, {{"type", "batch"}}, ExpBuckets{0.001, 2, 20})) \ + F(type_lock, {{"type", "lock"}}, ExpBuckets{0.001, 2, 20}), \ + F(type_delete, {{"type", "delete"}}, ExpBuckets{0.001, 2, 20})) \ M(tiflash_disaggregated_read_tasks_count, "Total number of storage engine disaggregated read tasks", Counter) \ M(tiflash_disaggregated_breakdown_duration_seconds, "", Histogram, \ F(type_rpc_establish, {{"type", "rpc_establish"}}, ExpBuckets{0.01, 2, 20}), \ @@ -319,8 +319,26 @@ namespace DB M(tiflash_storage_remote_stats, "The file stats on remote store", Gauge, \ F(type_total_size, {"type", "total_size"}), F(type_valid_size, {"type", "valid_size"}), \ F(type_num_files, {"type", "num_files"})) \ + M(tiflash_storage_checkpoint_seconds, "PageStorage checkpoint elapsed time", \ + Histogram, /* these command usually cost several seconds, increase the start bucket to 50ms */ \ + F(type_dump_checkpoint_snapshot, {{"type", "dump_checkpoint_snapshot"}}, ExpBuckets{0.05, 2, 20}), \ + F(type_dump_checkpoint_data, {{"type", "dump_checkpoint_data"}}, ExpBuckets{0.05, 2, 20}), \ + F(type_upload_checkpoint, {{"type", "upload_checkpoint"}}, ExpBuckets{0.05, 2, 20}), \ + F(type_copy_checkpoint_info, {{"type", "copy_checkpoint_info"}}, ExpBuckets{0.05, 2, 20})) \ M(tiflash_storage_checkpoint_flow, "The bytes flow cause by remote checkpoint", Counter, \ F(type_incremental, {"type", "incremental"}), F(type_compaction, {"type", "compaction"})) \ + M(tiflash_storage_checkpoint_keys_by_types, "The keys flow cause by remote checkpoint", Counter, \ + F(type_raftengine, {"type", "raftengine"}), F(type_kvengine, {"type", "kvengine"}), F(type_kvstore, {"type", "kvstore"}), \ + F(type_data, {"type", "data"}), F(type_log, {"type", "log"}), F(type_meta, {"type", "kvstore"}), \ + F(type_unknown, {"type", "unknown"})) \ + M(tiflash_storage_checkpoint_flow_by_types, "The bytes flow cause by remote checkpoint", Counter, \ + F(type_raftengine, {"type", "raftengine"}), F(type_kvengine, {"type", "kvengine"}), F(type_kvstore, {"type", "kvstore"}), \ + F(type_data, {"type", "data"}), F(type_log, {"type", "log"}), F(type_meta, {"type", "kvstore"}), \ + F(type_unknown, {"type", "unknown"})) \ + M(tiflash_storage_page_data_by_types, "The existing bytes stored in UniPageStorage", Gauge, \ + F(type_raftengine, {"type", "raftengine"}), F(type_kvengine, {"type", "kvengine"}), F(type_kvstore, {"type", "kvstore"}), \ + F(type_data, {"type", "data"}), F(type_log, {"type", "log"}), F(type_meta, {"type", "kvstore"}), \ + F(type_unknown, {"type", "unknown"})) \ M(tiflash_storage_s3_request_seconds, "S3 request duration in seconds", Histogram, \ F(type_put_object, {{"type", "put_object"}}, ExpBuckets{0.001, 2, 20}), \ F(type_copy_object, {{"type", "copy_object"}}, ExpBuckets{0.001, 2, 20}), \ @@ -331,6 +349,11 @@ namespace DB F(type_list_objects, {{"type", "list_objects"}}, ExpBuckets{0.001, 2, 20}), \ F(type_delete_object, {{"type", "delete_object"}}, ExpBuckets{0.001, 2, 20}), \ F(type_head_object, {{"type", "head_object"}}, ExpBuckets{0.001, 2, 20})) \ + M(tiflash_storage_s3_gc_status, "S3 GC status", Gauge, \ + F(type_lifecycle_added, {{"type", "lifecycle_added"}}), \ + F(type_lifecycle_failed, {{"type", "lifecycle_failed"}}), \ + F(type_owner, {{"type", "owner"}}), \ + F(type_running, {{"type", "running"}})) \ M(tiflash_storage_s3_gc_seconds, "S3 GC subprocess duration in seconds", \ Histogram, /* these command usually cost several seconds, increase the start bucket to 500ms */ \ F(type_total, {{"type", "total"}}, ExpBuckets{0.5, 2, 20}), \ @@ -339,11 +362,7 @@ namespace DB F(type_clean_locks, {{"type", "clean_locks"}}, ExpBuckets{0.5, 2, 20}), \ F(type_clean_manifests, {{"type", "clean_manifests"}}, ExpBuckets{0.5, 2, 20}), \ F(type_scan_then_clean_data_files, {{"type", "scan_then_clean_data_files"}}, ExpBuckets{0.5, 2, 20}), \ - F(type_clean_one_lock, {{"type", "clean_one_lock"}}, ExpBuckets{0.5, 2, 20})) \ - M(tiflash_storage_checkpoint_seconds, "PageStorage checkpoint elapsed time", Histogram, \ - F(type_dump_checkpoint_snapshot, {{"type", "dump_checkpoint_snapshot"}}, ExpBuckets{0.001, 2, 20}), \ - F(type_dump_checkpoint_data, {{"type", "dump_checkpoint_data"}}, ExpBuckets{0.001, 2, 20}), \ - F(type_upload_checkpoint, {{"type", "upload_checkpoint"}}, ExpBuckets{0.001, 2, 20})) + F(type_clean_one_lock, {{"type", "clean_one_lock"}}, ExpBuckets{0.5, 2, 20})) // clang-format on diff --git a/dbms/src/Common/UnifiedLogFormatter.cpp b/dbms/src/Common/UnifiedLogFormatter.cpp index 18ded330cf0..5a9e88f9a67 100644 --- a/dbms/src/Common/UnifiedLogFormatter.cpp +++ b/dbms/src/Common/UnifiedLogFormatter.cpp @@ -23,55 +23,100 @@ #include #include +#define RESET_FONT "\033[0m" +#define BOLD_FONT "\033[1m" +#define BLACK "\033[30m" +#define RED "\033[31m" +#define GREEN "\033[32m" +#define YELLOW "\033[33m" +#define BLUE "\033[34m" +#define MAGENTA "\033[35m" +#define CYAN "\033[36m" +#define WHITE "\033[37m" +#define BRIGHT_BLACK "\033[90m" +#define BRIGHT_RED "\033[91m" +#define BRIGHT_GREEN "\033[92m" +#define BRIGHT_YELLOW "\033[93m" +#define BRIGHT_BLUE "\033[94m" +#define BRIGHT_MAGENTA "\033[95m" +#define BRIGHT_CYAN "\033[96m" +#define BRIGHT_WHITE "\033[97m" + namespace DB { -void UnifiedLogFormatter::format(const Poco::Message & msg, std::string & text) + +enum class JsonEncodeKind { - FmtBuffer buf; + /** + * No need to encode, just copy the text + */ + DirectCopy, - // Timestamp - { - buf.append('['); - writeTimestamp(buf); - buf.append("] "); - } - // Priority - { - buf.append('['); - writePriority(buf, msg.getPriority()); - buf.append("] "); - } - // Source File - { - if (unlikely(!msg.getSourceFile())) - buf.append("[] "); - else - buf.fmtAppend(FMT_COMPILE("[{}:{}] "), msg.getSourceFile(), msg.getSourceLine()); - } - // Message - { - buf.append('['); - writeEscapedString(buf, msg.getText()); - buf.append("] "); - } - // Source and Identifiers + /** + * Add double quotes around the text is sufficient + */ + AddQuoteAndCopy, + + /** + * Need full JSON string encode + */ + Encode, +}; + +void writePriorityColor(FmtBuffer & buf, const Poco::Message::Priority & priority) +{ + switch (priority) { - const std::string & source = msg.getSource(); - if (!source.empty()) - { - buf.append("[source="); - writeEscapedString(buf, source); - buf.append("] "); - } + case Poco::Message::Priority::PRIO_TRACE: + buf.append(BRIGHT_BLACK); + break; + case Poco::Message::Priority::PRIO_DEBUG: + buf.append(BRIGHT_BLACK); + break; + case Poco::Message::Priority::PRIO_INFORMATION: + buf.append(BRIGHT_BLUE); + break; + case Poco::Message::Priority::PRIO_WARNING: + buf.append(YELLOW); + break; + case Poco::Message::Priority::PRIO_ERROR: + buf.append(RED); + break; + case Poco::Message::Priority::PRIO_FATAL: + buf.append(RED); + break; + case Poco::Message::Priority::PRIO_CRITICAL: + buf.append(RED); + break; + case Poco::Message::Priority::PRIO_NOTICE: + break; + default: + break; } - // Thread ID +} + +void writeMessageColor(FmtBuffer & buf, const Poco::Message::Priority & priority) +{ + switch (priority) { - buf.fmtAppend(FMT_COMPILE("[thread_id={}]"), Poco::ThreadNumber::get()); + case Poco::Message::Priority::PRIO_WARNING: + buf.append(YELLOW); + break; + case Poco::Message::Priority::PRIO_ERROR: + buf.append(RED); + break; + case Poco::Message::Priority::PRIO_FATAL: + buf.append(RED); + break; + case Poco::Message::Priority::PRIO_CRITICAL: + buf.append(RED); + break; + default: + break; } - text = buf.toString(); } -void UnifiedLogFormatter::writePriority(FmtBuffer & buf, const Poco::Message::Priority & priority) +void writePriority(FmtBuffer & buf, const Poco::Message::Priority & priority) { switch (priority) { @@ -105,7 +150,7 @@ void UnifiedLogFormatter::writePriority(FmtBuffer & buf, const Poco::Message::Pr } } -void UnifiedLogFormatter::writeTimestamp(FmtBuffer & buf) +void writeTimestamp(FmtBuffer & buf) { // The format is "yyyy/MM/dd HH:mm:ss.SSS ZZZZZ" auto time_point = std::chrono::system_clock::now(); @@ -151,26 +196,7 @@ void UnifiedLogFormatter::writeTimestamp(FmtBuffer & buf) buf.fmtAppend(FMT_COMPILE("{0:02d}:{1:02d}"), offset_tm->tm_hour, offset_tm->tm_min); } -void UnifiedLogFormatter::writeEscapedString(FmtBuffer & buf, const std::string & str) -{ - auto encode_kind = needJsonEncode(str); - switch (encode_kind) - { - case JsonEncodeKind::DirectCopy: - buf.append(str); - break; - case JsonEncodeKind::AddQuoteAndCopy: - buf.append('"'); - buf.append(str); - buf.append('"'); - break; - case JsonEncodeKind::Encode: - writeJSONString(buf, str); - break; - } -} - -UnifiedLogFormatter::JsonEncodeKind UnifiedLogFormatter::needJsonEncode(const std::string & src) +JsonEncodeKind needJsonEncode(const std::string & src) { bool needs_quote = false; bool json_encode_cannot_copy = false; @@ -192,7 +218,7 @@ UnifiedLogFormatter::JsonEncodeKind UnifiedLogFormatter::needJsonEncode(const st return JsonEncodeKind::DirectCopy; } -void UnifiedLogFormatter::writeJSONString(FmtBuffer & buf, const std::string & str) +void writeJSONString(FmtBuffer & buf, const std::string & str) { buf.append('"'); @@ -226,4 +252,110 @@ void UnifiedLogFormatter::writeJSONString(FmtBuffer & buf, const std::string & s buf.append('"'); } +void writeEscapedString(FmtBuffer & buf, const std::string & str) +{ + auto encode_kind = needJsonEncode(str); + switch (encode_kind) + { + case JsonEncodeKind::DirectCopy: + buf.append(str); + break; + case JsonEncodeKind::AddQuoteAndCopy: + buf.append('"'); + buf.append(str); + buf.append('"'); + break; + case JsonEncodeKind::Encode: + writeJSONString(buf, str); + break; + } +} + +template +void UnifiedLogFormatter::format(const Poco::Message & msg, std::string & text) +{ + FmtBuffer buf; + + // Timestamp + { + if constexpr (enable_color) + buf.append(BRIGHT_GREEN); + + buf.append('['); + writeTimestamp(buf); + buf.append("] "); + + if constexpr (enable_color) + buf.append(RESET_FONT); + } + + // Priority + { + if constexpr (enable_color) + { + buf.append(BOLD_FONT); + writePriorityColor(buf, msg.getPriority()); + } + + buf.append('['); + writePriority(buf, msg.getPriority()); + buf.append("] "); + + if constexpr (enable_color) + buf.append(RESET_FONT); + } + + // Source File + { + if constexpr (enable_color) + buf.append(MAGENTA); + + if (unlikely(!msg.getSourceFile())) + buf.append("[] "); + else + buf.fmtAppend(FMT_COMPILE("[{}:{}] "), msg.getSourceFile(), msg.getSourceLine()); + + if constexpr (enable_color) + buf.append(RESET_FONT); + } + // Message + { + if constexpr (enable_color) + writeMessageColor(buf, msg.getPriority()); + + buf.append('['); + writeEscapedString(buf, msg.getText()); + buf.append("] "); + + if constexpr (enable_color) + buf.append(RESET_FONT); + } + + // Source and Identifiers + if constexpr (enable_color) + buf.append(BRIGHT_BLACK); + + { + const std::string & source = msg.getSource(); + if (!source.empty()) + { + buf.append("[source="); + writeEscapedString(buf, source); + buf.append("] "); + } + } + // Thread ID + { + buf.fmtAppend(FMT_COMPILE("[thread_id={}]"), Poco::ThreadNumber::get()); + } + + if constexpr (enable_color) + buf.append(RESET_FONT); + + text = buf.toString(); +} + +template class UnifiedLogFormatter; +template class UnifiedLogFormatter; + } // namespace DB diff --git a/dbms/src/Common/UnifiedLogFormatter.h b/dbms/src/Common/UnifiedLogFormatter.h index 89b6ed09019..471b0c6917c 100644 --- a/dbms/src/Common/UnifiedLogFormatter.h +++ b/dbms/src/Common/UnifiedLogFormatter.h @@ -22,44 +22,15 @@ namespace DB { -class WriteBuffer; /// https://github.com/tikv/rfcs/blob/ed764d7d014c420ee0cbcde99597020c4f75346d/text/0018-unified-log-format.md +template class UnifiedLogFormatter : public Poco::Formatter { public: UnifiedLogFormatter() = default; void format(const Poco::Message & msg, std::string & text) override; - -private: - enum class JsonEncodeKind - { - /** - * No need to encode, just copy the text - */ - DirectCopy, - - /** - * Add double quotes around the text is sufficient - */ - AddQuoteAndCopy, - - /** - * Need full JSON string encode - */ - Encode, - }; - - static void writePriority(FmtBuffer & buf, const Poco::Message::Priority & priority); - - static void writeTimestamp(FmtBuffer & buf); - - static JsonEncodeKind needJsonEncode(const std::string & src); - - static void writeJSONString(FmtBuffer & buf, const std::string & str); - - static void writeEscapedString(FmtBuffer & buf, const std::string & str); }; } // namespace DB diff --git a/dbms/src/Common/tests/bench_logger.cpp b/dbms/src/Common/tests/bench_logger.cpp index 4c2b0b02cfc..f420dbbcdd6 100644 --- a/dbms/src/Common/tests/bench_logger.cpp +++ b/dbms/src/Common/tests/bench_logger.cpp @@ -72,7 +72,7 @@ class LogBM : public benchmark::Fixture void SetUp(const ::benchmark::State & /*state*/) { Poco::AutoPtr channel(new TestChannel()); - Poco::AutoPtr formatter(new UnifiedLogFormatter()); + Poco::AutoPtr formatter(new UnifiedLogFormatter()); Poco::AutoPtr formatting_channel(new Poco::FormattingChannel(formatter, channel)); Poco::Logger::root().setChannel(formatting_channel); Poco::Logger::root().setLevel(Poco::Message::PRIO_TRACE); diff --git a/dbms/src/Common/tests/gtest_logger.cpp b/dbms/src/Common/tests/gtest_logger.cpp index 3538e4b7e0e..49aef68054d 100644 --- a/dbms/src/Common/tests/gtest_logger.cpp +++ b/dbms/src/Common/tests/gtest_logger.cpp @@ -33,7 +33,7 @@ class LogMacroTest : public testing::Test void SetUp() override { RUNTIME_CHECK(channel_backup == nullptr); - Poco::AutoPtr formatter(new UnifiedLogFormatter()); + Poco::AutoPtr formatter(new UnifiedLogFormatter()); Poco::AutoPtr formatting_channel(new Poco::FormattingChannel(formatter, channel)); channel_backup = Poco::Logger::root().getChannel(); Poco::Logger::root().setChannel(formatting_channel); diff --git a/dbms/src/DataStreams/NonJoinedBlockInputStream.cpp b/dbms/src/DataStreams/NonJoinedBlockInputStream.cpp index cb947c8be06..d46868ef037 100644 --- a/dbms/src/DataStreams/NonJoinedBlockInputStream.cpp +++ b/dbms/src/DataStreams/NonJoinedBlockInputStream.cpp @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include #include #include @@ -200,12 +201,12 @@ void NonJoinedBlockInputStream::fillColumnsUsingCurrentPartition( } if (parent.strictness == ASTTableJoin::Strictness::Any) { - switch (parent.type) + switch (parent.join_map_method) { -#define M(TYPE) \ - case JoinType::TYPE: \ +#define M(METHOD) \ + case JoinMapMethod::METHOD: \ fillColumns( \ - *partition->maps_any_full.TYPE, \ + *partition->maps_any_full.METHOD, \ num_columns_left, \ mutable_columns_left, \ num_columns_right, \ @@ -221,12 +222,12 @@ void NonJoinedBlockInputStream::fillColumnsUsingCurrentPartition( } else if (parent.strictness == ASTTableJoin::Strictness::All) { - switch (parent.type) + switch (parent.join_map_method) { -#define M(TYPE) \ - case JoinType::TYPE: \ +#define M(METHOD) \ + case JoinMapMethod::METHOD: \ fillColumns( \ - *partition->maps_all_full.TYPE, \ + *partition->maps_all_full.METHOD, \ num_columns_left, \ mutable_columns_left, \ num_columns_right, \ diff --git a/dbms/src/Encryption/FileProvider.cpp b/dbms/src/Encryption/FileProvider.cpp index 767678e8f85..33677f51bbf 100644 --- a/dbms/src/Encryption/FileProvider.cpp +++ b/dbms/src/Encryption/FileProvider.cpp @@ -43,7 +43,11 @@ RandomAccessFilePtr FileProvider::newRandomAccessFile( } else { - RUNTIME_CHECK(Poco::Path(file_path_).isAbsolute(), file_path_); + // Unrecognized xx:// protocol. + RUNTIME_CHECK_MSG( + file_path_.find("://") == std::string::npos, + "Unsupported protocol in path {}", + file_path_); file = std::make_shared(file_path_, flags, read_limiter); } auto encryption_info = key_manager->getFile(encryption_path_.full_path); @@ -70,7 +74,11 @@ WritableFilePtr FileProvider::newWritableFile( } else { - RUNTIME_CHECK(Poco::Path(file_path_).isAbsolute(), file_path_); + // Unrecognized xx:// protocol. + RUNTIME_CHECK_MSG( + file_path_.find("://") == std::string::npos, + "Unsupported protocol in path {}", + file_path_); file = std::make_shared(file_path_, truncate_if_exists_, flags, mode, write_limiter_); } if (encryption_enabled && create_new_encryption_info_) diff --git a/dbms/src/Flash/Disaggregated/S3LockService.cpp b/dbms/src/Flash/Disaggregated/S3LockService.cpp index f022580f5df..2249f971f9e 100644 --- a/dbms/src/Flash/Disaggregated/S3LockService.cpp +++ b/dbms/src/Flash/Disaggregated/S3LockService.cpp @@ -248,6 +248,7 @@ bool S3LockService::tryMarkDeleteImpl(const String & data_file_key, disaggregate { const S3FilenameView key_view = S3FilenameView::fromKey(data_file_key); RUNTIME_CHECK(key_view.isDataFile(), data_file_key); + GET_METRIC(tiflash_disaggregated_object_lock_request_count, type_delete).Increment(); if (!gc_owner->isOwner()) { diff --git a/dbms/src/Interpreters/AsynchronousMetrics.cpp b/dbms/src/Interpreters/AsynchronousMetrics.cpp index e08725ea91d..6a49a8410ae 100644 --- a/dbms/src/Interpreters/AsynchronousMetrics.cpp +++ b/dbms/src/Interpreters/AsynchronousMetrics.cpp @@ -15,8 +15,10 @@ #include #include #include +#include #include #include +#include #include #include #include @@ -125,27 +127,62 @@ static void calculateMaxAndSum(Max & max, Sum & sum, T x) FileUsageStatistics AsynchronousMetrics::getPageStorageFileUsage() { - RUNTIME_ASSERT(!(context.getSharedContextDisagg()->isDisaggregatedComputeMode() && context.getSharedContextDisagg()->use_autoscaler)); - // Get from RegionPersister - auto & tmt = context.getTMTContext(); - auto & kvstore = tmt.getKVStore(); - FileUsageStatistics usage = kvstore->getFileUsageStatistics(); - - // Get the blob file status from all PS V3 instances - if (auto global_storage_pool = context.getGlobalStoragePool(); global_storage_pool != nullptr) + FileUsageStatistics usage; + switch (context.getSharedContextDisagg()->disaggregated_mode) { - const auto log_usage = global_storage_pool->log_storage->getFileUsageStatistics(); - const auto meta_usage = global_storage_pool->meta_storage->getFileUsageStatistics(); - const auto data_usage = global_storage_pool->data_storage->getFileUsageStatistics(); + case DisaggregatedMode::None: + { + if (auto uni_ps = context.tryGetWriteNodePageStorage(); uni_ps != nullptr) + { + /// When format_version=5 is enabled, then all data are stored in the `uni_ps` + usage.merge(uni_ps->getFileUsageStatistics()); + } + else + { + /// When format_version < 5, then there are multiple PageStorage instances - usage.merge(log_usage) - .merge(meta_usage) - .merge(data_usage); - } + // Get from RegionPersister + auto & tmt = context.getTMTContext(); + auto & kvstore = tmt.getKVStore(); + usage = kvstore->getFileUsageStatistics(); + + // Get the blob file status from all PS V3 instances + if (auto global_storage_pool = context.getGlobalStoragePool(); global_storage_pool != nullptr) + { + const auto log_usage = global_storage_pool->log_storage->getFileUsageStatistics(); + const auto meta_usage = global_storage_pool->meta_storage->getFileUsageStatistics(); + const auto data_usage = global_storage_pool->data_storage->getFileUsageStatistics(); - if (auto ps_cache = context.getSharedContextDisagg()->rn_page_cache_storage; ps_cache != nullptr) + usage.merge(log_usage) + .merge(meta_usage) + .merge(data_usage); + } + } + break; + } + case DisaggregatedMode::Storage: + { + // disagg write node, all data are stored in the `uni_ps` + if (auto uni_ps = context.getWriteNodePageStorage(); uni_ps != nullptr) + { + usage.merge(uni_ps->getFileUsageStatistics()); + } + break; + } + case DisaggregatedMode::Compute: { - usage.merge(ps_cache->getUniversalPageStorage()->getFileUsageStatistics()); + // disagg compute node without auto-scaler, the proxy data are stored in the `uni_ps` + if (auto uni_ps = context.getWriteNodePageStorage(); uni_ps != nullptr) + { + usage.merge(uni_ps->getFileUsageStatistics()); + } + // disagg compute node, all cache page data are stored in the `ps_cache` + if (auto ps_cache = context.getSharedContextDisagg()->rn_page_cache_storage; ps_cache != nullptr) + { + usage.merge(ps_cache->getUniversalPageStorage()->getFileUsageStatistics()); + } + break; + } } return usage; @@ -206,7 +243,6 @@ void AsynchronousMetrics::update() set("MaxDTBackgroundTasksLength", max_dt_background_tasks_length); } - if (!(context.getSharedContextDisagg()->isDisaggregatedComputeMode() && context.getSharedContextDisagg()->use_autoscaler)) { const FileUsageStatistics usage = getPageStorageFileUsage(); set("BlobFileNums", usage.total_file_num); @@ -217,6 +253,15 @@ void AsynchronousMetrics::update() set("PagesInMem", usage.num_pages); } + if (context.getSharedContextDisagg()->isDisaggregatedStorageMode()) + { + auto & tmt = context.getTMTContext(); + if (auto s3_gc_owner = tmt.getS3GCOwnerManager(); s3_gc_owner->isOwner()) + { + GET_METRIC(tiflash_storage_s3_gc_status, type_owner).Set(1.0); + } + } + #if USE_MIMALLOC #define MI_STATS_SET(X) set("mimalloc." #X, X) @@ -256,7 +301,7 @@ void AsynchronousMetrics::update() M("background_thread.num_runs", uint64_t) \ M("background_thread.run_interval", uint64_t) -#define GET_METRIC(NAME, TYPE) \ +#define GET_JEMALLOC_METRIC(NAME, TYPE) \ do \ { \ TYPE value{}; \ @@ -265,9 +310,9 @@ void AsynchronousMetrics::update() set("jemalloc." NAME, value); \ } while (0); - FOR_EACH_METRIC(GET_METRIC); + FOR_EACH_METRIC(GET_JEMALLOC_METRIC); -#undef GET_METRIC +#undef GET_JEMALLOC_METRIC #undef FOR_EACH_METRIC } #endif diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index 6c09f056716..6eea2d6a913 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -620,7 +620,6 @@ ConfigurationPtr Context::getUsersConfig() void Context::setSecurityConfig(Poco::Util::AbstractConfiguration & config, const LoggerPtr & log) { - LOG_INFO(log, "Setting secuirty config."); auto lock = getLock(); shared->security_config = std::make_shared(log); shared->security_config->init(config); diff --git a/dbms/src/Interpreters/Expand.cpp b/dbms/src/Interpreters/Expand.cpp index 3910efec76a..8a226978f9f 100644 --- a/dbms/src/Interpreters/Expand.cpp +++ b/dbms/src/Interpreters/Expand.cpp @@ -12,8 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include -#include +#include #include #include #include @@ -22,16 +21,6 @@ namespace DB { -namespace -{ -void convertColumnToNullable(ColumnWithTypeAndName & column) -{ - column.type = makeNullable(column.type); - if (column.column) - column.column = makeNullable(column.column); -} -} // namespace - Expand::Expand(const DB::GroupingSets & gss) : group_sets_names(gss) { diff --git a/dbms/src/Interpreters/ExternalLoaderConfigRepository.cpp b/dbms/src/Interpreters/ExternalLoaderConfigRepository.cpp index ae923c76397..0b291818d9d 100644 --- a/dbms/src/Interpreters/ExternalLoaderConfigRepository.cpp +++ b/dbms/src/Interpreters/ExternalLoaderConfigRepository.cpp @@ -49,7 +49,7 @@ ExternalLoaderConfigRepository::Files ExternalLoaderConfigRepository::list( Poco::Glob::glob(pattern, files, 0); } - for (Files::iterator it = files.begin(); it != files.end();) + for (auto it = files.begin(); it != files.end();) { if (ConfigProcessor::isPreprocessedFile(*it)) files.erase(it++); @@ -76,7 +76,6 @@ Poco::AutoPtr ExternalLoaderConfigRepository: { ConfigProcessor config_processor{config_file}; ConfigProcessor::LoadedConfig preprocessed = config_processor.loadConfig(); - config_processor.savePreprocessedConfig(preprocessed); return preprocessed.configuration; } diff --git a/dbms/src/Interpreters/Join.cpp b/dbms/src/Interpreters/Join.cpp index 9d948e2a428..e1f08ca161a 100644 --- a/dbms/src/Interpreters/Join.cpp +++ b/dbms/src/Interpreters/Join.cpp @@ -12,10 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include -#include -#include -#include +#include #include #include #include @@ -111,13 +108,6 @@ ColumnRawPtrs extractAndMaterializeKeyColumns(const Block & block, Columns & mat const std::string Join::match_helper_prefix = "__left-semi-join-match-helper"; const DataTypePtr Join::match_helper_type = makeNullable(std::make_shared()); -void convertColumnToNullable(ColumnWithTypeAndName & column) -{ - column.type = makeNullable(column.type); - if (column.column) - column.column = makeNullable(column.column); -} - Join::Join( const Names & key_names_left_, const Names & key_names_right_, @@ -191,97 +181,11 @@ void Join::meetErrorImpl(const String & error_message_, std::unique_lock(column) - || (column->isColumnConst() && typeid_cast(&static_cast(column)->getDataColumn())); -} - -JoinType Join::chooseMethod(const ColumnRawPtrs & key_columns, Sizes & key_sizes_) const -{ - const size_t keys_size = key_columns.size(); - - if (keys_size == 0) - return JoinType::CROSS; - - bool all_fixed = true; - size_t keys_bytes = 0; - key_sizes_.resize(keys_size); - for (size_t j = 0; j < keys_size; ++j) - { - if (!key_columns[j]->isFixedAndContiguous()) - { - all_fixed = false; - break; - } - key_sizes_[j] = key_columns[j]->sizeOfValueIfFixed(); - keys_bytes += key_sizes_[j]; - } - - /// If there is one numeric key that fits in 64 bits - if (keys_size == 1 && key_columns[0]->isNumeric()) - { - size_t size_of_field = key_columns[0]->sizeOfValueIfFixed(); - if (size_of_field == 1) - return JoinType::key8; - if (size_of_field == 2) - return JoinType::key16; - if (size_of_field == 4) - return JoinType::key32; - if (size_of_field == 8) - return JoinType::key64; - if (size_of_field == 16) - return JoinType::keys128; - throw Exception("Logical error: numeric column has sizeOfField not in 1, 2, 4, 8, 16.", ErrorCodes::LOGICAL_ERROR); - } - - /// If the keys fit in N bits, we will use a hash table for N-bit-packed keys - if (all_fixed && keys_bytes <= 16) - return JoinType::keys128; - if (all_fixed && keys_bytes <= 32) - return JoinType::keys256; - - /// If there is single string key, use hash table of it's values. - if (keys_size == 1 && CanAsColumnString(key_columns[0])) - { - if (collators.empty() || !collators[0]) - return JoinType::key_strbin; - else - { - switch (collators[0]->getCollatorType()) - { - case TiDB::ITiDBCollator::CollatorType::UTF8MB4_BIN: - case TiDB::ITiDBCollator::CollatorType::UTF8_BIN: - case TiDB::ITiDBCollator::CollatorType::LATIN1_BIN: - case TiDB::ITiDBCollator::CollatorType::ASCII_BIN: - { - return JoinType::key_strbinpadding; - } - case TiDB::ITiDBCollator::CollatorType::BINARY: - { - return JoinType::key_strbin; - } - default: - { - // for CI COLLATION, use original way - return JoinType::key_string; - } - } - } - } - - if (keys_size == 1 && typeid_cast(key_columns[0])) - return JoinType::key_fixed_string; - - /// Otherwise, use serialized values as the key. - return JoinType::serialized; -} - size_t Join::getTotalRowCount() const { size_t res = 0; - if (type == JoinType::CROSS) + if (join_map_method == JoinMapMethod::CROSS) { res = total_input_build_rows; } @@ -304,7 +208,7 @@ size_t Join::getTotalByteCount() } else { - if (type == JoinType::CROSS) + if (join_map_method == JoinMapMethod::CROSS) { for (const auto & block : blocks) res += block.bytes(); @@ -344,7 +248,7 @@ void Join::setBuildConcurrencyAndInitJoinPartition(size_t build_concurrency_) partitions.reserve(build_concurrency); for (size_t i = 0; i < getBuildConcurrency(); ++i) { - partitions.push_back(std::make_unique(type, kind, strictness, max_block_size, log)); + partitions.push_back(std::make_unique(join_map_method, kind, strictness, max_block_size, log)); } } @@ -412,13 +316,13 @@ void Join::initBuild(const Block & sample_block, size_t build_concurrency_) if (unlikely(initialized)) throw Exception("Logical error: Join has been initialized", ErrorCodes::LOGICAL_ERROR); initialized = true; - type = chooseMethod(getKeyColumns(key_names_right, sample_block), key_sizes); + join_map_method = chooseJoinMapMethod(getKeyColumns(key_names_right, sample_block), key_sizes, collators); setBuildConcurrencyAndInitJoinPartition(build_concurrency_); build_sample_block = sample_block; build_spiller = std::make_unique(build_spill_config, false, build_concurrency_, build_sample_block, log); if (max_bytes_before_external_join > 0) { - if (type == JoinType::CROSS) + if (join_map_method == JoinMapMethod::CROSS) { /// todo support spill for cross join max_bytes_before_external_join = 0; @@ -1353,24 +1257,10 @@ Block Join::joinBlockNullAware(ProbeProcessInfo & probe_process_info) const { Block block = probe_process_info.block; - size_t keys_size = key_names_left.size(); - ColumnRawPtrs key_columns(keys_size); - /// Rare case, when keys are constant. To avoid code bloat, simply materialize them. /// Note: this variable can't be removed because it will take smart pointers' lifecycle to the end of this function. Columns materialized_columns; - - /// Memoize key columns to work with. - for (size_t i = 0; i < keys_size; ++i) - { - key_columns[i] = block.getByName(key_names_left[i]).column.get(); - - if (ColumnPtr converted = key_columns[i]->convertToFullColumnIfConst()) - { - materialized_columns.emplace_back(converted); - key_columns[i] = materialized_columns.back().get(); - } - } + ColumnRawPtrs key_columns = extractAndMaterializeKeyColumns(block, materialized_columns, key_names_left); /// Note that `extractAllKeyNullMap` must be done before `extractNestedColumnsAndNullMap` /// because `extractNestedColumnsAndNullMap` will change the nullable column to its nested column. diff --git a/dbms/src/Interpreters/Join.h b/dbms/src/Interpreters/Join.h index 846dc2a796c..e23ec428508 100644 --- a/dbms/src/Interpreters/Join.h +++ b/dbms/src/Interpreters/Join.h @@ -23,7 +23,7 @@ #include #include #include -#include +#include #include #include @@ -291,9 +291,7 @@ class Join bool has_build_data_in_memory = false; private: - JoinType type = JoinType::EMPTY; - - JoinType chooseMethod(const ColumnRawPtrs & key_columns, Sizes & key_sizes) const; + JoinMapMethod join_map_method = JoinMapMethod::EMPTY; Sizes key_sizes; @@ -395,6 +393,4 @@ struct RestoreInfo , probe_stream(probe_stream_){}; }; -void convertColumnToNullable(ColumnWithTypeAndName & column); - } // namespace DB diff --git a/dbms/src/Interpreters/JoinHashMap.cpp b/dbms/src/Interpreters/JoinHashMap.cpp new file mode 100644 index 00000000000..10f95ddbcde --- /dev/null +++ b/dbms/src/Interpreters/JoinHashMap.cpp @@ -0,0 +1,111 @@ +// Copyright 2023 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 + +namespace DB +{ +namespace +{ +bool canAsColumnString(const IColumn * column) +{ + return typeid_cast(column) + || (column->isColumnConst() && typeid_cast(&static_cast(column)->getDataColumn())); +} +} // namespace + +JoinMapMethod chooseJoinMapMethod(const ColumnRawPtrs & key_columns, Sizes & key_sizes, const TiDB::TiDBCollators & collators) +{ + const size_t keys_size = key_columns.size(); + + if (keys_size == 0) + return JoinMapMethod::CROSS; + + bool all_fixed = true; + size_t keys_bytes = 0; + key_sizes.resize(keys_size); + for (size_t j = 0; j < keys_size; ++j) + { + if (!key_columns[j]->isFixedAndContiguous()) + { + all_fixed = false; + break; + } + key_sizes[j] = key_columns[j]->sizeOfValueIfFixed(); + keys_bytes += key_sizes[j]; + } + + /// If there is one numeric key that fits in 64 bits + if (keys_size == 1 && key_columns[0]->isNumeric()) + { + size_t size_of_field = key_columns[0]->sizeOfValueIfFixed(); + if (size_of_field == 1) + return JoinMapMethod::key8; + if (size_of_field == 2) + return JoinMapMethod::key16; + if (size_of_field == 4) + return JoinMapMethod::key32; + if (size_of_field == 8) + return JoinMapMethod::key64; + if (size_of_field == 16) + return JoinMapMethod::keys128; + throw Exception("Logical error: numeric column has sizeOfField not in 1, 2, 4, 8, 16.", ErrorCodes::LOGICAL_ERROR); + } + + /// If the keys fit in N bits, we will use a hash table for N-bit-packed keys + if (all_fixed && keys_bytes <= 16) + return JoinMapMethod::keys128; + if (all_fixed && keys_bytes <= 32) + return JoinMapMethod::keys256; + + /// If there is single string key, use hash table of it's values. + if (keys_size == 1 && canAsColumnString(key_columns[0])) + { + if (collators.empty() || !collators[0]) + return JoinMapMethod::key_strbin; + else + { + switch (collators[0]->getCollatorType()) + { + case TiDB::ITiDBCollator::CollatorType::UTF8MB4_BIN: + case TiDB::ITiDBCollator::CollatorType::UTF8_BIN: + case TiDB::ITiDBCollator::CollatorType::LATIN1_BIN: + case TiDB::ITiDBCollator::CollatorType::ASCII_BIN: + { + return JoinMapMethod::key_strbinpadding; + } + case TiDB::ITiDBCollator::CollatorType::BINARY: + { + return JoinMapMethod::key_strbin; + } + default: + { + // for CI COLLATION, use original way + return JoinMapMethod::key_string; + } + } + } + } + + if (keys_size == 1 && typeid_cast(key_columns[0])) + return JoinMapMethod::key_fixed_string; + + /// Otherwise, use serialized values as the key. + return JoinMapMethod::serialized; +} +} // namespace DB diff --git a/dbms/src/Interpreters/JoinHashTable.h b/dbms/src/Interpreters/JoinHashMap.h similarity index 97% rename from dbms/src/Interpreters/JoinHashTable.h rename to dbms/src/Interpreters/JoinHashMap.h index ae233f27403..a33129531a7 100644 --- a/dbms/src/Interpreters/JoinHashTable.h +++ b/dbms/src/Interpreters/JoinHashMap.h @@ -19,6 +19,7 @@ namespace DB { +using Sizes = std::vector; /// Reference to the row in block. struct RowRef { @@ -84,7 +85,7 @@ struct WithUsedFlag : Base M(keys256) \ M(serialized) -enum class JoinType +enum class JoinMapMethod { EMPTY, CROSS, @@ -165,4 +166,6 @@ using MapsAny = MapsTemplate>; using MapsAll = MapsTemplate>; using MapsAnyFull = MapsTemplate>; using MapsAllFull = MapsTemplate>; + +JoinMapMethod chooseJoinMapMethod(const ColumnRawPtrs & key_columns, Sizes & key_sizes, const TiDB::TiDBCollators & collators); } // namespace DB diff --git a/dbms/src/Interpreters/JoinPartition.cpp b/dbms/src/Interpreters/JoinPartition.cpp index ef731eea570..27aebb9b26a 100644 --- a/dbms/src/Interpreters/JoinPartition.cpp +++ b/dbms/src/Interpreters/JoinPartition.cpp @@ -63,18 +63,18 @@ void insertRowToList(RowRefList * list, RowRefList * elem, Block * stored_block, } template -static void initImpl(Maps & maps, JoinType type) +static void initImpl(Maps & maps, JoinMapMethod method) { - switch (type) + switch (method) { - case JoinType::EMPTY: + case JoinMapMethod::EMPTY: break; - case JoinType::CROSS: + case JoinMapMethod::CROSS: break; -#define M(TYPE) \ - case JoinType::TYPE: \ - maps.TYPE = std::make_unique(); \ +#define M(METHOD) \ + case JoinMapMethod::METHOD: \ + maps.METHOD = std::make_unique(); \ break; APPLY_FOR_JOIN_VARIANTS(M) #undef M @@ -85,18 +85,18 @@ static void initImpl(Maps & maps, JoinType type) } template -static Map & getMapImpl(Maps & maps, JoinType type) +static Map & getMapImpl(Maps & maps, JoinMapMethod method) { void * ret = nullptr; - switch (type) + switch (method) { - case JoinType::EMPTY: - case JoinType::CROSS: + case JoinMapMethod::EMPTY: + case JoinMapMethod::CROSS: throw Exception("Should not reach here"); -#define M(TYPE) \ - case JoinType::TYPE: \ - ret = maps.TYPE.get(); \ +#define M(METHOD) \ + case JoinMapMethod::METHOD: \ + ret = maps.METHOD.get(); \ break; APPLY_FOR_JOIN_VARIANTS(M) #undef M @@ -108,17 +108,17 @@ static Map & getMapImpl(Maps & maps, JoinType type) } template -static size_t getRowCountImpl(const Maps & maps, JoinType type) +static size_t getRowCountImpl(const Maps & maps, JoinMapMethod method) { - switch (type) + switch (method) { - case JoinType::EMPTY: + case JoinMapMethod::EMPTY: return 0; - case JoinType::CROSS: + case JoinMapMethod::CROSS: return 0; -#define M(NAME) \ - case JoinType::NAME: \ +#define M(NAME) \ + case JoinMapMethod::NAME: \ return maps.NAME ? maps.NAME->size() : 0; APPLY_FOR_JOIN_VARIANTS(M) #undef M @@ -129,17 +129,17 @@ static size_t getRowCountImpl(const Maps & maps, JoinType type) } template -static size_t getByteCountImpl(const Maps & maps, JoinType type) +static size_t getByteCountImpl(const Maps & maps, JoinMapMethod method) { - switch (type) + switch (method) { - case JoinType::EMPTY: + case JoinMapMethod::EMPTY: return 0; - case JoinType::CROSS: + case JoinMapMethod::CROSS: return 0; -#define M(NAME) \ - case JoinType::NAME: \ +#define M(NAME) \ + case JoinMapMethod::NAME: \ return maps.NAME ? maps.NAME->getBufferSizeInBytes() : 0; APPLY_FOR_JOIN_VARIANTS(M) #undef M @@ -150,17 +150,17 @@ static size_t getByteCountImpl(const Maps & maps, JoinType type) } template -static size_t clearMaps(Maps & maps, JoinType type) +static size_t clearMaps(Maps & maps, JoinMapMethod method) { size_t ret = 0; - switch (type) + switch (method) { - case JoinType::EMPTY: - case JoinType::CROSS: + case JoinMapMethod::EMPTY: + case JoinMapMethod::CROSS: ret = 0; break; #define M(NAME) \ - case JoinType::NAME: \ + case JoinMapMethod::NAME: \ if (maps.NAME) \ { \ ret = maps.NAME->getBufferSizeInBytes(); \ @@ -179,20 +179,20 @@ static size_t clearMaps(Maps & maps, JoinType type) size_t JoinPartition::getRowCount() { size_t ret = 0; - ret += getRowCountImpl(maps_any, join_type); - ret += getRowCountImpl(maps_all, join_type); - ret += getRowCountImpl(maps_any_full, join_type); - ret += getRowCountImpl(maps_all_full, join_type); + ret += getRowCountImpl(maps_any, join_map_method); + ret += getRowCountImpl(maps_all, join_map_method); + ret += getRowCountImpl(maps_any_full, join_map_method); + ret += getRowCountImpl(maps_all_full, join_map_method); return ret; } size_t JoinPartition::getHashMapAndPoolByteCount() { size_t ret = 0; - ret += getByteCountImpl(maps_any, join_type); - ret += getByteCountImpl(maps_all, join_type); - ret += getByteCountImpl(maps_any_full, join_type); - ret += getByteCountImpl(maps_all_full, join_type); + ret += getByteCountImpl(maps_any, join_map_method); + ret += getByteCountImpl(maps_all, join_map_method); + ret += getByteCountImpl(maps_any_full, join_map_method); + ret += getByteCountImpl(maps_all_full, join_map_method); ret += pool->size(); return ret; } @@ -205,16 +205,16 @@ void JoinPartition::initMap() if (!getFullness(kind)) { if (strictness == ASTTableJoin::Strictness::Any) - initImpl(maps_any, join_type); + initImpl(maps_any, join_map_method); else - initImpl(maps_all, join_type); + initImpl(maps_all, join_map_method); } else { if (strictness == ASTTableJoin::Strictness::Any) - initImpl(maps_any_full, join_type); + initImpl(maps_any_full, join_map_method); else - initImpl(maps_all_full, join_type); + initImpl(maps_all_full, join_map_method); } } @@ -264,10 +264,10 @@ void JoinPartition::releasePartitionPoolAndHashMap(std::unique_lock { size_t released_bytes = pool->size(); pool.reset(); - released_bytes += clearMaps(maps_any, join_type); - released_bytes += clearMaps(maps_all, join_type); - released_bytes += clearMaps(maps_any_full, join_type); - released_bytes += clearMaps(maps_all_full, join_type); + released_bytes += clearMaps(maps_any, join_map_method); + released_bytes += clearMaps(maps_all, join_map_method); + released_bytes += clearMaps(maps_any_full, join_map_method); + released_bytes += clearMaps(maps_all_full, join_map_method); subMemoryUsage(released_bytes); } @@ -308,73 +308,73 @@ Blocks JoinPartition::trySpillProbePartition(bool force, size_t max_cached_data_ namespace { /// code for hash map insertion -template +template struct KeyGetterForTypeImpl; template -struct KeyGetterForTypeImpl +struct KeyGetterForTypeImpl { using Type = ColumnsHashing::HashMethodOneNumber; }; template -struct KeyGetterForTypeImpl +struct KeyGetterForTypeImpl { using Type = ColumnsHashing::HashMethodOneNumber; }; template -struct KeyGetterForTypeImpl +struct KeyGetterForTypeImpl { using Type = ColumnsHashing::HashMethodOneNumber; }; template -struct KeyGetterForTypeImpl +struct KeyGetterForTypeImpl { using Type = ColumnsHashing::HashMethodOneNumber; }; template -struct KeyGetterForTypeImpl +struct KeyGetterForTypeImpl { using Type = ColumnsHashing::HashMethodString; }; template -struct KeyGetterForTypeImpl +struct KeyGetterForTypeImpl { using Type = ColumnsHashing::HashMethodStringBin; }; template -struct KeyGetterForTypeImpl +struct KeyGetterForTypeImpl { using Type = ColumnsHashing::HashMethodStringBin; }; template -struct KeyGetterForTypeImpl +struct KeyGetterForTypeImpl { using Type = ColumnsHashing::HashMethodFixedString; }; template -struct KeyGetterForTypeImpl +struct KeyGetterForTypeImpl { using Type = ColumnsHashing::HashMethodKeysFixed; }; template -struct KeyGetterForTypeImpl +struct KeyGetterForTypeImpl { using Type = ColumnsHashing::HashMethodKeysFixed; }; template -struct KeyGetterForTypeImpl +struct KeyGetterForTypeImpl { using Type = ColumnsHashing::HashMethodSerialized; }; -template +template struct KeyGetterForType { using Value = typename Data::value_type; using Mapped_t = typename Data::mapped_type; using Mapped = std::conditional_t, const Mapped_t, Mapped_t>; - using Type = typename KeyGetterForTypeImpl::Type; + using Type = typename KeyGetterForTypeImpl::Type; }; /// Inserting an element into a hash table of the form `key -> reference to a string`, which will then be used by JOIN. @@ -616,27 +616,27 @@ void insertBlockIntoMapsImpl( bool enable_fine_grained_shuffle, bool enable_join_spill) { - switch (join_partitions[stream_index]->getJoinType()) + switch (join_partitions[stream_index]->getJoinMapMethod()) { - case JoinType::EMPTY: + case JoinMapMethod::EMPTY: break; - case JoinType::CROSS: + case JoinMapMethod::CROSS: break; /// Do nothing. We have already saved block, and it is enough. -#define M(TYPE) \ - case JoinType::TYPE: \ - insertBlockIntoMapsImplType::Type, typename Maps::TYPE##Type>( \ - join_partitions, \ - rows, \ - key_columns, \ - key_sizes, \ - collators, \ - stored_block, \ - null_map, \ - stream_index, \ - insert_concurrency, \ - enable_fine_grained_shuffle, \ - enable_join_spill); \ +#define M(METHOD) \ + case JoinMapMethod::METHOD: \ + insertBlockIntoMapsImplType::Type, typename Maps::METHOD##Type>( \ + join_partitions, \ + rows, \ + key_columns, \ + key_sizes, \ + collators, \ + stored_block, \ + null_map, \ + stream_index, \ + insert_concurrency, \ + enable_fine_grained_shuffle, \ + enable_join_spill); \ break; APPLY_FOR_JOIN_VARIANTS(M) #undef M @@ -654,16 +654,16 @@ Map & JoinPartition::getHashMap() if (getFullness(kind)) { if (strictness == ASTTableJoin::Strictness::Any) - return getMapImpl(maps_any_full, join_type); + return getMapImpl(maps_any_full, join_map_method); else - return getMapImpl(maps_all_full, join_type); + return getMapImpl(maps_all_full, join_map_method); } else { if (strictness == ASTTableJoin::Strictness::Any) - return getMapImpl(maps_any, join_type); + return getMapImpl(maps_any, join_map_method); else - return getMapImpl(maps_all, join_type); + return getMapImpl(maps_all, join_map_method); } } @@ -1354,25 +1354,25 @@ void JoinPartition::probeBlockImpl( ProbeProcessInfo & probe_process_info) { const auto & current_join_partition = join_partitions[probe_process_info.partition_index]; - auto type = current_join_partition->join_type; - switch (type) + auto method = current_join_partition->join_map_method; + switch (method) { -#define M(TYPE) \ - case JoinType::TYPE: \ - probeBlockImplType::Type, typename Maps::TYPE##Type>( \ - join_partitions, \ - rows, \ - key_columns, \ - key_sizes, \ - added_columns, \ - null_map, \ - filter, \ - current_offset, \ - offsets_to_replicate, \ - right_indexes, \ - collators, \ - join_build_info, \ - probe_process_info); \ +#define M(METHOD) \ + case JoinMapMethod::METHOD: \ + probeBlockImplType::Type, typename Maps::METHOD##Type>( \ + join_partitions, \ + rows, \ + key_columns, \ + key_sizes, \ + added_columns, \ + null_map, \ + filter, \ + current_offset, \ + offsets_to_replicate, \ + right_indexes, \ + collators, \ + join_build_info, \ + probe_process_info); \ break; APPLY_FOR_JOIN_VARIANTS(M) #undef M @@ -1392,18 +1392,18 @@ std::pair>, std::listjoin_type; - switch (type) + auto method = join_partitions[0]->join_map_method; + switch (method) { -#define M(TYPE) \ - case JoinType::TYPE: \ - return probeBlockNullAwareType::Type, typename Maps::TYPE##Type>( \ - join_partitions, \ - block, \ - key_columns, \ - key_sizes, \ - collators, \ - left_side_info, \ +#define M(METHOD) \ + case JoinMapMethod::METHOD: \ + return probeBlockNullAwareType::Type, typename Maps::METHOD##Type>( \ + join_partitions, \ + block, \ + key_columns, \ + key_sizes, \ + collators, \ + left_side_info, \ right_side_info); APPLY_FOR_JOIN_VARIANTS(M) #undef M diff --git a/dbms/src/Interpreters/JoinPartition.h b/dbms/src/Interpreters/JoinPartition.h index 2ca1582920a..53ae98285f2 100644 --- a/dbms/src/Interpreters/JoinPartition.h +++ b/dbms/src/Interpreters/JoinPartition.h @@ -17,7 +17,7 @@ #include #include #include -#include +#include #include #include #include @@ -73,10 +73,10 @@ using JoinPartitions = std::vector>; class JoinPartition { public: - JoinPartition(JoinType join_type_, ASTTableJoin::Kind kind_, ASTTableJoin::Strictness strictness_, size_t max_block_size, const LoggerPtr & log_) + JoinPartition(JoinMapMethod join_map_type_, ASTTableJoin::Kind kind_, ASTTableJoin::Strictness strictness_, size_t max_block_size, const LoggerPtr & log_) : kind(kind_) , strictness(strictness_) - , join_type(join_type_) + , join_map_method(join_map_type_) , pool(std::make_shared()) , spill(false) , log(log_) @@ -131,7 +131,7 @@ class JoinPartition } bool isSpill() const { return spill; } void markSpill() { spill = true; } - JoinType getJoinType() const { return join_type; } + JoinMapMethod getJoinMapMethod() const { return join_map_method; } ASTTableJoin::Kind getJoinKind() const { return kind; } Block * getLastBuildBlock() { return &build_partition.blocks.back(); } ArenaPtr & getPartitionPool() @@ -212,7 +212,7 @@ class JoinPartition ASTTableJoin::Kind kind; ASTTableJoin::Strictness strictness; - JoinType join_type; + JoinMapMethod join_map_method; MapsAny maps_any; /// For ANY LEFT|INNER JOIN MapsAll maps_all; /// For ALL LEFT|INNER JOIN MapsAnyFull maps_any_full; /// For ANY RIGHT|FULL JOIN diff --git a/dbms/src/Server/DTTool/DTTool.h b/dbms/src/Server/DTTool/DTTool.h index fe0d4c16af4..51e57dab31f 100644 --- a/dbms/src/Server/DTTool/DTTool.h +++ b/dbms/src/Server/DTTool/DTTool.h @@ -127,7 +127,7 @@ class ImitativeEnv static void setupLogger() { Poco::AutoPtr channel = new Poco::ConsoleChannel(std::cout); - Poco::AutoPtr formatter(new UnifiedLogFormatter()); + Poco::AutoPtr formatter(new UnifiedLogFormatter()); Poco::AutoPtr formatting_channel(new Poco::FormattingChannel(formatter, channel)); Poco::Logger::root().setChannel(formatting_channel); Poco::Logger::root().setLevel("trace"); diff --git a/dbms/src/Server/RaftConfigParser.cpp b/dbms/src/Server/RaftConfigParser.cpp index c53a0c44a30..c2dc1e78532 100644 --- a/dbms/src/Server/RaftConfigParser.cpp +++ b/dbms/src/Server/RaftConfigParser.cpp @@ -35,8 +35,14 @@ TiFlashRaftConfig TiFlashRaftConfig::parseSettings(Poco::Util::AbstractConfigura TiFlashRaftConfig res; res.flash_server_addr = config.getString("flash.service_addr", "0.0.0.0:3930"); - if (!config.has("raft")) - return res; + { + // Check by `raft` prefix instead of check by `config.has("raft")`, + // because when sub keys are set from cli args, `raft` will not exist. + Poco::Util::AbstractConfiguration::Keys keys; + config.keys("raft", keys); + if (keys.empty()) + return res; + } if (config.has("raft.pd_addr")) { diff --git a/dbms/src/Server/Server.cpp b/dbms/src/Server/Server.cpp index 239ec3e795b..432aa3c1aec 100644 --- a/dbms/src/Server/Server.cpp +++ b/dbms/src/Server/Server.cpp @@ -60,6 +60,7 @@ #include #include #include +#include #include #include #include @@ -236,6 +237,31 @@ void Server::initialize(Poco::Util::Application & self) logger().information("starting up"); } +void Server::defineOptions(Poco::Util::OptionSet & options) +{ + options.addOption( + Poco::Util::Option("help", "h", "show help and exit") + .required(false) + .repeatable(false) + .binding("help")); + BaseDaemon::defineOptions(options); +} + +int Server::run() +{ + if (config().hasOption("help")) + { + Poco::Util::HelpFormatter help_formatter(Server::options()); + auto header_str = fmt::format("{} server [OPTION] [-- [POSITIONAL_ARGS]...]\n" + "POSITIONAL_ARGS can be used to rewrite config properties, for example, --http_port=8010", + commandName()); + help_formatter.setHeader(header_str); + help_formatter.format(std::cout); + return 0; + } + return BaseDaemon::run(); +} + std::string Server::getDefaultCorePath() const { return getCanonicalPath(config().getString("path")) + "cores"; @@ -243,19 +269,11 @@ std::string Server::getDefaultCorePath() const struct TiFlashProxyConfig { - static const std::string config_prefix; std::vector args; std::unordered_map val_map; bool is_proxy_runnable = false; // TiFlash Proxy will set the default value of "flash.proxy.addr", so we don't need to set here. - const String engine_store_version = "engine-version"; - const String engine_store_git_hash = "engine-git-hash"; - const String engine_store_address = "engine-addr"; - const String engine_store_advertise_address = "advertise-engine-addr"; - const String pd_endpoints = "pd-endpoints"; - const String engine_label = "engine-label"; - const String engine_role_label = "engine-role-label"; void addExtraArgs(const std::string & k, const std::string & v) { @@ -273,38 +291,41 @@ struct TiFlashProxyConfig // tiflash_compute doesn't need proxy. // todo: remove after AutoScaler is stable. if (disaggregated_mode == DisaggregatedMode::Compute && useAutoScaler(config)) + { + LOG_WARNING(Logger::get(), "TiFlash Proxy will not start because AutoScale Disaggregated Compute Mode is specified."); return; + } + + Poco::Util::AbstractConfiguration::Keys keys; + config.keys("flash.proxy", keys); + + if (!config.has("raft.pd_addr")) + { + LOG_WARNING(Logger::get(), "TiFlash Proxy will not start because `raft.pd_addr` is not configured."); + if (!keys.empty()) + LOG_WARNING(Logger::get(), "`flash.proxy.*` is ignored because TiFlash Proxy will not start."); - if (!config.has(config_prefix)) return; + } - Poco::Util::AbstractConfiguration::Keys keys; - config.keys(config_prefix, keys); { std::unordered_map args_map; for (const auto & key : keys) - { - const auto k = config_prefix + "." + key; - args_map[key] = config.getString(k); - } - args_map[pd_endpoints] = config.getString("raft.pd_addr"); - args_map[engine_store_version] = TiFlashBuildInfo::getReleaseVersion(); - args_map[engine_store_git_hash] = TiFlashBuildInfo::getGitHash(); - if (!args_map.count(engine_store_address)) - args_map[engine_store_address] = config.getString("flash.service_addr"); - else - args_map[engine_store_advertise_address] = args_map[engine_store_address]; + args_map[key] = config.getString("flash.proxy." + key); - args_map[engine_label] = getProxyLabelByDisaggregatedMode(disaggregated_mode); + args_map["pd-endpoints"] = config.getString("raft.pd_addr"); + args_map["engine-version"] = TiFlashBuildInfo::getReleaseVersion(); + args_map["engine-git-hash"] = TiFlashBuildInfo::getGitHash(); + if (!args_map.contains("engine-addr")) + args_map["engine-addr"] = config.getString("flash.service_addr", "0.0.0.0:3930"); + else + args_map["advertise-engine-addr"] = args_map["engine-addr"]; + args_map["engine-label"] = getProxyLabelByDisaggregatedMode(disaggregated_mode); if (disaggregated_mode != DisaggregatedMode::Compute && has_s3_config) - { - args_map[engine_role_label] = DISAGGREGATED_MODE_WRITE_ENGINE_ROLE; - } + args_map["engine-role-label"] = DISAGGREGATED_MODE_WRITE_ENGINE_ROLE; for (auto && [k, v] : args_map) - { val_map.emplace("--" + k, std::move(v)); - } } args.push_back("TiFlash Proxy"); @@ -317,8 +338,6 @@ struct TiFlashProxyConfig } }; -const std::string TiFlashProxyConfig::config_prefix = "flash.proxy"; - pingcap::ClusterConfig getClusterConfig(TiFlashSecurityConfigPtr security_config, const TiFlashRaftConfig & raft_config, const int api_version, const LoggerPtr & log) { pingcap::ClusterConfig config; @@ -984,6 +1003,10 @@ int Server::main(const std::vector & /*args*/) else LOG_INFO(log, "encryption is disabled"); } + else + { + LOG_WARNING(log, "Skipped initialize TiFlash Proxy"); + } SCOPE_EXIT({ if (!proxy_conf.is_proxy_runnable) @@ -1327,7 +1350,9 @@ int Server::main(const std::vector & /*args*/) /// Reload config in SYSTEM RELOAD CONFIG query. global_context->setConfigReloadCallback([&]() { main_config_reloader->reload(); - users_config_reloader->reload(); + + if (users_config_reloader) + users_config_reloader->reload(); }); /// Limit on total number of concurrently executed queries. @@ -1529,7 +1554,8 @@ int Server::main(const std::vector & /*args*/) main_config_reloader->addConfigObject(global_context->getSecurityConfig()); main_config_reloader->start(); - users_config_reloader->start(); + if (users_config_reloader) + users_config_reloader->start(); { // on ARM processors it can show only enabled at current moment cores diff --git a/dbms/src/Server/Server.h b/dbms/src/Server/Server.h index fc8141dbd9d..0f28e6c6858 100644 --- a/dbms/src/Server/Server.h +++ b/dbms/src/Server/Server.h @@ -34,6 +34,8 @@ class Server : public BaseDaemon , public IServer { public: + using ServerApplication::run; + Poco::Util::LayeredConfiguration & config() const override { return BaseDaemon::config(); @@ -54,7 +56,11 @@ class Server : public BaseDaemon return BaseDaemon::isCancelled(); } + void defineOptions(Poco::Util::OptionSet & _options) override; + protected: + int run() override; + void initialize(Application & self) override; void uninitialize() override; diff --git a/dbms/src/Server/StorageConfigParser.cpp b/dbms/src/Server/StorageConfigParser.cpp index 338ed179a03..d1617d31aea 100644 --- a/dbms/src/Server/StorageConfigParser.cpp +++ b/dbms/src/Server/StorageConfigParser.cpp @@ -270,13 +270,13 @@ bool TiFlashStorageConfig::parseFromDeprecatedConfiguration(Poco::Util::LayeredC if (!config.has("path")) return false; - LOG_WARNING(log, "The configuration \"path\" is deprecated. Check [storage] section for new style."); + LOG_WARNING(log, "The configuration `path` is deprecated. Check [storage] section for new style."); String paths = config.getString("path"); Poco::trimInPlace(paths); if (paths.empty()) throw Exception( - fmt::format("The configuration \"path\" is empty! [path={}]", config.getString("path")), + fmt::format("The configuration `path` is empty! [path={}]", config.getString("path")), ErrorCodes::INVALID_CONFIG_PARAMETER); Strings all_normal_path; Poco::StringTokenizer string_tokens(paths, ","); @@ -309,7 +309,7 @@ bool TiFlashStorageConfig::parseFromDeprecatedConfiguration(Poco::Util::LayeredC String str_kvstore_path; if (config.has("raft.kvstore_path")) { - LOG_WARNING(log, "The configuration \"raft.kvstore_path\" is deprecated. Check [storage.raft] section for new style."); + LOG_WARNING(log, "The configuration `raft.kvstore_path` is deprecated. Check [storage.raft] section for new style."); str_kvstore_path = config.getString("raft.kvstore_path"); } if (str_kvstore_path.empty()) @@ -348,9 +348,9 @@ std::tuple TiFlashStorageConfig::parseSettings(Poc if (config.has("storage.main")) { if (config.has("path")) - LOG_WARNING(log, "The configuration \"path\" is ignored when \"storage\" is defined."); + LOG_WARNING(log, "The configuration `path` is ignored when `storage` is defined."); if (config.has("capacity")) - LOG_WARNING(log, "The configuration \"capacity\" is ignored when \"storage\" is defined."); + LOG_WARNING(log, "The configuration `capacity` is ignored when `storage` is defined."); storage_config.parseStoragePath(config.getString("storage"), log); @@ -360,7 +360,7 @@ std::tuple TiFlashStorageConfig::parseSettings(Poc String deprecated_kvstore_path = config.getString("raft.kvstore_path"); if (!deprecated_kvstore_path.empty()) { - LOG_WARNING(log, "The configuration \"raft.kvstore_path\" is deprecated. Check \"storage.raft.dir\" for new style."); + LOG_WARNING(log, "The configuration `raft.kvstore_path` is deprecated. Check `storage.raft.dir` for new style."); kvstore_paths.clear(); kvstore_paths.emplace_back(getNormalizedPath(deprecated_kvstore_path)); for (auto & kvstore_path : kvstore_paths) @@ -379,7 +379,7 @@ std::tuple TiFlashStorageConfig::parseSettings(Poc // capacity if (config.has("capacity")) { - LOG_WARNING(log, "The configuration \"capacity\" is deprecated. Check [storage] section for new style."); + LOG_WARNING(log, "The configuration `capacity` is deprecated. Check [storage] section for new style."); // TODO: support human readable format for capacity, mark_cache_size, minmax_index_cache_size // eg. 100GiB, 10MiB String capacities = config.getString("capacity"); @@ -402,7 +402,7 @@ std::tuple TiFlashStorageConfig::parseSettings(Poc if (!storage_config.parseFromDeprecatedConfiguration(config, log)) { // Can not parse from the deprecated configuration "path". - String msg = "The configuration \"storage.main\" section is not defined. Please check your configuration file."; + String msg = "The configuration `storage.main` section is not defined. Please check your configuration file."; LOG_ERROR(log, "{}", msg); throw Exception(msg, ErrorCodes::INVALID_CONFIG_PARAMETER); } diff --git a/dbms/src/Server/StorageConfigParser.h b/dbms/src/Server/StorageConfigParser.h index 727f11b0d63..6802ef1b2b7 100644 --- a/dbms/src/Server/StorageConfigParser.h +++ b/dbms/src/Server/StorageConfigParser.h @@ -125,7 +125,7 @@ struct StorageRemoteCacheConfig String dir; UInt64 capacity = 0; UInt64 dtfile_level = 100; - double delta_rate = 0.3; + double delta_rate = 0.1; bool isCacheEnabled() const; void initCacheDir() const; diff --git a/dbms/src/Server/UserConfigParser.cpp b/dbms/src/Server/UserConfigParser.cpp index 07f22790b14..0edde9ce598 100644 --- a/dbms/src/Server/UserConfigParser.cpp +++ b/dbms/src/Server/UserConfigParser.cpp @@ -67,6 +67,12 @@ ConfigReloaderPtr parseSettings( if (load_from_main_config_path) users_config_path = config_path; + if (users_config_path.empty()) + { + global_context->setUsersConfig(new Poco::Util::LayeredConfiguration()); + return nullptr; + } + LOG_INFO(log, "Set users config file to: {}", users_config_path); return std::make_unique( diff --git a/dbms/src/Storages/DeltaMerge/File/DMFile.cpp b/dbms/src/Storages/DeltaMerge/File/DMFile.cpp index dba3a36a218..ad79c05e37a 100644 --- a/dbms/src/Storages/DeltaMerge/File/DMFile.cpp +++ b/dbms/src/Storages/DeltaMerge/File/DMFile.cpp @@ -156,7 +156,11 @@ DMFilePtr DMFile::restore( auto is_s3_file = S3::S3FilenameView::fromKeyWithPrefix(parent_path).isDataFile(); if (!is_s3_file) { - RUNTIME_CHECK(Poco::Path(parent_path).isAbsolute(), parent_path); + // Unrecognized xx:// protocol. + RUNTIME_CHECK_MSG( + parent_path.find("://") == std::string::npos, + "Unsupported protocol in path {}", + parent_path); String path = getPathByStatus(parent_path, file_id, DMFile::Status::READABLE); // The path may be dropped by another thread in some cases auto poco_file = Poco::File(path); diff --git a/dbms/src/Storages/Page/PageConstants.h b/dbms/src/Storages/Page/PageConstants.h index 0e5e4a091a3..f8b0364702e 100644 --- a/dbms/src/Storages/Page/PageConstants.h +++ b/dbms/src/Storages/Page/PageConstants.h @@ -24,10 +24,15 @@ static constexpr UInt64 GB = MB * 1024; enum class StorageType { + Unknown = 0, Log = 1, Data = 2, Meta = 3, KVStore = 4, + RaftEngine = 5, + KVEngine = 6, + + _MAX_STORAGE_TYPE_, // NOLINT(bugprone-reserved-identifier) }; enum class PageStorageRunMode : UInt8 diff --git a/dbms/src/Storages/Page/V3/CheckpointFile/CPDataFileStat.h b/dbms/src/Storages/Page/V3/CheckpointFile/CPDataFileStat.h index 7c5bf352d23..1b20c262b8a 100644 --- a/dbms/src/Storages/Page/V3/CheckpointFile/CPDataFileStat.h +++ b/dbms/src/Storages/Page/V3/CheckpointFile/CPDataFileStat.h @@ -26,12 +26,6 @@ namespace DB::PS::V3 { -struct CPDataWriteStats -{ - bool has_new_data = false; - size_t incremental_data_bytes = 0; - size_t compact_data_bytes = 0; -}; using RemoteFileValidSizes = std::unordered_map; diff --git a/dbms/src/Storages/Page/V3/CheckpointFile/CPDumpStat.cpp b/dbms/src/Storages/Page/V3/CheckpointFile/CPDumpStat.cpp new file mode 100644 index 00000000000..176eb42c8e3 --- /dev/null +++ b/dbms/src/Storages/Page/V3/CheckpointFile/CPDumpStat.cpp @@ -0,0 +1,85 @@ +// Copyright 2023 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 + +namespace DB::PS::V3 +{ + +void SetMetrics(const CPDataDumpStats & stats) +{ + for (size_t i = 0; i < static_cast(DB::StorageType::_MAX_STORAGE_TYPE_); ++i) + { + auto type = static_cast(i); + switch (type) + { + case DB::StorageType::Unknown: + { + GET_METRIC(tiflash_storage_checkpoint_keys_by_types, type_unknown).Increment(stats.num_keys[i]); + GET_METRIC(tiflash_storage_checkpoint_flow_by_types, type_unknown).Increment(stats.num_bytes[i]); + GET_METRIC(tiflash_storage_page_data_by_types, type_unknown).Set(stats.num_existing_bytes[i]); + break; + } + case DB::StorageType::RaftEngine: + { + GET_METRIC(tiflash_storage_checkpoint_keys_by_types, type_raftengine).Increment(stats.num_keys[i]); + GET_METRIC(tiflash_storage_checkpoint_flow_by_types, type_raftengine).Increment(stats.num_bytes[i]); + GET_METRIC(tiflash_storage_page_data_by_types, type_raftengine).Set(stats.num_existing_bytes[i]); + break; + } + case DB::StorageType::KVEngine: + { + GET_METRIC(tiflash_storage_checkpoint_keys_by_types, type_kvengine).Increment(stats.num_keys[i]); + GET_METRIC(tiflash_storage_checkpoint_flow_by_types, type_kvengine).Increment(stats.num_bytes[i]); + GET_METRIC(tiflash_storage_page_data_by_types, type_kvengine).Set(stats.num_existing_bytes[i]); + break; + } + case DB::StorageType::KVStore: + { + GET_METRIC(tiflash_storage_checkpoint_keys_by_types, type_kvstore).Increment(stats.num_keys[i]); + GET_METRIC(tiflash_storage_checkpoint_flow_by_types, type_kvstore).Increment(stats.num_bytes[i]); + GET_METRIC(tiflash_storage_page_data_by_types, type_kvstore).Set(stats.num_existing_bytes[i]); + break; + } + case DB::StorageType::Data: + { + GET_METRIC(tiflash_storage_checkpoint_keys_by_types, type_data).Increment(stats.num_keys[i]); + GET_METRIC(tiflash_storage_checkpoint_flow_by_types, type_data).Increment(stats.num_bytes[i]); + GET_METRIC(tiflash_storage_page_data_by_types, type_data).Set(stats.num_existing_bytes[i]); + break; + } + case DB::StorageType::Log: + { + GET_METRIC(tiflash_storage_checkpoint_keys_by_types, type_log).Increment(stats.num_keys[i]); + GET_METRIC(tiflash_storage_checkpoint_flow_by_types, type_log).Increment(stats.num_bytes[i]); + GET_METRIC(tiflash_storage_page_data_by_types, type_log).Set(stats.num_existing_bytes[i]); + break; + } + case DB::StorageType::Meta: + { + GET_METRIC(tiflash_storage_checkpoint_keys_by_types, type_meta).Increment(stats.num_keys[i]); + GET_METRIC(tiflash_storage_checkpoint_flow_by_types, type_meta).Increment(stats.num_bytes[i]); + GET_METRIC(tiflash_storage_page_data_by_types, type_meta).Set(stats.num_existing_bytes[i]); + break; + } + default: + __builtin_unreachable(); + } + } +} + +} // namespace DB::PS::V3 diff --git a/dbms/src/Storages/Page/V3/CheckpointFile/CPDumpStat.h b/dbms/src/Storages/Page/V3/CheckpointFile/CPDumpStat.h new file mode 100644 index 00000000000..5174e0b2e9f --- /dev/null +++ b/dbms/src/Storages/Page/V3/CheckpointFile/CPDumpStat.h @@ -0,0 +1,108 @@ +// Copyright 2023 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::PS::V3 +{ + +struct CPDataDumpStats +{ + bool has_new_data = false; + + size_t incremental_data_bytes = 0; + size_t compact_data_bytes = 0; + + // The number of keys uploaded in this checkpoint + std::array(StorageType::_MAX_STORAGE_TYPE_)> num_keys{}; + // The number of bytes uploaded in this checkpoint + std::array(StorageType::_MAX_STORAGE_TYPE_)> num_bytes{}; + + // The number of bytes this checkpoint is holding. Some of the data are already uploaded + // in the previous checkpoint data file. + std::array(StorageType::_MAX_STORAGE_TYPE_)> num_existing_bytes{}; + + // Total number of records in this checkpoint + size_t num_records = 0; + // Number of Pages that already uploaded to S3 + // and is not changed in this checkpoint + size_t num_pages_unchanged = 0; + // Number of Pages that already uploaded to S3 + // but picked by compaction in this checkpoint + size_t num_pages_compact = 0; + // Number of incremental Pages since last checkpoint + size_t num_pages_incremental = 0; + // Number of ExternalPages + size_t num_ext_pages = 0; + // Number of RefPages + size_t num_ref_pages = 0; + // Number of delete records + size_t num_delete_records = 0; + // Number of other records other than Pages/ExternalPages + size_t num_other_records = 0; +}; + +void SetMetrics(const CPDataDumpStats & stats); + +} // namespace DB::PS::V3 + +template <> +struct fmt::formatter +{ + static constexpr auto parse(format_parse_context & ctx) { return ctx.begin(); } + + template + auto format(const DB::PS::V3::CPDataDumpStats & value, FormatContext & ctx) const -> decltype(ctx.out()) + { + auto it = format_to( + ctx.out(), + "CPDataDumpStats{{" + "incremental_data_bytes={} compact_data_bytes={}" + " n_records{{total={}" + " pages_unchanged={} pages_compact={} pages_incremental={} ext_pages={} ref_pages={}" + " delete={} other={}}}", + value.incremental_data_bytes, + value.compact_data_bytes, + value.num_records, + value.num_pages_unchanged, + value.num_pages_compact, + value.num_pages_incremental, + value.num_ext_pages, + value.num_ref_pages, + value.num_delete_records, + value.num_other_records); + it = format_to(it, " types["); + for (size_t i = 0; i < static_cast(DB::StorageType::_MAX_STORAGE_TYPE_); ++i) + { + if (i != 0) + it = format_to(it, " "); + it = format_to( + it, + "{{type={} keys={} bytes={}}}", + magic_enum::enum_name(static_cast(i)), + value.num_keys[i], + value.num_bytes[i]); + } + return format_to( + it, + "]" // end of "keys" + "}}" // end of "CPDataDumpStats" + ); + } +}; diff --git a/dbms/src/Storages/Page/V3/CheckpointFile/CPFilesWriter.cpp b/dbms/src/Storages/Page/V3/CheckpointFile/CPFilesWriter.cpp index 6e4c0a02b70..fd8f226ce53 100644 --- a/dbms/src/Storages/Page/V3/CheckpointFile/CPFilesWriter.cpp +++ b/dbms/src/Storages/Page/V3/CheckpointFile/CPFilesWriter.cpp @@ -12,8 +12,11 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include #include +#include #include +#include #include #include @@ -62,7 +65,7 @@ void CPFilesWriter::writePrefix(const CPFilesWriter::PrefixInfo & info) write_stage = WriteStage::WritingEdits; } -CPDataWriteStats CPFilesWriter::writeEditsAndApplyCheckpointInfo( +CPDataDumpStats CPFilesWriter::writeEditsAndApplyCheckpointInfo( universal::PageEntriesEdit & edits, const std::unordered_set & file_ids_to_compact) { @@ -72,14 +75,31 @@ CPDataWriteStats CPFilesWriter::writeEditsAndApplyCheckpointInfo( if (records.empty()) return {.has_new_data = false}; - CPDataWriteStats write_down_stats; + CPDataDumpStats write_down_stats; + for (size_t i = 0; i < static_cast(StorageType::_MAX_STORAGE_TYPE_); ++i) + { + write_down_stats.num_keys[i] = 0; + write_down_stats.num_bytes[i] = 0; + } + std::unordered_map compact_stats; bool last_page_is_raft_data = true; // 1. Iterate all edits, find these entry edits without the checkpoint info // and collect the lock files from applied entries. + write_down_stats.num_records = records.size(); for (auto & rec_edit : records) { + StorageType id_storage_type = StorageType::Unknown; + { + id_storage_type = UniversalPageIdFormat::getUniversalPageIdType(rec_edit.page_id); + // all keys are included in the manifest + write_down_stats.num_keys[static_cast(id_storage_type)] += 1; + // this is the page data size of all latest version keys, including some uploaded in the + // previous checkpoint + write_down_stats.num_existing_bytes[static_cast(id_storage_type)] += rec_edit.entry.size; + } + if (rec_edit.type == EditRecordType::VAR_EXTERNAL) { RUNTIME_CHECK_MSG( @@ -90,11 +110,26 @@ CPDataWriteStats CPFilesWriter::writeEditsAndApplyCheckpointInfo( rec_edit); // for example, the s3 fullpath of external id locked_files.emplace(*rec_edit.entry.checkpoint_info.data_location.data_file_id); + write_down_stats.num_ext_pages += 1; continue; } if (rec_edit.type != EditRecordType::VAR_ENTRY) + { + if (rec_edit.type == EditRecordType::VAR_REF) + { + write_down_stats.num_ref_pages += 1; + } + else if (rec_edit.type == EditRecordType::VAR_DELETE) + { + write_down_stats.num_delete_records += 1; + } + else + { + write_down_stats.num_other_records += 1; + } continue; + } bool is_compaction = false; if (rec_edit.entry.checkpoint_info.has_value()) @@ -104,6 +139,7 @@ CPDataWriteStats CPFilesWriter::writeEditsAndApplyCheckpointInfo( { // for example, the s3 fullpath that was written in the previous uploaded CheckpointDataFile locked_files.emplace(file_id); + write_down_stats.num_pages_unchanged += 1; continue; } // else we rewrite this entry data to the data file generated by this checkpoint, so that @@ -113,7 +149,7 @@ CPDataWriteStats CPFilesWriter::writeEditsAndApplyCheckpointInfo( compact_stats.try_emplace(file_id, 0).first->second += rec_edit.entry.size; } - bool current_page_is_raft_data = rec_edit.page_id.isRaftData(); + bool current_page_is_raft_data = (id_storage_type == StorageType::RaftEngine); if (current_write_size > 0 // If current_write_size is 0, data_writer is a empty file, not need to create a new one. && (current_page_is_raft_data != last_page_is_raft_data // Data type changed || (max_data_file_size != 0 && current_write_size >= max_data_file_size))) // or reach size limit. @@ -131,6 +167,8 @@ CPDataWriteStats CPFilesWriter::writeEditsAndApplyCheckpointInfo( rec_edit.version, page.data.begin(), page.data.size()); + // the page data size uploaded in this checkpoint + write_down_stats.num_bytes[static_cast(id_storage_type)] += rec_edit.entry.size; current_write_size += data_location.size_in_file; RUNTIME_CHECK(page.data.size() == rec_edit.entry.size, page.data.size(), rec_edit.entry.size); bool is_local_data_reclaimed = rec_edit.entry.checkpoint_info.has_value() && rec_edit.entry.checkpoint_info.is_local_data_reclaimed; @@ -143,10 +181,12 @@ CPDataWriteStats CPFilesWriter::writeEditsAndApplyCheckpointInfo( if (is_compaction) { write_down_stats.compact_data_bytes += rec_edit.entry.size; + write_down_stats.num_pages_compact += 1; } else { write_down_stats.incremental_data_bytes += rec_edit.entry.size; + write_down_stats.num_pages_incremental += 1; } } diff --git a/dbms/src/Storages/Page/V3/CheckpointFile/CPFilesWriter.h b/dbms/src/Storages/Page/V3/CheckpointFile/CPFilesWriter.h index d69f5ec2e49..97a4b7fa163 100644 --- a/dbms/src/Storages/Page/V3/CheckpointFile/CPFilesWriter.h +++ b/dbms/src/Storages/Page/V3/CheckpointFile/CPFilesWriter.h @@ -17,6 +17,7 @@ #include #include #include +#include #include #include #include @@ -80,7 +81,7 @@ class CPFilesWriter : private boost::noncopyable * * You must call `writeSuffix` finally, if you don't plan to write edits anymore. */ - CPDataWriteStats writeEditsAndApplyCheckpointInfo( + CPDataDumpStats writeEditsAndApplyCheckpointInfo( universal::PageEntriesEdit & edit, const std::unordered_set & file_ids_to_compact = {}); diff --git a/dbms/src/Storages/Page/V3/Universal/UniversalPageId.h b/dbms/src/Storages/Page/V3/Universal/UniversalPageId.h index 3d6266d6906..d2247af98bd 100644 --- a/dbms/src/Storages/Page/V3/Universal/UniversalPageId.h +++ b/dbms/src/Storages/Page/V3/Universal/UniversalPageId.h @@ -70,13 +70,7 @@ class UniversalPageId final friend bool operator==(const String & lhs, const UniversalPageId & rhs); - bool isRaftData() const - { - return !id.empty() && id[0] == raft_prefix; - } - private: - static constexpr char raft_prefix = 0x01; String id; }; diff --git a/dbms/src/Storages/Page/V3/Universal/UniversalPageIdFormatImpl.h b/dbms/src/Storages/Page/V3/Universal/UniversalPageIdFormatImpl.h index 1b89faef8b0..c87c3dbd980 100644 --- a/dbms/src/Storages/Page/V3/Universal/UniversalPageIdFormatImpl.h +++ b/dbms/src/Storages/Page/V3/Universal/UniversalPageIdFormatImpl.h @@ -18,6 +18,7 @@ #include #include #include +#include #include #include #include @@ -92,12 +93,15 @@ struct UniversalPageIdFormat return toFullPageId(getSubPrefix(StorageType::KVStore), region_id); } + static constexpr char RAFT_PREFIX = 0x01; + static constexpr char KV_PREFIX = 0x02; + // data is in kv engine, so it is prepended by KV_PREFIX // KV_PREFIX LOCAL_PREFIX REGION_RAFT_PREFIX region_id APPLY_STATE_SUFFIX static UniversalPageId toRaftApplyStateKeyInKVEngine(UInt64 region_id) { WriteBufferFromOwnString buff; - writeChar(0x02, buff); + writeChar(KV_PREFIX, buff); writeChar(0x01, buff); writeChar(0x02, buff); encodeUInt64(region_id, buff); @@ -110,7 +114,7 @@ struct UniversalPageIdFormat static UniversalPageId toRegionLocalStateKeyInKVEngine(UInt64 region_id) { WriteBufferFromOwnString buff; - writeChar(0x02, buff); + writeChar(KV_PREFIX, buff); writeChar(0x01, buff); writeChar(0x03, buff); encodeUInt64(region_id, buff); @@ -122,7 +126,7 @@ struct UniversalPageIdFormat static String toFullRaftLogPrefix(UInt64 region_id) { WriteBufferFromOwnString buff; - writeChar(0x01, buff); + writeChar(RAFT_PREFIX, buff); writeChar(0x01, buff); writeChar(0x02, buff); encodeUInt64(region_id, buff); @@ -134,7 +138,7 @@ struct UniversalPageIdFormat static String toFullRaftLogScanEnd(UInt64 region_id) { WriteBufferFromOwnString buff; - writeChar(0x01, buff); + writeChar(RAFT_PREFIX, buff); writeChar(0x01, buff); writeChar(0x02, buff); encodeUInt64(region_id, buff); @@ -148,7 +152,7 @@ struct UniversalPageIdFormat static String getStoreIdentIdInKVEngine() { WriteBufferFromOwnString buff; - writeChar(0x02, buff); + writeChar(KV_PREFIX, buff); writeChar(0x01, buff); writeChar(0x01, buff); return buff.releaseStr(); @@ -158,7 +162,7 @@ struct UniversalPageIdFormat static String getStoreIdentId() { WriteBufferFromOwnString buff; - writeChar(0x01, buff); + writeChar(RAFT_PREFIX, buff); writeChar(0x01, buff); writeChar(0x01, buff); return buff.releaseStr(); @@ -190,6 +194,43 @@ struct UniversalPageIdFormat return page_id_without_keyspace.starts_with(getSubPrefix(type)); } + static inline StorageType getUniversalPageIdType(const UniversalPageId & page_id) + { + if (page_id.empty()) + return StorageType::Unknown; + + const auto & page_id_str = page_id.asStr(); + if (page_id_str[0] == RAFT_PREFIX) + { + return StorageType::RaftEngine; + } + else if (page_id_str[0] == KV_PREFIX) + { + return StorageType::KVEngine; + } + else + { + auto page_id_without_keyspace = TiKVKeyspaceID::removeKeyspaceID(std::string_view(page_id_str.data(), page_id_str.size())); + if (page_id_without_keyspace.starts_with(getSubPrefix(StorageType::Log))) + { + return StorageType::Log; + } + if (page_id_without_keyspace.starts_with(getSubPrefix(StorageType::Data))) + { + return StorageType::Data; + } + if (page_id_without_keyspace.starts_with(getSubPrefix(StorageType::Meta))) + { + return StorageType::Meta; + } + if (page_id_without_keyspace.starts_with(getSubPrefix(StorageType::KVStore))) + { + return StorageType::KVStore; + } + } + return StorageType::Unknown; + } + private: static inline void encodeUInt64(const UInt64 x, WriteBuffer & ss) { diff --git a/dbms/src/Storages/Page/V3/Universal/UniversalPageStorage.cpp b/dbms/src/Storages/Page/V3/Universal/UniversalPageStorage.cpp index fc3afe8cfee..5b5bb68cf18 100644 --- a/dbms/src/Storages/Page/V3/Universal/UniversalPageStorage.cpp +++ b/dbms/src/Storages/Page/V3/Universal/UniversalPageStorage.cpp @@ -420,7 +420,7 @@ bool UniversalPageStorage::canSkipCheckpoint() const return snap->sequence == last_checkpoint_sequence; } -PS::V3::CPDataWriteStats UniversalPageStorage::dumpIncrementalCheckpoint(const UniversalPageStorage::DumpCheckpointOptions & options) +PS::V3::CPDataDumpStats UniversalPageStorage::dumpIncrementalCheckpoint(const UniversalPageStorage::DumpCheckpointOptions & options) { std::scoped_lock lock(checkpoint_mu); Stopwatch sw; @@ -469,7 +469,7 @@ PS::V3::CPDataWriteStats UniversalPageStorage::dumpIncrementalCheckpoint(const U file_ids_to_compact = options.compact_getter(); } // get the remote file ids that need to be compacted - auto write_stats = writer->writeEditsAndApplyCheckpointInfo(edit_from_mem, file_ids_to_compact); + const auto checkpoint_dump_stats = writer->writeEditsAndApplyCheckpointInfo(edit_from_mem, file_ids_to_compact); auto data_file_paths = writer->writeSuffix(); writer.reset(); auto dump_data_seconds = sw.elapsedMillisecondsFromLastTime() / 1000.0; @@ -502,30 +502,33 @@ PS::V3::CPDataWriteStats UniversalPageStorage::dumpIncrementalCheckpoint(const U // TODO: Currently, even when has_new_data == false, // something will be written to DataFile (i.e., the file prefix). // This can be avoided, as its content is useless. - if (write_stats.has_new_data) + if (checkpoint_dump_stats.has_new_data) { // Copy back the checkpoint info to the current PageStorage. // New checkpoint infos are attached in `writeEditsAndApplyCheckpointInfo`. page_directory->copyCheckpointInfoFromEdit(edit_from_mem); } + auto copy_checkpoint_info_seconds = sw.elapsedMillisecondsFromLastTime() / 1000.0; last_checkpoint_sequence = snap->sequence; GET_METRIC(tiflash_storage_checkpoint_seconds, type_dump_checkpoint_snapshot).Observe(dump_snapshot_seconds); GET_METRIC(tiflash_storage_checkpoint_seconds, type_dump_checkpoint_data).Observe(dump_data_seconds); GET_METRIC(tiflash_storage_checkpoint_seconds, type_upload_checkpoint).Observe(upload_seconds); - LOG_DEBUG(log, - "Checkpoint result: files={}, dump_snapshot={:.3f}s, dump_data={:.3f}s, upload={:.3f}s, " - "total={:.3f}s, sequence={}, incremental_data_bytes={}, compact_data_bytes={}", - data_file_paths, - dump_snapshot_seconds, - dump_data_seconds, - upload_seconds, - sw.elapsedSeconds(), - sequence, - write_stats.incremental_data_bytes, - write_stats.compact_data_bytes); - return write_stats; + GET_METRIC(tiflash_storage_checkpoint_seconds, type_copy_checkpoint_info).Observe(copy_checkpoint_info_seconds); + LOG_INFO(log, + "Checkpoint result: files={} dump_snapshot={:.3f}s dump_data={:.3f}s upload={:.3f}s copy_checkpoint_info={:.3f}s " + "total={:.3f}s sequence={} {}", + data_file_paths, + dump_snapshot_seconds, + dump_data_seconds, + upload_seconds, + copy_checkpoint_info_seconds, + sw.elapsedSeconds(), + sequence, + checkpoint_dump_stats); + SetMetrics(checkpoint_dump_stats); + return checkpoint_dump_stats; } } // namespace DB diff --git a/dbms/src/Storages/Page/V3/Universal/UniversalPageStorage.h b/dbms/src/Storages/Page/V3/Universal/UniversalPageStorage.h index 75eddfef87e..d720b1ff008 100644 --- a/dbms/src/Storages/Page/V3/Universal/UniversalPageStorage.h +++ b/dbms/src/Storages/Page/V3/Universal/UniversalPageStorage.h @@ -25,6 +25,7 @@ #include #include #include +#include #include #include #include @@ -198,7 +199,7 @@ class UniversalPageStorage final UInt64 max_edit_records_per_part = 100000; }; - PS::V3::CPDataWriteStats dumpIncrementalCheckpoint(const DumpCheckpointOptions & options); + PS::V3::CPDataDumpStats dumpIncrementalCheckpoint(const DumpCheckpointOptions & options); PS::V3::CPDataFilesStatCache::CacheMap getRemoteDataFilesStatCache() const { diff --git a/dbms/src/Storages/Page/workload/PSStressEnv.cpp b/dbms/src/Storages/Page/workload/PSStressEnv.cpp index 49a651e7388..236bf837b33 100644 --- a/dbms/src/Storages/Page/workload/PSStressEnv.cpp +++ b/dbms/src/Storages/Page/workload/PSStressEnv.cpp @@ -35,7 +35,7 @@ Poco::Logger * StressEnv::logger; void StressEnv::initGlobalLogger() { Poco::AutoPtr channel = new Poco::ConsoleChannel(std::cerr); - Poco::AutoPtr formatter(new DB::UnifiedLogFormatter); + Poco::AutoPtr formatter(new DB::UnifiedLogFormatter()); Poco::AutoPtr formatting_channel(new Poco::FormattingChannel(formatter, channel)); Poco::Logger::root().setChannel(formatting_channel); Poco::Logger::root().setLevel("trace"); diff --git a/dbms/src/Storages/S3/PocoHTTPClient.cpp b/dbms/src/Storages/S3/PocoHTTPClient.cpp index f1d678d34fc..dae0248a715 100644 --- a/dbms/src/Storages/S3/PocoHTTPClient.cpp +++ b/dbms/src/Storages/S3/PocoHTTPClient.cpp @@ -31,11 +31,11 @@ #include #include #include +#include #include #include #include -#include #include #include diff --git a/dbms/src/Storages/S3/S3Common.cpp b/dbms/src/Storages/S3/S3Common.cpp index 2ecf07fe6bf..34d73ddbf34 100644 --- a/dbms/src/Storages/S3/S3Common.cpp +++ b/dbms/src/Storages/S3/S3Common.cpp @@ -662,7 +662,7 @@ void rewriteObjectWithTagging(const TiFlashS3Client & client, const String & key LOG_DEBUG(client.log, "rewrite object key={} cost={:.2f}s", key, elapsed_seconds); } -void ensureLifecycleRuleExist(const TiFlashS3Client & client, Int32 expire_days) +bool ensureLifecycleRuleExist(const TiFlashS3Client & client, Int32 expire_days) { bool lifecycle_rule_has_been_set = false; Aws::Vector old_rules; @@ -679,7 +679,13 @@ void ensureLifecycleRuleExist(const TiFlashS3Client & client, Int32 expire_days) { break; } - throw fromS3Error(outcome.GetError(), "GetBucketLifecycle fail"); + LOG_WARNING( + client.log, + "GetBucketLifecycle fail, please check the bucket lifecycle configuration or create the lifecycle rule manually" + ", bucket={} {}", + client.bucket(), + S3ErrorMessage(error)); + return false; } auto res = outcome.GetResultWithOwnership(); @@ -712,17 +718,9 @@ void ensureLifecycleRuleExist(const TiFlashS3Client & client, Int32 expire_days) if (lifecycle_rule_has_been_set) { LOG_INFO(client.log, "The lifecycle rule has been set, n_rules={} filter={}", old_rules.size(), TaggingObjectIsDeleted); - return; - } - else - { - UNUSED(expire_days); - LOG_WARNING(client.log, "The lifecycle rule with filter \"{}\" has not been set, please check the bucket lifecycle configuration", TaggingObjectIsDeleted); - return; + return true; } -#if 0 - // Adding rule by AWS SDK is failed, don't know why // Reference: https://docs.aws.amazon.com/AmazonS3/latest/userguide/S3OutpostsLifecycleCLIJava.html LOG_INFO(client.log, "The lifecycle rule with filter \"{}\" has not been added, n_rules={}", TaggingObjectIsDeleted, old_rules.size()); static_assert(TaggingObjectIsDeleted == "tiflash_deleted=true"); @@ -746,16 +744,24 @@ void ensureLifecycleRuleExist(const TiFlashS3Client & client, Int32 expire_days) .WithRules(old_rules); Aws::S3::Model::PutBucketLifecycleConfigurationRequest request; - request.WithBucket(bucket) + request.WithBucket(client.bucket()) .WithLifecycleConfiguration(lifecycle_config); auto outcome = client.PutBucketLifecycleConfiguration(request); if (!outcome.IsSuccess()) { - throw fromS3Error(outcome.GetError(), "PutBucketLifecycle fail"); + const auto & error = outcome.GetError(); + LOG_WARNING( + client.log, + "Create lifecycle rule with filter \"{}\" failed, please check the bucket lifecycle configuration or create the lifecycle rule manually" + ", bucket={} {}", + TaggingObjectIsDeleted, + client.bucket(), + S3ErrorMessage(error)); + return false; } LOG_INFO(client.log, "The lifecycle rule has been added, new_n_rules={} tag={}", old_rules.size(), TaggingObjectIsDeleted); -#endif + return true; } void listPrefix( diff --git a/dbms/src/Storages/S3/S3Common.h b/dbms/src/Storages/S3/S3Common.h index f4f1bb1d3bf..9dc4ec63ce1 100644 --- a/dbms/src/Storages/S3/S3Common.h +++ b/dbms/src/Storages/S3/S3Common.h @@ -159,7 +159,7 @@ bool objectExists(const TiFlashS3Client & client, const String & key); void uploadFile(const TiFlashS3Client & client, const String & local_fname, const String & remote_fname, int max_retry_times = 3); constexpr std::string_view TaggingObjectIsDeleted = "tiflash_deleted=true"; -void ensureLifecycleRuleExist(const TiFlashS3Client & client, Int32 expire_days); +bool ensureLifecycleRuleExist(const TiFlashS3Client & client, Int32 expire_days); /** * tagging is the tag-set for the object. The tag-set must be encoded as URL Query diff --git a/dbms/src/Storages/S3/S3GCManager.cpp b/dbms/src/Storages/S3/S3GCManager.cpp index 1224686f19b..334a453c4ad 100644 --- a/dbms/src/Storages/S3/S3GCManager.cpp +++ b/dbms/src/Storages/S3/S3GCManager.cpp @@ -92,11 +92,25 @@ bool S3GCManager::runOnAllStores() return false; } + GET_METRIC(tiflash_storage_s3_gc_status, type_running).Set(1.0); + SCOPE_EXIT({ + GET_METRIC(tiflash_storage_s3_gc_status, type_running).Set(0.0); + }); + if (config.method == S3GCMethod::Lifecycle && !lifecycle_has_been_set) { auto client = S3::ClientFactory::instance().sharedTiFlashClient(); - ensureLifecycleRuleExist(*client, /*expire_days*/ 1); - lifecycle_has_been_set = true; + lifecycle_has_been_set = ensureLifecycleRuleExist(*client, /*expire_days*/ 1); + if (lifecycle_has_been_set) + { + GET_METRIC(tiflash_storage_s3_gc_status, type_lifecycle_added).Set(1.0); + GET_METRIC(tiflash_storage_s3_gc_status, type_lifecycle_failed).Set(0.0); + } + else + { + GET_METRIC(tiflash_storage_s3_gc_status, type_lifecycle_added).Set(0.0); + GET_METRIC(tiflash_storage_s3_gc_status, type_lifecycle_failed).Set(1.0); + } } Stopwatch watch; @@ -301,6 +315,7 @@ void S3GCManager::cleanOneLock(const String & lock_key, const S3FilenameView & l RUNTIME_CHECK(unlocked_datafilename_view.isDataFile()); const auto unlocked_datafile_key = unlocked_datafilename_view.toFullKey(); const auto unlocked_datafile_delmark_key = unlocked_datafilename_view.getDelMarkKey(); + auto sub_logger = log->getChild(fmt::format("remove_key={}", unlocked_datafile_key)); // delete S3 lock file auto client = S3::ClientFactory::instance().sharedTiFlashClient(); @@ -340,7 +355,7 @@ void S3GCManager::cleanOneLock(const String & lock_key, const S3FilenameView & l } if (ok) { - LOG_INFO(log, "delmark created, key={}", unlocked_datafile_key); + LOG_INFO(sub_logger, "delmark created, key={}", unlocked_datafile_key); switch (config.method) { case S3GCMethod::Lifecycle: @@ -352,10 +367,10 @@ void S3GCManager::cleanOneLock(const String & lock_key, const S3FilenameView & l // However, After the lock key is not seen in the manifest file after // 1 day, we consider it is long enough for no other write node try // access to the data file. - lifecycleMarkDataFileDeleted(unlocked_datafile_key); + lifecycleMarkDataFileDeleted(unlocked_datafile_key, sub_logger); elapsed_lifecycle_mark_delete = watch.elapsedMillisecondsFromLastTime() / 1000.0; LOG_INFO( - log, + sub_logger, "cleanOneLock done, method={} key={} remove_lock={:.3f} get_delmark={:.3f} mark_delete={:.3f} lifecycle_mark_delete={:.3f}", magic_enum::enum_name(config.method), unlocked_datafile_key, @@ -372,7 +387,7 @@ void S3GCManager::cleanOneLock(const String & lock_key, const S3FilenameView & l else { LOG_INFO( - log, + sub_logger, "delmark create failed, method={} key={} reason={} remove_lock={:.3f} get_delmark={:.3f} mark_delete={:.3f} lifecycle_mark_delete={:.3f}", magic_enum::enum_name(config.method), unlocked_datafile_key, @@ -397,9 +412,9 @@ void S3GCManager::cleanOneLock(const String & lock_key, const S3FilenameView & l { const auto elapsed_scan_try_remove_datafile = watch.elapsedMillisecondsFromLastTime() / 1000.0; // delmark exist, check whether we need to physical remove the datafile - removeDataFileIfDelmarkExpired(unlocked_datafile_key, unlocked_datafile_delmark_key, timepoint, delmark_object_info.last_modification_time); + removeDataFileIfDelmarkExpired(unlocked_datafile_key, unlocked_datafile_delmark_key, timepoint, delmark_object_info.last_modification_time, sub_logger); LOG_INFO( - log, + sub_logger, "cleanOneLock done, method={} key={} remove_lock={:.3f} get_delmark={:.3f} mark_delete={:.3f} scan_try_physical_remove={:.3f}", magic_enum::enum_name(config.method), unlocked_datafile_key, @@ -416,7 +431,8 @@ void S3GCManager::removeDataFileIfDelmarkExpired( const String & datafile_key, const String & delmark_key, const Aws::Utils::DateTime & timepoint, - const Aws::Utils::DateTime & delmark_mtime) + const Aws::Utils::DateTime & delmark_mtime, + const LoggerPtr & sub_logger) const { // delmark exist bool expired = false; @@ -428,7 +444,7 @@ void S3GCManager::removeDataFileIfDelmarkExpired( expired = true; } LOG_INFO( - log, + sub_logger, "delmark exist, datafile={} mark_mtime={} now={} diff_sec={:.3f} expired={}", datafile_key, delmark_mtime.ToGmtString(Aws::Utils::DateFormat::ISO_8601), @@ -444,11 +460,11 @@ void S3GCManager::removeDataFileIfDelmarkExpired( // physical delete. // It is safe to ignore if datafile_key not exist and S3 won't report // error when the key is not exist - physicalRemoveDataFile(datafile_key); + physicalRemoveDataFile(datafile_key, sub_logger); auto client = S3::ClientFactory::instance().sharedTiFlashClient(); deleteObject(*client, delmark_key); - LOG_INFO(log, "datafile delmark deleted, key={}", delmark_key); + LOG_INFO(sub_logger, "datafile delmark deleted, key={}", delmark_key); } void S3GCManager::tryCleanExpiredDataFiles(UInt64 gc_store_id, const Aws::Utils::DateTime & timepoint) @@ -474,21 +490,21 @@ void S3GCManager::tryCleanExpiredDataFiles(UInt64 gc_store_id, const Aws::Utils: // Only remove the data file with expired delmark if (!filename_view.isDelMark()) break; - auto datafile_key = filename_view.asDataFile().toFullKey(); - removeDataFileIfDelmarkExpired(datafile_key, delmark_key, timepoint, object.GetLastModified()); + const auto datafile_key = filename_view.asDataFile().toFullKey(); + auto sub_logger = log->getChild(fmt::format("remove_key={}", datafile_key)); + removeDataFileIfDelmarkExpired(datafile_key, delmark_key, timepoint, object.GetLastModified(), sub_logger); } while (false); return PageResult{.num_keys = 1, .more = true}; }); } -void S3GCManager::lifecycleMarkDataFileDeleted(const String & datafile_key) +void S3GCManager::lifecycleMarkDataFileDeleted(const String & datafile_key, const LoggerPtr & sub_logger) { assert(config.method == S3GCMethod::Lifecycle); auto view = S3FilenameView::fromKey(datafile_key); RUNTIME_CHECK(view.isDataFile(), magic_enum::enum_name(view.type), datafile_key); auto client = S3::ClientFactory::instance().sharedTiFlashClient(); - auto sub_logger = log->getChild(fmt::format("remove_key={}", datafile_key)); if (!view.isDMFile()) { // CheckpointDataFile is a single object, add tagging for it and update its mtime @@ -520,14 +536,13 @@ void S3GCManager::lifecycleMarkDataFileDeleted(const String & datafile_key) } } -void S3GCManager::physicalRemoveDataFile(const String & datafile_key) +void S3GCManager::physicalRemoveDataFile(const String & datafile_key, const LoggerPtr & sub_logger) const { assert(config.method == S3GCMethod::ScanThenDelete); auto view = S3FilenameView::fromKey(datafile_key); RUNTIME_CHECK(view.isDataFile(), magic_enum::enum_name(view.type), datafile_key); auto client = S3::ClientFactory::instance().sharedTiFlashClient(); - auto sub_logger = log->getChild(fmt::format("remove_key={}", datafile_key)); if (!view.isDMFile()) { // CheckpointDataFile is a single object, remove it. diff --git a/dbms/src/Storages/S3/S3GCManager.h b/dbms/src/Storages/S3/S3GCManager.h index 06954a2dc5d..fda4e209311 100644 --- a/dbms/src/Storages/S3/S3GCManager.h +++ b/dbms/src/Storages/S3/S3GCManager.h @@ -117,10 +117,11 @@ class S3GCManager const String & datafile_key, const String & delmark_key, const Aws::Utils::DateTime & timepoint, - const Aws::Utils::DateTime & delmark_mtime); + const Aws::Utils::DateTime & delmark_mtime, + const LoggerPtr & sub_logger) const; - void lifecycleMarkDataFileDeleted(const String & datafile_key); - void physicalRemoveDataFile(const String & datafile_key); + void lifecycleMarkDataFileDeleted(const String & datafile_key, const LoggerPtr & sub_logger); + void physicalRemoveDataFile(const String & datafile_key, const LoggerPtr & sub_logger) const; static std::vector getAllStoreIds(); diff --git a/dbms/src/Storages/S3/tests/gtest_s3gcmanager.cpp b/dbms/src/Storages/S3/tests/gtest_s3gcmanager.cpp index b2fabc3676b..2e782701a77 100644 --- a/dbms/src/Storages/S3/tests/gtest_s3gcmanager.cpp +++ b/dbms/src/Storages/S3/tests/gtest_s3gcmanager.cpp @@ -223,7 +223,7 @@ try // delmark expired auto delmark_mtime = timepoint - std::chrono::milliseconds(3601 * 1000); - gc_mgr->removeDataFileIfDelmarkExpired(df_key, delmark_key, timepoint, delmark_mtime); + gc_mgr->removeDataFileIfDelmarkExpired(df_key, delmark_key, timepoint, delmark_mtime, log); // removed ASSERT_FALSE(S3::objectExists(*mock_s3_client, df_key)); @@ -238,7 +238,7 @@ try // delmark not expired auto delmark_mtime = timepoint - std::chrono::milliseconds(3599 * 1000); - gc_mgr->removeDataFileIfDelmarkExpired(df_key, delmark_key, timepoint, delmark_mtime); + gc_mgr->removeDataFileIfDelmarkExpired(df_key, delmark_key, timepoint, delmark_mtime, log); // not removed ASSERT_TRUE(S3::objectExists(*mock_s3_client, df_key)); @@ -269,7 +269,7 @@ try // delmark expired auto delmark_mtime = timepoint - std::chrono::milliseconds(3601 * 1000); - gc_mgr->removeDataFileIfDelmarkExpired(df2_key, delmark_key, timepoint, delmark_mtime); + gc_mgr->removeDataFileIfDelmarkExpired(df2_key, delmark_key, timepoint, delmark_mtime, log); // removed ASSERT_FALSE(S3::objectExists(*mock_s3_client, df2_key + "/meta")); @@ -290,7 +290,7 @@ try // delmark not expired auto delmark_mtime = timepoint - std::chrono::milliseconds(3599 * 1000); - gc_mgr->removeDataFileIfDelmarkExpired(df2_key, delmark_key, timepoint, delmark_mtime); + gc_mgr->removeDataFileIfDelmarkExpired(df2_key, delmark_key, timepoint, delmark_mtime, log); // not removed ASSERT_TRUE(S3::objectExists(*mock_s3_client, df2_key + "/meta")); diff --git a/dbms/src/Storages/Transaction/ProxyFFI.cpp b/dbms/src/Storages/Transaction/ProxyFFI.cpp index 912646e7eed..a837a6395e1 100644 --- a/dbms/src/Storages/Transaction/ProxyFFI.cpp +++ b/dbms/src/Storages/Transaction/ProxyFFI.cpp @@ -298,7 +298,7 @@ CppStrWithView HandleReadPage(const EngineStoreServerWrap * server, BaseBuffView else { LOG_TRACE(&Poco::Logger::get("ProxyFFI"), fmt::format("FFI read page {} fail", UniversalPageId(page_id.data, page_id.len))); - return CppStrWithView{.inner = GenRawCppPtr(), .view = BaseBuffView{}}; + return CppStrWithView{.inner = GenRawCppPtr(), .view = BaseBuffView{nullptr, 0}}; } } catch (...) @@ -361,7 +361,7 @@ CppStrWithView HandleGetLowerBound(const EngineStoreServerWrap * server, BaseBuf else { LOG_TRACE(&Poco::Logger::get("ProxyFFI"), fmt::format("FFI get lower bound for page {} fail", UniversalPageId(raw_page_id.data, raw_page_id.len))); - return CppStrWithView{.inner = GenRawCppPtr(), .view = BaseBuffView{}}; + return CppStrWithView{.inner = GenRawCppPtr(), .view = BaseBuffView{nullptr, 0}}; } } catch (...) @@ -779,7 +779,7 @@ CppStrWithView GetConfig(EngineStoreServerWrap * server, [[maybe_unused]] uint8_ config_file_path = server->tmt->getContext().getConfigRef().getString("config-file"); std::ifstream stream(config_file_path); if (!stream) - return CppStrWithView{.inner = GenRawCppPtr(), .view = BaseBuffView{}}; + return CppStrWithView{.inner = GenRawCppPtr(), .view = BaseBuffView{nullptr, 0}}; auto * s = RawCppString::New((std::istreambuf_iterator(stream)), std::istreambuf_iterator()); stream.close(); @@ -793,7 +793,7 @@ CppStrWithView GetConfig(EngineStoreServerWrap * server, [[maybe_unused]] uint8_ } catch (...) { - return CppStrWithView{.inner = GenRawCppPtr(), .view = BaseBuffView{}}; + return CppStrWithView{.inner = GenRawCppPtr(), .view = BaseBuffView{nullptr, 0}}; } } diff --git a/dbms/src/Storages/Transaction/ProxyFFIStatusService.cpp b/dbms/src/Storages/Transaction/ProxyFFIStatusService.cpp index 46d77e6f23c..5fa3f1c0bc2 100644 --- a/dbms/src/Storages/Transaction/ProxyFFIStatusService.cpp +++ b/dbms/src/Storages/Transaction/ProxyFFIStatusService.cpp @@ -47,7 +47,7 @@ HttpRequestRes HandleHttpRequestSyncStatus( { LOG_ERROR(log, "invalid SyncStatus request: {}", query); status = HttpRequestStatus::ErrorParam; - return HttpRequestRes{.status = status, .res = CppStrWithView{.inner = GenRawCppPtr(), .view = BaseBuffView{}}}; + return HttpRequestRes{.status = status, .res = CppStrWithView{.inner = GenRawCppPtr(), .view = BaseBuffView{nullptr, 0}}}; } @@ -69,7 +69,7 @@ HttpRequestRes HandleHttpRequestSyncStatus( } if (status != HttpRequestStatus::Ok) - return HttpRequestRes{.status = status, .res = CppStrWithView{.inner = GenRawCppPtr(), .view = BaseBuffView{}}}; + return HttpRequestRes{.status = status, .res = CppStrWithView{.inner = GenRawCppPtr(), .view = BaseBuffView{nullptr, 0}}}; } std::stringstream ss; @@ -168,7 +168,7 @@ HttpRequestRes HandleHttpRequest(EngineStoreServerWrap * server, BaseBuffView pa return method(server, path, str, std::string_view(query.data, query.len), std::string_view(body.data, body.len)); } } - return HttpRequestRes{.status = HttpRequestStatus::ErrorParam, .res = CppStrWithView{.inner = GenRawCppPtr(), .view = BaseBuffView{}}}; + return HttpRequestRes{.status = HttpRequestStatus::ErrorParam, .res = CppStrWithView{.inner = GenRawCppPtr(), .view = BaseBuffView{nullptr, 0}}}; } } // namespace DB diff --git a/dbms/src/TestUtils/TiFlashTestEnv.cpp b/dbms/src/TestUtils/TiFlashTestEnv.cpp index 5a031f001d1..c6b8f73a963 100644 --- a/dbms/src/TestUtils/TiFlashTestEnv.cpp +++ b/dbms/src/TestUtils/TiFlashTestEnv.cpp @@ -210,7 +210,7 @@ void TiFlashTestEnv::shutdown() void TiFlashTestEnv::setupLogger(const String & level, std::ostream & os) { Poco::AutoPtr channel = new Poco::ConsoleChannel(os); - Poco::AutoPtr formatter(new UnifiedLogFormatter()); + Poco::AutoPtr formatter(new UnifiedLogFormatter()); Poco::AutoPtr formatting_channel(new Poco::FormattingChannel(formatter, channel)); Poco::Logger::root().setChannel(formatting_channel); Poco::Logger::root().setLevel(level); diff --git a/libs/libdaemon/src/BaseDaemon.cpp b/libs/libdaemon/src/BaseDaemon.cpp index bfe7cbc125d..630324c4f70 100644 --- a/libs/libdaemon/src/BaseDaemon.cpp +++ b/libs/libdaemon/src/BaseDaemon.cpp @@ -695,6 +695,14 @@ static std::string normalize(const std::string & log_level) void BaseDaemon::reloadConfiguration() { + // when config-file is not specified and config.toml does not exist, we do not load config. + if (!config().has("config-file")) + { + Poco::File f("config.toml"); + if (!f.exists()) + return; + } + /** If the program is not run in daemon mode and 'config-file' is not specified, * then we use config from 'config.toml' file in current directory, * but will log to console (or use parameters --log-file, --errorlog-file from command line) @@ -749,12 +757,12 @@ void BaseDaemon::wakeup() void BaseDaemon::buildLoggers(Poco::Util::AbstractConfiguration & config) { - auto current_logger = config.getString("logger"); + auto current_logger = config.getString("logger", "(null)"); if (config_logger == current_logger) return; config_logger = current_logger; - bool is_daemon = config.getBool("application.runAsDaemon", true); + bool is_daemon = config.getBool("application.runAsDaemon", false); // Split log, error log and tracing log. Poco::AutoPtr split = new Poco::ReloadableSplitterChannel; @@ -767,7 +775,7 @@ void BaseDaemon::buildLoggers(Poco::Util::AbstractConfiguration & config) std::cerr << "Logging " << log_level << " to " << log_path << std::endl; // Set up two channel chains. - Poco::AutoPtr pf = new DB::UnifiedLogFormatter(); + Poco::AutoPtr pf = new DB::UnifiedLogFormatter(); Poco::AutoPtr log = new FormattingChannel(pf); log_file = new Poco::TiFlashLogFileChannel; log_file->setProperty(Poco::FileChannel::PROP_PATH, Poco::Path(log_path).absolute().toString()); @@ -790,7 +798,7 @@ void BaseDaemon::buildLoggers(Poco::Util::AbstractConfiguration & config) std::cerr << "Logging errors to " << errorlog_path << std::endl; Poco::AutoPtr level = new Poco::LevelFilterChannel; level->setLevel(Message::PRIO_NOTICE); - Poco::AutoPtr pf = new DB::UnifiedLogFormatter(); + Poco::AutoPtr pf = new DB::UnifiedLogFormatter(); Poco::AutoPtr errorlog = new FormattingChannel(pf); error_log_file = new Poco::TiFlashLogFileChannel; error_log_file->setProperty(Poco::FileChannel::PROP_PATH, Poco::Path(errorlog_path).absolute().toString()); @@ -815,7 +823,7 @@ void BaseDaemon::buildLoggers(Poco::Util::AbstractConfiguration & config) /// to filter the tracing log. Poco::AutoPtr source = new Poco::SourceFilterChannel; source->setSource(DB::tracing_log_source); - Poco::AutoPtr pf = new DB::UnifiedLogFormatter(); + Poco::AutoPtr pf = new DB::UnifiedLogFormatter(); Poco::AutoPtr tracing_log = new FormattingChannel(pf); tracing_log_file = new Poco::TiFlashLogFileChannel; tracing_log_file->setProperty(Poco::FileChannel::PROP_PATH, Poco::Path(tracing_log_path).absolute().toString()); @@ -846,14 +854,20 @@ void BaseDaemon::buildLoggers(Poco::Util::AbstractConfiguration & config) syslog_channel->open(); } - if (config.getBool("logger.console", false) || (!config.hasProperty("logger.console") && !is_daemon && (isatty(STDIN_FILENO) || isatty(STDERR_FILENO)))) + bool should_log_to_console = isatty(STDIN_FILENO) || isatty(STDERR_FILENO); + bool enable_colors = isatty(STDERR_FILENO); + + if (config.getBool("logger.console", false) + || (!config.hasProperty("logger.console") && !is_daemon && should_log_to_console)) { Poco::AutoPtr file = new ConsoleChannel; - Poco::AutoPtr pf = new OwnPatternFormatter(this); - pf->setProperty("times", "local"); + Poco::AutoPtr pf; + if (enable_colors) + pf = new DB::UnifiedLogFormatter(); + else + pf = new DB::UnifiedLogFormatter(); Poco::AutoPtr log = new FormattingChannel(pf); log->setChannel(file); - logger().warning("Logging " + log_level + " to console"); split->addChannel(log); } @@ -1002,8 +1016,6 @@ void BaseDaemon::initialize(Application & self) umask(umask_num); } - ConfigProcessor(config_path).savePreprocessedConfig(loaded_config); - /// Write core dump on crash. { struct rlimit rlim diff --git a/metrics/grafana/tiflash_summary.json b/metrics/grafana/tiflash_summary.json index 43711fbda64..f78541a55ae 100644 --- a/metrics/grafana/tiflash_summary.json +++ b/metrics/grafana/tiflash_summary.json @@ -52,7 +52,7 @@ "gnetId": null, "graphTooltip": 1, "id": null, - "iteration": 1680849886690, + "iteration": 1681106846245, "links": [], "panels": [ { @@ -969,7 +969,7 @@ "h": 7, "w": 12, "x": 0, - "y": 25 + "y": 2 }, "hiddenSeries": false, "id": 141, @@ -1081,7 +1081,7 @@ "h": 7, "w": 12, "x": 12, - "y": 25 + "y": 2 }, "hiddenSeries": false, "id": 154, @@ -1211,7 +1211,7 @@ "h": 7, "w": 12, "x": 0, - "y": 32 + "y": 9 }, "hiddenSeries": false, "id": 145, @@ -1341,7 +1341,7 @@ "h": 7, "w": 12, "x": 12, - "y": 32 + "y": 9 }, "hiddenSeries": false, "id": 147, @@ -1471,7 +1471,7 @@ "h": 7, "w": 12, "x": 0, - "y": 39 + "y": 16 }, "hiddenSeries": false, "id": 155, @@ -1601,7 +1601,7 @@ "h": 7, "w": 12, "x": 12, - "y": 39 + "y": 16 }, "hiddenSeries": false, "id": 153, @@ -1731,7 +1731,7 @@ "h": 7, "w": 12, "x": 0, - "y": 46 + "y": 23 }, "hiddenSeries": false, "id": 151, @@ -1861,7 +1861,7 @@ "h": 7, "w": 12, "x": 12, - "y": 46 + "y": 23 }, "hiddenSeries": false, "id": 156, @@ -1991,7 +1991,7 @@ "h": 7, "w": 12, "x": 0, - "y": 53 + "y": 30 }, "hiddenSeries": false, "id": 149, @@ -2121,7 +2121,7 @@ "h": 7, "w": 12, "x": 12, - "y": 53 + "y": 30 }, "hiddenSeries": false, "id": 159, @@ -2251,7 +2251,7 @@ "h": 7, "w": 12, "x": 0, - "y": 60 + "y": 37 }, "hiddenSeries": false, "id": 161, @@ -7704,7 +7704,7 @@ "h": 8, "w": 24, "x": 0, - "y": 7 + "y": 86 }, "hiddenSeries": false, "id": 62, @@ -7823,7 +7823,7 @@ "h": 8, "w": 12, "x": 0, - "y": 15 + "y": 94 }, "height": "", "hiddenSeries": false, @@ -7942,7 +7942,7 @@ "h": 8, "w": 12, "x": 12, - "y": 15 + "y": 94 }, "height": "", "hiddenSeries": false, @@ -8059,7 +8059,7 @@ "h": 9, "w": 24, "x": 0, - "y": 23 + "y": 102 }, "height": "", "hiddenSeries": false, @@ -8181,7 +8181,7 @@ "h": 9, "w": 24, "x": 0, - "y": 32 + "y": 111 }, "hiddenSeries": false, "id": 90, @@ -9215,7 +9215,8 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "PageStorage Checkpoint Duration", + "decimals": 1, + "description": "", "fieldConfig": { "defaults": {}, "overrides": [] @@ -9228,17 +9229,21 @@ "x": 0, "y": 40 }, + "height": "", "hiddenSeries": false, - "id": 177, + "id": 198, "legend": { "alignAsTable": true, "avg": false, "current": true, + "hideEmpty": false, + "hideZero": true, "max": true, "min": false, "rightSide": true, "show": true, - "sort": "max", + "sideWidth": null, + "sort": "current", "sortDesc": true, "total": false, "values": true @@ -9246,7 +9251,7 @@ "lines": true, "linewidth": 1, "links": [], - "nullPointMode": "null as zero", + "nullPointMode": "null", "options": { "alertThreshold": true }, @@ -9255,6 +9260,7 @@ "pointradius": 5, "points": false, "renderer": "flot", + "repeatedByRow": true, "seriesOverrides": [], "spaceLength": 10, "stack": false, @@ -9262,44 +9268,21 @@ "targets": [ { "exemplar": true, - "expr": "histogram_quantile(1.00, sum(rate(tiflash_storage_checkpoint_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "expr": "sum(tiflash_storage_page_data_by_types{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (type)", "format": "time_series", "hide": false, "interval": "", - "intervalFactor": 1, - "legendFormat": "{{type}}-max", - "refId": "A" - }, - { - "exemplar": true, - "expr": "histogram_quantile(0.99, sum(rate(tiflash_storage_checkpoint_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", - "hide": false, - "interval": "", - "legendFormat": "{{type}}-99", - "refId": "C" - }, - { - "exemplar": true, - "expr": "histogram_quantile(0.90, sum(rate(tiflash_storage_checkpoint_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", - "hide": false, - "interval": "", - "legendFormat": "{{type}}-90", - "refId": "D" - }, - { - "exemplar": true, - "expr": "histogram_quantile(0.80, sum(rate(tiflash_storage_checkpoint_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", - "hide": true, - "interval": "", - "legendFormat": "{{type}}-80", - "refId": "E" + "intervalFactor": 2, + "legendFormat": "{{type}}", + "refId": "A", + "step": 10 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "PageStorage Checkpoint Duration", + "title": "PageStorage stored bytes by type", "tooltip": { "shared": true, "sort": 2, @@ -9315,7 +9298,7 @@ }, "yaxes": [ { - "format": "s", + "format": "bytes", "label": null, "logBase": 1, "max": null, @@ -9327,7 +9310,7 @@ "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true } ], @@ -11179,7 +11162,7 @@ "fillGradient": 0, "gridPos": { "h": 8, - "w": 12, + "w": 24, "x": 0, "y": 11 }, @@ -11231,7 +11214,7 @@ "title": "Read Duration Breakdown", "tooltip": { "shared": true, - "sort": 0, + "sort": 2, "value_type": "individual" }, "type": "graph", @@ -11271,8 +11254,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": 1, - "description": "The flow of checkpoint operations", + "description": "PageStorage Checkpoint Duration", "fieldConfig": { "defaults": {}, "overrides": [] @@ -11282,24 +11264,20 @@ "gridPos": { "h": 8, "w": 12, - "x": 12, - "y": 11 + "x": 0, + "y": 19 }, - "height": "", "hiddenSeries": false, - "id": 174, + "id": 187, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": false, - "hideZero": true, "max": true, "min": false, "rightSide": true, "show": true, - "sideWidth": null, - "sort": "current", + "sort": "max", "sortDesc": true, "total": false, "values": true @@ -11307,7 +11285,7 @@ "lines": true, "linewidth": 1, "links": [], - "nullPointMode": "null", + "nullPointMode": "null as zero", "options": { "alertThreshold": true }, @@ -11316,7 +11294,6 @@ "pointradius": 5, "points": false, "renderer": "flot", - "repeatedByRow": true, "seriesOverrides": [], "spaceLength": 10, "stack": false, @@ -11324,30 +11301,44 @@ "targets": [ { "exemplar": true, - "expr": "sum(rate(tiflash_storage_checkpoint_flow{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"incremental\"}[1m]))", + "expr": "histogram_quantile(1.00, sum(rate(tiflash_storage_checkpoint_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", "format": "time_series", "hide": false, "interval": "", - "intervalFactor": 2, - "legendFormat": "incremental", - "refId": "A", - "step": 10 + "intervalFactor": 1, + "legendFormat": "{{type}}", + "refId": "A" }, { "exemplar": true, - "expr": "sum(rate(tiflash_storage_checkpoint_flow{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"compaction\"}[1m]))", - "format": "time_series", + "expr": "histogram_quantile(0.99, sum(rate(tiflash_storage_checkpoint_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "hide": true, "interval": "", - "intervalFactor": 1, - "legendFormat": "compaction", - "refId": "B" + "legendFormat": "{{type}}-99", + "refId": "C" + }, + { + "exemplar": true, + "expr": "histogram_quantile(0.90, sum(rate(tiflash_storage_checkpoint_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "hide": true, + "interval": "", + "legendFormat": "{{type}}-90", + "refId": "D" + }, + { + "exemplar": true, + "expr": "histogram_quantile(0.80, sum(rate(tiflash_storage_checkpoint_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "hide": true, + "interval": "", + "legendFormat": "{{type}}-80", + "refId": "E" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Checkpoint flow", + "title": "Checkpoint Upload Duration", "tooltip": { "shared": true, "sort": 2, @@ -11363,7 +11354,7 @@ }, "yaxes": [ { - "format": "binBps", + "format": "s", "label": null, "logBase": 1, "max": null, @@ -11375,7 +11366,7 @@ "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true } ], @@ -11391,40 +11382,42 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, - "description": "The number of files of owned by each TiFlash node", - "editable": true, - "error": false, + "description": "The flow of checkpoint operations", "fieldConfig": { "defaults": {}, "overrides": [] }, - "fill": 0, + "fill": 1, "fillGradient": 0, - "grid": {}, "gridPos": { "h": 8, "w": 12, - "x": 0, + "x": 12, "y": 19 }, + "height": "", "hiddenSeries": false, - "id": 176, + "id": 174, "legend": { "alignAsTable": true, "avg": false, "current": true, - "max": false, + "hideEmpty": false, + "hideZero": true, + "max": true, "min": false, "rightSide": true, "show": true, "sideWidth": null, + "sort": "current", + "sortDesc": true, "total": false, "values": true }, "lines": true, "linewidth": 1, "links": [], - "nullPointMode": "null as zero", + "nullPointMode": "null", "options": { "alertThreshold": true }, @@ -11433,6 +11426,7 @@ "pointradius": 5, "points": false, "renderer": "flot", + "repeatedByRow": true, "seriesOverrides": [], "spaceLength": 10, "stack": false, @@ -11440,23 +11434,31 @@ "targets": [ { "exemplar": true, - "expr": "sum(tiflash_storage_remote_stats{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"num_files\"}) by (instance)", + "expr": "sum(rate(tiflash_storage_checkpoint_flow{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"incremental\"}[1m]))", "format": "time_series", "hide": false, "interval": "", "intervalFactor": 2, - "legendFormat": "checkpoint_data-{{instance}}", + "legendFormat": "incremental", "refId": "A", "step": 10 + }, + { + "exemplar": true, + "expr": "sum(rate(tiflash_storage_checkpoint_flow{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"compaction\"}[1m]))", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "compaction", + "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Remote File Num", + "title": "Checkpoint Upload flow", "tooltip": { - "msResolution": false, "shared": true, "sort": 2, "value_type": "individual" @@ -11471,7 +11473,7 @@ }, "yaxes": [ { - "format": "short", + "format": "binBps", "label": null, "logBase": 1, "max": null, @@ -11479,10 +11481,10 @@ "show": true }, { - "format": "percentunit", + "format": "short", "label": null, "logBase": 1, - "max": "1.1", + "max": null, "min": "0", "show": true } @@ -11499,40 +11501,42 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, - "description": "The remote store usage owned by each TiFlash node", - "editable": true, - "error": false, + "description": "The keys of checkpoint operations. All keys are uploaded in the checkpoint. Grouped by key types.", "fieldConfig": { "defaults": {}, "overrides": [] }, - "fill": 0, + "fill": 1, "fillGradient": 0, - "grid": {}, "gridPos": { "h": 8, "w": 12, - "x": 12, - "y": 19 + "x": 0, + "y": 27 }, + "height": "", "hiddenSeries": false, - "id": 175, + "id": 196, "legend": { "alignAsTable": true, "avg": false, "current": true, - "max": false, + "hideEmpty": false, + "hideZero": true, + "max": true, "min": false, "rightSide": true, "show": true, "sideWidth": null, + "sort": "current", + "sortDesc": true, "total": false, "values": true }, "lines": true, "linewidth": 1, "links": [], - "nullPointMode": "null as zero", + "nullPointMode": "null", "options": { "alertThreshold": true }, @@ -11541,54 +11545,30 @@ "pointradius": 5, "points": false, "renderer": "flot", - "seriesOverrides": [ - { - "alias": "/^valid_rate/", - "yaxis": 2 - }, - { - "alias": "/size/", - "linewidth": 3 - } - ], + "repeatedByRow": true, + "seriesOverrides": [], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { "exemplar": true, - "expr": "sum(tiflash_storage_remote_stats{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"total_size\"}) by (instance)", - "hide": false, - "interval": "", - "intervalFactor": 2, - "legendFormat": "remote_size-{{instance}}", - "refId": "A" - }, - { - "exemplar": true, - "expr": "sum(tiflash_storage_remote_stats{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"valid_size\"}) by (instance)", + "expr": "sum(rate(tiflash_storage_checkpoint_keys_by_types{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", + "format": "time_series", "hide": false, "interval": "", "intervalFactor": 2, - "legendFormat": "valid_size-{{instance}}", - "refId": "B" - }, - { - "exemplar": true, - "expr": "sum((tiflash_storage_remote_stats{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"valid_size\"}) / (tiflash_storage_remote_stats{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"total_size\"})) by (instance)", - "hide": true, - "interval": "", - "legendFormat": "valid_rate-{{instance}}", - "refId": "C" + "legendFormat": "{{type}}", + "refId": "A", + "step": 10 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Remote Store Usage", + "title": "Checkpoint Upload keys by type (all)", "tooltip": { - "msResolution": false, "shared": true, "sort": 2, "value_type": "individual" @@ -11603,7 +11583,7 @@ }, "yaxes": [ { - "format": "bytes", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -11611,10 +11591,10 @@ "show": true }, { - "format": "percentunit", + "format": "short", "label": null, "logBase": 1, - "max": "1.1", + "max": null, "min": "0", "show": true } @@ -11630,36 +11610,43 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "S3 read/write throughput", + "decimals": 1, + "description": "The flow of checkpoint operations. Group by key types", "fieldConfig": { "defaults": {}, "overrides": [] }, - "fill": 0, + "fill": 1, "fillGradient": 0, "gridPos": { "h": 8, "w": 12, - "x": 0, + "x": 12, "y": 27 }, + "height": "", "hiddenSeries": false, - "id": 178, + "id": 197, "legend": { - "alignAsTable": false, + "alignAsTable": true, "avg": false, - "current": false, - "max": false, + "current": true, + "hideEmpty": false, + "hideZero": true, + "max": true, "min": false, - "rightSide": false, + "rightSide": true, "show": true, + "sideWidth": null, + "sort": "current", + "sortDesc": true, "total": false, - "values": false + "values": true }, "lines": true, "linewidth": 1, "links": [], - "nullPointMode": "null as zero", + "nullPointMode": "null", "options": { "alertThreshold": true }, @@ -11668,6 +11655,7 @@ "pointradius": 5, "points": false, "renderer": "flot", + "repeatedByRow": true, "seriesOverrides": [], "spaceLength": 10, "stack": false, @@ -11675,31 +11663,24 @@ "targets": [ { "exemplar": true, - "expr": "sum(rate(tiflash_system_profile_event_S3WriteBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", + "expr": "sum(rate(tiflash_storage_checkpoint_flow_by_types{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", "format": "time_series", "hide": false, "interval": "", - "intervalFactor": 1, - "legendFormat": "S3WriteBytes", - "refId": "B" - }, - { - "exemplar": true, - "expr": "sum(rate(tiflash_system_profile_event_S3ReadBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", - "hide": false, - "interval": "", - "legendFormat": "S3ReadBytes", - "refId": "A" + "intervalFactor": 2, + "legendFormat": "{{type}}", + "refId": "A", + "step": 10 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "S3 Bytes", + "title": "Checkpoint Upload flow by type (incremental+compaction)", "tooltip": { "shared": true, - "sort": 0, + "sort": 2, "value_type": "individual" }, "type": "graph", @@ -11712,8 +11693,7 @@ }, "yaxes": [ { - "decimals": null, - "format": "bytes", + "format": "binBps", "label": null, "logBase": 1, "max": null, @@ -11721,7 +11701,7 @@ "show": true }, { - "format": "opm", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -11740,31 +11720,36 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "S3 OPS", + "decimals": 1, + "description": "The number of files of owned by each TiFlash node", + "editable": true, + "error": false, "fieldConfig": { "defaults": {}, "overrides": [] }, "fill": 0, "fillGradient": 0, + "grid": {}, "gridPos": { "h": 8, "w": 12, - "x": 12, - "y": 27 + "x": 0, + "y": 35 }, "hiddenSeries": false, - "id": 179, + "id": 176, "legend": { - "alignAsTable": false, + "alignAsTable": true, "avg": false, - "current": false, + "current": true, "max": false, "min": false, - "rightSide": false, + "rightSide": true, "show": true, + "sideWidth": null, "total": false, - "values": false + "values": true }, "lines": true, "linewidth": 1, @@ -11785,87 +11770,591 @@ "targets": [ { "exemplar": true, - "expr": "sum(rate(tiflash_system_profile_event_S3PutObject{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", + "expr": "sum(tiflash_storage_remote_stats{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"num_files\"}) by (instance)", "format": "time_series", "hide": false, "interval": "", - "intervalFactor": 1, - "legendFormat": "S3PutObject", - "refId": "B" - }, + "intervalFactor": 2, + "legendFormat": "checkpoint_data-{{instance}}", + "refId": "A", + "step": 10 + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Remote File Num", + "tooltip": { + "msResolution": false, + "shared": true, + "sort": 2, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ { - "exemplar": true, - "expr": "sum(rate(tiflash_system_profile_event_S3GetObject{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", - "hide": false, - "interval": "", - "legendFormat": "S3GetObject", - "refId": "A" + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true }, { - "exemplar": true, - "expr": "sum(rate(tiflash_system_profile_event_S3HeadObject{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", - "hide": false, - "interval": "", - "legendFormat": "S3HeadObject", - "refId": "C" - }, + "format": "percentunit", + "label": null, + "logBase": 1, + "max": "1.1", + "min": "0", + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "decimals": 1, + "description": "The remote store usage owned by each TiFlash node", + "editable": true, + "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 0, + "fillGradient": 0, + "grid": {}, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 35 + }, + "hiddenSeries": false, + "id": 175, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "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": [ { - "exemplar": true, - "expr": "sum(rate(tiflash_system_profile_event_S3ListObjects{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", - "hide": false, - "interval": "", - "legendFormat": "S3ListObjects", - "refId": "D" + "alias": "/^valid_rate/", + "yaxis": 2 }, { - "exemplar": true, - "expr": "sum(rate(tiflash_system_profile_event_S3DeleteObject{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", - "hide": false, - "interval": "", - "legendFormat": "S3DeleteObject", - "refId": "E" - }, + "alias": "/size/", + "linewidth": 3 + } + ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ { "exemplar": true, - "expr": "sum(rate(tiflash_system_profile_event_S3CopyObject{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", + "expr": "sum(tiflash_storage_remote_stats{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"total_size\"}) by (instance)", "hide": false, "interval": "", - "legendFormat": "S3CopyObject", - "refId": "F" + "intervalFactor": 2, + "legendFormat": "remote_size-{{instance}}", + "refId": "A" }, { "exemplar": true, - "expr": "sum(rate(tiflash_system_profile_event_S3CreateMultipartUpload{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", + "expr": "sum(tiflash_storage_remote_stats{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"valid_size\"}) by (instance)", "hide": false, "interval": "", - "legendFormat": "S3CreateMultipartUpload", - "refId": "G" + "intervalFactor": 2, + "legendFormat": "valid_size-{{instance}}", + "refId": "B" }, { "exemplar": true, - "expr": "sum(rate(tiflash_system_profile_event_S3UploadPart{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", - "hide": false, + "expr": "sum((tiflash_storage_remote_stats{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"valid_size\"}) / (tiflash_storage_remote_stats{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"total_size\"})) by (instance)", + "hide": true, "interval": "", - "legendFormat": "S3UploadPart", - "refId": "H" + "legendFormat": "valid_rate-{{instance}}", + "refId": "C" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Remote Store Usage", + "tooltip": { + "msResolution": false, + "shared": true, + "sort": 2, + "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": "percentunit", + "label": null, + "logBase": 1, + "max": "1.1", + "min": "0", + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 0, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 43 + }, + "hiddenSeries": false, + "id": 189, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "hideZero": false, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "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_disaggregated_object_lock_request_count{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": "Remote Object Lock Request QPS", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "decimals": null, + "format": "none", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "none", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 0, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 43 + }, + "hiddenSeries": false, + "id": 191, + "legend": { + "alignAsTable": true, + "avg": false, + "current": false, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "sort": "max", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.11", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "histogram_quantile(0.99, sum(rate(tiflash_disaggregated_object_lock_request_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "interval": "", + "legendFormat": "99%-{{type}}", + "queryType": "randomWalk", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Remote Object Lock Duration", + "tooltip": { + "shared": true, + "sort": 2, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "s", + "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, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 0, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 51 + }, + "hiddenSeries": false, + "id": 193, + "legend": { + "alignAsTable": true, + "avg": false, + "current": false, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "sort": "max", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.11", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [ + { + "alias": "/total/", + "yaxis": 2 + }, + { + "alias": "/one_store/", + "yaxis": 2 + }, + { + "alias": "/clean_locks/", + "yaxis": 2 + } + ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "histogram_quantile(0.99, sum(rate(tiflash_storage_s3_gc_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "interval": "", + "legendFormat": "99%-{{type}}", + "queryType": "randomWalk", + "refId": "A" + }, + { + "exemplar": true, + "expr": "histogram_quantile(0.90, sum(rate(tiflash_storage_s3_gc_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "hide": true, + "interval": "", + "legendFormat": "90%-{{type}}", + "refId": "B" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Remote GC Duration Breakdown", + "tooltip": { + "shared": true, + "sort": 2, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "decimals": 1, + "description": "", + "editable": true, + "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 0, + "fillGradient": 0, + "grid": {}, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 51 + }, + "hiddenSeries": false, + "id": 195, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": false, + "hideZero": false, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "sort": "current", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null as zero", + "options": { + "alertThreshold": false + }, + "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_system_profile_event_S3CompleteMultipartUpload{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", - "hide": false, + "expr": "sum(tiflash_storage_s3_gc_status{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance,type)", + "format": "time_series", "interval": "", - "legendFormat": "S3CompleteMultipartUpload", - "refId": "I" + "intervalFactor": 2, + "legendFormat": "{{instance}}-{{type}}", + "refId": "A", + "step": 10 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "S3 OPS", + "title": "Remote GC Status", "tooltip": { + "msResolution": false, "shared": true, - "sort": 0, + "sort": 2, "value_type": "individual" }, "type": "graph", @@ -11878,8 +12367,7 @@ }, "yaxes": [ { - "decimals": null, - "format": "ops", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -11887,11 +12375,11 @@ "show": true }, { - "format": "opm", + "format": "short", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true } ], @@ -11899,40 +12387,53 @@ "align": false, "alignLevel": null } - }, + } + ], + "title": "Disaggregated", + "type": "row" + }, + { + "collapsed": true, + "datasource": null, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 11 + }, + "id": 172, + "panels": [ { "aliasColors": {}, "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "S3 Request Duration", + "description": "S3 read/write throughput", "fieldConfig": { "defaults": {}, "overrides": [] }, - "fill": 1, + "fill": 0, "fillGradient": 0, "gridPos": { "h": 8, "w": 12, "x": 0, - "y": 35 + "y": 52 }, "hiddenSeries": false, - "id": 180, + "id": 178, "legend": { - "alignAsTable": true, + "alignAsTable": false, "avg": false, - "current": true, - "max": true, + "current": false, + "max": false, "min": false, - "rightSide": true, + "rightSide": false, "show": true, - "sort": "max", - "sortDesc": true, "total": false, - "values": true + "values": false }, "lines": true, "linewidth": 1, @@ -11953,47 +12454,31 @@ "targets": [ { "exemplar": true, - "expr": "histogram_quantile(1.0, sum(rate(tiflash_storage_s3_request_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "expr": "sum(rate(tiflash_system_profile_event_S3WriteBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", "format": "time_series", "hide": false, "interval": "", "intervalFactor": 1, - "legendFormat": "{{type}}-max", - "refId": "A" - }, - { - "exemplar": true, - "expr": "histogram_quantile(0.99, sum(rate(tiflash_storage_s3_request_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", - "hide": false, - "interval": "", - "legendFormat": "{{type}}-99", - "refId": "C" + "legendFormat": "S3WriteBytes", + "refId": "B" }, { "exemplar": true, - "expr": "histogram_quantile(0.90, sum(rate(tiflash_storage_s3_request_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "expr": "sum(rate(tiflash_system_profile_event_S3ReadBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", "hide": false, "interval": "", - "legendFormat": "{{type}}-90", - "refId": "D" - }, - { - "exemplar": true, - "expr": "histogram_quantile(0.80, sum(rate(tiflash_storage_s3_request_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", - "hide": true, - "interval": "", - "legendFormat": "{{type}}-80", - "refId": "E" + "legendFormat": "S3ReadBytes", + "refId": "A" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "S3 Request Duration", + "title": "S3 Bytes", "tooltip": { "shared": true, - "sort": 2, + "sort": 0, "value_type": "individual" }, "type": "graph", @@ -12006,7 +12491,8 @@ }, "yaxes": [ { - "format": "s", + "decimals": null, + "format": "bytes", "label": null, "logBase": 1, "max": null, @@ -12014,11 +12500,11 @@ "show": true }, { - "format": "short", + "format": "opm", "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true } ], @@ -12026,22 +12512,7 @@ "align": false, "alignLevel": null } - } - ], - "title": "Disaggregated", - "type": "row" - }, - { - "collapsed": true, - "datasource": null, - "gridPos": { - "h": 1, - "w": 24, - "x": 0, - "y": 10 - }, - "id": 172, - "panels": [ + }, { "aliasColors": {}, "bars": false, @@ -12059,7 +12530,7 @@ "h": 8, "w": 12, "x": 12, - "y": 72 + "y": 52 }, "hiddenSeries": false, "id": 179, @@ -12214,7 +12685,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "S3 read/write throughput", + "description": "S3 Retry OPS", "fieldConfig": { "defaults": {}, "overrides": [] @@ -12225,10 +12696,10 @@ "h": 8, "w": 12, "x": 0, - "y": 80 + "y": 60 }, "hiddenSeries": false, - "id": 178, + "id": 182, "legend": { "alignAsTable": false, "avg": false, @@ -12259,20 +12730,20 @@ "targets": [ { "exemplar": true, - "expr": "sum(rate(tiflash_system_profile_event_S3WriteBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", + "expr": "sum(rate(tiflash_system_profile_event_S3GetObjectRetry{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", "format": "time_series", "hide": false, "interval": "", "intervalFactor": 1, - "legendFormat": "S3WriteBytes", + "legendFormat": "S3GetObjectRetry", "refId": "B" }, { "exemplar": true, - "expr": "sum(rate(tiflash_system_profile_event_S3ReadBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", + "expr": "sum(rate(tiflash_system_profile_event_S3PutObjectRetry{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", "hide": false, "interval": "", - "legendFormat": "S3ReadBytes", + "legendFormat": "S3PutObjectRetry", "refId": "A" } ], @@ -12280,7 +12751,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "S3 Bytes", + "title": "S3 Retry OPS", "tooltip": { "shared": true, "sort": 0, @@ -12296,9 +12767,8 @@ }, "yaxes": [ { - "$$hashKey": "object:263", "decimals": null, - "format": "bytes", + "format": "ops", "label": null, "logBase": 1, "max": null, @@ -12306,7 +12776,6 @@ "show": true }, { - "$$hashKey": "object:264", "format": "opm", "label": null, "logBase": 1, @@ -12337,7 +12806,7 @@ "h": 8, "w": 12, "x": 12, - "y": 80 + "y": 60 }, "hiddenSeries": false, "id": 180, @@ -12392,7 +12861,7 @@ { "exemplar": true, "expr": "histogram_quantile(0.90, sum(rate(tiflash_storage_s3_request_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", - "hide": false, + "hide": true, "interval": "", "legendFormat": "{{type}}-90", "refId": "D" @@ -12426,7 +12895,6 @@ }, "yaxes": [ { - "$$hashKey": "object:195", "format": "s", "label": null, "logBase": 1, @@ -12435,7 +12903,6 @@ "show": true }, { - "$$hashKey": "object:196", "format": "short", "label": null, "logBase": 1, @@ -12449,118 +12916,6 @@ "alignLevel": null } }, - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "S3 Retry OPS", - "fieldConfig": { - "defaults": {}, - "overrides": [] - }, - "fill": 0, - "fillGradient": 0, - "gridPos": { - "h": 8, - "w": 12, - "x": 0, - "y": 80 - }, - "hiddenSeries": false, - "id": 182, - "legend": { - "alignAsTable": false, - "avg": false, - "current": false, - "max": false, - "min": false, - "rightSide": false, - "show": true, - "total": false, - "values": false - }, - "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_system_profile_event_S3GetObjectRetry{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", - "format": "time_series", - "hide": false, - "interval": "", - "intervalFactor": 1, - "legendFormat": "S3GetObjectRetry", - "refId": "B" - }, - { - "exemplar": true, - "expr": "sum(rate(tiflash_system_profile_event_S3PutObjectRetry{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", - "hide": false, - "interval": "", - "legendFormat": "S3PutObjectRetry", - "refId": "A" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "S3 Retry OPS", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "$$hashKey": "object:263", - "decimals": null, - "format": "ops", - "label": null, - "logBase": 1, - "max": null, - "min": "0", - "show": true - }, - { - "$$hashKey": "object:264", - "format": "opm", - "label": null, - "logBase": 1, - "max": null, - "min": "0", - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } - }, { "aliasColors": {}, "bars": false, @@ -12578,7 +12933,7 @@ "h": 8, "w": 12, "x": 0, - "y": 80 + "y": 68 }, "hiddenSeries": false, "id": 185, @@ -12657,7 +13012,6 @@ }, "yaxes": [ { - "$$hashKey": "object:263", "decimals": null, "format": "ops", "label": null, @@ -12667,7 +13021,6 @@ "show": true }, { - "$$hashKey": "object:264", "format": "opm", "label": null, "logBase": 1, @@ -12808,4 +13161,4 @@ "title": "Test-Cluster-TiFlash-Summary", "uid": "SVbh2xUWk", "version": 1 -} \ No newline at end of file +}