Merge branch 'master' of github.com:yandex/ClickHouse

This commit is contained in:
Alexey Milovidov 2019-08-05 01:28:26 +03:00
commit cc2744ebb9
29 changed files with 449 additions and 257 deletions

View File

@ -273,17 +273,13 @@ int Server::main(const std::vector<std::string> & /*args*/)
* table engines could use Context on destroy.
*/
LOG_INFO(log, "Shutting down storages.");
if (text_log)
text_log->shutdown();
global_context->shutdown();
LOG_DEBUG(log, "Shutted down storages.");
/** Explicitly destroy Context. It is more convenient than in destructor of Server, because logger is still available.
* At this moment, no one could own shared part of Context.
*/
text_log.reset();
global_context.reset();
LOG_DEBUG(log, "Destroyed global context.");
});
@ -413,7 +409,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
main_config_zk_changed_event,
[&](ConfigurationPtr config)
{
setTextLog(text_log);
setTextLog(global_context->getTextLog());
buildLoggers(*config, logger());
global_context->setClustersConfig(config);
global_context->setMacros(std::make_unique<Macros>(*config, "macros"));
@ -500,14 +496,11 @@ int Server::main(const std::vector<std::string> & /*args*/)
LOG_INFO(log, "Loading metadata from " + path);
/// Create text_log instance
text_log = createSystemLog<TextLog>(*global_context, "system", "text_log", global_context->getConfigRef(), "text_log");
try
{
loadMetadataSystem(*global_context);
/// After attaching system databases we can initialize system log.
global_context->initializeSystemLogs(text_log);
global_context->initializeSystemLogs();
/// After the system database is created, attach virtual system tables (in addition to query_log and part_log)
attachSystemTablesServer(*global_context->getDatabase("system"), has_zookeeper);
/// Then, load remaining databases

View File

@ -57,7 +57,6 @@ protected:
private:
std::unique_ptr<Context> global_context;
std::shared_ptr<TextLog> text_log;
};
}

View File

@ -1,15 +0,0 @@
#include "FunctionsConsistentHashing.h"
#include <Functions/FunctionFactory.h>
namespace DB
{
void registerFunctionsConsistentHashing(FunctionFactory & factory)
{
factory.registerFunction<FunctionYandexConsistentHash>();
factory.registerFunction<FunctionJumpConsistentHash>();
factory.registerFunction<FunctionSumburConsistentHash>();
}
}

View File

@ -8,9 +8,6 @@
#include <Common/typeid_cast.h>
#include <common/likely.h>
#include <sumbur.h>
#include <consistent_hashing.h>
namespace DB
{
@ -23,69 +20,6 @@ namespace ErrorCodes
}
/// An O(1) time and space consistent hash algorithm by Konstantin Oblakov
struct YandexConsistentHashImpl
{
static constexpr auto name = "yandexConsistentHash";
using HashType = UInt64;
/// Actually it supports UInt64, but it is efficient only if n <= 32768
using ResultType = UInt16;
using BucketsType = ResultType;
static constexpr auto max_buckets = 32768;
static inline ResultType apply(UInt64 hash, BucketsType n)
{
return ConsistentHashing(hash, n);
}
};
/// Code from https://arxiv.org/pdf/1406.2294.pdf
static inline int32_t JumpConsistentHash(uint64_t key, int32_t num_buckets)
{
int64_t b = -1, j = 0;
while (j < num_buckets)
{
b = j;
key = key * 2862933555777941757ULL + 1;
j = static_cast<int64_t>((b + 1) * (double(1LL << 31) / double((key >> 33) + 1)));
}
return static_cast<int32_t>(b);
}
struct JumpConsistentHashImpl
{
static constexpr auto name = "jumpConsistentHash";
using HashType = UInt64;
using ResultType = Int32;
using BucketsType = ResultType;
static constexpr auto max_buckets = static_cast<UInt64>(std::numeric_limits<BucketsType>::max());
static inline ResultType apply(UInt64 hash, BucketsType n)
{
return JumpConsistentHash(hash, n);
}
};
struct SumburConsistentHashImpl
{
static constexpr auto name = "sumburConsistentHash";
using HashType = UInt32;
using ResultType = UInt16;
using BucketsType = ResultType;
static constexpr auto max_buckets = static_cast<UInt64>(std::numeric_limits<BucketsType>::max());
static inline ResultType apply(HashType hash, BucketsType n)
{
return static_cast<ResultType>(sumburConsistentHash(hash, n));
}
};
template <typename Impl>
class FunctionConsistentHashImpl : public IFunction
{
@ -221,10 +155,4 @@ private:
}
};
using FunctionYandexConsistentHash = FunctionConsistentHashImpl<YandexConsistentHashImpl>;
using FunctionJumpConsistentHash = FunctionConsistentHashImpl<JumpConsistentHashImpl>;
using FunctionSumburConsistentHash = FunctionConsistentHashImpl<SumburConsistentHashImpl>;
}

View File

@ -1,38 +0,0 @@
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionsVisitParam.h>
#include <Functions/FunctionsStringSearch.h>
namespace DB
{
struct NameVisitParamHas { static constexpr auto name = "visitParamHas"; };
struct NameVisitParamExtractUInt { static constexpr auto name = "visitParamExtractUInt"; };
struct NameVisitParamExtractInt { static constexpr auto name = "visitParamExtractInt"; };
struct NameVisitParamExtractFloat { static constexpr auto name = "visitParamExtractFloat"; };
struct NameVisitParamExtractBool { static constexpr auto name = "visitParamExtractBool"; };
struct NameVisitParamExtractRaw { static constexpr auto name = "visitParamExtractRaw"; };
struct NameVisitParamExtractString { static constexpr auto name = "visitParamExtractString"; };
using FunctionVisitParamHas = FunctionsStringSearch<ExtractParamImpl<HasParam>, NameVisitParamHas>;
using FunctionVisitParamExtractUInt = FunctionsStringSearch<ExtractParamImpl<ExtractNumericType<UInt64>>, NameVisitParamExtractUInt>;
using FunctionVisitParamExtractInt = FunctionsStringSearch<ExtractParamImpl<ExtractNumericType<Int64>>, NameVisitParamExtractInt>;
using FunctionVisitParamExtractFloat = FunctionsStringSearch<ExtractParamImpl<ExtractNumericType<Float64>>, NameVisitParamExtractFloat>;
using FunctionVisitParamExtractBool = FunctionsStringSearch<ExtractParamImpl<ExtractBool>, NameVisitParamExtractBool>;
using FunctionVisitParamExtractRaw = FunctionsStringSearchToString<ExtractParamToStringImpl<ExtractRaw>, NameVisitParamExtractRaw>;
using FunctionVisitParamExtractString = FunctionsStringSearchToString<ExtractParamToStringImpl<ExtractString>, NameVisitParamExtractString>;
void registerFunctionsVisitParam(FunctionFactory & factory)
{
factory.registerFunction<FunctionVisitParamHas>();
factory.registerFunction<FunctionVisitParamExtractUInt>();
factory.registerFunction<FunctionVisitParamExtractInt>();
factory.registerFunction<FunctionVisitParamExtractFloat>();
factory.registerFunction<FunctionVisitParamExtractBool>();
factory.registerFunction<FunctionVisitParamExtractRaw>();
factory.registerFunction<FunctionVisitParamExtractString>();
}
}

View File

@ -1,14 +1,9 @@
#pragma once
#include <Poco/UTF8Encoding.h>
#include <Poco/Unicode.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeFixedString.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnArray.h>
#include <Columns/ColumnFixedString.h>
#include <Common/hex.h>
#include <Common/Volnitsky.h>
#include <Functions/IFunction.h>
#include <Functions/FunctionHelpers.h>
@ -43,15 +38,6 @@ namespace ErrorCodes
extern const int ILLEGAL_COLUMN;
}
struct HasParam
{
using ResultType = UInt8;
static UInt8 extract(const UInt8 *, const UInt8 *)
{
return true;
}
};
template <typename NumericType>
struct ExtractNumericType
@ -78,77 +64,6 @@ struct ExtractNumericType
}
};
struct ExtractBool
{
using ResultType = UInt8;
static UInt8 extract(const UInt8 * begin, const UInt8 * end)
{
return begin + 4 <= end && 0 == strncmp(reinterpret_cast<const char *>(begin), "true", 4);
}
};
struct ExtractRaw
{
using ExpectChars = PODArrayWithStackMemory<char, 64>;
static void extract(const UInt8 * pos, const UInt8 * end, ColumnString::Chars & res_data)
{
ExpectChars expects_end;
UInt8 current_expect_end = 0;
for (auto extract_begin = pos; pos != end; ++pos)
{
if (current_expect_end && *pos == current_expect_end)
{
expects_end.pop_back();
current_expect_end = expects_end.empty() ? 0 : expects_end.back();
}
else
{
switch (*pos)
{
case '[':
current_expect_end = ']';
expects_end.push_back(current_expect_end);
break;
case '{':
current_expect_end = '}';
expects_end.push_back(current_expect_end);
break;
case '"' :
current_expect_end = '"';
expects_end.push_back(current_expect_end);
break;
case '\\':
/// skip backslash
if (pos + 1 < end && pos[1] == '"')
pos++;
break;
default:
if (!current_expect_end && (*pos == ',' || *pos == '}'))
{
res_data.insert(extract_begin, pos);
return;
}
}
}
}
}
};
struct ExtractString
{
static void extract(const UInt8 * pos, const UInt8 * end, ColumnString::Chars & res_data)
{
size_t old_size = res_data.size();
ReadBufferFromMemory in(pos, end - pos);
if (!tryReadJSONStringInto(res_data, in))
res_data.resize(old_size);
}
};
/** Searches for occurrences of a field in the visit parameter and calls ParamExtractor
* for each occurrence of the field, passing it a pointer to the part of the string,
@ -285,6 +200,4 @@ struct ExtractParamToStringImpl
}
};
}

View File

@ -232,10 +232,9 @@ bool PreparedFunctionImpl::defaultImplementationForConstantArguments(Block & blo
const ColumnWithTypeAndName & column = block.getByPosition(args[arg_num]);
if (arguments_to_remain_constants.end() != std::find(arguments_to_remain_constants.begin(), arguments_to_remain_constants.end(), arg_num))
if (column.column->empty())
temporary_block.insert({column.column->cloneResized(1), column.type, column.name});
else
temporary_block.insert(column);
{
temporary_block.insert({column.column->cloneResized(1), column.type, column.name});
}
else
{
have_converted_columns = true;

View File

@ -111,8 +111,6 @@ void FunctionArrayReduce::executeImpl(Block & block, const ColumnNumbers & argum
std::unique_ptr<Arena> arena = agg_func.allocatesMemoryInArena() ? std::make_unique<Arena>() : nullptr;
size_t rows = input_rows_count;
/// Aggregate functions do not support constant columns. Therefore, we materialize them.
std::vector<ColumnPtr> materialized_columns;
@ -124,6 +122,7 @@ void FunctionArrayReduce::executeImpl(Block & block, const ColumnNumbers & argum
for (size_t i = 0; i < num_arguments_columns; ++i)
{
const IColumn * col = block.getByPosition(arguments[i + 1]).column.get();
const ColumnArray::Offsets * offsets_i = nullptr;
if (const ColumnArray * arr = checkAndGetColumn<ColumnArray>(col))
{
@ -159,7 +158,7 @@ void FunctionArrayReduce::executeImpl(Block & block, const ColumnNumbers & argum
+ block.getByPosition(result).type->getName(), ErrorCodes::ILLEGAL_COLUMN);
ColumnArray::Offset current_offset = 0;
for (size_t i = 0; i < rows; ++i)
for (size_t i = 0; i < input_rows_count; ++i)
{
agg_func.create(place);
ColumnArray::Offset next_offset = (*offsets)[i];

View File

@ -0,0 +1,44 @@
#include "FunctionsConsistentHashing.h"
#include <Functions/FunctionFactory.h>
namespace DB
{
/// Code from https://arxiv.org/pdf/1406.2294.pdf
static inline int32_t JumpConsistentHash(uint64_t key, int32_t num_buckets)
{
int64_t b = -1, j = 0;
while (j < num_buckets)
{
b = j;
key = key * 2862933555777941757ULL + 1;
j = static_cast<int64_t>((b + 1) * (double(1LL << 31) / double((key >> 33) + 1)));
}
return static_cast<int32_t>(b);
}
struct JumpConsistentHashImpl
{
static constexpr auto name = "jumpConsistentHash";
using HashType = UInt64;
using ResultType = Int32;
using BucketsType = ResultType;
static constexpr auto max_buckets = static_cast<UInt64>(std::numeric_limits<BucketsType>::max());
static inline ResultType apply(UInt64 hash, BucketsType n)
{
return JumpConsistentHash(hash, n);
}
};
using FunctionJumpConsistentHash = FunctionConsistentHashImpl<JumpConsistentHashImpl>;
void registerFunctionJumpConsistentHash(FunctionFactory & factory)
{
factory.registerFunction<FunctionJumpConsistentHash>();
}
}

View File

@ -20,7 +20,6 @@ void registerFunctionsExternalDictionaries(FunctionFactory &);
void registerFunctionsExternalModels(FunctionFactory &);
void registerFunctionsFormatting(FunctionFactory &);
void registerFunctionsHashing(FunctionFactory &);
void registerFunctionsConsistentHashing(FunctionFactory &);
void registerFunctionsHigherOrder(FunctionFactory &);
void registerFunctionsLogical(FunctionFactory &);
void registerFunctionsMiscellaneous(FunctionFactory &);
@ -41,6 +40,7 @@ void registerFunctionsNull(FunctionFactory &);
void registerFunctionsFindCluster(FunctionFactory &);
void registerFunctionsJSON(FunctionFactory &);
void registerFunctionsIntrospection(FunctionFactory &);
void registerFunctionsConsistentHashing(FunctionFactory & factory);
void registerFunctions()
{
@ -60,7 +60,6 @@ void registerFunctions()
registerFunctionsExternalModels(factory);
registerFunctionsFormatting(factory);
registerFunctionsHashing(factory);
registerFunctionsConsistentHashing(factory);
registerFunctionsHigherOrder(factory);
registerFunctionsLogical(factory);
registerFunctionsMiscellaneous(factory);
@ -80,6 +79,7 @@ void registerFunctions()
registerFunctionsFindCluster(factory);
registerFunctionsJSON(factory);
registerFunctionsIntrospection(factory);
registerFunctionsConsistentHashing(factory);
}
}

View File

@ -0,0 +1,18 @@
namespace DB
{
class FunctionFactory;
void registerFunctionYandexConsistentHash(FunctionFactory & factory);
void registerFunctionJumpConsistentHash(FunctionFactory & factory);
void registerFunctionSumburConsistentHash(FunctionFactory & factory);
void registerFunctionsConsistentHashing(FunctionFactory & factory)
{
registerFunctionYandexConsistentHash(factory);
registerFunctionJumpConsistentHash(factory);
registerFunctionSumburConsistentHash(factory);
}
}

View File

@ -0,0 +1,25 @@
namespace DB
{
class FunctionFactory;
void registerFunctionVisitParamHas(FunctionFactory & factory);
void registerFunctionVisitParamExtractUInt(FunctionFactory & factory);
void registerFunctionVisitParamExtractInt(FunctionFactory & factory);
void registerFunctionVisitParamExtractFloat(FunctionFactory & factory);
void registerFunctionVisitParamExtractBool(FunctionFactory & factory);
void registerFunctionVisitParamExtractRaw(FunctionFactory & factory);
void registerFunctionVisitParamExtractString(FunctionFactory & factory);
void registerFunctionsVisitParam(FunctionFactory & factory)
{
registerFunctionVisitParamHas(factory);
registerFunctionVisitParamExtractUInt(factory);
registerFunctionVisitParamExtractInt(factory);
registerFunctionVisitParamExtractFloat(factory);
registerFunctionVisitParamExtractBool(factory);
registerFunctionVisitParamExtractRaw(factory);
registerFunctionVisitParamExtractString(factory);
}
}

View File

@ -0,0 +1,34 @@
#include "FunctionsConsistentHashing.h"
#include <Functions/FunctionFactory.h>
#include <sumbur.h>
namespace DB
{
struct SumburConsistentHashImpl
{
static constexpr auto name = "sumburConsistentHash";
using HashType = UInt32;
using ResultType = UInt16;
using BucketsType = ResultType;
static constexpr auto max_buckets = static_cast<UInt64>(std::numeric_limits<BucketsType>::max());
static inline ResultType apply(HashType hash, BucketsType n)
{
return static_cast<ResultType>(sumburConsistentHash(hash, n));
}
};
using FunctionSumburConsistentHash = FunctionConsistentHashImpl<SumburConsistentHashImpl>;
void registerFunctionSumburConsistentHash(FunctionFactory & factory)
{
factory.registerFunction<FunctionSumburConsistentHash>();
}
}

View File

@ -0,0 +1,28 @@
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionsVisitParam.h>
#include <Functions/FunctionsStringSearch.h>
namespace DB
{
struct ExtractBool
{
using ResultType = UInt8;
static UInt8 extract(const UInt8 * begin, const UInt8 * end)
{
return begin + 4 <= end && 0 == strncmp(reinterpret_cast<const char *>(begin), "true", 4);
}
};
struct NameVisitParamExtractBool { static constexpr auto name = "visitParamExtractBool"; };
using FunctionVisitParamExtractBool = FunctionsStringSearch<ExtractParamImpl<ExtractBool>, NameVisitParamExtractBool>;
void registerFunctionVisitParamExtractBool(FunctionFactory & factory)
{
factory.registerFunction<FunctionVisitParamExtractBool>();
}
}

View File

@ -0,0 +1,18 @@
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionsVisitParam.h>
#include <Functions/FunctionsStringSearch.h>
namespace DB
{
struct NameVisitParamExtractFloat { static constexpr auto name = "visitParamExtractFloat"; };
using FunctionVisitParamExtractFloat = FunctionsStringSearch<ExtractParamImpl<ExtractNumericType<Float64>>, NameVisitParamExtractFloat>;
void registerFunctionVisitParamExtractFloat(FunctionFactory & factory)
{
factory.registerFunction<FunctionVisitParamExtractFloat>();
}
}

View File

@ -0,0 +1,18 @@
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionsVisitParam.h>
#include <Functions/FunctionsStringSearch.h>
namespace DB
{
struct NameVisitParamExtractInt { static constexpr auto name = "visitParamExtractInt"; };
using FunctionVisitParamExtractInt = FunctionsStringSearch<ExtractParamImpl<ExtractNumericType<Int64>>, NameVisitParamExtractInt>;
void registerFunctionVisitParamExtractInt(FunctionFactory & factory)
{
factory.registerFunction<FunctionVisitParamExtractInt>();
}
}

View File

@ -0,0 +1,67 @@
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionsVisitParam.h>
#include <Functions/FunctionsStringSearch.h>
namespace DB
{
struct ExtractRaw
{
using ExpectChars = PODArrayWithStackMemory<char, 64>;
static void extract(const UInt8 * pos, const UInt8 * end, ColumnString::Chars & res_data)
{
ExpectChars expects_end;
UInt8 current_expect_end = 0;
for (auto extract_begin = pos; pos != end; ++pos)
{
if (current_expect_end && *pos == current_expect_end)
{
expects_end.pop_back();
current_expect_end = expects_end.empty() ? 0 : expects_end.back();
}
else
{
switch (*pos)
{
case '[':
current_expect_end = ']';
expects_end.push_back(current_expect_end);
break;
case '{':
current_expect_end = '}';
expects_end.push_back(current_expect_end);
break;
case '"' :
current_expect_end = '"';
expects_end.push_back(current_expect_end);
break;
case '\\':
/// skip backslash
if (pos + 1 < end && pos[1] == '"')
pos++;
break;
default:
if (!current_expect_end && (*pos == ',' || *pos == '}'))
{
res_data.insert(extract_begin, pos);
return;
}
}
}
}
}
};
struct NameVisitParamExtractRaw { static constexpr auto name = "visitParamExtractRaw"; };
using FunctionVisitParamExtractRaw = FunctionsStringSearchToString<ExtractParamToStringImpl<ExtractRaw>, NameVisitParamExtractRaw>;
void registerFunctionVisitParamExtractRaw(FunctionFactory & factory)
{
factory.registerFunction<FunctionVisitParamExtractRaw>();
}
}

View File

@ -0,0 +1,29 @@
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionsVisitParam.h>
#include <Functions/FunctionsStringSearch.h>
namespace DB
{
struct ExtractString
{
static void extract(const UInt8 * pos, const UInt8 * end, ColumnString::Chars & res_data)
{
size_t old_size = res_data.size();
ReadBufferFromMemory in(pos, end - pos);
if (!tryReadJSONStringInto(res_data, in))
res_data.resize(old_size);
}
};
struct NameVisitParamExtractString { static constexpr auto name = "visitParamExtractString"; };
using FunctionVisitParamExtractString = FunctionsStringSearchToString<ExtractParamToStringImpl<ExtractString>, NameVisitParamExtractString>;
void registerFunctionVisitParamExtractString(FunctionFactory & factory)
{
factory.registerFunction<FunctionVisitParamExtractString>();
}
}

View File

@ -0,0 +1,18 @@
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionsVisitParam.h>
#include <Functions/FunctionsStringSearch.h>
namespace DB
{
struct NameVisitParamExtractUInt { static constexpr auto name = "visitParamExtractUInt"; };
using FunctionVisitParamExtractUInt = FunctionsStringSearch<ExtractParamImpl<ExtractNumericType<UInt64>>, NameVisitParamExtractUInt>;
void registerFunctionVisitParamExtractUInt(FunctionFactory & factory)
{
factory.registerFunction<FunctionVisitParamExtractUInt>();
}
}

View File

@ -0,0 +1,28 @@
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionsVisitParam.h>
#include <Functions/FunctionsStringSearch.h>
namespace DB
{
struct HasParam
{
using ResultType = UInt8;
static UInt8 extract(const UInt8 *, const UInt8 *)
{
return true;
}
};
struct NameVisitParamHas { static constexpr auto name = "visitParamHas"; };
using FunctionVisitParamHas = FunctionsStringSearch<ExtractParamImpl<HasParam>, NameVisitParamHas>;
void registerFunctionVisitParamHas(FunctionFactory & factory)
{
factory.registerFunction<FunctionVisitParamHas>();
}
}

View File

@ -0,0 +1,34 @@
#include "FunctionsConsistentHashing.h"
#include <Functions/FunctionFactory.h>
#include <consistent_hashing.h>
namespace DB
{
/// An O(1) time and space consistent hash algorithm by Konstantin Oblakov
struct YandexConsistentHashImpl
{
static constexpr auto name = "yandexConsistentHash";
using HashType = UInt64;
/// Actually it supports UInt64, but it is efficient only if n <= 32768
using ResultType = UInt16;
using BucketsType = ResultType;
static constexpr auto max_buckets = 32768;
static inline ResultType apply(UInt64 hash, BucketsType n)
{
return ConsistentHashing(hash, n);
}
};
using FunctionYandexConsistentHash = FunctionConsistentHashImpl<YandexConsistentHashImpl>;
void registerFunctionYandexConsistentHash(FunctionFactory & factory)
{
factory.registerFunction<FunctionYandexConsistentHash>();
}
}

View File

@ -1645,11 +1645,10 @@ Compiler & Context::getCompiler()
}
void Context::initializeSystemLogs(std::shared_ptr<TextLog> text_log)
void Context::initializeSystemLogs()
{
auto lock = getLock();
shared->system_logs.emplace(*global_context, getConfigRef());
shared->system_logs->text_log = text_log;
}
bool Context::hasTraceCollector()
@ -1716,11 +1715,10 @@ std::shared_ptr<TextLog> Context::getTextLog()
{
auto lock = getLock();
if (!shared->system_logs)
if (auto log = shared->system_logs->text_log.lock())
return log;
if (!shared->system_logs || !shared->system_logs->text_log)
return {};
return {};
return shared->system_logs->text_log;
}

View File

@ -424,7 +424,7 @@ public:
Compiler & getCompiler();
/// Call after initialization before using system logs. Call for global context.
void initializeSystemLogs(std::shared_ptr<TextLog> text_log);
void initializeSystemLogs();
void initializeTraceCollector();
bool hasTraceCollector();

View File

@ -47,6 +47,7 @@ SystemLogs::SystemLogs(Context & global_context, const Poco::Util::AbstractConfi
query_thread_log = createSystemLog<QueryThreadLog>(global_context, "system", "query_thread_log", config, "query_thread_log");
part_log = createSystemLog<PartLog>(global_context, "system", "part_log", config, "part_log");
trace_log = createSystemLog<TraceLog>(global_context, "system", "trace_log", config, "trace_log");
text_log = createSystemLog<TextLog>(global_context, "system", "text_log", config, "text_log");
part_log_database = config.getString("part_log.database", "system");
}
@ -67,6 +68,8 @@ void SystemLogs::shutdown()
part_log->shutdown();
if (trace_log)
trace_log->shutdown();
if (text_log)
text_log->shutdown();
}
}

View File

@ -75,8 +75,7 @@ struct SystemLogs
std::shared_ptr<QueryThreadLog> query_thread_log; /// Used to log query threads.
std::shared_ptr<PartLog> part_log; /// Used to log operations with parts
std::shared_ptr<TraceLog> trace_log; /// Used to log traces from query profiler
std::weak_ptr<TextLog> text_log; /// Used to log all text. We use weak_ptr, because this log is
/// a server's field.
std::shared_ptr<TextLog> text_log; /// Used to log all text messages.
String part_log_database;
};

View File

@ -0,0 +1,43 @@
0 ([100,200],[30,30])
0 ([100,200],[30,30])
1 ([100,200],[30,30])
2 ([100,200],[30,30])
3 ([100,200],[30,30])
4 ([100,200],[30,30])
5 ([100,200],[30,30])
6 ([100,200],[30,30])
7 ([100,200],[30,30])
8 ([100,200],[30,30])
9 ([100,200],[30,30])
0 ([100,200],[30,30])
1 ([100,200],[30,30])
2 ([100,200],[30,30])
3 ([100,200],[30,30])
4 ([100,200],[30,30])
5 ([100,200],[30,30])
6 ([100,200],[30,30])
7 ([100,200],[30,30])
8 ([100,200],[30,30])
9 ([100,200],[30,30])
0 400
1 400
2 400
3 400
4 400
5 400
6 400
7 400
8 400
9 400
0 [100,100,200]
1 [100,100,200]
2 [100,100,200]
3 [100,100,200]
4 [100,100,200]
5 [100,100,200]
6 [100,100,200]
7 [100,100,200]
8 [100,100,200]
9 [100,100,200]
Array(Array(UInt8)), Const(size = 2, Array(size = 1, UInt64(size = 1), Array(size = 1, UInt64(size = 1), UInt8(size = 2)))) ([1,2],[1,2])
Array(Array(UInt8)), Const(size = 2, Array(size = 1, UInt64(size = 1), Array(size = 1, UInt64(size = 1), UInt8(size = 2)))) ([1,2],[1,2])

View File

@ -0,0 +1,7 @@
select number, arrayReduce( 'sumMap', [a],[b] ) from (select [100,100,200] a,[10,20,30] b, number from numbers(1));
select number, arrayReduce( 'sumMap', [a],[b] ) from (select materialize([100,100,200]) a,materialize([10,20,30]) b, number from numbers(10));
select number, arrayReduce( 'sumMap', [a],[b] ) from (select [100,100,200] a,[10,20,30] b, number from numbers(10));
select number, arrayReduce( 'sum', a) from (select materialize([100,100,200]) a, number from numbers(10));
select number, arrayReduce( 'max', [a] ) from (select materialize([100,100,200]) a, number from numbers(10));
select dumpColumnStructure([a]), arrayReduce('sumMap', [a], [a]) from (select [1, 2] a FROM numbers(2));

View File

@ -17,6 +17,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
) ENGINE = ReplacingMergeTree([ver])
[PARTITION BY expr]
[ORDER BY expr]
[PRIMARY KEY expr]
[SAMPLE BY expr]
[SETTINGS name=value, ...]
```

View File

@ -20,23 +20,25 @@ using DB::CurrentThread;
/// Logs a message to a specified logger with that level.
#define LOG_SIMPLE(logger, message, priority, PRIORITY) do \
{ \
const bool is_clients_log = (CurrentThread::getGroup() != nullptr) && \
(CurrentThread::getGroup()->client_logs_level >= (priority)); \
if ((logger)->is((PRIORITY)) || is_clients_log) { \
std::stringstream oss_internal_rare; \
oss_internal_rare << message; \
if (auto channel = (logger)->getChannel()) { \
std::string file_function; \
file_function += __FILE__; \
file_function += ", "; \
file_function += __PRETTY_FUNCTION__; \
Message poco_message((logger)->name(), oss_internal_rare.str(), \
(PRIORITY), file_function.c_str(), __LINE__); \
channel->log(poco_message); \
} \
} \
#define LOG_SIMPLE(logger, message, priority, PRIORITY) do \
{ \
const bool is_clients_log = (CurrentThread::getGroup() != nullptr) && \
(CurrentThread::getGroup()->client_logs_level >= (priority)); \
if ((logger)->is((PRIORITY)) || is_clients_log) \
{ \
std::stringstream oss_internal_rare; \
oss_internal_rare << message; \
if (auto channel = (logger)->getChannel()) \
{ \
std::string file_function; \
file_function += __FILE__; \
file_function += "; "; \
file_function += __PRETTY_FUNCTION__; \
Message poco_message((logger)->name(), oss_internal_rare.str(), \
(PRIORITY), file_function.c_str(), __LINE__); \
channel->log(poco_message); \
} \
} \
} while (false)