Skip to content

Commit

Permalink
This is an automated cherry-pick of pingcap#6844
Browse files Browse the repository at this point in the history
Signed-off-by: ti-chi-bot <ti-community-prow-bot@tidb.io>
  • Loading branch information
JaySon-Huang authored and ti-chi-bot committed Feb 21, 2023
1 parent b1fb85f commit 33893c1
Show file tree
Hide file tree
Showing 45 changed files with 942 additions and 200 deletions.
6 changes: 6 additions & 0 deletions dbms/src/Common/TiFlashException.h
Original file line number Diff line number Diff line change
Expand Up @@ -251,6 +251,12 @@ class TiFlashException : public Exception
, error(_error)
{}

template <typename... Args>
TiFlashException(const TiFlashError & _error, const std::string & fmt, Args &&... args)
: Exception(FmtBuffer().fmtAppend(fmt, std::forward<Args>(args)...).toString())
, error(_error)
{}

const char * name() const throw() override { return "DB::TiFlashException"; }
const char * className() const throw() override { return "DB::TiFlashException"; }

Expand Down
7 changes: 4 additions & 3 deletions dbms/src/Common/tests/gtest_logger.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@
#include <Common/formatReadable.h>
#include <Common/tests/TestChannel.h>
#include <Poco/FormattingChannel.h>
#include <Poco/Message.h>
#include <TestUtils/TiFlashTestBasic.h>
#include <common/logger_useful.h>

Expand Down Expand Up @@ -56,7 +57,7 @@ TEST_F(LogMacroTest, Poco)

ASSERT_EQ(
channel->getLastMessage().getText().substr(32), // length of timestamp is 32
R"raw( [INFO] [gtest_logger.cpp:55] ["float-number: 3.1416, 3.14159, size: 9.01 GiB"] [source=LoggerTest] [thread_id=1])raw");
R"raw( [INFO] [gtest_logger.cpp:56] ["float-number: 3.1416, 3.14159, size: 9.01 GiB"] [source=LoggerTest] [thread_id=1])raw");
}

TEST_F(LogMacroTest, PropsLogger)
Expand All @@ -66,7 +67,7 @@ TEST_F(LogMacroTest, PropsLogger)

ASSERT_EQ(
channel->getLastMessage().getText().substr(32), // length of timestamp is 32
R"raw( [INFO] [gtest_logger.cpp:65] ["float-number: 3.1416, 3.14159, size: 9.01 GiB"] [source="props=foo"] [thread_id=1])raw");
R"raw( [INFO] [gtest_logger.cpp:66] ["float-number: 3.1416, 3.14159, size: 9.01 GiB"] [source="props=foo"] [thread_id=1])raw");
}

TEST_F(LogMacroTest, PureMessage)
Expand All @@ -76,7 +77,7 @@ TEST_F(LogMacroTest, PureMessage)

ASSERT_EQ(
channel->getLastMessage().getText().substr(32), // length of timestamp is 32
R"raw( [INFO] [gtest_logger.cpp:75] ["some arbitrary message {"] [thread_id=1])raw");
R"raw( [INFO] [gtest_logger.cpp:76] ["some arbitrary message {"] [thread_id=1])raw");
}

TEST(LogIdTest, Basic)
Expand Down
4 changes: 2 additions & 2 deletions dbms/src/Encryption/RateLimiter.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -704,15 +704,15 @@ IOLimitTuner::IOLimitTuner(
, bg_read_stat(std::move(bg_read_stat_))
, fg_read_stat(std::move(fg_read_stat_))
, io_config(io_config_)
, log(Logger::get("IOLimitTuner"))
, log(Logger::get())
{}

IOLimitTuner::TuneResult IOLimitTuner::tune() const
{
auto msg = fmt::format("limiter {} write {} read {}", limiterCount(), writeLimiterCount(), readLimiterCount());
if (limiterCount() < 2)
{
LOG_INFO(log, "{} NOT need to tune.", msg);
LOG_DEBUG(log, "{} NOT need to tune.", msg);
return {0, 0, false, 0, 0, false};
}
LOG_INFO(log, "{} need to tune.", msg);
Expand Down
6 changes: 4 additions & 2 deletions dbms/src/Flash/LogSearch.h
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@

#pragma once

#include <Common/Logger.h>
#include <Poco/File.h>
#include <common/logger_useful.h>
#include <re2/re2.h>
Expand All @@ -37,6 +38,7 @@

namespace DB
{

class LogIterator : private boost::noncopyable
{
public:
Expand All @@ -51,7 +53,7 @@ class LogIterator : private boost::noncopyable
, levels(_levels)
, patterns(_patterns)
, log_input_stream(_log_input_stream)
, log(&Poco::Logger::get("LogIterator"))
, log(Logger::get())
, cur_lineno(0)
{
init();
Expand Down Expand Up @@ -128,7 +130,7 @@ class LogIterator : private boost::noncopyable
std::istream & log_input_stream;
std::string line;

Poco::Logger * log;
LoggerPtr log;

uint32_t cur_lineno;
std::optional<std::pair<uint32_t, Error::Type>> err_info; // <lineno, Error::Type>
Expand Down
14 changes: 1 addition & 13 deletions dbms/src/Storages/DeltaMerge/DeltaTree.h
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@
#include <Core/Types.h>
#include <IO/WriteHelpers.h>
#include <Storages/DeltaMerge/Tuple.h>
#include <common/logger_useful.h>

#include <algorithm>
#include <cstddef>
Expand Down Expand Up @@ -773,8 +772,6 @@ class DeltaTree
Allocator * allocator = nullptr;
size_t bytes = 0;

Poco::Logger * log = nullptr;

public:
// For test cases only.
ValueSpacePtr insert_value_space;
Expand Down Expand Up @@ -888,14 +885,10 @@ class DeltaTree
{
allocator = new Allocator();

log = &Poco::Logger::get("DeltaTree");

insert_value_space = insert_value_space_;

root = createNode<Leaf>();
left_leaf = right_leaf = as(Leaf, root);

LOG_TRACE(log, "create");
}

public:
Expand Down Expand Up @@ -928,8 +921,6 @@ class DeltaTree
std::swap(num_deletes, other.num_deletes);
std::swap(num_entries, other.num_entries);

std::swap(log, other.log);

std::swap(allocator, allocator);

insert_value_space.swap(other.insert_value_space);
Expand All @@ -946,8 +937,6 @@ class DeltaTree
}

delete allocator;

LOG_TRACE(log, "free");
}

void checkAll() const
Expand Down Expand Up @@ -1004,7 +993,6 @@ DT_CLASS::DeltaTree(const DT_CLASS::Self & o)
, num_deletes(o.num_deletes)
, num_entries(o.num_entries)
, allocator(new Allocator())
, log(&Poco::Logger::get("DeltaTree"))
{
NodePtr my_root;
if (isLeaf(o.root))
Expand Down Expand Up @@ -1453,4 +1441,4 @@ typename DT_CLASS::InternPtr DT_CLASS::afterNodeUpdated(T * node)
#undef DT_CLASS

} // namespace DM
} // namespace DB
} // namespace DB
5 changes: 1 addition & 4 deletions dbms/src/Storages/DeltaMerge/DeltaTree.ipp
Original file line number Diff line number Diff line change
Expand Up @@ -34,8 +34,6 @@ __attribute__((noinline, flatten)) typename DT_CLASS::InternPtr DT_CLASS::TIFLAS
as(Intern, root)->parent = nullptr;
--height;

LOG_TRACE(log, "height {} -> {}", (height + 1), height);

return {};
}

Expand All @@ -57,7 +55,6 @@ __attribute__((noinline, flatten)) typename DT_CLASS::InternPtr DT_CLASS::TIFLAS

++height;

LOG_TRACE(log, "height {} -> {}", (height - 1), height);
}

auto pos = parent->searchChild(asNode(node));
Expand Down Expand Up @@ -162,4 +159,4 @@ __attribute__((noinline, flatten)) typename DT_CLASS::InternPtr DT_CLASS::TIFLAS
return parent;
else
return {};
}
}
3 changes: 1 addition & 2 deletions dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -42,9 +42,8 @@ class SegmentReader

~SegmentReader()
{
LOG_DEBUG(log, "Stop begin");
t.join();
LOG_DEBUG(log, "Stop end");
LOG_DEBUG(log, "Stopped");
}

std::thread::id getId() const
Expand Down
8 changes: 5 additions & 3 deletions dbms/src/Storages/DeltaMerge/SSTFilesToBlockInputStream.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -87,17 +87,19 @@ void SSTFilesToBlockInputStream::readPrefix()
break;
}
}

// Pass the log to SSTReader inorder to filter logs by table_id suffix
if (!ssts_default.empty())
{
default_cf_reader = std::make_unique<MultiSSTReader<MonoSSTReader, SSTView>>(proxy_helper, ColumnFamilyType::Default, make_inner_func, ssts_default);
default_cf_reader = std::make_unique<MultiSSTReader<MonoSSTReader, SSTView>>(proxy_helper, ColumnFamilyType::Default, make_inner_func, ssts_default, log);
}
if (!ssts_write.empty())
{
write_cf_reader = std::make_unique<MultiSSTReader<MonoSSTReader, SSTView>>(proxy_helper, ColumnFamilyType::Write, make_inner_func, ssts_write);
write_cf_reader = std::make_unique<MultiSSTReader<MonoSSTReader, SSTView>>(proxy_helper, ColumnFamilyType::Write, make_inner_func, ssts_write, log);
}
if (!ssts_lock.empty())
{
lock_cf_reader = std::make_unique<MultiSSTReader<MonoSSTReader, SSTView>>(proxy_helper, ColumnFamilyType::Lock, make_inner_func, ssts_lock);
lock_cf_reader = std::make_unique<MultiSSTReader<MonoSSTReader, SSTView>>(proxy_helper, ColumnFamilyType::Lock, make_inner_func, ssts_lock, log);
}
LOG_INFO(log, "Finish Construct MultiSSTReader, write {} lock {} default {} region {}", ssts_write.size(), ssts_lock.size(), ssts_default.size(), this->region->id());

Expand Down
3 changes: 2 additions & 1 deletion dbms/src/Storages/GCManager.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@
// See the License for the specific language governing permissions and
// limitations under the License.

#include <Common/Logger.h>
#include <Storages/DeltaMerge/GCOptions.h>
#include <Storages/GCManager.h>
#include <Storages/IManageableStorage.h>
Expand All @@ -26,7 +27,7 @@ extern const int TABLE_IS_DROPPED;

GCManager::GCManager(Context & context)
: global_context{context.getGlobalContext()}
, log(&Poco::Logger::get("GCManager"))
, log(Logger::get())
{
}

Expand Down
9 changes: 3 additions & 6 deletions dbms/src/Storages/GCManager.h
Original file line number Diff line number Diff line change
Expand Up @@ -17,13 +17,10 @@
#include <Common/Stopwatch.h>
#include <Storages/Transaction/Types.h>

namespace Poco
{
class Logger;
}

namespace DB
{
class Logger;
using LoggerPtr = std::shared_ptr<Logger>;
class Context;

class GCManager
Expand All @@ -42,6 +39,6 @@ class GCManager

AtomicStopwatch gc_check_stop_watch;

Poco::Logger * log;
LoggerPtr log;
};
} // namespace DB
113 changes: 113 additions & 0 deletions dbms/src/Storages/Page/V3/Blob/GCInfo.cpp
Original file line number Diff line number Diff line change
@@ -0,0 +1,113 @@
// 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 <Common/FmtUtils.h>
#include <Storages/Page/V3/Blob/GCInfo.h>

#include <magic_enum.hpp>


template <>
struct fmt::formatter<DB::PS::V3::BlobFileGCInfo>
{
static constexpr auto parse(format_parse_context & ctx) -> decltype(ctx.begin())
{
const auto * it = ctx.begin();
const auto * end = ctx.end();
if (it != end && *it != '}')
throw format_error("invalid format");
return it;
}

template <typename FormatContext>
auto format(const DB::PS::V3::BlobFileGCInfo & i, FormatContext & ctx) const -> decltype(ctx.out())
{
return format_to(ctx.out(), "<id:{} rate:{:.2f}>", i.blob_id, i.valid_rate);
}
};
template <>
struct fmt::formatter<DB::PS::V3::BlobFileTruncateInfo>
{
static constexpr auto parse(format_parse_context & ctx) -> decltype(ctx.begin())
{
const auto * it = ctx.begin();
const auto * end = ctx.end();
if (it != end && *it != '}')
throw format_error("invalid format");
return it;
}

template <typename FormatContext>
auto format(const DB::PS::V3::BlobFileTruncateInfo & i, FormatContext & ctx) const -> decltype(ctx.out())
{
return format_to(ctx.out(), "<id:{} origin:{} truncate:{} rate:{:.2f}>", i.blob_id, i.origin_size, i.truncated_size, i.valid_rate);
}
};

namespace DB::PS::V3
{

Poco::Message::Priority BlobStoreGCInfo::getLoggingLevel() const
{
if (blob_gc_info[FullGC].empty() && blob_gc_truncate_info.empty())
return Poco::Message::PRIO_DEBUG;
return Poco::Message::PRIO_INFORMATION;
}

String BlobStoreGCInfo::toString() const
{
return fmt::format("{} {} {} {}",
toTypeString(ReadOnly),
toTypeString(Unchanged),
toTypeString(FullGC),
toTypeTruncateString(Truncated));
}

String BlobStoreGCInfo::toTypeString(const Type type_index) const
{
if (blob_gc_info[type_index].empty())
return fmt::format("{{{}: [null]}}", magic_enum::enum_name(type_index));

// e.g. {FullGC: [<id:4 rate:0.16>]}}
FmtBuffer fmt_buf;
fmt_buf.fmtAppend("{{{}: [", magic_enum::enum_name(type_index))
.joinStr(
blob_gc_info[type_index].begin(),
blob_gc_info[type_index].end(),
[](const auto & i, FmtBuffer & fb) {
fb.fmtAppend("{}", i);
},
", ")
.append("]}}");
return fmt_buf.toString();
}

String BlobStoreGCInfo::toTypeTruncateString(const Type type_index) const
{
if (blob_gc_truncate_info.empty())
return fmt::format("{{{}: [null]}}", magic_enum::enum_name(type_index));

FmtBuffer fmt_buf;
fmt_buf.fmtAppend("{{{}: [", type_index)
.joinStr(
blob_gc_truncate_info.begin(),
blob_gc_truncate_info.end(),
[](const auto & i, FmtBuffer & fb) {
fb.fmtAppend("{}", i);
},
", ")
.append("]}}");
return fmt_buf.toString();
}
} // namespace DB::PS::V3
Loading

0 comments on commit 33893c1

Please sign in to comment.