Skip to content

Commit

Permalink
Remove useless code (#5004)
Browse files Browse the repository at this point in the history
close #4998
  • Loading branch information
hongyunyan authored May 31, 2022
1 parent 580821a commit 6afdd74
Show file tree
Hide file tree
Showing 17 changed files with 135 additions and 446 deletions.
12 changes: 6 additions & 6 deletions dbms/src/Debug/dbgFuncRegion.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -40,7 +40,7 @@ extern const int UNKNOWN_TABLE;
// put_region(region_id, start, end, database_name, table_name[, partition-name])
void dbgFuncPutRegion(Context & context, const ASTs & args, DBGInvoker::Printer output)
{
RegionID region_id = static_cast<RegionID>(safeGet<UInt64>(typeid_cast<const ASTLiteral &>(*args[0]).value));
auto region_id = static_cast<RegionID>(safeGet<UInt64>(typeid_cast<const ASTLiteral &>(*args[0]).value));
bool has_partition_id = false;
size_t args_size = args.size();
if (dynamic_cast<ASTLiteral *>(args[args_size - 1].get()) != nullptr)
Expand Down Expand Up @@ -81,8 +81,8 @@ void dbgFuncPutRegion(Context & context, const ASTs & args, DBGInvoker::Printer
}
else
{
HandleID start = static_cast<HandleID>(safeGet<UInt64>(typeid_cast<const ASTLiteral &>(*args[1]).value));
HandleID end = static_cast<HandleID>(safeGet<UInt64>(typeid_cast<const ASTLiteral &>(*args[2]).value));
auto start = static_cast<HandleID>(safeGet<UInt64>(typeid_cast<const ASTLiteral &>(*args[1]).value));
auto end = static_cast<HandleID>(safeGet<UInt64>(typeid_cast<const ASTLiteral &>(*args[2]).value));

TMTContext & tmt = context.getTMTContext();
RegionPtr region = RegionBench::createRegion(table_id, region_id, start, end);
Expand All @@ -107,7 +107,7 @@ void dbgFuncTryFlushRegion(Context & context, const ASTs & args, DBGInvoker::Pri
throw Exception("Args not matched, should be: region-id", ErrorCodes::BAD_ARGUMENTS);
}

RegionID region_id = static_cast<RegionID>(safeGet<UInt64>(typeid_cast<const ASTLiteral &>(*args[0]).value));
auto region_id = static_cast<RegionID>(safeGet<UInt64>(typeid_cast<const ASTLiteral &>(*args[0]).value));

TMTContext & tmt = context.getTMTContext();
tmt.getRegionTable().tryFlushRegion(region_id);
Expand Down Expand Up @@ -160,7 +160,7 @@ void dbgFuncDumpAllRegion(Context & context, const ASTs & args, DBGInvoker::Prin
if (args.empty())
throw Exception("Args not matched, should be: table_id", ErrorCodes::BAD_ARGUMENTS);

TableID table_id = static_cast<TableID>(safeGet<UInt64>(typeid_cast<const ASTLiteral &>(*args[0]).value));
auto table_id = static_cast<TableID>(safeGet<UInt64>(typeid_cast<const ASTLiteral &>(*args[0]).value));

bool ignore_none = false;
if (args.size() > 1)
Expand Down Expand Up @@ -190,7 +190,7 @@ void dbgFuncRemoveRegion(Context & context, const ASTs & args, DBGInvoker::Print
if (args.empty())
throw Exception("Args not matched, should be: region_id", ErrorCodes::BAD_ARGUMENTS);

RegionID region_id = static_cast<RegionID>(safeGet<UInt64>(typeid_cast<const ASTLiteral &>(*args[0]).value));
auto region_id = static_cast<RegionID>(safeGet<UInt64>(typeid_cast<const ASTLiteral &>(*args[0]).value));

TMTContext & tmt = context.getTMTContext();
KVStorePtr & kvstore = tmt.getKVStore();
Expand Down
6 changes: 1 addition & 5 deletions dbms/src/Server/RaftConfigParser.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -92,11 +92,7 @@ TiFlashRaftConfig TiFlashRaftConfig::parseSettings(Poco::Util::LayeredConfigurat
{
String snapshot_method = config.getString("raft.snapshot.method");
std::transform(snapshot_method.begin(), snapshot_method.end(), snapshot_method.begin(), [](char ch) { return std::tolower(ch); });
if (snapshot_method == "block")
{
res.snapshot_apply_method = TiDB::SnapshotApplyMethod::Block;
}
else if (snapshot_method == "file1")
if (snapshot_method == "file1")
{
res.snapshot_apply_method = TiDB::SnapshotApplyMethod::DTFile_Directory;
}
Expand Down
92 changes: 4 additions & 88 deletions dbms/src/Storages/Transaction/ApplySnapshot.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -261,75 +261,6 @@ void KVStore::onSnapshot(const RegionPtrWrap & new_region_wrap, RegionPtr old_re

extern RegionPtrWithBlock::CachePtr GenRegionPreDecodeBlockData(const RegionPtr &, Context &);

/// `preHandleSnapshotToBlock` read data from SSTFiles and predoced the data as a block
RegionPreDecodeBlockDataPtr KVStore::preHandleSnapshotToBlock(
RegionPtr new_region,
const SSTViewVec snaps,
uint64_t /*index*/,
uint64_t /*term*/,
TMTContext & tmt)
{
RegionPreDecodeBlockDataPtr cache{nullptr};
{
decltype(bg_gc_region_data)::value_type tmp;
std::lock_guard lock(bg_gc_region_data_mutex);
if (!bg_gc_region_data.empty())
{
tmp.swap(bg_gc_region_data.back());
bg_gc_region_data.pop_back();
}
}

Stopwatch watch;
auto & ctx = tmt.getContext();
SCOPE_EXIT({ GET_METRIC(tiflash_raft_command_duration_seconds, type_apply_snapshot_predecode).Observe(watch.elapsedSeconds()); });

{
LOG_FMT_INFO(log, "Pre-handle snapshot {} with {} TiKV sst files", new_region->toString(false), snaps.len);
// Iterator over all SST files and insert key-values into `new_region`
for (UInt64 i = 0; i < snaps.len; ++i)
{
const auto & snapshot = snaps.views[i];
auto sst_reader = SSTReader{proxy_helper, snapshot};

uint64_t kv_size = 0;
while (sst_reader.remained())
{
auto key = sst_reader.key();
auto value = sst_reader.value();
new_region->insert(snaps.views[i].type, TiKVKey(key.data, key.len), TiKVValue(value.data, value.len));
++kv_size;
sst_reader.next();
}

LOG_FMT_INFO(log,
"Decode {} got [cf: {}, kv size: {}]",
std::string_view(snapshot.path.data, snapshot.path.len),
CFToName(snapshot.type),
kv_size);
// Note that number of keys in different cf will be aggregated into one metrics
GET_METRIC(tiflash_raft_process_keys, type_apply_snapshot).Increment(kv_size);
}
{
LOG_FMT_INFO(log, "Start to pre-decode {} into block", new_region->toString());
auto block_cache = GenRegionPreDecodeBlockData(new_region, ctx);
if (block_cache)
{
std::stringstream ss;
block_cache->toString(ss);
LOG_FMT_INFO(log, "Got pre-decode block cache {}", ss.str());
}
else
LOG_FMT_INFO(log, "Got empty pre-decode block cache");

cache = std::move(block_cache);
}
LOG_FMT_INFO(log, "Pre-handle snapshot {} cost {}ms", new_region->toString(false), watch.elapsedMilliseconds());
}

return cache;
}

std::vector<UInt64> KVStore::preHandleSnapshotToFiles(
RegionPtr new_region,
const SSTViewVec snaps,
Expand Down Expand Up @@ -473,8 +404,8 @@ void KVStore::handlePreApplySnapshot(const RegionPtrWrap & new_region, TMTContex
LOG_FMT_INFO(log, "{} apply snapshot success", new_region->toString(false));
}

template void KVStore::handlePreApplySnapshot<RegionPtrWithBlock>(const RegionPtrWithBlock &, TMTContext &);
template void KVStore::handlePreApplySnapshot<RegionPtrWithSnapshotFiles>(const RegionPtrWithSnapshotFiles &, TMTContext &);

template void KVStore::checkAndApplySnapshot<RegionPtrWithBlock>(const RegionPtrWithBlock &, TMTContext &);
template void KVStore::checkAndApplySnapshot<RegionPtrWithSnapshotFiles>(const RegionPtrWithSnapshotFiles &, TMTContext &);
template void KVStore::onSnapshot<RegionPtrWithBlock>(const RegionPtrWithBlock &, RegionPtr, UInt64, TMTContext &);
Expand Down Expand Up @@ -521,10 +452,7 @@ void KVStore::handleApplySnapshot(
TMTContext & tmt)
{
auto new_region = genRegionPtr(std::move(region), peer_id, index, term);
if (snapshot_apply_method == TiDB::SnapshotApplyMethod::Block)
handlePreApplySnapshot(RegionPtrWithBlock{new_region, preHandleSnapshotToBlock(new_region, snaps, index, term, tmt)}, tmt);
else
handlePreApplySnapshot(RegionPtrWithSnapshotFiles{new_region, preHandleSnapshotToFiles(new_region, snaps, index, term, tmt)}, tmt);
handlePreApplySnapshot(RegionPtrWithSnapshotFiles{new_region, preHandleSnapshotToFiles(new_region, snaps, index, term, tmt)}, tmt);
}

EngineStoreApplyRes KVStore::handleIngestSST(UInt64 region_id, const SSTViewVec snaps, UInt64 index, UInt64 term, TMTContext & tmt)
Expand All @@ -543,15 +471,12 @@ EngineStoreApplyRes KVStore::handleIngestSST(UInt64 region_id, const SSTViewVec

fiu_do_on(FailPoints::force_set_sst_decode_rand, {
static int num_call = 0;
switch (num_call++ % 3)
switch (num_call++ % 2)
{
case 0:
snapshot_apply_method = TiDB::SnapshotApplyMethod::Block;
break;
case 1:
snapshot_apply_method = TiDB::SnapshotApplyMethod::DTFile_Directory;
break;
case 2:
case 1:
snapshot_apply_method = TiDB::SnapshotApplyMethod::DTFile_Single;
break;
default:
Expand All @@ -576,15 +501,6 @@ EngineStoreApplyRes KVStore::handleIngestSST(UInt64 region_id, const SSTViewVec
}
};

if (snapshot_apply_method == TiDB::SnapshotApplyMethod::Block)
{
// try to flush remain data in memory.
func_try_flush();
region->handleIngestSSTInMemory(snaps, index, term);
// after `handleIngestSSTInMemory`, all data are stored in `region`, try to flush committed data into storage
func_try_flush();
}
else
{
// try to flush remain data in memory.
func_try_flush();
Expand Down
7 changes: 1 addition & 6 deletions dbms/src/Storages/Transaction/KVStore.h
Original file line number Diff line number Diff line change
Expand Up @@ -109,12 +109,7 @@ class KVStore final : private boost::noncopyable
EngineStoreApplyRes handleWriteRaftCmd(const WriteCmdsView & cmds, UInt64 region_id, UInt64 index, UInt64 term, TMTContext & tmt);

void handleApplySnapshot(metapb::Region && region, uint64_t peer_id, const SSTViewVec, uint64_t index, uint64_t term, TMTContext & tmt);
RegionPreDecodeBlockDataPtr preHandleSnapshotToBlock(
RegionPtr new_region,
const SSTViewVec,
uint64_t index,
uint64_t term,
TMTContext & tmt);

std::vector<UInt64> /* */ preHandleSnapshotToFiles(
RegionPtr new_region,
const SSTViewVec,
Expand Down
54 changes: 0 additions & 54 deletions dbms/src/Storages/Transaction/PartitionStreams.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -353,60 +353,6 @@ void RegionTable::writeBlockByRegion(
data_list_to_remove = std::move(*data_list_read);
}

RegionTable::ReadBlockByRegionRes RegionTable::readBlockByRegion(const TiDB::TableInfo & table_info,
const ColumnsDescription & columns [[maybe_unused]],
const Names & column_names_to_read,
const RegionPtr & region,
RegionVersion region_version,
RegionVersion conf_version,
bool resolve_locks,
Timestamp start_ts,
const std::unordered_set<UInt64> * bypass_lock_ts,
RegionScanFilterPtr scan_filter)
{
if (!region)
throw Exception(std::string(__PRETTY_FUNCTION__) + ": region is null", ErrorCodes::LOGICAL_ERROR);

// Tiny optimization for queries that need only handle, tso, delmark.
bool need_value = column_names_to_read.size() != 3;
auto region_data_lock = resolveLocksAndReadRegionData(
table_info.id,
region,
start_ts,
bypass_lock_ts,
region_version,
conf_version,
resolve_locks,
need_value);

return std::visit(variant_op::overloaded{
[&](RegionDataReadInfoList & data_list_read) -> ReadBlockByRegionRes {
/// Read region data as block.
Block block;
// FIXME: remove this deprecated function
assert(0);
{
auto reader = RegionBlockReader(nullptr);
bool ok = reader.setStartTs(start_ts)
.setFilter(scan_filter)
.read(block, data_list_read, /*force_decode*/ true);
if (!ok)
// TODO: Enrich exception message.
throw Exception("Read region " + std::to_string(region->id()) + " of table "
+ std::to_string(table_info.id) + " failed",
ErrorCodes::LOGICAL_ERROR);
}
return block;
},
[&](LockInfoPtr & lock_value) -> ReadBlockByRegionRes {
assert(lock_value);
throw LockException(region->id(), std::move(lock_value));
},
[](RegionException::RegionReadStatus & s) -> ReadBlockByRegionRes { return s; },
},
region_data_lock);
}

RegionTable::ResolveLocksAndWriteRegionRes RegionTable::resolveLocksAndWriteRegion(TMTContext & tmt,
const TiDB::TableID table_id,
const RegionPtr & region,
Expand Down
40 changes: 3 additions & 37 deletions dbms/src/Storages/Transaction/ProxyFFI.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -304,25 +304,13 @@ RawRustPtrWrap::~RawRustPtrWrap()
RustGcHelper::instance().gcRustPtr(ptr, type);
}
RawRustPtrWrap::RawRustPtrWrap(RawRustPtrWrap && src)
: RawRustPtr()
{
RawRustPtr & tar = (*this);
tar = src;
src.ptr = nullptr;
}

struct PreHandledSnapshotWithBlock
{
~PreHandledSnapshotWithBlock() { CurrentMetrics::sub(CurrentMetrics::RaftNumSnapshotsPendingApply); }
PreHandledSnapshotWithBlock(const RegionPtr & region_, RegionPtrWithBlock::CachePtr && cache_)
: region(region_)
, cache(std::move(cache_))
{
CurrentMetrics::add(CurrentMetrics::RaftNumSnapshotsPendingApply);
}
RegionPtr region;
RegionPtrWithBlock::CachePtr cache;
};

struct PreHandledSnapshotWithFiles
{
~PreHandledSnapshotWithFiles() { CurrentMetrics::sub(CurrentMetrics::RaftNumSnapshotsPendingApply); }
Expand Down Expand Up @@ -362,13 +350,6 @@ RawCppPtr PreHandleSnapshot(

switch (kvstore->applyMethod())
{
case TiDB::SnapshotApplyMethod::Block:
{
// Pre-decode as a block
auto new_region_block_cache = kvstore->preHandleSnapshotToBlock(new_region, snaps, index, term, tmt);
auto * res = new PreHandledSnapshotWithBlock{new_region, std::move(new_region_block_cache)};
return GenRawCppPtr(res, RawCppPtrTypeImpl::PreHandledSnapshotWithBlock);
}
case TiDB::SnapshotApplyMethod::DTFile_Directory:
case TiDB::SnapshotApplyMethod::DTFile_Single:
{
Expand All @@ -391,18 +372,12 @@ RawCppPtr PreHandleSnapshot(
template <typename PreHandledSnapshot>
void ApplyPreHandledSnapshot(EngineStoreServerWrap * server, PreHandledSnapshot * snap)
{
static_assert(
std::is_same_v<PreHandledSnapshot, PreHandledSnapshotWithBlock> || std::is_same_v<PreHandledSnapshot, PreHandledSnapshotWithFiles>,
"Unknown pre-handled snapshot type");
static_assert(std::is_same_v<PreHandledSnapshot, PreHandledSnapshotWithFiles>, "Unknown pre-handled snapshot type");

try
{
auto & kvstore = server->tmt->getKVStore();
if constexpr (std::is_same_v<PreHandledSnapshot, PreHandledSnapshotWithBlock>)
{
kvstore->handlePreApplySnapshot(RegionPtrWithBlock{snap->region, std::move(snap->cache)}, *server->tmt);
}
else if constexpr (std::is_same_v<PreHandledSnapshot, PreHandledSnapshotWithFiles>)
if constexpr (std::is_same_v<PreHandledSnapshot, PreHandledSnapshotWithFiles>)
{
kvstore->handlePreApplySnapshot(RegionPtrWithSnapshotFiles{snap->region, std::move(snap->ingest_ids)}, *server->tmt);
}
Expand All @@ -418,12 +393,6 @@ void ApplyPreHandledSnapshot(EngineStoreServerWrap * server, RawVoidPtr res, Raw
{
switch (static_cast<RawCppPtrTypeImpl>(type))
{
case RawCppPtrTypeImpl::PreHandledSnapshotWithBlock:
{
auto * snap = reinterpret_cast<PreHandledSnapshotWithBlock *>(res);
ApplyPreHandledSnapshot(server, snap);
break;
}
case RawCppPtrTypeImpl::PreHandledSnapshotWithFiles:
{
auto * snap = reinterpret_cast<PreHandledSnapshotWithFiles *>(res);
Expand All @@ -445,9 +414,6 @@ void GcRawCppPtr(RawVoidPtr ptr, RawCppPtrType type)
case RawCppPtrTypeImpl::String:
delete reinterpret_cast<RawCppStringPtr>(ptr);
break;
case RawCppPtrTypeImpl::PreHandledSnapshotWithBlock:
delete reinterpret_cast<PreHandledSnapshotWithBlock *>(ptr);
break;
case RawCppPtrTypeImpl::PreHandledSnapshotWithFiles:
delete reinterpret_cast<PreHandledSnapshotWithFiles *>(ptr);
break;
Expand Down
1 change: 0 additions & 1 deletion dbms/src/Storages/Transaction/ProxyFFI.h
Original file line number Diff line number Diff line change
Expand Up @@ -56,7 +56,6 @@ enum class RawCppPtrTypeImpl : RawCppPtrType
{
None = 0,
String,
PreHandledSnapshotWithBlock,
PreHandledSnapshotWithFiles,
WakerNotifier,
};
Expand Down
Loading

0 comments on commit 6afdd74

Please sign in to comment.