mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 16:42:05 +00:00
Merge branch 'master' into merge_tree_deduplication
This commit is contained in:
commit
0ceb95aa55
@ -464,6 +464,7 @@ find_contrib_lib(double-conversion) # Must be before parquet
|
||||
include (cmake/find/ssl.cmake)
|
||||
include (cmake/find/ldap.cmake) # after ssl
|
||||
include (cmake/find/icu.cmake)
|
||||
include (cmake/find/xz.cmake)
|
||||
include (cmake/find/zlib.cmake)
|
||||
include (cmake/find/zstd.cmake)
|
||||
include (cmake/find/ltdl.cmake) # for odbc
|
||||
|
27
cmake/find/xz.cmake
Normal file
27
cmake/find/xz.cmake
Normal file
@ -0,0 +1,27 @@
|
||||
option (USE_INTERNAL_XZ_LIBRARY "Set to OFF to use system xz (lzma) library instead of bundled" ${NOT_UNBUNDLED})
|
||||
|
||||
if(NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/xz/src/liblzma/api/lzma.h")
|
||||
if(USE_INTERNAL_XZ_LIBRARY)
|
||||
message(WARNING "submodule contrib/xz is missing. to fix try run: \n git submodule update --init --recursive")
|
||||
message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal xz (lzma) library")
|
||||
set(USE_INTERNAL_XZ_LIBRARY 0)
|
||||
endif()
|
||||
set(MISSING_INTERNAL_XZ_LIBRARY 1)
|
||||
endif()
|
||||
|
||||
if (NOT USE_INTERNAL_XZ_LIBRARY)
|
||||
find_library (XZ_LIBRARY lzma)
|
||||
find_path (XZ_INCLUDE_DIR NAMES lzma.h PATHS ${XZ_INCLUDE_PATHS})
|
||||
if (NOT XZ_LIBRARY OR NOT XZ_INCLUDE_DIR)
|
||||
message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system xz (lzma) library")
|
||||
endif ()
|
||||
endif ()
|
||||
|
||||
if (XZ_LIBRARY AND XZ_INCLUDE_DIR)
|
||||
elseif (NOT MISSING_INTERNAL_XZ_LIBRARY)
|
||||
set (USE_INTERNAL_XZ_LIBRARY 1)
|
||||
set (XZ_LIBRARY liblzma)
|
||||
set (XZ_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/contrib/xz/src/liblzma/api)
|
||||
endif ()
|
||||
|
||||
message (STATUS "Using xz (lzma): ${XZ_INCLUDE_DIR} : ${XZ_LIBRARY}")
|
@ -69,11 +69,25 @@ function watchdog
|
||||
killall -9 clickhouse-client ||:
|
||||
}
|
||||
|
||||
function filter_exists
|
||||
{
|
||||
local path
|
||||
for path in "$@"; do
|
||||
if [ -e "$path" ]; then
|
||||
echo "$path"
|
||||
else
|
||||
echo "'$path' does not exists" >&2
|
||||
fi
|
||||
done
|
||||
}
|
||||
|
||||
function fuzz
|
||||
{
|
||||
# Obtain the list of newly added tests. They will be fuzzed in more extreme way than other tests.
|
||||
# Don't overwrite the NEW_TESTS_OPT so that it can be set from the environment.
|
||||
NEW_TESTS="$(grep -P 'tests/queries/0_stateless/.*\.sql' ci-changed-files.txt | sed -r -e 's!^!ch/!' | sort -R)"
|
||||
# ci-changed-files.txt contains also files that has been deleted/renamed, filter them out.
|
||||
NEW_TESTS="$(filter_exists $NEW_TESTS)"
|
||||
if [[ -n "$NEW_TESTS" ]]
|
||||
then
|
||||
NEW_TESTS_OPT="${NEW_TESTS_OPT:---interleave-queries-file ${NEW_TESTS}}"
|
||||
|
@ -986,7 +986,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
///
|
||||
/// Look at compiler-rt/lib/sanitizer_common/sanitizer_stacktrace.h
|
||||
///
|
||||
#if USE_UNWIND && !WITH_COVERAGE && !defined(SANITIZER)
|
||||
#if USE_UNWIND && !WITH_COVERAGE && !defined(SANITIZER) && defined(__x86_64__)
|
||||
/// Profilers cannot work reliably with any other libunwind or without PHDR cache.
|
||||
if (hasPHDRCache())
|
||||
{
|
||||
@ -1023,6 +1023,10 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
" when two different stack unwinding methods will interfere with each other.");
|
||||
#endif
|
||||
|
||||
#if !defined(__x86_64__)
|
||||
LOG_INFO(log, "Query Profiler is only tested on x86_64. It also known to not work under qemu-user.");
|
||||
#endif
|
||||
|
||||
if (!hasPHDRCache())
|
||||
LOG_INFO(log, "Query Profiler and TraceCollector are disabled because they require PHDR cache to be created"
|
||||
" (otherwise the function 'dl_iterate_phdr' is not lock free and not async-signal safe).");
|
||||
|
@ -375,11 +375,9 @@ if (ZSTD_LIBRARY)
|
||||
endif ()
|
||||
endif()
|
||||
|
||||
set (LZMA_LIBRARY liblzma)
|
||||
set (LZMA_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/contrib/xz/src/liblzma/api)
|
||||
if (LZMA_LIBRARY)
|
||||
target_link_libraries (clickhouse_common_io PUBLIC ${LZMA_LIBRARY})
|
||||
target_include_directories (clickhouse_common_io SYSTEM BEFORE PUBLIC ${LZMA_INCLUDE_DIR})
|
||||
if (XZ_LIBRARY)
|
||||
target_link_libraries (clickhouse_common_io PUBLIC ${XZ_LIBRARY})
|
||||
target_include_directories (clickhouse_common_io SYSTEM BEFORE PUBLIC ${XZ_INCLUDE_DIR})
|
||||
endif()
|
||||
|
||||
if (USE_ICU)
|
||||
|
@ -59,13 +59,16 @@ public:
|
||||
|
||||
StringRef serializeValueIntoArena(size_t /*n*/, Arena & arena, char const *& begin) const override
|
||||
{
|
||||
return { arena.allocContinue(0, begin), 0 };
|
||||
/// Has to put one useless byte into Arena, because serialization into zero number of bytes is ambiguous.
|
||||
char * res = arena.allocContinue(1, begin);
|
||||
*res = 0;
|
||||
return { res, 1 };
|
||||
}
|
||||
|
||||
const char * deserializeAndInsertFromArena(const char * pos) override
|
||||
{
|
||||
++s;
|
||||
return pos;
|
||||
return pos + 1;
|
||||
}
|
||||
|
||||
const char * skipSerializedInArena(const char * pos) const override
|
||||
|
@ -547,6 +547,7 @@
|
||||
M(577, INVALID_SHARD_ID) \
|
||||
M(578, INVALID_FORMAT_INSERT_QUERY_WITH_DATA) \
|
||||
M(579, INCORRECT_PART_TYPE) \
|
||||
M(580, CANNOT_SET_ROUNDING_MODE) \
|
||||
\
|
||||
M(998, POSTGRESQL_CONNECTION_FAILURE) \
|
||||
M(999, KEEPER_EXCEPTION) \
|
||||
|
@ -18,97 +18,72 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
static const size_t MAX_CONNECTIONS = 16;
|
||||
|
||||
inline static UInt16 getPortFromContext(const Context & context, bool secure)
|
||||
namespace ErrorCodes
|
||||
{
|
||||
return secure ? context.getTCPPortSecure().value_or(0) : context.getTCPPort();
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
static ConnectionPoolWithFailoverPtr createPool(
|
||||
const std::string & host,
|
||||
UInt16 port,
|
||||
bool secure,
|
||||
const std::string & db,
|
||||
const std::string & user,
|
||||
const std::string & password)
|
||||
namespace
|
||||
{
|
||||
ConnectionPoolPtrs pools;
|
||||
pools.emplace_back(std::make_shared<ConnectionPool>(
|
||||
MAX_CONNECTIONS,
|
||||
host,
|
||||
port,
|
||||
db,
|
||||
user,
|
||||
password,
|
||||
"", /* cluster */
|
||||
"", /* cluster_secret */
|
||||
"ClickHouseDictionarySource",
|
||||
Protocol::Compression::Enable,
|
||||
secure ? Protocol::Secure::Enable : Protocol::Secure::Disable));
|
||||
return std::make_shared<ConnectionPoolWithFailover>(pools, LoadBalancing::RANDOM);
|
||||
}
|
||||
constexpr size_t MAX_CONNECTIONS = 16;
|
||||
|
||||
inline UInt16 getPortFromContext(const Context & context, bool secure)
|
||||
{
|
||||
return secure ? context.getTCPPortSecure().value_or(0) : context.getTCPPort();
|
||||
}
|
||||
|
||||
ConnectionPoolWithFailoverPtr createPool(const ClickHouseDictionarySource::Configuration & configuration)
|
||||
{
|
||||
if (configuration.is_local)
|
||||
return nullptr;
|
||||
|
||||
ConnectionPoolPtrs pools;
|
||||
pools.emplace_back(std::make_shared<ConnectionPool>(
|
||||
MAX_CONNECTIONS,
|
||||
configuration.host,
|
||||
configuration.port,
|
||||
configuration.db,
|
||||
configuration.user,
|
||||
configuration.password,
|
||||
"", /* cluster */
|
||||
"", /* cluster_secret */
|
||||
"ClickHouseDictionarySource",
|
||||
Protocol::Compression::Enable,
|
||||
configuration.secure ? Protocol::Secure::Enable : Protocol::Secure::Disable));
|
||||
|
||||
return std::make_shared<ConnectionPoolWithFailover>(pools, LoadBalancing::RANDOM);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
ClickHouseDictionarySource::ClickHouseDictionarySource(
|
||||
const DictionaryStructure & dict_struct_,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & path_to_settings,
|
||||
const std::string & config_prefix,
|
||||
const Configuration & configuration_,
|
||||
const Block & sample_block_,
|
||||
const Context & context_,
|
||||
const std::string & default_database)
|
||||
const Context & context_)
|
||||
: update_time{std::chrono::system_clock::from_time_t(0)}
|
||||
, dict_struct{dict_struct_}
|
||||
, secure(config.getBool(config_prefix + ".secure", false))
|
||||
, host{config.getString(config_prefix + ".host", "localhost")}
|
||||
, port(config.getInt(config_prefix + ".port", getPortFromContext(context_, secure)))
|
||||
, user{config.getString(config_prefix + ".user", "default")}
|
||||
, password{config.getString(config_prefix + ".password", "")}
|
||||
, db{config.getString(config_prefix + ".db", default_database)}
|
||||
, table{config.getString(config_prefix + ".table")}
|
||||
, where{config.getString(config_prefix + ".where", "")}
|
||||
, update_field{config.getString(config_prefix + ".update_field", "")}
|
||||
, invalidate_query{config.getString(config_prefix + ".invalidate_query", "")}
|
||||
, query_builder{dict_struct, db, "", table, where, IdentifierQuotingStyle::Backticks}
|
||||
, configuration{configuration_}
|
||||
, query_builder{dict_struct, configuration.db, "", configuration.table, configuration.where, IdentifierQuotingStyle::Backticks}
|
||||
, sample_block{sample_block_}
|
||||
, context(context_)
|
||||
, is_local{isLocalAddress({host, port}, getPortFromContext(context_, secure))}
|
||||
, pool{is_local ? nullptr : createPool(host, port, secure, db, user, password)}
|
||||
, context{context_}
|
||||
, pool{createPool(configuration)}
|
||||
, load_all_query{query_builder.composeLoadAllQuery()}
|
||||
{
|
||||
/// We should set user info even for the case when the dictionary is loaded in-process (without TCP communication).
|
||||
if (is_local)
|
||||
{
|
||||
context.setUser(user, password, Poco::Net::SocketAddress("127.0.0.1", 0));
|
||||
context = copyContextAndApplySettings(path_to_settings, context, config);
|
||||
}
|
||||
|
||||
/// Query context is needed because some code in executeQuery function may assume it exists.
|
||||
/// Current example is Context::getSampleBlockCache from InterpreterSelectWithUnionQuery::getSampleBlock.
|
||||
context.makeQueryContext();
|
||||
}
|
||||
|
||||
|
||||
ClickHouseDictionarySource::ClickHouseDictionarySource(const ClickHouseDictionarySource & other)
|
||||
: update_time{other.update_time}
|
||||
, dict_struct{other.dict_struct}
|
||||
, secure{other.secure}
|
||||
, host{other.host}
|
||||
, port{other.port}
|
||||
, user{other.user}
|
||||
, password{other.password}
|
||||
, db{other.db}
|
||||
, table{other.table}
|
||||
, where{other.where}
|
||||
, update_field{other.update_field}
|
||||
, invalidate_query{other.invalidate_query}
|
||||
, configuration{other.configuration}
|
||||
, invalidate_query_response{other.invalidate_query_response}
|
||||
, query_builder{dict_struct, db, "", table, where, IdentifierQuotingStyle::Backticks}
|
||||
, query_builder{dict_struct, configuration.db, "", configuration.table, configuration.where, IdentifierQuotingStyle::Backticks}
|
||||
, sample_block{other.sample_block}
|
||||
, context(other.context)
|
||||
, is_local{other.is_local}
|
||||
, pool{is_local ? nullptr : createPool(host, port, secure, db, user, password)}
|
||||
, context{other.context}
|
||||
, pool{createPool(configuration)}
|
||||
, load_all_query{other.load_all_query}
|
||||
{
|
||||
context.makeQueryContext();
|
||||
@ -121,7 +96,7 @@ std::string ClickHouseDictionarySource::getUpdateFieldAndDate()
|
||||
time_t hr_time = std::chrono::system_clock::to_time_t(update_time) - 1;
|
||||
std::string str_time = DateLUT::instance().timeToString(hr_time);
|
||||
update_time = std::chrono::system_clock::now();
|
||||
return query_builder.composeUpdateQuery(update_field, str_time);
|
||||
return query_builder.composeUpdateQuery(configuration.update_field, str_time);
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -155,9 +130,9 @@ BlockInputStreamPtr ClickHouseDictionarySource::loadKeys(const Columns & key_col
|
||||
|
||||
bool ClickHouseDictionarySource::isModified() const
|
||||
{
|
||||
if (!invalidate_query.empty())
|
||||
if (!configuration.invalidate_query.empty())
|
||||
{
|
||||
auto response = doInvalidateQuery(invalidate_query);
|
||||
auto response = doInvalidateQuery(configuration.invalidate_query);
|
||||
LOG_TRACE(log, "Invalidate query has returned: {}, previous value: {}", response, invalidate_query_response);
|
||||
if (invalidate_query_response == response)
|
||||
return false;
|
||||
@ -168,21 +143,21 @@ bool ClickHouseDictionarySource::isModified() const
|
||||
|
||||
bool ClickHouseDictionarySource::hasUpdateField() const
|
||||
{
|
||||
return !update_field.empty();
|
||||
return !configuration.update_field.empty();
|
||||
}
|
||||
|
||||
std::string ClickHouseDictionarySource::toString() const
|
||||
{
|
||||
return "ClickHouse: " + db + '.' + table + (where.empty() ? "" : ", where: " + where);
|
||||
const std::string & where = configuration.where;
|
||||
return "ClickHouse: " + configuration.db + '.' + configuration.table + (where.empty() ? "" : ", where: " + where);
|
||||
}
|
||||
|
||||
|
||||
BlockInputStreamPtr ClickHouseDictionarySource::createStreamForQuery(const String & query)
|
||||
{
|
||||
/// Sample block should not contain first row default values
|
||||
auto empty_sample_block = sample_block.cloneEmpty();
|
||||
|
||||
if (is_local)
|
||||
if (configuration.is_local)
|
||||
{
|
||||
auto stream = executeQuery(query, context, true).getInputStream();
|
||||
stream = std::make_shared<ConvertingBlockInputStream>(stream, empty_sample_block, ConvertingBlockInputStream::MatchColumnsMode::Position);
|
||||
@ -195,7 +170,7 @@ BlockInputStreamPtr ClickHouseDictionarySource::createStreamForQuery(const Strin
|
||||
std::string ClickHouseDictionarySource::doInvalidateQuery(const std::string & request) const
|
||||
{
|
||||
LOG_TRACE(log, "Performing invalidate query");
|
||||
if (is_local)
|
||||
if (configuration.is_local)
|
||||
{
|
||||
Context query_context = context;
|
||||
auto input_block = executeQuery(request, query_context, true).getInputStream();
|
||||
@ -210,7 +185,6 @@ std::string ClickHouseDictionarySource::doInvalidateQuery(const std::string & re
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void registerDictionarySourceClickHouse(DictionarySourceFactory & factory)
|
||||
{
|
||||
auto create_table_source = [=](const DictionaryStructure & dict_struct,
|
||||
@ -218,12 +192,48 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory)
|
||||
const std::string & config_prefix,
|
||||
Block & sample_block,
|
||||
const Context & context,
|
||||
const std::string & default_database,
|
||||
const std::string & default_database [[maybe_unused]],
|
||||
bool /* check_config */) -> DictionarySourcePtr
|
||||
{
|
||||
return std::make_unique<ClickHouseDictionarySource>(
|
||||
dict_struct, config, config_prefix, config_prefix + ".clickhouse", sample_block, context, default_database);
|
||||
bool secure = config.getBool(config_prefix + ".secure", false);
|
||||
Context context_copy = context;
|
||||
|
||||
UInt16 default_port = getPortFromContext(context_copy, secure);
|
||||
|
||||
std::string settings_config_prefix = config_prefix + ".clickhouse";
|
||||
std::string host = config.getString(settings_config_prefix + ".host", "localhost");
|
||||
UInt16 port = static_cast<UInt16>(config.getUInt(settings_config_prefix + ".port", default_port));
|
||||
|
||||
ClickHouseDictionarySource::Configuration configuration {
|
||||
.secure = config.getBool(settings_config_prefix + ".secure", false),
|
||||
.host = host,
|
||||
.port = port,
|
||||
.user = config.getString(settings_config_prefix + ".user", "default"),
|
||||
.password = config.getString(settings_config_prefix + ".password", ""),
|
||||
.db = config.getString(settings_config_prefix + ".db", default_database),
|
||||
.table = config.getString(settings_config_prefix + ".table"),
|
||||
.where = config.getString(settings_config_prefix + ".where", ""),
|
||||
.update_field = config.getString(settings_config_prefix + ".update_field", ""),
|
||||
.invalidate_query = config.getString(settings_config_prefix + ".invalidate_query", ""),
|
||||
.is_local = isLocalAddress({host, port}, default_port)
|
||||
};
|
||||
|
||||
/// We should set user info even for the case when the dictionary is loaded in-process (without TCP communication).
|
||||
if (configuration.is_local)
|
||||
{
|
||||
context_copy.setUser(configuration.user, configuration.password, Poco::Net::SocketAddress("127.0.0.1", 0));
|
||||
context_copy = copyContextAndApplySettings(config_prefix, context_copy, config);
|
||||
}
|
||||
|
||||
String dictionary_name = config.getString(".dictionary.name", "");
|
||||
String dictionary_database = config.getString(".dictionary.database", "");
|
||||
|
||||
if (dictionary_name == configuration.table && dictionary_database == configuration.db)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "ClickHouseDictionarySource table cannot be dictionary table");
|
||||
|
||||
return std::make_unique<ClickHouseDictionarySource>(dict_struct, configuration, sample_block, context_copy);
|
||||
};
|
||||
|
||||
factory.registerSource("clickhouse", create_table_source);
|
||||
}
|
||||
|
||||
|
@ -18,14 +18,26 @@ namespace DB
|
||||
class ClickHouseDictionarySource final : public IDictionarySource
|
||||
{
|
||||
public:
|
||||
struct Configuration
|
||||
{
|
||||
const bool secure;
|
||||
const std::string host;
|
||||
const UInt16 port;
|
||||
const std::string user;
|
||||
const std::string password;
|
||||
const std::string db;
|
||||
const std::string table;
|
||||
const std::string where;
|
||||
const std::string update_field;
|
||||
const std::string invalidate_query;
|
||||
const bool is_local;
|
||||
};
|
||||
|
||||
ClickHouseDictionarySource(
|
||||
const DictionaryStructure & dict_struct_,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & path_to_settings,
|
||||
const std::string & config_prefix,
|
||||
const Configuration & configuration_,
|
||||
const Block & sample_block_,
|
||||
const Context & context,
|
||||
const std::string & default_database);
|
||||
const Context & context);
|
||||
|
||||
/// copy-constructor is provided in order to support cloneability
|
||||
ClickHouseDictionarySource(const ClickHouseDictionarySource & other);
|
||||
@ -50,7 +62,7 @@ public:
|
||||
|
||||
/// Used for detection whether the hashtable should be preallocated
|
||||
/// (since if there is WHERE then it can filter out too much)
|
||||
bool hasWhere() const { return !where.empty(); }
|
||||
bool hasWhere() const { return !configuration.where.empty(); }
|
||||
|
||||
private:
|
||||
std::string getUpdateFieldAndDate();
|
||||
@ -61,21 +73,11 @@ private:
|
||||
|
||||
std::chrono::time_point<std::chrono::system_clock> update_time;
|
||||
const DictionaryStructure dict_struct;
|
||||
const bool secure;
|
||||
const std::string host;
|
||||
const UInt16 port;
|
||||
const std::string user;
|
||||
const std::string password;
|
||||
const std::string db;
|
||||
const std::string table;
|
||||
const std::string where;
|
||||
const std::string update_field;
|
||||
std::string invalidate_query;
|
||||
const Configuration configuration;
|
||||
mutable std::string invalidate_query_response;
|
||||
ExternalQueryBuilder query_builder;
|
||||
Block sample_block;
|
||||
Context context;
|
||||
const bool is_local;
|
||||
ConnectionPoolWithFailoverPtr pool;
|
||||
const std::string load_all_query;
|
||||
Poco::Logger * log = &Poco::Logger::get("ClickHouseDictionarySource");
|
||||
|
@ -200,7 +200,7 @@ ColumnPtr HashedDictionary<dictionary_key_type, sparse>::getHierarchy(ColumnPtr
|
||||
const auto & dictionary_attribute = dict_struct.attributes[hierarchical_attribute_index];
|
||||
const auto & hierarchical_attribute = attributes[hierarchical_attribute_index];
|
||||
|
||||
const UInt64 null_value = dictionary_attribute.null_value.get<UInt64>();
|
||||
const UInt64 null_value = dictionary_attribute.null_value.template get<UInt64>();
|
||||
const CollectionType<UInt64> & parent_keys_map = std::get<CollectionType<UInt64>>(hierarchical_attribute.container);
|
||||
|
||||
auto is_key_valid_func = [&](auto & key) { return parent_keys_map.find(key) != parent_keys_map.end(); };
|
||||
@ -246,7 +246,7 @@ ColumnUInt8::Ptr HashedDictionary<dictionary_key_type, sparse>::isInHierarchy(
|
||||
const auto & dictionary_attribute = dict_struct.attributes[hierarchical_attribute_index];
|
||||
auto & hierarchical_attribute = attributes[hierarchical_attribute_index];
|
||||
|
||||
const UInt64 null_value = dictionary_attribute.null_value.get<UInt64>();
|
||||
const UInt64 null_value = dictionary_attribute.null_value.template get<UInt64>();
|
||||
const CollectionType<UInt64> & parent_keys_map = std::get<CollectionType<UInt64>>(hierarchical_attribute.container);
|
||||
|
||||
auto is_key_valid_func = [&](auto & key) { return parent_keys_map.find(key) != parent_keys_map.end(); };
|
||||
@ -327,14 +327,14 @@ void HashedDictionary<dictionary_key_type, sparse>::createAttributes()
|
||||
{
|
||||
string_arena = std::make_unique<Arena>();
|
||||
|
||||
const auto & string_null_value = dictionary_attribute.null_value.get<String>();
|
||||
const auto & string_null_value = dictionary_attribute.null_value.template get<String>();
|
||||
const size_t string_null_value_size = string_null_value.size();
|
||||
|
||||
const char * string_in_arena = string_arena->insert(string_null_value.data(), string_null_value_size);
|
||||
default_value = {string_in_arena, string_null_value_size};
|
||||
}
|
||||
else
|
||||
default_value = dictionary_attribute.null_value.get<NearestFieldType<ValueType>>();
|
||||
default_value = dictionary_attribute.null_value.template get<NearestFieldType<ValueType>>();
|
||||
|
||||
Attribute attribute{dictionary_attribute.underlying_type, std::move(is_nullable_set), default_value, CollectionType<ValueType>(), std::move(string_arena)};
|
||||
attributes.emplace_back(std::move(attribute));
|
||||
|
@ -21,6 +21,8 @@
|
||||
|
||||
#ifdef __SSE4_1__
|
||||
#include <smmintrin.h>
|
||||
#else
|
||||
#include <fenv.h>
|
||||
#endif
|
||||
|
||||
|
||||
@ -34,6 +36,7 @@ namespace ErrorCodes
|
||||
extern const int ARGUMENT_OUT_OF_BOUND;
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
extern const int BAD_ARGUMENTS;
|
||||
extern const int CANNOT_SET_ROUNDING_MODE;
|
||||
}
|
||||
|
||||
|
||||
@ -231,7 +234,7 @@ inline float roundWithMode(float x, RoundingMode mode)
|
||||
{
|
||||
switch (mode)
|
||||
{
|
||||
case RoundingMode::Round: return roundf(x);
|
||||
case RoundingMode::Round: return nearbyintf(x);
|
||||
case RoundingMode::Floor: return floorf(x);
|
||||
case RoundingMode::Ceil: return ceilf(x);
|
||||
case RoundingMode::Trunc: return truncf(x);
|
||||
@ -244,7 +247,7 @@ inline double roundWithMode(double x, RoundingMode mode)
|
||||
{
|
||||
switch (mode)
|
||||
{
|
||||
case RoundingMode::Round: return round(x);
|
||||
case RoundingMode::Round: return nearbyint(x);
|
||||
case RoundingMode::Floor: return floor(x);
|
||||
case RoundingMode::Ceil: return ceil(x);
|
||||
case RoundingMode::Trunc: return trunc(x);
|
||||
@ -595,6 +598,15 @@ public:
|
||||
return false;
|
||||
};
|
||||
|
||||
#if !defined(__SSE4_1__)
|
||||
/// In case of "nearbyint" function is used, we should ensure the expected rounding mode for the Banker's rounding.
|
||||
/// Actually it is by default. But we will set it just in case.
|
||||
|
||||
if constexpr (rounding_mode == RoundingMode::Round)
|
||||
if (0 != fesetround(FE_TONEAREST))
|
||||
throw Exception("Cannot set floating point rounding mode", ErrorCodes::CANNOT_SET_ROUNDING_MODE);
|
||||
#endif
|
||||
|
||||
if (!callOnIndexAndDataType<void>(column.type->getTypeId(), call))
|
||||
{
|
||||
throw Exception("Illegal column " + column.name + " of argument of function " + getName(),
|
||||
|
@ -357,6 +357,8 @@ void TinyLogBlockOutputStream::writeSuffix()
|
||||
for (const auto & file : column_files)
|
||||
storage.file_checker.update(file);
|
||||
storage.file_checker.save();
|
||||
|
||||
lock.unlock();
|
||||
}
|
||||
|
||||
|
||||
|
@ -70,7 +70,7 @@ private:
|
||||
Files files;
|
||||
|
||||
FileChecker file_checker;
|
||||
mutable std::shared_timed_mutex rwlock;
|
||||
std::shared_timed_mutex rwlock;
|
||||
|
||||
Poco::Logger * log;
|
||||
|
||||
|
@ -122,8 +122,8 @@ def test_attach_without_fetching(start_cluster):
|
||||
|
||||
with PartitionManager() as pm:
|
||||
# If something goes wrong and replica 2 wants to fetch data, the test will fail.
|
||||
pm.partition_instances(node_2, node_1)
|
||||
pm.partition_instances(node_1, node_3)
|
||||
pm.partition_instances(node_2, node_1, action='REJECT --reject-with tcp-reset')
|
||||
pm.partition_instances(node_1, node_3, action='REJECT --reject-with tcp-reset')
|
||||
|
||||
node_1.query("ALTER TABLE test ATTACH PART '0_0_0_0'")
|
||||
|
||||
|
@ -1,11 +1,11 @@
|
||||
0
|
||||
4416930539393268817 1241149650 9
|
||||
4761183170873013810 4249604106 0
|
||||
7766709361750702608 3902320246 4
|
||||
9624464864560415994 1298551497 3
|
||||
10577349846663553072 1343103100 1
|
||||
11700034558374135620 1618865725 8
|
||||
18198135717204167749 1996614413 2
|
||||
4761183170873013810 4249604106 0
|
||||
12742043333840853032 1295823179 6
|
||||
9624464864560415994 1298551497 3
|
||||
4416930539393268817 1241149650 9
|
||||
15228578409069794350 2641603337 5
|
||||
13365811232860260488 3844986530 7
|
||||
15228578409069794350 2641603337 5
|
||||
18198135717204167749 1996614413 2
|
||||
|
@ -3,4 +3,4 @@ SELECT value FROM system.one ANY LEFT JOIN (SELECT dummy, dummy AS value) js2 US
|
||||
SELECT value1, value2, sum(number)
|
||||
FROM (SELECT number, intHash64(number) AS value1 FROM system.numbers LIMIT 10) js1
|
||||
ANY LEFT JOIN (SELECT number, intHash32(number) AS value2 FROM system.numbers LIMIT 10) js2
|
||||
USING number GROUP BY value1, value2;
|
||||
USING number GROUP BY value1, value2 ORDER BY value1, value2;
|
||||
|
@ -1,5 +1,5 @@
|
||||
SELECT emptyArrayToSingle(arrayFilter(x -> x != 99, arrayJoin([[1, 2], [99], [4, 5, 6]])));
|
||||
SELECT emptyArrayToSingle(emptyArrayString()), emptyArrayToSingle(emptyArrayDate()), emptyArrayToSingle(emptyArrayDateTime());
|
||||
SELECT emptyArrayToSingle(emptyArrayString()), emptyArrayToSingle(emptyArrayDate()), emptyArrayToSingle(arrayFilter(x -> 0, [now('Europe/Moscow')]));
|
||||
|
||||
SELECT
|
||||
emptyArrayToSingle(range(number % 3)),
|
||||
|
@ -20,51 +20,51 @@
|
||||
170.21 MiB 170.21 MiB 170.21 MiB
|
||||
462.69 MiB 462.69 MiB 462.69 MiB
|
||||
1.23 GiB 1.23 GiB 1.23 GiB
|
||||
3.34 GiB 3.34 GiB -2.00 GiB
|
||||
9.08 GiB 9.08 GiB -2.00 GiB
|
||||
24.67 GiB 24.67 GiB -2.00 GiB
|
||||
67.06 GiB 67.06 GiB -2.00 GiB
|
||||
182.29 GiB 182.29 GiB -2.00 GiB
|
||||
495.51 GiB 495.51 GiB -2.00 GiB
|
||||
1.32 TiB 1.32 TiB -2.00 GiB
|
||||
3.58 TiB 3.58 TiB -2.00 GiB
|
||||
9.72 TiB 9.72 TiB -2.00 GiB
|
||||
26.42 TiB 26.42 TiB -2.00 GiB
|
||||
71.82 TiB 71.82 TiB -2.00 GiB
|
||||
195.22 TiB 195.22 TiB -2.00 GiB
|
||||
530.66 TiB 530.66 TiB -2.00 GiB
|
||||
1.41 PiB 1.41 PiB -2.00 GiB
|
||||
3.83 PiB 3.83 PiB -2.00 GiB
|
||||
10.41 PiB 10.41 PiB -2.00 GiB
|
||||
28.29 PiB 28.29 PiB -2.00 GiB
|
||||
76.91 PiB 76.91 PiB -2.00 GiB
|
||||
209.06 PiB 209.06 PiB -2.00 GiB
|
||||
568.30 PiB 568.30 PiB -2.00 GiB
|
||||
1.51 EiB 1.51 EiB -2.00 GiB
|
||||
4.10 EiB 4.10 EiB -2.00 GiB
|
||||
11.15 EiB 11.15 EiB -2.00 GiB
|
||||
30.30 EiB 0.00 B -2.00 GiB
|
||||
82.37 EiB 0.00 B -2.00 GiB
|
||||
223.89 EiB 0.00 B -2.00 GiB
|
||||
608.60 EiB 0.00 B -2.00 GiB
|
||||
1.62 ZiB 0.00 B -2.00 GiB
|
||||
4.39 ZiB 0.00 B -2.00 GiB
|
||||
11.94 ZiB 0.00 B -2.00 GiB
|
||||
32.45 ZiB 0.00 B -2.00 GiB
|
||||
88.21 ZiB 0.00 B -2.00 GiB
|
||||
239.77 ZiB 0.00 B -2.00 GiB
|
||||
651.77 ZiB 0.00 B -2.00 GiB
|
||||
1.73 YiB 0.00 B -2.00 GiB
|
||||
4.70 YiB 0.00 B -2.00 GiB
|
||||
12.78 YiB 0.00 B -2.00 GiB
|
||||
34.75 YiB 0.00 B -2.00 GiB
|
||||
94.46 YiB 0.00 B -2.00 GiB
|
||||
256.78 YiB 0.00 B -2.00 GiB
|
||||
698.00 YiB 0.00 B -2.00 GiB
|
||||
1897.37 YiB 0.00 B -2.00 GiB
|
||||
5157.59 YiB 0.00 B -2.00 GiB
|
||||
14019.80 YiB 0.00 B -2.00 GiB
|
||||
38109.75 YiB 0.00 B -2.00 GiB
|
||||
103593.05 YiB 0.00 B -2.00 GiB
|
||||
281595.11 YiB 0.00 B -2.00 GiB
|
||||
765454.88 YiB 0.00 B -2.00 GiB
|
||||
3.34 GiB 3.34 GiB 2.00 GiB
|
||||
9.08 GiB 9.08 GiB 2.00 GiB
|
||||
24.67 GiB 24.67 GiB 2.00 GiB
|
||||
67.06 GiB 67.06 GiB 2.00 GiB
|
||||
182.29 GiB 182.29 GiB 2.00 GiB
|
||||
495.51 GiB 495.51 GiB 2.00 GiB
|
||||
1.32 TiB 1.32 TiB 2.00 GiB
|
||||
3.58 TiB 3.58 TiB 2.00 GiB
|
||||
9.72 TiB 9.72 TiB 2.00 GiB
|
||||
26.42 TiB 26.42 TiB 2.00 GiB
|
||||
71.82 TiB 71.82 TiB 2.00 GiB
|
||||
195.22 TiB 195.22 TiB 2.00 GiB
|
||||
530.66 TiB 530.66 TiB 2.00 GiB
|
||||
1.41 PiB 1.41 PiB 2.00 GiB
|
||||
3.83 PiB 3.83 PiB 2.00 GiB
|
||||
10.41 PiB 10.41 PiB 2.00 GiB
|
||||
28.29 PiB 28.29 PiB 2.00 GiB
|
||||
76.91 PiB 76.91 PiB 2.00 GiB
|
||||
209.06 PiB 209.06 PiB 2.00 GiB
|
||||
568.30 PiB 568.30 PiB 2.00 GiB
|
||||
1.51 EiB 1.51 EiB 2.00 GiB
|
||||
4.10 EiB 4.10 EiB 2.00 GiB
|
||||
11.15 EiB 11.15 EiB 2.00 GiB
|
||||
30.30 EiB 16.00 EiB 2.00 GiB
|
||||
82.37 EiB 16.00 EiB 2.00 GiB
|
||||
223.89 EiB 16.00 EiB 2.00 GiB
|
||||
608.60 EiB 16.00 EiB 2.00 GiB
|
||||
1.62 ZiB 16.00 EiB 2.00 GiB
|
||||
4.39 ZiB 16.00 EiB 2.00 GiB
|
||||
11.94 ZiB 16.00 EiB 2.00 GiB
|
||||
32.45 ZiB 16.00 EiB 2.00 GiB
|
||||
88.21 ZiB 16.00 EiB 2.00 GiB
|
||||
239.77 ZiB 16.00 EiB 2.00 GiB
|
||||
651.77 ZiB 16.00 EiB 2.00 GiB
|
||||
1.73 YiB 16.00 EiB 2.00 GiB
|
||||
4.70 YiB 16.00 EiB 2.00 GiB
|
||||
12.78 YiB 16.00 EiB 2.00 GiB
|
||||
34.75 YiB 16.00 EiB 2.00 GiB
|
||||
94.46 YiB 16.00 EiB 2.00 GiB
|
||||
256.78 YiB 16.00 EiB 2.00 GiB
|
||||
698.00 YiB 16.00 EiB 2.00 GiB
|
||||
1897.37 YiB 16.00 EiB 2.00 GiB
|
||||
5157.59 YiB 16.00 EiB 2.00 GiB
|
||||
14019.80 YiB 16.00 EiB 2.00 GiB
|
||||
38109.75 YiB 16.00 EiB 2.00 GiB
|
||||
103593.05 YiB 16.00 EiB 2.00 GiB
|
||||
281595.11 YiB 16.00 EiB 2.00 GiB
|
||||
765454.88 YiB 16.00 EiB 2.00 GiB
|
||||
|
@ -1,4 +1,4 @@
|
||||
WITH round(exp(number), 6) AS x, toUInt64(x) AS y, toInt32(x) AS z
|
||||
WITH round(exp(number), 6) AS x, x > 0xFFFFFFFFFFFFFFFF ? 0xFFFFFFFFFFFFFFFF : toUInt64(x) AS y, x > 0x7FFFFFFF ? 0x7FFFFFFF : toInt32(x) AS z
|
||||
SELECT formatReadableSize(x), formatReadableSize(y), formatReadableSize(z)
|
||||
FROM system.numbers
|
||||
LIMIT 70;
|
||||
|
@ -3,5 +3,3 @@
|
||||
a
|
||||
a
|
||||
---
|
||||
a
|
||||
a
|
||||
|
@ -43,7 +43,7 @@ SELECT * FROM d;
|
||||
SELECT '---';
|
||||
|
||||
INSERT INTO m VALUES ('b');
|
||||
SELECT v FROM d ORDER BY v; -- { clientError 36 }
|
||||
SELECT toString(v) FROM (SELECT v FROM d ORDER BY v) FORMAT Null; -- { serverError 36 }
|
||||
|
||||
|
||||
DROP TABLE m;
|
||||
|
@ -5,7 +5,7 @@
|
||||
"host": "clickhouse-test-host-001.clickhouse.com",
|
||||
"home": "clickhouse",
|
||||
"detail": "clickhouse",
|
||||
"row_number": "999998"
|
||||
"row_number": "99998"
|
||||
},
|
||||
{
|
||||
"datetime": "2020-12-12",
|
||||
@ -13,11 +13,11 @@
|
||||
"host": "clickhouse-test-host-001.clickhouse.com",
|
||||
"home": "clickhouse",
|
||||
"detail": "clickhouse",
|
||||
"row_number": "999999"
|
||||
"row_number": "99999"
|
||||
}
|
||||
],
|
||||
|
||||
"rows": 1000000,
|
||||
"rows": 100000,
|
||||
|
||||
"rows_before_limit_at_least": 1048080,
|
||||
"rows_before_limit_at_least": 131010,
|
||||
|
||||
|
@ -4,4 +4,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
${CLICKHOUSE_CURL} -sS -H 'Accept-Encoding: gzip' "${CLICKHOUSE_URL}&enable_http_compression=1&http_zlib_compression_level=1" -d "SELECT toDate('2020-12-12') as datetime, 'test-pipeline' as pipeline, 'clickhouse-test-host-001.clickhouse.com' as host, 'clickhouse' as home, 'clickhouse' as detail, number as row_number FROM numbers(1000000) FORMAT JSON" | gzip -d | tail -n30 | head -n23
|
||||
${CLICKHOUSE_CURL} -sS -H 'Accept-Encoding: gzip' "${CLICKHOUSE_URL}&enable_http_compression=1&http_zlib_compression_level=1" -d "SELECT toDate('2020-12-12') as datetime, 'test-pipeline' as pipeline, 'clickhouse-test-host-001.clickhouse.com' as host, 'clickhouse' as home, 'clickhouse' as detail, number as row_number FROM numbers(100000) FORMAT JSON" | gzip -d | tail -n30 | head -n23
|
||||
|
@ -0,0 +1,3 @@
|
||||
1 1
|
||||
2 2
|
||||
3 3
|
@ -0,0 +1,53 @@
|
||||
DROP DATABASE IF EXISTS 01780_db;
|
||||
CREATE DATABASE 01780_db;
|
||||
|
||||
DROP DICTIONARY IF EXISTS dict1;
|
||||
CREATE DICTIONARY dict1
|
||||
(
|
||||
id UInt64,
|
||||
value String
|
||||
)
|
||||
PRIMARY KEY id
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 TABLE 'dict1'))
|
||||
LAYOUT(DIRECT());
|
||||
|
||||
SELECT * FROM dict1; --{serverError 36}
|
||||
|
||||
DROP DICTIONARY dict1;
|
||||
|
||||
DROP DICTIONARY IF EXISTS dict2;
|
||||
CREATE DICTIONARY 01780_db.dict2
|
||||
(
|
||||
id UInt64,
|
||||
value String
|
||||
)
|
||||
PRIMARY KEY id
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 DATABASE '01780_db' TABLE 'dict2'))
|
||||
LAYOUT(DIRECT());
|
||||
|
||||
SELECT * FROM 01780_db.dict2; --{serverError 36}
|
||||
DROP DICTIONARY 01780_db.dict2;
|
||||
|
||||
DROP TABLE IF EXISTS 01780_db.dict3_source;
|
||||
CREATE TABLE 01780_db.dict3_source
|
||||
(
|
||||
id UInt64,
|
||||
value String
|
||||
) ENGINE = TinyLog;
|
||||
|
||||
INSERT INTO 01780_db.dict3_source VALUES (1, '1'), (2, '2'), (3, '3');
|
||||
|
||||
CREATE DICTIONARY 01780_db.dict3
|
||||
(
|
||||
id UInt64,
|
||||
value String
|
||||
)
|
||||
PRIMARY KEY id
|
||||
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 TABLE 'dict3_source' DATABASE '01780_db'))
|
||||
LAYOUT(DIRECT());
|
||||
|
||||
SELECT * FROM 01780_db.dict3;
|
||||
|
||||
DROP DICTIONARY 01780_db.dict3;
|
||||
|
||||
DROP DATABASE 01780_db;
|
@ -0,0 +1 @@
|
||||
1
|
@ -0,0 +1 @@
|
||||
SELECT 1 GROUP BY emptyArrayToSingle(arrayFilter(x -> 1, []));
|
@ -696,6 +696,7 @@
|
||||
"01685_ssd_cache_dictionary_complex_key",
|
||||
"01760_system_dictionaries",
|
||||
"01760_polygon_dictionaries",
|
||||
"01778_hierarchical_dictionaries"
|
||||
"01778_hierarchical_dictionaries",
|
||||
"01780_clickhouse_dictionary_source_loop"
|
||||
]
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user