Merge branch 'master' into merge_tree_deduplication

This commit is contained in:
alesapin 2021-04-04 18:32:27 +03:00
commit 0ceb95aa55
28 changed files with 309 additions and 178 deletions

View File

@ -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
View 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}")

View File

@ -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}}"

View File

@ -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).");

View File

@ -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)

View File

@ -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

View File

@ -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) \

View File

@ -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);
}

View File

@ -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");

View File

@ -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));

View File

@ -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(),

View File

@ -357,6 +357,8 @@ void TinyLogBlockOutputStream::writeSuffix()
for (const auto & file : column_files)
storage.file_checker.update(file);
storage.file_checker.save();
lock.unlock();
}

View File

@ -70,7 +70,7 @@ private:
Files files;
FileChecker file_checker;
mutable std::shared_timed_mutex rwlock;
std::shared_timed_mutex rwlock;
Poco::Logger * log;

View File

@ -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'")

View File

@ -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

View File

@ -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;

View File

@ -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)),

View File

@ -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

View File

@ -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;

View File

@ -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;

View File

@ -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,

View File

@ -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

View File

@ -0,0 +1,3 @@
1 1
2 2
3 3

View File

@ -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;

View File

@ -0,0 +1 @@
SELECT 1 GROUP BY emptyArrayToSingle(arrayFilter(x -> 1, []));

View File

@ -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"
]
}