Skip to content

Commit

Permalink
Merge branch 'master' into mpmsqueue-trypop
Browse files Browse the repository at this point in the history
  • Loading branch information
gengliqi authored Jul 7, 2022
2 parents fc8a972 + 47657d3 commit 9098b94
Show file tree
Hide file tree
Showing 16 changed files with 588 additions and 48 deletions.
3 changes: 2 additions & 1 deletion dbms/src/Columns/ColumnConst.h
Original file line number Diff line number Diff line change
Expand Up @@ -233,7 +233,8 @@ class ColumnConst final : public COWPtrHelper<IColumn, ColumnConst>
template <typename T>
T getValue() const
{
return getField().safeGet<typename NearestFieldType<T>::Type>();
auto && tmp = getField();
return std::move(tmp.safeGet<typename NearestFieldType<T>::Type>());
}
};

Expand Down
2 changes: 2 additions & 0 deletions dbms/src/Debug/DBGInvoker.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -120,6 +120,8 @@ DBGInvoker::DBGInvoker()
regSchemafulFunc("query_mapped", dbgFuncQueryMapped);
regSchemalessFunc("get_tiflash_replica_count", dbgFuncGetTiflashReplicaCount);
regSchemalessFunc("get_partition_tables_tiflash_replica_count", dbgFuncGetPartitionTablesTiflashReplicaCount);
regSchemalessFunc("get_tiflash_mode", dbgFuncGetTiflashMode);
regSchemalessFunc("get_partition_tables_tiflash_mode", dbgFuncGetPartitionTablesTiflashMode);

regSchemalessFunc("search_log_for_key", dbgFuncSearchLogForKey);
regSchemalessFunc("tidb_dag", dbgFuncTiDBQueryFromNaturalDag);
Expand Down
52 changes: 52 additions & 0 deletions dbms/src/Debug/dbgFuncSchemaName.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -181,4 +181,56 @@ void dbgFuncGetPartitionTablesTiflashReplicaCount(Context & context, const ASTs
output(fmt_buf.toString());
}

void dbgFuncGetTiflashMode(Context & context, const ASTs & args, DBGInvoker::Printer output)
{
if (args.empty() || args.size() != 2)
throw Exception("Args not matched, should be: database-name[, table-name]", ErrorCodes::BAD_ARGUMENTS);

const String & database_name = typeid_cast<const ASTIdentifier &>(*args[0]).name;
FmtBuffer fmt_buf;

const String & table_name = typeid_cast<const ASTIdentifier &>(*args[1]).name;
auto mapped = mappedTable(context, database_name, table_name);
auto storage = context.getTable(mapped->first, mapped->second);
auto managed_storage = std::dynamic_pointer_cast<IManageableStorage>(storage);
if (!managed_storage)
throw Exception(database_name + "." + table_name + " is not ManageableStorage", ErrorCodes::BAD_ARGUMENTS);

fmt_buf.append((TiFlashModeToString(managed_storage->getTableInfo().tiflash_mode)));

output(fmt_buf.toString());
}

void dbgFuncGetPartitionTablesTiflashMode(Context & context, const ASTs & args, DBGInvoker::Printer output)
{
if (args.empty() || args.size() != 2)
throw Exception("Args not matched, should be: database-name[, table-name]", ErrorCodes::BAD_ARGUMENTS);

const String & database_name = typeid_cast<const ASTIdentifier &>(*args[0]).name;
FmtBuffer fmt_buf;

const String & table_name = typeid_cast<const ASTIdentifier &>(*args[1]).name;
auto mapped = mappedTable(context, database_name, table_name);
auto storage = context.getTable(mapped->first, mapped->second);
auto managed_storage = std::dynamic_pointer_cast<IManageableStorage>(storage);
if (!managed_storage)
throw Exception(database_name + "." + table_name + " is not ManageableStorage", ErrorCodes::BAD_ARGUMENTS);

auto table_info = managed_storage->getTableInfo();

if (!table_info.isLogicalPartitionTable())
throw Exception(database_name + "." + table_name + " is not logical partition table", ErrorCodes::BAD_ARGUMENTS);

SchemaNameMapper name_mapper;
for (const auto & part_def : table_info.partition.definitions)
{
auto paritition_table_info = table_info.producePartitionTableInfo(part_def.id, name_mapper);
auto partition_storage = context.getTMTContext().getStorages().get(paritition_table_info->id);
fmt_buf.append((TiFlashModeToString(partition_storage->getTableInfo().tiflash_mode)));
fmt_buf.append("/");
}

output(fmt_buf.toString());
}

} // namespace DB
10 changes: 10 additions & 0 deletions dbms/src/Debug/dbgFuncSchemaName.h
Original file line number Diff line number Diff line change
Expand Up @@ -50,4 +50,14 @@ void dbgFuncGetTiflashReplicaCount(Context & context, const ASTs & args, DBGInvo
// ./storage-client.sh "DBGInvoke get_partition_tables_tiflash_replica_count(db_name, table_name)"
void dbgFuncGetPartitionTablesTiflashReplicaCount(Context & context, const ASTs & args, DBGInvoker::Printer output);

// Get table's tiflash mode with mapped table name
// Usage:
// ./storage-client.sh "DBGInvoke get_tiflash_mode(db_name, table_name)"
void dbgFuncGetTiflashMode(Context & context, const ASTs & args, DBGInvoker::Printer output);

// Get the logical table's partition tables' tiflash replica counts with mapped table name
// Usage:
// ./storage-client.sh "DBGInvoke get_partition_tables_tiflash_mode(db_name, table_name)"
void dbgFuncGetPartitionTablesTiflashMode(Context & context, const ASTs & args, DBGInvoker::Printer output);

} // namespace DB
210 changes: 210 additions & 0 deletions dbms/src/Functions/CollationOperatorOptimized.h
Original file line number Diff line number Diff line change
@@ -0,0 +1,210 @@
// 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.

#pragma once

#include <Columns/ColumnString.h>
#include <Core/AccurateComparison.h>
#include <Functions/StringUtil.h>
#include <common/StringRef.h>
#include <common/defines.h>

#include <cstddef>
#include <string_view>


namespace DB
{

template <typename T>
ALWAYS_INLINE inline int signum(T val)
{
return (0 < val) - (val < 0);
}

// Check equality is much faster than other comparison.
// - check size first
// - return 0 if equal else 1
__attribute__((flatten, always_inline, pure)) inline uint8_t RawStrEqualCompare(const std::string_view & lhs, const std::string_view & rhs)
{
return StringRef(lhs) == StringRef(rhs) ? 0 : 1;
}

// Compare str view by memcmp
__attribute__((flatten, always_inline, pure)) inline int RawStrCompare(const std::string_view & v1, const std::string_view & v2)
{
return signum(v1.compare(v2));
}

constexpr char SPACE = ' ';

// Remove tail space
__attribute__((flatten, always_inline, pure)) inline std::string_view RightTrim(const std::string_view & v)
{
if (likely(v.empty() || v.back() != SPACE))
return v;
size_t end = v.find_last_not_of(SPACE);
return end == std::string_view::npos ? std::string_view{} : std::string_view(v.data(), end + 1);
}

__attribute__((flatten, always_inline, pure)) inline int RtrimStrCompare(const std::string_view & va, const std::string_view & vb)
{
return RawStrCompare(RightTrim(va), RightTrim(vb));
}

// If true, only need to check equal or not.
template <typename T>
struct IsEqualRelated
{
static constexpr const bool value = false;
};

// For `EqualsOp` and `NotEqualsOp`, value is true.
template <typename... A>
struct IsEqualRelated<DB::EqualsOp<A...>>
{
static constexpr const bool value = true;
};
template <typename... A>
struct IsEqualRelated<DB::NotEqualsOp<A...>>
{
static constexpr const bool value = true;
};

// Loop columns and invoke callback for each pair.
template <typename F>
__attribute__((flatten, always_inline)) inline void LoopTwoColumns(
const ColumnString::Chars_t & a_data,
const ColumnString::Offsets & a_offsets,
const ColumnString::Chars_t & b_data,
const ColumnString::Offsets & b_offsets,
size_t size,
F && func)
{
for (size_t i = 0; i < size; ++i)
{
size_t a_size = StringUtil::sizeAt(a_offsets, i) - 1;
size_t b_size = StringUtil::sizeAt(b_offsets, i) - 1;
const auto * a_ptr = reinterpret_cast<const char *>(&a_data[StringUtil::offsetAt(a_offsets, i)]);
const auto * b_ptr = reinterpret_cast<const char *>(&b_data[StringUtil::offsetAt(b_offsets, i)]);

func({a_ptr, a_size}, {b_ptr, b_size}, i);
}
}

// Loop one column and invoke callback for each pair.
template <typename F>
__attribute__((flatten, always_inline)) inline void LoopOneColumn(
const ColumnString::Chars_t & a_data,
const ColumnString::Offsets & a_offsets,
size_t size,
F && func)
{
for (size_t i = 0; i < size; ++i)
{
size_t a_size = StringUtil::sizeAt(a_offsets, i) - 1;
const auto * a_ptr = reinterpret_cast<const char *>(&a_data[StringUtil::offsetAt(a_offsets, i)]);

func({a_ptr, a_size}, i);
}
}

// Handle str-column compare str-column.
// - Optimize UTF8_BIN and UTF8MB4_BIN
// - Check if columns do NOT contain tail space
// - If Op is `EqualsOp` or `NotEqualsOp`, optimize comparison by faster way
template <typename Op, typename Result>
ALWAYS_INLINE inline bool StringVectorStringVector(
const ColumnString::Chars_t & a_data,
const ColumnString::Offsets & a_offsets,
const ColumnString::Chars_t & b_data,
const ColumnString::Offsets & b_offsets,
const TiDB::TiDBCollatorPtr & collator,
Result & c)
{
bool use_optimized_path = false;

switch (collator->getCollatorId())
{
case TiDB::ITiDBCollator::UTF8MB4_BIN:
case TiDB::ITiDBCollator::UTF8_BIN:
{
size_t size = a_offsets.size();

LoopTwoColumns(a_data, a_offsets, b_data, b_offsets, size, [&c](const std::string_view & va, const std::string_view & vb, size_t i) {
if constexpr (IsEqualRelated<Op>::value)
{
c[i] = Op::apply(RawStrEqualCompare(RightTrim(va), RightTrim(vb)), 0);
}
else
{
c[i] = Op::apply(RtrimStrCompare(va, vb), 0);
}
});

use_optimized_path = true;

break;
}
default:
break;
}
return use_optimized_path;
}

// Handle str-column compare const-str.
// - Optimize UTF8_BIN and UTF8MB4_BIN
// - Right trim const-str first
// - Check if column does NOT contain tail space
// - If Op is `EqualsOp` or `NotEqualsOp`, optimize comparison by faster way
template <typename Op, typename Result>
ALWAYS_INLINE inline bool StringVectorConstant(
const ColumnString::Chars_t & a_data,
const ColumnString::Offsets & a_offsets,
const std::string_view & b,
const TiDB::TiDBCollatorPtr & collator,
Result & c)
{
bool use_optimized_path = false;

switch (collator->getCollatorId())
{
case TiDB::ITiDBCollator::UTF8MB4_BIN:
case TiDB::ITiDBCollator::UTF8_BIN:
{
size_t size = a_offsets.size();

std::string_view tar_str_view = RightTrim(b); // right trim const-str first

LoopOneColumn(a_data, a_offsets, size, [&c, &tar_str_view](const std::string_view & view, size_t i) {
if constexpr (IsEqualRelated<Op>::value)
{
c[i] = Op::apply(RawStrEqualCompare(RightTrim(view), tar_str_view), 0);
}
else
{
c[i] = Op::apply(RawStrCompare(RightTrim(view), tar_str_view), 0);
}
});

use_optimized_path = true;
break;
}
default:
break;
}
return use_optimized_path;
}

} // namespace DB
Loading

0 comments on commit 9098b94

Please sign in to comment.