Merge branch 'Avogar-patch-1' of github.com:ClickHouse/ClickHouse into Avogar-patch-1

This commit is contained in:
avogar 2022-07-21 08:58:13 +00:00
commit 076d1c0bb1
19 changed files with 159 additions and 81 deletions

View File

@ -1,3 +1,19 @@
if (_CLICKHOUSE_TOOLCHAIN_FILE_LOADED)
# During first run of cmake the toolchain file will be loaded twice,
# - /usr/share/cmake-3.23/Modules/CMakeDetermineSystem.cmake
# - /bld/CMakeFiles/3.23.2/CMakeSystem.cmake
#
# But once you already have non-empty cmake cache it will be loaded only
# once:
# - /bld/CMakeFiles/3.23.2/CMakeSystem.cmake
#
# This has no harm except for double load of toolchain will add
# --gcc-toolchain multiple times that will not allow ccache to reuse the
# cache.
return()
endif()
set (_CLICKHOUSE_TOOLCHAIN_FILE_LOADED ON)
set (CMAKE_TRY_COMPILE_TARGET_TYPE STATIC_LIBRARY)
set (CMAKE_SYSTEM_NAME "Linux")

View File

@ -45,7 +45,7 @@ void LRUFileCache::initialize()
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
return;
throw;
}
}
else
@ -841,7 +841,11 @@ void LRUFileCache::loadCacheInfoIntoMemory(std::lock_guard<std::mutex> & cache_l
/// cache_base_path / key_prefix / key / offset
if (!files.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cache already initialized");
throw Exception(
ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR,
"Cache initialization is partially made. "
"This can be a result of a failed first attempt to initialize cache. "
"Please, check log for error messages");
fs::directory_iterator key_prefix_it{cache_base_path};
for (; key_prefix_it != fs::directory_iterator(); ++key_prefix_it)

View File

@ -342,6 +342,23 @@ OptimizedRegularExpressionImpl<thread_safe>::OptimizedRegularExpressionImpl(cons
}
}
template <bool thread_safe>
OptimizedRegularExpressionImpl<thread_safe>::OptimizedRegularExpressionImpl(OptimizedRegularExpressionImpl && rhs) noexcept
: is_trivial(rhs.is_trivial)
, required_substring_is_prefix(rhs.required_substring_is_prefix)
, is_case_insensitive(rhs.is_case_insensitive)
, required_substring(std::move(rhs.required_substring))
, re2(std::move(rhs.re2))
, number_of_subpatterns(rhs.number_of_subpatterns)
{
if (!required_substring.empty())
{
if (is_case_insensitive)
case_insensitive_substring_searcher.emplace(required_substring.data(), required_substring.size());
else
case_sensitive_substring_searcher.emplace(required_substring.data(), required_substring.size());
}
}
template <bool thread_safe>
bool OptimizedRegularExpressionImpl<thread_safe>::match(const char * subject, size_t subject_size) const

View File

@ -56,6 +56,9 @@ public:
using StringPieceType = std::conditional_t<thread_safe, re2::StringPiece, re2_st::StringPiece>;
OptimizedRegularExpressionImpl(const std::string & regexp_, int options = 0); /// NOLINT
/// StringSearcher store pointers to required_substring, it must be updated on move.
OptimizedRegularExpressionImpl(OptimizedRegularExpressionImpl && rhs) noexcept;
OptimizedRegularExpressionImpl(const OptimizedRegularExpressionImpl & rhs) = delete;
bool match(const std::string & subject) const
{

View File

@ -5,6 +5,7 @@
#include <Common/typeid_cast.h>
#include <DataTypes/IDataType.h>
#include <DataTypes/DataTypeDecimalBase.h>
#include <DataTypes/DataTypeDateTime64.h>
namespace DB
@ -13,6 +14,7 @@ namespace DB
namespace ErrorCodes
{
extern const int DECIMAL_OVERFLOW;
extern const int LOGICAL_ERROR;
}
/// Implements Decimal(P, S), where P is precision, S is scale.
@ -58,7 +60,7 @@ inline const DataTypeDecimal<T> * checkDecimal(const IDataType & data_type)
return typeid_cast<const DataTypeDecimal<T> *>(&data_type);
}
inline UInt32 getDecimalScale(const IDataType & data_type, UInt32 default_value = std::numeric_limits<UInt32>::max())
inline UInt32 getDecimalScale(const IDataType & data_type)
{
if (const auto * decimal_type = checkDecimal<Decimal32>(data_type))
return decimal_type->getScale();
@ -68,7 +70,10 @@ inline UInt32 getDecimalScale(const IDataType & data_type, UInt32 default_value
return decimal_type->getScale();
if (const auto * decimal_type = checkDecimal<Decimal256>(data_type))
return decimal_type->getScale();
return default_value;
if (const auto * date_time_type = typeid_cast<const DataTypeDateTime64 *>(&data_type))
return date_time_type->getScale();
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot get decimal scale from type {}", data_type.getName());
}
inline UInt32 getDecimalPrecision(const IDataType & data_type)
@ -81,7 +86,10 @@ inline UInt32 getDecimalPrecision(const IDataType & data_type)
return decimal_type->getPrecision();
if (const auto * decimal_type = checkDecimal<Decimal256>(data_type))
return decimal_type->getPrecision();
return 0;
if (const auto * date_time_type = typeid_cast<const DataTypeDateTime64 *>(&data_type))
return date_time_type->getPrecision();
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot get decimal precision from type {}", data_type.getName());
}
template <typename T>

View File

@ -554,7 +554,11 @@ DataTypePtr getLeastSupertype(const DataTypes & types)
UInt32 max_scale = 0;
for (const auto & type : types)
{
UInt32 scale = getDecimalScale(*type, 0);
auto type_id = type->getTypeId();
if (type_id != TypeIndex::Decimal32 && type_id != TypeIndex::Decimal64 && type_id != TypeIndex::Decimal128)
continue;
UInt32 scale = getDecimalScale(*type);
if (scale > max_scale)
max_scale = scale;
}

View File

@ -23,20 +23,6 @@ ActionLocksManager::ActionLocksManager(ContextPtr context_) : WithContext(contex
{
}
template <typename F>
inline void forEachTable(F && f, ContextPtr context)
{
for (auto & elem : DatabaseCatalog::instance().getDatabases())
for (auto iterator = elem.second->getTablesIterator(context); iterator->isValid(); iterator->next())
if (auto table = iterator->table())
f(table);
}
void ActionLocksManager::add(StorageActionBlockType action_type, ContextPtr context_)
{
forEachTable([&](const StoragePtr & table) { add(table, action_type); }, context_);
}
void ActionLocksManager::add(const StorageID & table_id, StorageActionBlockType action_type)
{
if (auto table = DatabaseCatalog::instance().tryGetTable(table_id, getContext()))
@ -54,14 +40,6 @@ void ActionLocksManager::add(const StoragePtr & table, StorageActionBlockType ac
}
}
void ActionLocksManager::remove(StorageActionBlockType action_type)
{
std::lock_guard lock(mutex);
for (auto & storage_elem : storage_locks)
storage_elem.second.erase(action_type);
}
void ActionLocksManager::remove(const StorageID & table_id, StorageActionBlockType action_type)
{
if (auto table = DatabaseCatalog::instance().tryGetTable(table_id, getContext()))

View File

@ -20,14 +20,10 @@ class ActionLocksManager : WithContext
public:
explicit ActionLocksManager(ContextPtr context);
/// Adds new locks for each table
void add(StorageActionBlockType action_type, ContextPtr context);
/// Add new lock for a table if it has not been already added
void add(const StorageID & table_id, StorageActionBlockType action_type);
void add(const StoragePtr & table, StorageActionBlockType action_type);
/// Remove locks for all tables
void remove(StorageActionBlockType action_type);
/// Removes a lock for a table if it exists
void remove(const StorageID & table_id, StorageActionBlockType action_type);
void remove(const StoragePtr & table, StorageActionBlockType action_type);

View File

@ -146,14 +146,14 @@ namespace
struct QueryASTSettings
{
bool graph = false;
bool rewrite = false;
bool optimize = false;
constexpr static char name[] = "AST";
std::unordered_map<std::string, std::reference_wrapper<bool>> boolean_settings =
{
{"graph", graph},
{"rewrite", rewrite}
{"optimize", optimize}
};
};
@ -280,7 +280,7 @@ QueryPipeline InterpreterExplainQuery::executeImpl()
case ASTExplainQuery::ParsedAST:
{
auto settings = checkAndGetSettings<QueryASTSettings>(ast.getSettings());
if (settings.rewrite)
if (settings.optimize)
{
ExplainAnalyzedSyntaxVisitor::Data data(getContext());
ExplainAnalyzedSyntaxVisitor(data).visit(query);

View File

@ -30,6 +30,7 @@ namespace DB
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int INCORRECT_DATA;
}
CapnProtoRowInputFormat::CapnProtoRowInputFormat(ReadBuffer & in_, Block header, Params params_, const FormatSchemaInfo & info, const FormatSettings & format_settings_)
@ -264,20 +265,20 @@ bool CapnProtoRowInputFormat::readRow(MutableColumns & columns, RowReadExtension
if (in->eof())
return false;
auto array = readMessage();
#if CAPNP_VERSION >= 7000 && CAPNP_VERSION < 8000
capnp::UnalignedFlatArrayMessageReader msg(array);
#else
capnp::FlatArrayMessageReader msg(array);
#endif
auto root_reader = msg.getRoot<capnp::DynamicStruct>(root);
for (size_t i = 0; i != columns.size(); ++i)
try
{
auto value = getReaderByColumnName(root_reader, column_names[i]);
insertValue(*columns[i], column_types[i], value, format_settings.capn_proto.enum_comparing_mode);
auto array = readMessage();
capnp::FlatArrayMessageReader msg(array);
auto root_reader = msg.getRoot<capnp::DynamicStruct>(root);
for (size_t i = 0; i != columns.size(); ++i)
{
auto value = getReaderByColumnName(root_reader, column_names[i]);
insertValue(*columns[i], column_types[i], value, format_settings.capn_proto.enum_comparing_mode);
}
}
catch (const kj::Exception & e)
{
throw Exception(ErrorCodes::INCORRECT_DATA, "Cannot read row: {}", e.getDescription().cStr());
}
return true;

View File

@ -124,7 +124,7 @@ void MergeTreeBackgroundExecutor<Queue>::routine(TaskRuntimeDataPtr item)
/// All operations with queues are considered no to do any allocations
auto erase_from_active = [this, item]() TSA_REQUIRES(mutex)
auto erase_from_active = [this, &item]() TSA_REQUIRES(mutex)
{
active.erase(std::remove(active.begin(), active.end(), item), active.end());
};
@ -157,11 +157,10 @@ void MergeTreeBackgroundExecutor<Queue>::routine(TaskRuntimeDataPtr item)
if (need_execute_again)
{
std::lock_guard guard(mutex);
erase_from_active();
if (item->is_currently_deleting)
{
erase_from_active();
/// This is significant to order the destructors.
{
NOEXCEPT_SCOPE({
@ -179,7 +178,6 @@ void MergeTreeBackgroundExecutor<Queue>::routine(TaskRuntimeDataPtr item)
/// Otherwise the destruction of the task won't be ordered with the destruction of the
/// storage.
pending.push(std::move(item));
erase_from_active();
has_tasks.notify_one();
item = nullptr;
return;

View File

@ -0,0 +1,53 @@
-- { echoOn }
EXPLAIN AST optimize=0 SELECT * FROM numbers(0);
SelectWithUnionQuery (children 1)
ExpressionList (children 1)
SelectQuery (children 2)
ExpressionList (children 1)
Asterisk
TablesInSelectQuery (children 1)
TablesInSelectQueryElement (children 1)
TableExpression (children 1)
Function numbers (children 1)
ExpressionList (children 1)
Literal UInt64_0
EXPLAIN AST optimize=1 SELECT * FROM numbers(0);
SelectWithUnionQuery (children 1)
ExpressionList (children 1)
SelectQuery (children 2)
ExpressionList (children 1)
Identifier number
TablesInSelectQuery (children 1)
TablesInSelectQueryElement (children 1)
TableExpression (children 1)
Function numbers (children 1)
ExpressionList (children 1)
Literal UInt64_0
EXPLAIN AST optimize=0 SELECT countDistinct(number) FROM numbers(0);
SelectWithUnionQuery (children 1)
ExpressionList (children 1)
SelectQuery (children 2)
ExpressionList (children 1)
Function countDistinct (children 1)
ExpressionList (children 1)
Identifier number
TablesInSelectQuery (children 1)
TablesInSelectQueryElement (children 1)
TableExpression (children 1)
Function numbers (children 1)
ExpressionList (children 1)
Literal UInt64_0
EXPLAIN AST optimize=1 SELECT countDistinct(number) FROM numbers(0);
SelectWithUnionQuery (children 1)
ExpressionList (children 1)
SelectQuery (children 2)
ExpressionList (children 1)
Function uniqExact (children 1)
ExpressionList (children 1)
Identifier number
TablesInSelectQuery (children 1)
TablesInSelectQueryElement (children 1)
TableExpression (children 1)
Function numbers (children 1)
ExpressionList (children 1)
Literal UInt64_0

View File

@ -0,0 +1,6 @@
-- { echoOn }
EXPLAIN AST optimize=0 SELECT * FROM numbers(0);
EXPLAIN AST optimize=1 SELECT * FROM numbers(0);
EXPLAIN AST optimize=0 SELECT countDistinct(number) FROM numbers(0);
EXPLAIN AST optimize=1 SELECT countDistinct(number) FROM numbers(0);
-- { echoOff }

View File

@ -1,25 +0,0 @@
-- { echoOn }
EXPLAIN AST rewrite=0 SELECT * FROM numbers(0);
SelectWithUnionQuery (children 1)
ExpressionList (children 1)
SelectQuery (children 2)
ExpressionList (children 1)
Asterisk
TablesInSelectQuery (children 1)
TablesInSelectQueryElement (children 1)
TableExpression (children 1)
Function numbers (children 1)
ExpressionList (children 1)
Literal UInt64_0
EXPLAIN AST rewrite=1 SELECT * FROM numbers(0);
SelectWithUnionQuery (children 1)
ExpressionList (children 1)
SelectQuery (children 2)
ExpressionList (children 1)
Identifier number
TablesInSelectQuery (children 1)
TablesInSelectQueryElement (children 1)
TableExpression (children 1)
Function numbers (children 1)
ExpressionList (children 1)
Literal UInt64_0

View File

@ -1,4 +0,0 @@
-- { echoOn }
EXPLAIN AST rewrite=0 SELECT * FROM numbers(0);
EXPLAIN AST rewrite=1 SELECT * FROM numbers(0);
-- { echoOff }

View File

@ -0,0 +1,2 @@
2022-01-24 02:30:00.008122000
1

View File

@ -0,0 +1,15 @@
create table dat (blockNum Decimal(10,0), eventTimestamp DateTime64(9)) Engine=MergeTree() primary key eventTimestamp;
insert into dat values (1, '2022-01-24 02:30:00.008122000');
CREATE DICTIONARY datDictionary
(
`blockNum` Decimal(10, 0),
`eventTimestamp` DateTime64(9)
)
PRIMARY KEY blockNum
SOURCE(CLICKHOUSE(TABLE 'dat'))
LIFETIME(MIN 0 MAX 1000)
LAYOUT(FLAT());
select (select eventTimestamp from datDictionary);
select count(*) from dat where eventTimestamp >= (select eventTimestamp from datDictionary);

View File

@ -0,0 +1 @@
{"a":"1","b":"2","c":"","d":"4"}{"a":"1","b":"2","c":"","d":"4"}{"a":"1","b":"2","c":"","d":"4"}{"a":"1","b":"2","c":"","d":"4"} ['a','b','c','d','a','b','c','d','a','b','c','d','a','b','c','d'] [':"',':"',':"',':"',':"',':"',':"',':"',':"',':"',':"',':"',':"',':"',':"',':"']

View File

@ -0,0 +1,5 @@
-- Regression for UB (stack-use-after-scope) in extactAll()
SELECT
'{"a":"1","b":"2","c":"","d":"4"}{"a":"1","b":"2","c":"","d":"4"}{"a":"1","b":"2","c":"","d":"4"}{"a":"1","b":"2","c":"","d":"4"}' AS json,
extractAll(json, '"([^"]*)":') AS keys,
extractAll(json, ':"\0[^"]*)"') AS values;