Skip to content

Commit

Permalink
Merge branch 'dtworkload-support-mixed-mode' of github.com:lidezhu/ti…
Browse files Browse the repository at this point in the history
…cs into dtworkload-support-mixed-mode
  • Loading branch information
lidezhu committed Jun 1, 2022
2 parents 05510f4 + 0d585ac commit ab85dd9
Show file tree
Hide file tree
Showing 89 changed files with 1,849 additions and 1,242 deletions.
1 change: 1 addition & 0 deletions dbms/CMakeLists.txt
Original file line number Diff line number Diff line change
Expand Up @@ -90,6 +90,7 @@ add_headers_and_sources(dbms src/Storages/Page/V2/mvcc)
add_headers_and_sources(dbms src/Storages/Page/V2/VersionSet)
add_headers_and_sources(dbms src/Storages/Page/V2/gc)
add_headers_and_sources(dbms src/WindowFunctions)
add_headers_and_sources(dbms src/TiDB/Schema)
if (ENABLE_V3_PAGESTORAGE)
add_headers_and_sources(dbms src/Storages/Page/V3)
add_headers_and_sources(dbms src/Storages/Page/V3/LogFile)
Expand Down
31 changes: 27 additions & 4 deletions dbms/src/Common/Decimal.h
Original file line number Diff line number Diff line change
Expand Up @@ -306,10 +306,6 @@ using Decimal64 = Decimal<Int64>;
using Decimal128 = Decimal<Int128>;
using Decimal256 = Decimal<Int256>;

static constexpr PrecType minDecimalPrecision()
{
return 1;
}
template <typename T>
static constexpr PrecType maxDecimalPrecision()
{
Expand All @@ -336,6 +332,33 @@ constexpr PrecType maxDecimalPrecision<Decimal256>()
return 65;
}

template <typename T>
constexpr PrecType minDecimalPrecision()
{
/// return a invalid value
return maxDecimalPrecision<Decimal256>() + 1;
}
template <>
constexpr PrecType minDecimalPrecision<Decimal32>()
{
return 1;
}
template <>
constexpr PrecType minDecimalPrecision<Decimal64>()
{
return maxDecimalPrecision<Decimal32>() + 1;
}
template <>
constexpr PrecType minDecimalPrecision<Decimal128>()
{
return maxDecimalPrecision<Decimal64>() + 1;
}
template <>
constexpr PrecType minDecimalPrecision<Decimal256>()
{
return maxDecimalPrecision<Decimal128>() + 1;
}

template <typename T>
struct PromoteType
{
Expand Down
15 changes: 14 additions & 1 deletion dbms/src/Core/Field.h
Original file line number Diff line number Diff line change
Expand Up @@ -122,7 +122,20 @@ class DecimalField
}
if (cnt == 0)
cnt = 1;
return cnt;
return std::max(cnt, scale);
}

/// In TiFlash there are 4 subtype of decimal:
/// Decimal32, Decimal64, Decimal128 and Decimal256
/// they are not compatible with each other. So a DecimalField<Decimal32>
/// can not be inserted into a decimal column with DecimalType<Decimal64>
/// getPrecWithCurrentDecimalType will return the prec that fit
/// current decimal type, that is to say, current DecimalField can be
/// inserted into a decimal column with type `Decimal(getPrecWithCurrentDecimalType, getScale)`
UInt32 getPrecWithCurrentDecimalType() const
{
auto raw_prec = getPrec();
return std::max(raw_prec, minDecimalPrecision<T>());
}

template <typename U>
Expand Down
2 changes: 1 addition & 1 deletion dbms/src/DataTypes/DataTypeDecimal.h
Original file line number Diff line number Diff line change
Expand Up @@ -192,7 +192,7 @@ using DataTypeDecimal256 = DataTypeDecimal<Decimal256>;

inline DataTypePtr createDecimal(UInt64 prec, UInt64 scale)
{
if (prec < minDecimalPrecision() || prec > maxDecimalPrecision<Decimal256>())
if (prec < minDecimalPrecision<Decimal32>() || prec > maxDecimalPrecision<Decimal256>())
throw Exception("Wrong precision:" + DB::toString(prec), ErrorCodes::ARGUMENT_OUT_OF_BOUND);

if (static_cast<UInt64>(scale) > prec)
Expand Down
3 changes: 1 addition & 2 deletions dbms/src/DataTypes/FieldToDataType.h
Original file line number Diff line number Diff line change
Expand Up @@ -40,8 +40,7 @@ class FieldToDataType : public StaticVisitor<DataTypePtr>
template <typename T>
DataTypePtr operator()(const DecimalField<T> & x) const
{
PrecType prec = maxDecimalPrecision<T>();
return std::make_shared<DataTypeDecimal<T>>(prec, x.getScale());
return std::make_shared<DataTypeDecimal<T>>(x.getPrecWithCurrentDecimalType(), x.getScale());
}
};

Expand Down
100 changes: 100 additions & 0 deletions dbms/src/DataTypes/tests/gtest_decimal_literal_datatype.cpp
Original file line number Diff line number Diff line change
@@ -0,0 +1,100 @@
// Copyright 2022 PingCAP, Ltd.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

#include <DataTypes/DataTypeFactory.h>
#include <DataTypes/FieldToDataType.h>
#include <TestUtils/TiFlashTestBasic.h>

namespace DB
{
namespace tests
{
TEST(DecimalLiteralDataTypeTest, getPrec)
try
{
/// Decimal32
ASSERT_TRUE(DecimalField<Decimal32>(0, 0).getPrec() == 1);
ASSERT_TRUE(DecimalField<Decimal32>(0, 1).getPrec() == 1);
ASSERT_TRUE(DecimalField<Decimal32>(0, 2).getPrec() == 2);
ASSERT_TRUE(DecimalField<Decimal32>(123, 0).getPrec() == 3);
ASSERT_TRUE(DecimalField<Decimal32>(123, 2).getPrec() == 3);
ASSERT_TRUE(DecimalField<Decimal32>(123, 4).getPrec() == 4);

/// Decimal64
ASSERT_TRUE(DecimalField<Decimal64>(0, 0).getPrec() == 1);
ASSERT_TRUE(DecimalField<Decimal64>(0, 1).getPrec() == 1);
ASSERT_TRUE(DecimalField<Decimal64>(0, 2).getPrec() == 2);
ASSERT_TRUE(DecimalField<Decimal64>(123, 0).getPrec() == 3);
ASSERT_TRUE(DecimalField<Decimal64>(123, 2).getPrec() == 3);
ASSERT_TRUE(DecimalField<Decimal64>(123, 4).getPrec() == 4);
ASSERT_TRUE(DecimalField<Decimal64>(1234567891011ll, 4).getPrec() == 13);

/// Decimal128
ASSERT_TRUE(DecimalField<Decimal128>(0, 0).getPrec() == 1);
ASSERT_TRUE(DecimalField<Decimal128>(0, 1).getPrec() == 1);
ASSERT_TRUE(DecimalField<Decimal128>(0, 2).getPrec() == 2);
ASSERT_TRUE(DecimalField<Decimal128>(123, 0).getPrec() == 3);
ASSERT_TRUE(DecimalField<Decimal128>(123, 2).getPrec() == 3);
ASSERT_TRUE(DecimalField<Decimal128>(123, 4).getPrec() == 4);
ASSERT_TRUE(DecimalField<Decimal128>(Int128(123123123123123ll) * 1000000, 4).getPrec() == 21);

/// Decimal256
ASSERT_TRUE(DecimalField<Decimal256>(Int256(0), 0).getPrec() == 1);
ASSERT_TRUE(DecimalField<Decimal256>(Int256(0), 1).getPrec() == 1);
ASSERT_TRUE(DecimalField<Decimal256>(Int256(0), 2).getPrec() == 2);
ASSERT_TRUE(DecimalField<Decimal256>(Int256(123), 0).getPrec() == 3);
ASSERT_TRUE(DecimalField<Decimal256>(Int256(123), 2).getPrec() == 3);
ASSERT_TRUE(DecimalField<Decimal256>(Int256(123), 4).getPrec() == 4);
ASSERT_TRUE(DecimalField<Decimal256>(Int256(123123123123123123ll) * Int256(1000000000ll) * Int256(100000000000000ll), 4).getPrec() == 41);
}
CATCH

TEST(DecimalLiteralDataTypeTest, fieldToDataType)
try
{
/// Decimal32
ASSERT_TRUE(DataTypeFactory::instance().get("Decimal(1,0)")->equals(*applyVisitor(FieldToDataType(), Field(DecimalField<Decimal32>(0, 0)))));
ASSERT_TRUE(DataTypeFactory::instance().get("Decimal(1,1)")->equals(*applyVisitor(FieldToDataType(), Field(DecimalField<Decimal32>(0, 1)))));
ASSERT_TRUE(DataTypeFactory::instance().get("Decimal(2,2)")->equals(*applyVisitor(FieldToDataType(), Field(DecimalField<Decimal32>(0, 2)))));
ASSERT_TRUE(DataTypeFactory::instance().get("Decimal(3,0)")->equals(*applyVisitor(FieldToDataType(), Field(DecimalField<Decimal32>(123, 0)))));
ASSERT_TRUE(DataTypeFactory::instance().get("Decimal(3,2)")->equals(*applyVisitor(FieldToDataType(), Field(DecimalField<Decimal32>(123, 2)))));
ASSERT_TRUE(DataTypeFactory::instance().get("Decimal(4,4)")->equals(*applyVisitor(FieldToDataType(), Field(DecimalField<Decimal32>(123, 4)))));

/// Decimal64
ASSERT_TRUE(DataTypeFactory::instance().get("Decimal(10,0)")->equals(*applyVisitor(FieldToDataType(), Field(DecimalField<Decimal64>(0, 0)))));
ASSERT_TRUE(DataTypeFactory::instance().get("Decimal(10,1)")->equals(*applyVisitor(FieldToDataType(), Field(DecimalField<Decimal64>(0, 1)))));
ASSERT_TRUE(DataTypeFactory::instance().get("Decimal(10,2)")->equals(*applyVisitor(FieldToDataType(), Field(DecimalField<Decimal64>(0, 2)))));
ASSERT_TRUE(DataTypeFactory::instance().get("Decimal(10,0)")->equals(*applyVisitor(FieldToDataType(), Field(DecimalField<Decimal64>(123, 0)))));
ASSERT_TRUE(DataTypeFactory::instance().get("Decimal(10,2)")->equals(*applyVisitor(FieldToDataType(), Field(DecimalField<Decimal64>(123, 2)))));
ASSERT_TRUE(DataTypeFactory::instance().get("Decimal(13,4)")->equals(*applyVisitor(FieldToDataType(), Field(DecimalField<Decimal64>(1234567891011ll, 4)))));

/// Decimal128
ASSERT_TRUE(DataTypeFactory::instance().get("Decimal(19,0)")->equals(*applyVisitor(FieldToDataType(), Field(DecimalField<Decimal128>(0, 0)))));
ASSERT_TRUE(DataTypeFactory::instance().get("Decimal(19,1)")->equals(*applyVisitor(FieldToDataType(), Field(DecimalField<Decimal128>(0, 1)))));
ASSERT_TRUE(DataTypeFactory::instance().get("Decimal(19,2)")->equals(*applyVisitor(FieldToDataType(), Field(DecimalField<Decimal128>(0, 2)))));
ASSERT_TRUE(DataTypeFactory::instance().get("Decimal(19,0)")->equals(*applyVisitor(FieldToDataType(), Field(DecimalField<Decimal128>(123, 0)))));
ASSERT_TRUE(DataTypeFactory::instance().get("Decimal(19,2)")->equals(*applyVisitor(FieldToDataType(), Field(DecimalField<Decimal128>(123, 2)))));
ASSERT_TRUE(DataTypeFactory::instance().get("Decimal(21,4)")->equals(*applyVisitor(FieldToDataType(), Field(DecimalField<Decimal128>(Int128(123123123123123ll) * 1000000, 4)))));

/// Decimal256
ASSERT_TRUE(DataTypeFactory::instance().get("Decimal(39,0)")->equals(*applyVisitor(FieldToDataType(), Field(DecimalField<Decimal256>(Int256(0), 0)))));
ASSERT_TRUE(DataTypeFactory::instance().get("Decimal(39,1)")->equals(*applyVisitor(FieldToDataType(), Field(DecimalField<Decimal256>(Int256(0), 1)))));
ASSERT_TRUE(DataTypeFactory::instance().get("Decimal(39,2)")->equals(*applyVisitor(FieldToDataType(), Field(DecimalField<Decimal256>(Int256(0), 2)))));
ASSERT_TRUE(DataTypeFactory::instance().get("Decimal(39,0)")->equals(*applyVisitor(FieldToDataType(), Field(DecimalField<Decimal256>(Int256(123), 0)))));
ASSERT_TRUE(DataTypeFactory::instance().get("Decimal(39,2)")->equals(*applyVisitor(FieldToDataType(), Field(DecimalField<Decimal256>(Int256(123), 2)))));
ASSERT_TRUE(DataTypeFactory::instance().get("Decimal(41,4)")->equals(*applyVisitor(FieldToDataType(), Field(DecimalField<Decimal256>(Int256(123123123123123123ll) * Int256(1000000000ll) * Int256(100000000000000ll), 4)))));
}
CATCH
} // namespace tests
} // namespace DB
2 changes: 1 addition & 1 deletion dbms/src/Databases/test/gtest_database.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -25,11 +25,11 @@
#include <Storages/IManageableStorage.h>
#include <Storages/IStorage.h>
#include <Storages/MutableSupport.h>
#include <Storages/Transaction/SchemaNameMapper.h>
#include <Storages/Transaction/TMTContext.h>
#include <Storages/Transaction/TMTStorages.h>
#include <Storages/registerStorages.h>
#include <TestUtils/TiFlashTestBasic.h>
#include <TiDB/Schema/SchemaNameMapper.h>
#include <common/ThreadPool.h>
#include <common/logger_useful.h>

Expand Down
3 changes: 1 addition & 2 deletions dbms/src/Debug/MockSchemaGetter.h
Original file line number Diff line number Diff line change
Expand Up @@ -15,14 +15,13 @@
#pragma once

#include <Debug/MockTiDB.h>
#include <Storages/Transaction/SchemaGetter.h>
#include <TiDB/Schema/SchemaGetter.h>

namespace DB
{

struct MockSchemaGetter
{

TiDB::DBInfoPtr getDatabase(DatabaseID db_id) { return MockTiDB::instance().getDBInfoByID(db_id); }

Int64 getVersion() { return MockTiDB::instance().getVersion(); }
Expand Down
2 changes: 1 addition & 1 deletion dbms/src/Debug/MockSchemaNameMapper.h
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,7 @@

#pragma once

#include <Storages/Transaction/SchemaNameMapper.h>
#include <TiDB/Schema/SchemaNameMapper.h>

namespace DB
{
Expand Down
4 changes: 2 additions & 2 deletions dbms/src/Debug/MockTiDB.h
Original file line number Diff line number Diff line change
Expand Up @@ -15,10 +15,10 @@
#pragma once

#include <Storages/ColumnsDescription.h>
#include <Storages/Transaction/SchemaGetter.h>
#include <Storages/Transaction/SchemaSyncer.h>
#include <Storages/Transaction/TiDB.h>
#include <Storages/Transaction/Types.h>
#include <TiDB/Schema/SchemaGetter.h>
#include <TiDB/Schema/SchemaSyncer.h>

#include <atomic>

Expand Down
20 changes: 10 additions & 10 deletions dbms/src/Debug/dbgFuncCoprocessor.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -336,7 +336,7 @@ void dbgFuncTiDBQueryFromNaturalDag(Context & context, const ASTs & args, DBGInv
if (args.size() != 1)
throw Exception("Args not matched, should be: json_dag_path", ErrorCodes::BAD_ARGUMENTS);

String json_dag_path = safeGet<String>(typeid_cast<const ASTLiteral &>(*args[0]).value);
auto json_dag_path = safeGet<String>(typeid_cast<const ASTLiteral &>(*args[0]).value);
auto dag = NaturalDag(json_dag_path, &Poco::Logger::get("MockDAG"));
dag.init();
dag.build(context);
Expand Down Expand Up @@ -431,7 +431,7 @@ BlockInputStreamPtr dbgFuncTiDBQuery(Context & context, const ASTs & args)
if (args.empty() || args.size() > 3)
throw Exception("Args not matched, should be: query[, region-id, dag_prop_string]", ErrorCodes::BAD_ARGUMENTS);

String query = safeGet<String>(typeid_cast<const ASTLiteral &>(*args[0]).value);
auto query = safeGet<String>(typeid_cast<const ASTLiteral &>(*args[0]).value);
RegionID region_id = InvalidRegionID;
if (args.size() >= 2)
region_id = safeGet<RegionID>(typeid_cast<const ASTLiteral &>(*args[1]).value);
Expand Down Expand Up @@ -464,8 +464,8 @@ BlockInputStreamPtr dbgFuncMockTiDBQuery(Context & context, const ASTs & args)
if (args.size() < 2 || args.size() > 4)
throw Exception("Args not matched, should be: query, region-id[, start-ts, dag_prop_string]", ErrorCodes::BAD_ARGUMENTS);

String query = safeGet<String>(typeid_cast<const ASTLiteral &>(*args[0]).value);
RegionID region_id = safeGet<RegionID>(typeid_cast<const ASTLiteral &>(*args[1]).value);
auto query = safeGet<String>(typeid_cast<const ASTLiteral &>(*args[0]).value);
auto region_id = safeGet<RegionID>(typeid_cast<const ASTLiteral &>(*args[1]).value);
Timestamp start_ts = DEFAULT_MAX_READ_TSO;
if (args.size() >= 3)
start_ts = safeGet<Timestamp>(typeid_cast<const ASTLiteral &>(*args[2]).value);
Expand Down Expand Up @@ -671,14 +671,14 @@ const ASTTablesInSelectQueryElement * getJoin(ASTSelectQuery & ast_query)
if (!ast_query.tables)
return nullptr;

const ASTTablesInSelectQuery & tables_in_select_query = static_cast<const ASTTablesInSelectQuery &>(*ast_query.tables);
const auto & tables_in_select_query = static_cast<const ASTTablesInSelectQuery &>(*ast_query.tables);
if (tables_in_select_query.children.empty())
return nullptr;

const ASTTablesInSelectQueryElement * joined_table = nullptr;
for (const auto & child : tables_in_select_query.children)
{
const ASTTablesInSelectQueryElement & tables_element = static_cast<const ASTTablesInSelectQueryElement &>(*child);
const auto & tables_element = static_cast<const ASTTablesInSelectQueryElement &>(*child);
if (tables_element.table_join)
{
if (!joined_table)
Expand Down Expand Up @@ -737,7 +737,7 @@ std::pair<ExecutorPtr, bool> compileQueryBlock(
bool append_pk_column = false;
for (const auto & expr : ast_query.select_expression_list->children)
{
if (ASTIdentifier * identifier = typeid_cast<ASTIdentifier *>(expr.get()))
if (auto * identifier = typeid_cast<ASTIdentifier *>(expr.get()))
{
if (identifier->getColumnName() == MutableSupport::tidb_pk_column_name)
{
Expand All @@ -756,7 +756,7 @@ std::pair<ExecutorPtr, bool> compileQueryBlock(
String right_table_alias;
{
String database_name, table_name;
const ASTTableExpression & table_to_join = static_cast<const ASTTableExpression &>(*joined_table->table_expression);
const auto & table_to_join = static_cast<const ASTTableExpression &>(*joined_table->table_expression);
if (table_to_join.database_and_table_name)
{
auto identifier = static_cast<const ASTIdentifier &>(*table_to_join.database_and_table_name);
Expand Down Expand Up @@ -788,7 +788,7 @@ std::pair<ExecutorPtr, bool> compileQueryBlock(
bool right_append_pk_column = false;
for (const auto & expr : ast_query.select_expression_list->children)
{
if (ASTIdentifier * identifier = typeid_cast<ASTIdentifier *>(expr.get()))
if (auto * identifier = typeid_cast<ASTIdentifier *>(expr.get()))
{
auto names = splitQualifiedName(identifier->getColumnName());
if (names.second == MutableSupport::tidb_pk_column_name)
Expand Down Expand Up @@ -831,7 +831,7 @@ std::pair<ExecutorPtr, bool> compileQueryBlock(
bool has_agg_func = false;
for (const auto & child : ast_query.select_expression_list->children)
{
const ASTFunction * func = typeid_cast<const ASTFunction *>(child.get());
const auto * func = typeid_cast<const ASTFunction *>(child.get());
if (func && AggregateFunctionFactory::instance().isAggregateFunctionName(func->name))
{
has_agg_func = true;
Expand Down
2 changes: 1 addition & 1 deletion dbms/src/Debug/dbgFuncMockTiDBTable.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -24,8 +24,8 @@
#include <Parsers/ParserRenameQuery.h>
#include <Parsers/parseQuery.h>
#include <Storages/Transaction/KVStore.h>
#include <Storages/Transaction/SchemaSyncer.h>
#include <Storages/Transaction/TMTContext.h>
#include <TiDB/Schema/SchemaSyncer.h>
#include <fmt/core.h>

namespace DB
Expand Down
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
4 changes: 2 additions & 2 deletions dbms/src/Debug/dbgFuncSchema.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -22,10 +22,10 @@
#include <Parsers/ASTLiteral.h>
#include <Parsers/ParserCreateQuery.h>
#include <Storages/IManageableStorage.h>
#include <Storages/Transaction/SchemaSyncService.h>
#include <Storages/Transaction/SchemaSyncer.h>
#include <Storages/Transaction/TMTContext.h>
#include <Storages/Transaction/TiDB.h>
#include <TiDB/Schema/SchemaSyncService.h>
#include <TiDB/Schema/SchemaSyncer.h>
#include <fmt/core.h>

#include <ext/singleton.h>
Expand Down
Loading

0 comments on commit ab85dd9

Please sign in to comment.