mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-26 09:32:01 +00:00
Merge branch 'master' of github.com:yandex/ClickHouse
This commit is contained in:
commit
cc2744ebb9
@ -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
|
||||
|
@ -57,7 +57,6 @@ protected:
|
||||
|
||||
private:
|
||||
std::unique_ptr<Context> global_context;
|
||||
std::shared_ptr<TextLog> text_log;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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>();
|
||||
}
|
||||
|
||||
}
|
@ -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>;
|
||||
|
||||
|
||||
}
|
||||
|
@ -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>();
|
||||
}
|
||||
|
||||
}
|
@ -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
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
|
||||
}
|
||||
|
@ -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;
|
||||
|
@ -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];
|
||||
|
44
dbms/src/Functions/jumpConsistentHash.cpp
Normal file
44
dbms/src/Functions/jumpConsistentHash.cpp
Normal 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>();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
||||
|
18
dbms/src/Functions/registerFunctionsConsistentHashing.cpp
Normal file
18
dbms/src/Functions/registerFunctionsConsistentHashing.cpp
Normal 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);
|
||||
}
|
||||
|
||||
}
|
||||
|
25
dbms/src/Functions/registerFunctionsVisitParam.cpp
Normal file
25
dbms/src/Functions/registerFunctionsVisitParam.cpp
Normal 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);
|
||||
}
|
||||
|
||||
}
|
34
dbms/src/Functions/sumburConsistentHash.cpp
Normal file
34
dbms/src/Functions/sumburConsistentHash.cpp
Normal 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>();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
28
dbms/src/Functions/visitParamExtractBool.cpp
Normal file
28
dbms/src/Functions/visitParamExtractBool.cpp
Normal 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>();
|
||||
}
|
||||
|
||||
}
|
18
dbms/src/Functions/visitParamExtractFloat.cpp
Normal file
18
dbms/src/Functions/visitParamExtractFloat.cpp
Normal 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>();
|
||||
}
|
||||
|
||||
}
|
18
dbms/src/Functions/visitParamExtractInt.cpp
Normal file
18
dbms/src/Functions/visitParamExtractInt.cpp
Normal 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>();
|
||||
}
|
||||
|
||||
}
|
67
dbms/src/Functions/visitParamExtractRaw.cpp
Normal file
67
dbms/src/Functions/visitParamExtractRaw.cpp
Normal 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>();
|
||||
}
|
||||
|
||||
}
|
29
dbms/src/Functions/visitParamExtractString.cpp
Normal file
29
dbms/src/Functions/visitParamExtractString.cpp
Normal 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>();
|
||||
}
|
||||
|
||||
}
|
18
dbms/src/Functions/visitParamExtractUInt.cpp
Normal file
18
dbms/src/Functions/visitParamExtractUInt.cpp
Normal 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>();
|
||||
}
|
||||
|
||||
}
|
28
dbms/src/Functions/visitParamHas.cpp
Normal file
28
dbms/src/Functions/visitParamHas.cpp
Normal 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>();
|
||||
}
|
||||
|
||||
}
|
34
dbms/src/Functions/yandexConsistentHash.cpp
Normal file
34
dbms/src/Functions/yandexConsistentHash.cpp
Normal 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>();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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;
|
||||
}
|
||||
|
||||
|
||||
|
@ -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();
|
||||
|
@ -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();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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;
|
||||
};
|
||||
|
@ -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])
|
7
dbms/tests/queries/0_stateless/00978_sum_map_bugfix.sql
Normal file
7
dbms/tests/queries/0_stateless/00978_sum_map_bugfix.sql
Normal 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));
|
@ -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, ...]
|
||||
```
|
||||
|
@ -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)
|
||||
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user