Skip to content

Commit

Permalink
Merge branch 'release-6.5' into cherry-pick-6746-to-release-6.5
Browse files Browse the repository at this point in the history
  • Loading branch information
Ivy-YinSu authored Mar 3, 2023
2 parents a1b517b + 19ff4fd commit afc0556
Show file tree
Hide file tree
Showing 5 changed files with 32 additions and 14 deletions.
2 changes: 1 addition & 1 deletion contrib/tiflash-proxy
Submodule tiflash-proxy updated 66 files
+6 −4 Cargo.lock
+1 −1 Cargo.toml
+1 −1 components/backup-stream/Cargo.toml
+9 −5 components/backup-stream/src/checkpoint_manager.rs
+56 −1 components/backup-stream/src/endpoint.rs
+17 −1 components/backup-stream/src/errors.rs
+5 −0 components/backup-stream/src/metadata/client.rs
+1 −1 components/backup-stream/src/metadata/mod.rs
+340 −16 components/backup-stream/src/metadata/store/etcd.rs
+185 −35 components/backup-stream/src/metadata/store/lazy_etcd.rs
+2 −2 components/backup-stream/src/metadata/test.rs
+29 −3 components/backup-stream/src/observer.rs
+4 −0 components/backup-stream/src/utils.rs
+1 −0 components/backup/src/endpoint.rs
+1 −1 components/cdc/src/endpoint.rs
+2 −0 components/cloud/azure/Cargo.toml
+39 −5 components/cloud/azure/src/azblob.rs
+1 −0 components/error_code/src/storage.rs
+7 −16 components/raft_log_engine/src/engine.rs
+0 −3 components/raftstore-v2/src/worker/pd/update_max_timestamp.rs
+91 −72 components/raftstore/src/store/async_io/write.rs
+121 −3 components/raftstore/src/store/async_io/write_tests.rs
+1 −1 components/raftstore/src/store/config.rs
+6 −1 components/raftstore/src/store/fsm/apply.rs
+22 −9 components/raftstore/src/store/fsm/peer.rs
+8 −6 components/raftstore/src/store/peer.rs
+16 −1 components/raftstore/src/store/util.rs
+106 −2 components/raftstore/src/store/worker/read.rs
+144 −9 components/resolved_ts/src/advance.rs
+17 −6 components/resolved_ts/src/endpoint.rs
+1 −1 components/resolved_ts/src/resolver.rs
+4 −0 components/security/src/lib.rs
+1 −7 components/server/src/server.rs
+14 −3 components/snap_recovery/src/init_cluster.rs
+10 −0 components/sst_importer/src/metrics.rs
+303 −48 components/sst_importer/src/sst_importer.rs
+2 −1 components/tidb_query_expr/src/types/expr_builder.rs
+2 −2 components/tikv_kv/src/lib.rs
+2 −2 components/txn_types/src/types.rs
+10 −0 etc/error_code.toml
+4 −4 metrics/alertmanager/tikv.rules.yml
+4 −4 src/config.rs
+7 −1 src/coprocessor/endpoint.rs
+4 −0 src/coprocessor/mod.rs
+1 −1 src/import/mod.rs
+314 −334 src/import/sst_service.rs
+16 −0 src/server/gc_worker/compaction_filter.rs
+16 −0 src/server/gc_worker/mod.rs
+2 −2 src/server/raftkv/mod.rs
+9 −0 src/storage/errors.rs
+74 −6 src/storage/mod.rs
+146 −4 src/storage/txn/actions/acquire_pessimistic_lock.rs
+147 −72 src/storage/txn/actions/flashback_to_version.rs
+1 −1 src/storage/txn/commands/acquire_pessimistic_lock.rs
+2 −2 src/storage/txn/commands/flashback_to_version.rs
+44 −21 src/storage/txn/commands/flashback_to_version_read_phase.rs
+2 −2 src/storage/txn/commands/mod.rs
+1 −1 src/storage/txn/commands/prewrite.rs
+7 −0 src/storage/txn/mod.rs
+1 −1 src/storage/txn/scheduler.rs
+1 −1 tests/failpoints/cases/test_table_properties.rs
+30 −0 tests/integrations/backup/mod.rs
+35 −25 tests/integrations/coprocessor/test_select.rs
+1 −1 tests/integrations/pd/test_rpc_client.rs
+101 −13 tests/integrations/raftstore/test_flashback.rs
+67 −3 tests/integrations/server/kv_service.rs
6 changes: 4 additions & 2 deletions dbms/src/Flash/LogSearch.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -410,9 +410,11 @@ bool FilterFileByDatetime(
const int64_t start_time)
{
static const std::string date_format_example = "0000-00-00-00:00:00.000";
static const std::string raftstore_proxy_date_format_example = "0000-00-00-00:00:00.000000000";
static const char * date_format = "%d-%d-%d-%d:%d:%d.%d";

static const std::string raftstore_proxy_date_format_example = "0000-00-00T00:00:00.000.log";
static const char * raftstore_proxy_date_format = "%d-%d-%dT%d-%d-%d.%d";

for (const auto & ignore_log_file_prefix : ignore_log_file_prefixes)
{
if (!ignore_log_file_prefix.empty() && startsWith(path, ignore_log_file_prefix))
Expand Down Expand Up @@ -441,7 +443,7 @@ bool FilterFileByDatetime(
else
{
// filter proxy log end datetime
return filterLogEndDatetime(path, raftstore_proxy_date_format_example, date_format, start_time);
return filterLogEndDatetime(path, raftstore_proxy_date_format_example, raftstore_proxy_date_format, start_time);
}
}

Expand Down
2 changes: 1 addition & 1 deletion dbms/src/Flash/tests/gtest_log_search.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -123,7 +123,7 @@ TEST_F(LogSearchTest, SearchDir)
ASSERT_TRUE(FilterFileByDatetime("/1/server.log.2021-10-09-14:50:55.481.gz", {"/1/test-err.log"}, 1633855377000)); // 1633855377000 : 2021-10-10 16:42:57
ASSERT_FALSE(FilterFileByDatetime("/1/server.log.2021-10-10-16:43:57.123.gz", {"/1/test-err.log"}, 1633855377000));

ASSERT_TRUE(FilterFileByDatetime("/1/proxy.log.2021-10-09-14:50:55.123456789", {"/1/test-err.log"}, 1633855377000));
ASSERT_TRUE(FilterFileByDatetime("/1/tiflash_tikv.2021-10-09T14-50-55.123.log", {"/1/test-err.log"}, 1633855377000));

{
const std::string example_data = "[2020/04/23 13:11:02.329 +08:00] [DEBUG] [\"Application : Load metadata done.\"]\n";
Expand Down
32 changes: 24 additions & 8 deletions dbms/src/Server/MetricsPrometheus.cpp
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
// Copyright 2022 PingCAP, Ltd.
// 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.
Expand Down Expand Up @@ -103,7 +103,7 @@ class MetricHandlerFactory : public Poco::Net::HTTPRequestHandlerFactory
std::shared_ptr<Poco::Net::HTTPServer> getHTTPServer(
const TiFlashSecurityConfig & security_config,
const std::weak_ptr<prometheus::Collectable> & collectable,
const String & metrics_port)
const String & address)
{
Poco::Net::Context::Ptr context = new Poco::Net::Context(
Poco::Net::Context::TLSV1_2_SERVER_USE,
Expand All @@ -125,8 +125,7 @@ std::shared_ptr<Poco::Net::HTTPServer> getHTTPServer(
Poco::Net::SecureServerSocket socket(context);

Poco::Net::HTTPServerParams::Ptr http_params = new Poco::Net::HTTPServerParams;

Poco::Net::SocketAddress addr("0.0.0.0", std::stoi(metrics_port));
Poco::Net::SocketAddress addr = Poco::Net::SocketAddress(address);
socket.bind(addr, true);
socket.listen();
auto server = std::make_shared<Poco::Net::HTTPServer>(new MetricHandlerFactory(collectable), socket, http_params);
Expand All @@ -136,6 +135,17 @@ std::shared_ptr<Poco::Net::HTTPServer> getHTTPServer(
constexpr Int64 MILLISECOND = 1000;
constexpr Int64 INIT_DELAY = 5;

namespace
{
inline bool isIPv6(const String & input_address)
{
if (input_address.empty())
return false;
char str[INET6_ADDRSTRLEN];
return inet_pton(AF_INET6, input_address.c_str(), str) == 1;
}
} // namespace

MetricsPrometheus::MetricsPrometheus(
Context & context,
const AsynchronousMetrics & async_metrics_,
Expand Down Expand Up @@ -202,17 +212,23 @@ MetricsPrometheus::MetricsPrometheus(
if (conf.hasOption(status_metrics_port) || !conf.hasOption(status_metrics_addr))
{
auto metrics_port = conf.getString(status_metrics_port, DB::toString(DEFAULT_METRICS_PORT));
auto listen_host = conf.getString("listen_host", "0.0.0.0");
String addr;
if (isIPv6(listen_host))
addr = "[" + listen_host + "]:" + metrics_port;
else
addr = listen_host + ":" + metrics_port;
if (security_config.has_tls_config)
{
server = getHTTPServer(security_config, tiflash_metrics.registry, metrics_port);
server = getHTTPServer(security_config, tiflash_metrics.registry, addr);
server->start();
LOG_INFO(log, "Enable prometheus secure pull mode; Metrics Port = {}", metrics_port);
LOG_INFO(log, "Enable prometheus secure pull mode; Listen Host = {}, Metrics Port = {}", listen_host, metrics_port);
}
else
{
exposer = std::make_shared<prometheus::Exposer>(metrics_port);
exposer = std::make_shared<prometheus::Exposer>(addr);
exposer->RegisterCollectable(tiflash_metrics.registry);
LOG_INFO(log, "Enable prometheus pull mode; Metrics Port = {}", metrics_port);
LOG_INFO(log, "Enable prometheus pull mode; Listen Host = {}, Metrics Port = {}", listen_host, metrics_port);
}
}
else
Expand Down
4 changes: 2 additions & 2 deletions dbms/src/Storages/DeltaMerge/DeltaMergeStore_InternalBg.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -595,7 +595,7 @@ SegmentPtr DeltaMergeStore::gcTrySegmentMerge(const DMContextPtr & dm_context, c
auto new_segment = segmentMerge(*dm_context, segments_to_merge, SegmentMergeReason::BackgroundGCThread);
if (new_segment)
{
checkSegmentUpdate(dm_context, segment, ThreadType::BG_GC);
checkSegmentUpdate(dm_context, new_segment, ThreadType::BG_GC);
}

return new_segment;
Expand Down Expand Up @@ -718,7 +718,7 @@ SegmentPtr DeltaMergeStore::gcTrySegmentMergeDelta(const DMContextPtr & dm_conte
}

segment_snap = {};
checkSegmentUpdate(dm_context, segment, ThreadType::BG_GC);
checkSegmentUpdate(dm_context, new_segment, ThreadType::BG_GC);

return new_segment;
}
Expand Down

0 comments on commit afc0556

Please sign in to comment.