mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 09:32:06 +00:00
Merge remote-tracking branch 'upstream/master' into fix26
This commit is contained in:
commit
fb3abc2fe3
@ -26,7 +26,7 @@
|
||||
* Исправлено undefined behaviour в функции `dictIsIn` для словарей типа `cache`. [#4515](https://github.com/yandex/ClickHouse/pull/4515) ([alesapin](https://github.com/alesapin))
|
||||
* Исправлен deadlock в случае, если запрос SELECT блокирует одну и ту же таблицу несколько раз (например - из разных потоков, либо при выполнении разных подзапросов) и одновременно с этим производится DDL запрос. [#4535](https://github.com/yandex/ClickHouse/pull/4535) ([Alex Zatelepin](https://github.com/ztlpn))
|
||||
* Настройка `compile_expressions` выключена по-умолчанию до тех пор, пока мы не зафиксируем исходники используемой библиотеки `LLVM` и не будем проверять её под `ASan` (сейчас библиотека LLVM берётся из системы). [#4579](https://github.com/yandex/ClickHouse/pull/4579) ([alesapin](https://github.com/alesapin))
|
||||
* Исправлено падение по `std::terminate`, если `invalidate_query` для внешних словарей с истоником `clickhouse` вернул неправильный результат (пустой; более чем одну строку; более чем один столбец). Исправлена ошибка, из-за которой запрос `invalidate_query` производился каждые пять секунд, независимо от указанного `lifetime`. [#4583](https://github.com/yandex/ClickHouse/pull/4583) ([alexey-milovidov](https://github.com/alexey-milovidov))
|
||||
* Исправлено падение по `std::terminate`, если `invalidate_query` для внешних словарей с источником `clickhouse` вернул неправильный результат (пустой; более чем одну строку; более чем один столбец). Исправлена ошибка, из-за которой запрос `invalidate_query` производился каждые пять секунд, независимо от указанного `lifetime`. [#4583](https://github.com/yandex/ClickHouse/pull/4583) ([alexey-milovidov](https://github.com/alexey-milovidov))
|
||||
* Исправлен deadlock в случае, если запрос `invalidate_query` для внешнего словаря с источником `clickhouse` использовал таблицу `system.dictionaries` или базу данных типа `Dictionary` (редкий случай). [#4599](https://github.com/yandex/ClickHouse/pull/4599) ([alexey-milovidov](https://github.com/alexey-milovidov))
|
||||
* Исправлена работа CROSS JOIN с пустым WHERE [#4598](https://github.com/yandex/ClickHouse/pull/4598) ([Artem Zuikov](https://github.com/4ertus2))
|
||||
* Исправлен segfault в функции `replicate` с константным аргументом. [#4603](https://github.com/yandex/ClickHouse/pull/4603) ([alexey-milovidov](https://github.com/alexey-milovidov))
|
||||
|
@ -420,6 +420,7 @@ namespace ErrorCodes
|
||||
extern const int NO_COMMON_COLUMNS_WITH_PROTOBUF_SCHEMA = 443;
|
||||
extern const int UNKNOWN_PROTOBUF_FORMAT = 444;
|
||||
extern const int CANNOT_MPROTECT = 445;
|
||||
extern const int FUNCTION_NOT_ALLOWED = 446;
|
||||
|
||||
extern const int KEEPER_EXCEPTION = 999;
|
||||
extern const int POCO_EXCEPTION = 1000;
|
||||
|
@ -305,7 +305,9 @@ struct Settings
|
||||
M(SettingBool, allow_experimental_cross_to_join_conversion, true, "Convert CROSS JOIN to INNER JOIN if possible") \
|
||||
M(SettingBool, cancel_http_readonly_queries_on_client_close, false, "Cancel HTTP readonly queries when a client closes the connection without waiting for response.") \
|
||||
M(SettingBool, external_table_functions_use_nulls, true, "If it is set to true, external table functions will implicitly use Nullable type if needed. Otherwise NULLs will be substituted with default values. Currently supported only for 'mysql' table function.") \
|
||||
M(SettingBool, allow_experimental_data_skipping_indices, false, "If it is set to true, data skipping indices can be used in CREATE TABLE/ALTER TABLE queries.")\
|
||||
M(SettingBool, allow_experimental_data_skipping_indices, false, "If it is set to true, data skipping indices can be used in CREATE TABLE/ALTER TABLE queries.") \
|
||||
\
|
||||
M(SettingBool, allow_hyperscan, true, "Allow functions that use Hyperscan library. Disable to avoid potentially long compilation times and excessive resource usage.") \
|
||||
|
||||
#define DECLARE(TYPE, NAME, DEFAULT, DESCRIPTION) \
|
||||
TYPE NAME {DEFAULT};
|
||||
|
@ -340,6 +340,7 @@ template <typename Impl>
|
||||
struct MultiSearchImpl
|
||||
{
|
||||
using ResultType = UInt8;
|
||||
static constexpr bool is_using_hyperscan = false;
|
||||
|
||||
static void vector_constant(
|
||||
const ColumnString::Chars & haystack_data,
|
||||
@ -355,6 +356,7 @@ template <typename Impl>
|
||||
struct MultiSearchFirstPositionImpl
|
||||
{
|
||||
using ResultType = UInt64;
|
||||
static constexpr bool is_using_hyperscan = false;
|
||||
|
||||
static void vector_constant(
|
||||
const ColumnString::Chars & haystack_data,
|
||||
@ -374,6 +376,7 @@ template <typename Impl>
|
||||
struct MultiSearchFirstIndexImpl
|
||||
{
|
||||
using ResultType = UInt64;
|
||||
static constexpr bool is_using_hyperscan = false;
|
||||
|
||||
static void vector_constant(
|
||||
const ColumnString::Chars & haystack_data,
|
||||
@ -610,6 +613,7 @@ struct MultiMatchAnyImpl
|
||||
{
|
||||
static_assert(static_cast<int>(FindAny) + static_cast<int>(FindAnyIndex) == 1);
|
||||
using ResultType = Type;
|
||||
static constexpr bool is_using_hyperscan = true;
|
||||
|
||||
static void vector_constant(
|
||||
const ColumnString::Chars & haystack_data,
|
||||
|
@ -11,8 +11,10 @@
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <common/StringRef.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
/** Search and replace functions in strings:
|
||||
@ -67,6 +69,7 @@ namespace ErrorCodes
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
extern const int FUNCTION_NOT_ALLOWED;
|
||||
}
|
||||
|
||||
template <typename Impl, typename Name>
|
||||
@ -207,6 +210,8 @@ public:
|
||||
String getName() const override { return name; }
|
||||
|
||||
size_t getNumberOfArguments() const override { return 2; }
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
@ -285,11 +290,19 @@ class FunctionsMultiStringSearch : public IFunction
|
||||
|
||||
public:
|
||||
static constexpr auto name = Name::name;
|
||||
static FunctionPtr create(const Context &) { return std::make_shared<FunctionsMultiStringSearch>(); }
|
||||
static FunctionPtr create(const Context & context)
|
||||
{
|
||||
if (Impl::is_using_hyperscan && !context.getSettingsRef().allow_hyperscan)
|
||||
throw Exception("Hyperscan functions are disabled, because setting 'allow_hyperscan' is set to 0", ErrorCodes::FUNCTION_NOT_ALLOWED);
|
||||
|
||||
return std::make_shared<FunctionsMultiStringSearch>();
|
||||
}
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
size_t getNumberOfArguments() const override { return 2; }
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
|
@ -32,23 +32,54 @@ namespace ErrorCodes
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
}
|
||||
|
||||
static NameSet requiredRightKeys(const Names & key_names, const NamesAndTypesList & columns_added_by_join)
|
||||
{
|
||||
NameSet required;
|
||||
|
||||
static std::unordered_map<String, DataTypePtr> requiredRightKeys(const Names & key_names, const NamesAndTypesList & columns_added_by_join)
|
||||
{
|
||||
NameSet right_keys;
|
||||
for (const auto & name : key_names)
|
||||
right_keys.insert(name);
|
||||
|
||||
std::unordered_map<String, DataTypePtr> required;
|
||||
for (const auto & column : columns_added_by_join)
|
||||
{
|
||||
if (right_keys.count(column.name))
|
||||
required.insert(column.name);
|
||||
}
|
||||
required.insert({column.name, column.type});
|
||||
|
||||
return required;
|
||||
}
|
||||
|
||||
static void convertColumnToNullable(ColumnWithTypeAndName & column)
|
||||
{
|
||||
if (column.type->isNullable())
|
||||
return;
|
||||
|
||||
column.type = makeNullable(column.type);
|
||||
if (column.column)
|
||||
column.column = makeNullable(column.column);
|
||||
}
|
||||
|
||||
/// Converts column to nullable if needed. No backward convertion.
|
||||
static ColumnWithTypeAndName correctNullability(ColumnWithTypeAndName && column, bool nullable)
|
||||
{
|
||||
if (nullable)
|
||||
convertColumnToNullable(column);
|
||||
return std::move(column);
|
||||
}
|
||||
|
||||
static ColumnWithTypeAndName correctNullability(ColumnWithTypeAndName && column, bool nullable, const ColumnUInt8 & negative_null_map)
|
||||
{
|
||||
if (nullable)
|
||||
{
|
||||
convertColumnToNullable(column);
|
||||
if (negative_null_map.size())
|
||||
{
|
||||
MutableColumnPtr mutable_column = (*std::move(column.column)).mutate();
|
||||
static_cast<ColumnNullable &>(*mutable_column).applyNegatedNullMap(negative_null_map);
|
||||
column.column = std::move(mutable_column);
|
||||
}
|
||||
}
|
||||
return std::move(column);
|
||||
}
|
||||
|
||||
|
||||
Join::Join(const Names & key_names_right_, bool use_nulls_, const SizeLimits & limits,
|
||||
ASTTableJoin::Kind kind_, ASTTableJoin::Strictness strictness_, bool any_take_last_row_)
|
||||
@ -215,15 +246,6 @@ size_t Join::getTotalByteCount() const
|
||||
return res;
|
||||
}
|
||||
|
||||
|
||||
static void convertColumnToNullable(ColumnWithTypeAndName & column)
|
||||
{
|
||||
column.type = makeNullable(column.type);
|
||||
if (column.column)
|
||||
column.column = makeNullable(column.column);
|
||||
}
|
||||
|
||||
|
||||
void Join::setSampleBlock(const Block & block)
|
||||
{
|
||||
std::unique_lock lock(rwlock);
|
||||
@ -712,7 +734,7 @@ void Join::joinBlockImpl(
|
||||
|
||||
std::unique_ptr<IColumn::Offsets> offsets_to_replicate;
|
||||
|
||||
IColumn::Filter filter = switchJoinRightColumns<KIND, STRICTNESS>(
|
||||
IColumn::Filter row_filter = switchJoinRightColumns<KIND, STRICTNESS>(
|
||||
type, maps_, block.rows(), key_columns, key_sizes, added, null_map, offsets_to_replicate);
|
||||
|
||||
for (size_t i = 0; i < added.size(); ++i)
|
||||
@ -720,10 +742,16 @@ void Join::joinBlockImpl(
|
||||
|
||||
/// Filter & insert missing rows
|
||||
|
||||
NameSet needed_key_names_right = requiredRightKeys(key_names_right, columns_added_by_join);
|
||||
auto right_keys = requiredRightKeys(key_names_right, columns_added_by_join);
|
||||
|
||||
if constexpr (STRICTNESS == ASTTableJoin::Strictness::Any)
|
||||
{
|
||||
/// Some trash to represent IColumn::Filter as ColumnUInt8 needed for ColumnNullable::applyNullMap()
|
||||
auto null_map_filter_ptr = ColumnUInt8::create();
|
||||
ColumnUInt8 & null_map_filter = static_cast<ColumnUInt8 &>(*null_map_filter_ptr);
|
||||
null_map_filter.getData().swap(row_filter);
|
||||
const IColumn::Filter & filter = null_map_filter.getData();
|
||||
|
||||
constexpr bool inner_or_right = static_in_v<KIND, ASTTableJoin::Kind::Inner, ASTTableJoin::Kind::Right>;
|
||||
if constexpr (inner_or_right)
|
||||
{
|
||||
@ -737,10 +765,12 @@ void Join::joinBlockImpl(
|
||||
auto & right_name = key_names_right[i];
|
||||
auto & left_name = key_names_left[i];
|
||||
|
||||
if (needed_key_names_right.count(right_name) && !block.has(right_name))
|
||||
auto it = right_keys.find(right_name);
|
||||
if (it != right_keys.end() && !block.has(right_name))
|
||||
{
|
||||
const auto & col = block.getByName(left_name);
|
||||
block.insert({col.column, col.type, right_name});
|
||||
bool is_nullable = it->second->isNullable();
|
||||
block.insert(correctNullability({col.column, col.type, right_name}, is_nullable));
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -752,7 +782,8 @@ void Join::joinBlockImpl(
|
||||
auto & right_name = key_names_right[i];
|
||||
auto & left_name = key_names_left[i];
|
||||
|
||||
if (needed_key_names_right.count(right_name) && !block.has(right_name))
|
||||
auto it = right_keys.find(right_name);
|
||||
if (it != right_keys.end() && !block.has(right_name))
|
||||
{
|
||||
const auto & col = block.getByName(left_name);
|
||||
ColumnPtr column = col.column->convertToFullColumnIfConst();
|
||||
@ -766,13 +797,15 @@ void Join::joinBlockImpl(
|
||||
mut_column->insertDefault();
|
||||
}
|
||||
|
||||
block.insert({std::move(mut_column), col.type, right_name});
|
||||
bool is_nullable = use_nulls || it->second->isNullable();
|
||||
block.insert(correctNullability({std::move(mut_column), col.type, right_name}, is_nullable, null_map_filter));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
constexpr bool left_or_full = static_in_v<KIND, ASTTableJoin::Kind::Left, ASTTableJoin::Kind::Full>;
|
||||
if (!offsets_to_replicate)
|
||||
throw Exception("No data to filter columns", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
@ -782,7 +815,8 @@ void Join::joinBlockImpl(
|
||||
auto & right_name = key_names_right[i];
|
||||
auto & left_name = key_names_left[i];
|
||||
|
||||
if (needed_key_names_right.count(right_name) && !block.has(right_name))
|
||||
auto it = right_keys.find(right_name);
|
||||
if (it != right_keys.end() && !block.has(right_name))
|
||||
{
|
||||
const auto & col = block.getByName(left_name);
|
||||
ColumnPtr column = col.column->convertToFullColumnIfConst();
|
||||
@ -793,7 +827,7 @@ void Join::joinBlockImpl(
|
||||
{
|
||||
if (size_t to_insert = (*offsets_to_replicate)[row] - last_offset)
|
||||
{
|
||||
if (!filter[row])
|
||||
if (!row_filter[row])
|
||||
mut_column->insertDefault();
|
||||
else
|
||||
for (size_t dup = 0; dup < to_insert; ++dup)
|
||||
@ -803,7 +837,9 @@ void Join::joinBlockImpl(
|
||||
last_offset = (*offsets_to_replicate)[row];
|
||||
}
|
||||
|
||||
block.insert({std::move(mut_column), col.type, right_name});
|
||||
/// TODO: null_map_filter
|
||||
bool is_nullable = (use_nulls && left_or_full) || it->second->isNullable();
|
||||
block.insert(correctNullability({std::move(mut_column), col.type, right_name}, is_nullable));
|
||||
}
|
||||
}
|
||||
|
||||
@ -997,11 +1033,8 @@ struct AdderNonJoined;
|
||||
template <typename Mapped>
|
||||
struct AdderNonJoined<ASTTableJoin::Strictness::Any, Mapped>
|
||||
{
|
||||
static void add(const Mapped & mapped, size_t & rows_added, MutableColumns & columns_left, MutableColumns & columns_right)
|
||||
static void add(const Mapped & mapped, size_t & rows_added, MutableColumns & columns_right)
|
||||
{
|
||||
for (size_t j = 0; j < columns_left.size(); ++j)
|
||||
columns_left[j]->insertDefault();
|
||||
|
||||
for (size_t j = 0; j < columns_right.size(); ++j)
|
||||
columns_right[j]->insertFrom(*mapped.block->getByPosition(j).column.get(), mapped.row_num);
|
||||
|
||||
@ -1012,13 +1045,10 @@ struct AdderNonJoined<ASTTableJoin::Strictness::Any, Mapped>
|
||||
template <typename Mapped>
|
||||
struct AdderNonJoined<ASTTableJoin::Strictness::All, Mapped>
|
||||
{
|
||||
static void add(const Mapped & mapped, size_t & rows_added, MutableColumns & columns_left, MutableColumns & columns_right)
|
||||
static void add(const Mapped & mapped, size_t & rows_added, MutableColumns & columns_right)
|
||||
{
|
||||
for (auto current = &static_cast<const typename Mapped::Base_t &>(mapped); current != nullptr; current = current->next)
|
||||
{
|
||||
for (size_t j = 0; j < columns_left.size(); ++j)
|
||||
columns_left[j]->insertDefault();
|
||||
|
||||
for (size_t j = 0; j < columns_right.size(); ++j)
|
||||
columns_right[j]->insertFrom(*current->block->getByPosition(j).column.get(), current->row_num);
|
||||
|
||||
@ -1040,53 +1070,51 @@ public:
|
||||
* result_sample_block - keys, "left" columns, and "right" columns.
|
||||
*/
|
||||
|
||||
std::unordered_map<String, String> key_renames;
|
||||
makeResultSampleBlock(left_sample_block, key_names_left, columns_added_by_join, key_renames);
|
||||
|
||||
const Block & right_sample_block = parent.sample_block_with_columns_to_add;
|
||||
|
||||
size_t num_keys = key_names_left.size();
|
||||
size_t num_columns_left = left_sample_block.columns() - num_keys;
|
||||
size_t num_columns_right = right_sample_block.columns();
|
||||
|
||||
column_indices_left.reserve(num_columns_left);
|
||||
column_indices_keys_and_right.reserve(num_keys + num_columns_right);
|
||||
|
||||
std::vector<bool> is_left_key(left_sample_block.columns(), false);
|
||||
std::vector<size_t> key_positions_left;
|
||||
key_positions_left.reserve(key_names_left.size());
|
||||
|
||||
for (const std::string & key : key_names_left)
|
||||
{
|
||||
size_t key_pos = left_sample_block.getPositionByName(key);
|
||||
key_positions_left.push_back(key_pos);
|
||||
is_left_key[key_pos] = true;
|
||||
}
|
||||
|
||||
const Block & right_sample_block = parent.sample_block_with_columns_to_add;
|
||||
|
||||
std::unordered_map<size_t, size_t> left_to_right_key_map;
|
||||
makeResultSampleBlock(left_sample_block, right_sample_block, columns_added_by_join,
|
||||
key_positions_left, is_left_key, left_to_right_key_map);
|
||||
|
||||
column_indices_left.reserve(left_sample_block.columns() - key_names_left.size());
|
||||
column_indices_keys_and_right.reserve(key_names_left.size() + right_sample_block.columns());
|
||||
|
||||
/// Use right key columns if present. @note left & right key columns could have different nullability.
|
||||
for (size_t key_pos : key_positions_left)
|
||||
{
|
||||
/// Here we establish the mapping between key columns of the left- and right-side tables.
|
||||
/// key_pos index is inserted in the position corresponding to key column in parent.blocks
|
||||
/// (saved blocks of the right-side table) and points to the same key column
|
||||
/// in the left_sample_block and thus in the result_sample_block.
|
||||
column_indices_keys_and_right.push_back(key_pos);
|
||||
|
||||
auto it = key_renames.find(key);
|
||||
if (it != key_renames.end())
|
||||
key_renames_indices[key_pos] = result_sample_block.getPositionByName(it->second);
|
||||
}
|
||||
|
||||
size_t num_src_columns = left_sample_block.columns() + right_sample_block.columns();
|
||||
|
||||
for (size_t i = 0; i < result_sample_block.columns(); ++i)
|
||||
{
|
||||
if (i < left_sample_block.columns())
|
||||
auto it = left_to_right_key_map.find(key_pos);
|
||||
if (it != left_to_right_key_map.end())
|
||||
{
|
||||
if (!is_left_key[i])
|
||||
{
|
||||
column_indices_left.emplace_back(i);
|
||||
|
||||
/// If use_nulls, convert left columns to Nullable.
|
||||
if (parent.use_nulls)
|
||||
convertColumnToNullable(result_sample_block.getByPosition(i));
|
||||
}
|
||||
column_indices_keys_and_right.push_back(it->second);
|
||||
column_indices_left.push_back(key_pos);
|
||||
}
|
||||
else if (i < num_src_columns)
|
||||
column_indices_keys_and_right.emplace_back(i);
|
||||
else
|
||||
column_indices_keys_and_right.push_back(key_pos);
|
||||
}
|
||||
|
||||
for (size_t i = 0; i < left_sample_block.columns(); ++i)
|
||||
if (!is_left_key[i])
|
||||
column_indices_left.emplace_back(i);
|
||||
|
||||
size_t num_additional_keys = left_to_right_key_map.size();
|
||||
for (size_t i = left_sample_block.columns(); i < result_sample_block.columns() - num_additional_keys; ++i)
|
||||
column_indices_keys_and_right.emplace_back(i);
|
||||
}
|
||||
|
||||
String getName() const override { return "NonJoined"; }
|
||||
@ -1118,18 +1146,25 @@ private:
|
||||
/// Indices of key columns in result_sample_block or columns that come from the right-side table.
|
||||
/// Order is significant: it is the same as the order of columns in the blocks of the right-side table that are saved in parent.blocks.
|
||||
ColumnNumbers column_indices_keys_and_right;
|
||||
std::unordered_map<size_t, size_t> key_renames_indices;
|
||||
|
||||
std::unique_ptr<void, std::function<void(void *)>> position; /// type erasure
|
||||
|
||||
|
||||
void makeResultSampleBlock(const Block & left_sample_block, const Names & key_names_left,
|
||||
const NamesAndTypesList & columns_added_by_join, std::unordered_map<String, String> & key_renames)
|
||||
void makeResultSampleBlock(const Block & left_sample_block, const Block & right_sample_block,
|
||||
const NamesAndTypesList & columns_added_by_join,
|
||||
const std::vector<size_t> & key_positions_left, const std::vector<bool> & is_left_key,
|
||||
std::unordered_map<size_t, size_t> & left_to_right_key_map)
|
||||
{
|
||||
const Block & right_sample_block = parent.sample_block_with_columns_to_add;
|
||||
|
||||
result_sample_block = materializeBlock(left_sample_block);
|
||||
|
||||
/// Convert left columns to Nullable if allowed
|
||||
if (parent.use_nulls)
|
||||
{
|
||||
for (size_t i = 0; i < result_sample_block.columns(); ++i)
|
||||
if (!is_left_key[i])
|
||||
convertColumnToNullable(result_sample_block.getByPosition(i));
|
||||
}
|
||||
|
||||
/// Add columns from the right-side table to the block.
|
||||
for (size_t i = 0; i < right_sample_block.columns(); ++i)
|
||||
{
|
||||
@ -1139,20 +1174,23 @@ private:
|
||||
}
|
||||
|
||||
const auto & key_names_right = parent.key_names_right;
|
||||
NameSet needed_key_names_right = requiredRightKeys(key_names_right, columns_added_by_join);
|
||||
auto right_keys = requiredRightKeys(key_names_right, columns_added_by_join);
|
||||
|
||||
/// Add join key columns from right block if they has different name.
|
||||
for (size_t i = 0; i < key_names_right.size(); ++i)
|
||||
{
|
||||
auto & right_name = key_names_right[i];
|
||||
auto & left_name = key_names_left[i];
|
||||
size_t left_key_pos = key_positions_left[i];
|
||||
|
||||
if (needed_key_names_right.count(right_name) && !result_sample_block.has(right_name))
|
||||
auto it = right_keys.find(right_name);
|
||||
if (it != right_keys.end() && !result_sample_block.has(right_name))
|
||||
{
|
||||
const auto & col = result_sample_block.getByName(left_name);
|
||||
result_sample_block.insert({col.column, col.type, right_name});
|
||||
const auto & col = result_sample_block.getByPosition(left_key_pos);
|
||||
bool is_nullable = (parent.use_nulls && isFull(parent.kind)) || it->second->isNullable();
|
||||
result_sample_block.insert(correctNullability({col.column, col.type, right_name}, is_nullable));
|
||||
|
||||
key_renames[left_name] = right_name;
|
||||
size_t right_key_pos = result_sample_block.getPositionByName(right_name);
|
||||
left_to_right_key_map[left_key_pos] = right_key_pos;
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -1169,7 +1207,7 @@ private:
|
||||
{
|
||||
#define M(TYPE) \
|
||||
case Join::Type::TYPE: \
|
||||
rows_added = fillColumns<STRICTNESS>(*maps.TYPE, columns_left, columns_keys_and_right); \
|
||||
rows_added = fillColumns<STRICTNESS>(*maps.TYPE, columns_keys_and_right); \
|
||||
break;
|
||||
APPLY_FOR_JOIN_VARIANTS(M)
|
||||
#undef M
|
||||
@ -1183,32 +1221,12 @@ private:
|
||||
|
||||
Block res = result_sample_block.cloneEmpty();
|
||||
|
||||
/// @note it's possible to make ColumnConst here and materialize it later
|
||||
for (size_t i = 0; i < columns_left.size(); ++i)
|
||||
res.getByPosition(column_indices_left[i]).column = std::move(columns_left[i]);
|
||||
res.getByPosition(column_indices_left[i]).column = columns_left[i]->cloneResized(rows_added);
|
||||
|
||||
if (key_renames_indices.empty())
|
||||
{
|
||||
for (size_t i = 0; i < columns_keys_and_right.size(); ++i)
|
||||
res.getByPosition(column_indices_keys_and_right[i]).column = std::move(columns_keys_and_right[i]);
|
||||
}
|
||||
else
|
||||
{
|
||||
for (size_t i = 0; i < columns_keys_and_right.size(); ++i)
|
||||
{
|
||||
size_t key_idx = column_indices_keys_and_right[i];
|
||||
|
||||
auto it = key_renames_indices.find(key_idx);
|
||||
if (it != key_renames_indices.end())
|
||||
{
|
||||
auto & key_column = res.getByPosition(key_idx).column;
|
||||
if (key_column->empty())
|
||||
key_column = key_column->cloneResized(columns_keys_and_right[i]->size());
|
||||
res.getByPosition(it->second).column = std::move(columns_keys_and_right[i]);
|
||||
}
|
||||
else
|
||||
res.getByPosition(key_idx).column = std::move(columns_keys_and_right[i]);
|
||||
}
|
||||
}
|
||||
for (size_t i = 0; i < columns_keys_and_right.size(); ++i)
|
||||
res.getByPosition(column_indices_keys_and_right[i]).column = std::move(columns_keys_and_right[i]);
|
||||
|
||||
return res;
|
||||
}
|
||||
@ -1230,7 +1248,7 @@ private:
|
||||
}
|
||||
|
||||
template <ASTTableJoin::Strictness STRICTNESS, typename Map>
|
||||
size_t fillColumns(const Map & map, MutableColumns & columns_left, MutableColumns & columns_keys_and_right)
|
||||
size_t fillColumns(const Map & map, MutableColumns & columns_keys_and_right)
|
||||
{
|
||||
size_t rows_added = 0;
|
||||
|
||||
@ -1247,7 +1265,7 @@ private:
|
||||
if (it->getSecond().getUsed())
|
||||
continue;
|
||||
|
||||
AdderNonJoined<STRICTNESS, typename Map::mapped_type>::add(it->getSecond(), rows_added, columns_left, columns_keys_and_right);
|
||||
AdderNonJoined<STRICTNESS, typename Map::mapped_type>::add(it->getSecond(), rows_added, columns_keys_and_right);
|
||||
|
||||
if (rows_added >= max_block_size)
|
||||
{
|
||||
|
@ -39,9 +39,13 @@ ASTPtr ASTColumnDeclaration::clone() const
|
||||
void ASTColumnDeclaration::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
|
||||
{
|
||||
frame.need_parens = false;
|
||||
std::string indent_str = settings.one_line ? "" : std::string(4 * frame.indent, ' ');
|
||||
|
||||
settings.ostr << settings.nl_or_ws << indent_str << backQuoteIfNeed(name);
|
||||
if (!settings.one_line)
|
||||
settings.ostr << settings.nl_or_ws << std::string(4 * frame.indent, ' ');
|
||||
|
||||
/// We have to always backquote column names to avoid ambiguouty with INDEX and other declarations in CREATE query.
|
||||
settings.ostr << backQuote(name);
|
||||
|
||||
if (type)
|
||||
{
|
||||
settings.ostr << ' ';
|
||||
|
@ -25,7 +25,6 @@ const char * IAST::hilite_alias = "\033[0;32m";
|
||||
const char * IAST::hilite_none = "\033[0m";
|
||||
|
||||
|
||||
/// Quote the identifier with backquotes, if required.
|
||||
String backQuoteIfNeed(const String & x)
|
||||
{
|
||||
String res(x.size(), '\0');
|
||||
@ -36,6 +35,16 @@ String backQuoteIfNeed(const String & x)
|
||||
return res;
|
||||
}
|
||||
|
||||
String backQuote(const String & x)
|
||||
{
|
||||
String res(x.size(), '\0');
|
||||
{
|
||||
WriteBufferFromString wb(res);
|
||||
writeBackQuotedString(x, wb);
|
||||
}
|
||||
return res;
|
||||
}
|
||||
|
||||
|
||||
size_t IAST::checkSize(size_t max_size) const
|
||||
{
|
||||
|
@ -208,7 +208,9 @@ private:
|
||||
};
|
||||
|
||||
|
||||
/// Surrounds an identifier by back quotes if it is necessary.
|
||||
/// Quote the identifier with backquotes, if required.
|
||||
String backQuoteIfNeed(const String & x);
|
||||
/// Quote the identifier with backquotes.
|
||||
String backQuote(const String & x);
|
||||
|
||||
}
|
||||
|
@ -1,14 +1,14 @@
|
||||
d Date
|
||||
k UInt64
|
||||
i32 Int32
|
||||
CREATE TABLE test.alter ( d Date, k UInt64, i32 Int32) ENGINE = MergeTree(d, k, 8192)
|
||||
CREATE TABLE test.alter (`d` Date, `k` UInt64, `i32` Int32) ENGINE = MergeTree(d, k, 8192)
|
||||
2015-01-01 10 42
|
||||
d Date
|
||||
k UInt64
|
||||
i32 Int32
|
||||
n.ui8 Array(UInt8)
|
||||
n.s Array(String)
|
||||
CREATE TABLE test.alter ( d Date, k UInt64, i32 Int32, `n.ui8` Array(UInt8), `n.s` Array(String)) ENGINE = MergeTree(d, k, 8192)
|
||||
CREATE TABLE test.alter (`d` Date, `k` UInt64, `i32` Int32, `n.ui8` Array(UInt8), `n.s` Array(String)) ENGINE = MergeTree(d, k, 8192)
|
||||
2015-01-01 8 40 [1,2,3] ['12','13','14']
|
||||
2015-01-01 10 42 [] []
|
||||
d Date
|
||||
@ -17,7 +17,7 @@ i32 Int32
|
||||
n.ui8 Array(UInt8)
|
||||
n.s Array(String)
|
||||
n.d Array(Date)
|
||||
CREATE TABLE test.alter ( d Date, k UInt64, i32 Int32, `n.ui8` Array(UInt8), `n.s` Array(String), `n.d` Array(Date)) ENGINE = MergeTree(d, k, 8192)
|
||||
CREATE TABLE test.alter (`d` Date, `k` UInt64, `i32` Int32, `n.ui8` Array(UInt8), `n.s` Array(String), `n.d` Array(Date)) ENGINE = MergeTree(d, k, 8192)
|
||||
2015-01-01 7 39 [10,20,30] ['120','130','140'] ['2000-01-01','2000-01-01','2000-01-03']
|
||||
2015-01-01 8 40 [1,2,3] ['12','13','14'] ['0000-00-00','0000-00-00','0000-00-00']
|
||||
2015-01-01 10 42 [] [] []
|
||||
@ -28,7 +28,7 @@ n.ui8 Array(UInt8)
|
||||
n.s Array(String)
|
||||
n.d Array(Date)
|
||||
s String DEFAULT \'0\'
|
||||
CREATE TABLE test.alter ( d Date, k UInt64, i32 Int32, `n.ui8` Array(UInt8), `n.s` Array(String), `n.d` Array(Date), s String DEFAULT \'0\') ENGINE = MergeTree(d, k, 8192)
|
||||
CREATE TABLE test.alter (`d` Date, `k` UInt64, `i32` Int32, `n.ui8` Array(UInt8), `n.s` Array(String), `n.d` Array(Date), `s` String DEFAULT \'0\') ENGINE = MergeTree(d, k, 8192)
|
||||
2015-01-01 6 38 [10,20,30] ['asd','qwe','qwe'] ['2000-01-01','2000-01-01','2000-01-03'] 100500
|
||||
2015-01-01 7 39 [10,20,30] ['120','130','140'] ['2000-01-01','2000-01-01','2000-01-03'] 0
|
||||
2015-01-01 8 40 [1,2,3] ['12','13','14'] ['0000-00-00','0000-00-00','0000-00-00'] 0
|
||||
@ -39,7 +39,7 @@ i32 Int32
|
||||
n.ui8 Array(UInt8)
|
||||
n.s Array(String)
|
||||
s Int64
|
||||
CREATE TABLE test.alter ( d Date, k UInt64, i32 Int32, `n.ui8` Array(UInt8), `n.s` Array(String), s Int64) ENGINE = MergeTree(d, k, 8192)
|
||||
CREATE TABLE test.alter (`d` Date, `k` UInt64, `i32` Int32, `n.ui8` Array(UInt8), `n.s` Array(String), `s` Int64) ENGINE = MergeTree(d, k, 8192)
|
||||
2015-01-01 6 38 [10,20,30] ['asd','qwe','qwe'] 100500
|
||||
2015-01-01 7 39 [10,20,30] ['120','130','140'] 0
|
||||
2015-01-01 8 40 [1,2,3] ['12','13','14'] 0
|
||||
@ -51,7 +51,7 @@ n.ui8 Array(UInt8)
|
||||
n.s Array(String)
|
||||
s UInt32
|
||||
n.d Array(Date)
|
||||
CREATE TABLE test.alter ( d Date, k UInt64, i32 Int32, `n.ui8` Array(UInt8), `n.s` Array(String), s UInt32, `n.d` Array(Date)) ENGINE = MergeTree(d, k, 8192)
|
||||
CREATE TABLE test.alter (`d` Date, `k` UInt64, `i32` Int32, `n.ui8` Array(UInt8), `n.s` Array(String), `s` UInt32, `n.d` Array(Date)) ENGINE = MergeTree(d, k, 8192)
|
||||
2015-01-01 6 38 [10,20,30] ['asd','qwe','qwe'] 100500 ['0000-00-00','0000-00-00','0000-00-00']
|
||||
2015-01-01 7 39 [10,20,30] ['120','130','140'] 0 ['0000-00-00','0000-00-00','0000-00-00']
|
||||
2015-01-01 8 40 [1,2,3] ['12','13','14'] 0 ['0000-00-00','0000-00-00','0000-00-00']
|
||||
@ -65,7 +65,7 @@ k UInt64
|
||||
i32 Int32
|
||||
n.s Array(String)
|
||||
s UInt32
|
||||
CREATE TABLE test.alter ( d Date, k UInt64, i32 Int32, `n.s` Array(String), s UInt32) ENGINE = MergeTree(d, k, 8192)
|
||||
CREATE TABLE test.alter (`d` Date, `k` UInt64, `i32` Int32, `n.s` Array(String), `s` UInt32) ENGINE = MergeTree(d, k, 8192)
|
||||
2015-01-01 6 38 ['asd','qwe','qwe'] 100500
|
||||
2015-01-01 7 39 ['120','130','140'] 0
|
||||
2015-01-01 8 40 ['12','13','14'] 0
|
||||
@ -74,7 +74,7 @@ d Date
|
||||
k UInt64
|
||||
i32 Int32
|
||||
s UInt32
|
||||
CREATE TABLE test.alter ( d Date, k UInt64, i32 Int32, s UInt32) ENGINE = MergeTree(d, k, 8192)
|
||||
CREATE TABLE test.alter (`d` Date, `k` UInt64, `i32` Int32, `s` UInt32) ENGINE = MergeTree(d, k, 8192)
|
||||
2015-01-01 6 38 100500
|
||||
2015-01-01 7 39 0
|
||||
2015-01-01 8 40 0
|
||||
@ -85,7 +85,7 @@ i32 Int32
|
||||
s UInt32
|
||||
n.s Array(String)
|
||||
n.d Array(Date)
|
||||
CREATE TABLE test.alter ( d Date, k UInt64, i32 Int32, s UInt32, `n.s` Array(String), `n.d` Array(Date)) ENGINE = MergeTree(d, k, 8192)
|
||||
CREATE TABLE test.alter (`d` Date, `k` UInt64, `i32` Int32, `s` UInt32, `n.s` Array(String), `n.d` Array(Date)) ENGINE = MergeTree(d, k, 8192)
|
||||
2015-01-01 6 38 100500 [] []
|
||||
2015-01-01 7 39 0 [] []
|
||||
2015-01-01 8 40 0 [] []
|
||||
@ -94,7 +94,7 @@ d Date
|
||||
k UInt64
|
||||
i32 Int32
|
||||
s UInt32
|
||||
CREATE TABLE test.alter ( d Date, k UInt64, i32 Int32, s UInt32) ENGINE = MergeTree(d, k, 8192)
|
||||
CREATE TABLE test.alter (`d` Date, `k` UInt64, `i32` Int32, `s` UInt32) ENGINE = MergeTree(d, k, 8192)
|
||||
2015-01-01 6 38 100500
|
||||
2015-01-01 7 39 0
|
||||
2015-01-01 8 40 0
|
||||
|
@ -1,22 +1,22 @@
|
||||
d Date
|
||||
k UInt64
|
||||
i32 Int32
|
||||
CREATE TABLE test.replicated_alter1 ( d Date, k UInt64, i32 Int32) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r1\', d, k, 8192)
|
||||
CREATE TABLE test.replicated_alter1 (`d` Date, `k` UInt64, `i32` Int32) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r1\', d, k, 8192)
|
||||
d Date
|
||||
k UInt64
|
||||
i32 Int32
|
||||
CREATE TABLE test.replicated_alter2 ( d Date, k UInt64, i32 Int32) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r2\', d, k, 8192)
|
||||
CREATE TABLE test.replicated_alter2 (`d` Date, `k` UInt64, `i32` Int32) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r2\', d, k, 8192)
|
||||
2015-01-01 10 42
|
||||
d Date
|
||||
k UInt64
|
||||
i32 Int32
|
||||
dt DateTime
|
||||
CREATE TABLE test.replicated_alter1 ( d Date, k UInt64, i32 Int32, dt DateTime) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r1\', d, k, 8192)
|
||||
CREATE TABLE test.replicated_alter1 (`d` Date, `k` UInt64, `i32` Int32, `dt` DateTime) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r1\', d, k, 8192)
|
||||
d Date
|
||||
k UInt64
|
||||
i32 Int32
|
||||
dt DateTime
|
||||
CREATE TABLE test.replicated_alter2 ( d Date, k UInt64, i32 Int32, dt DateTime) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r2\', d, k, 8192)
|
||||
CREATE TABLE test.replicated_alter2 (`d` Date, `k` UInt64, `i32` Int32, `dt` DateTime) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r2\', d, k, 8192)
|
||||
2015-01-01 9 41 1992-01-01 08:00:00
|
||||
2015-01-01 10 42 0000-00-00 00:00:00
|
||||
d Date
|
||||
@ -25,14 +25,14 @@ i32 Int32
|
||||
dt DateTime
|
||||
n.ui8 Array(UInt8)
|
||||
n.s Array(String)
|
||||
CREATE TABLE test.replicated_alter1 ( d Date, k UInt64, i32 Int32, dt DateTime, `n.ui8` Array(UInt8), `n.s` Array(String)) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r1\', d, k, 8192)
|
||||
CREATE TABLE test.replicated_alter1 (`d` Date, `k` UInt64, `i32` Int32, `dt` DateTime, `n.ui8` Array(UInt8), `n.s` Array(String)) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r1\', d, k, 8192)
|
||||
d Date
|
||||
k UInt64
|
||||
i32 Int32
|
||||
dt DateTime
|
||||
n.ui8 Array(UInt8)
|
||||
n.s Array(String)
|
||||
CREATE TABLE test.replicated_alter2 ( d Date, k UInt64, i32 Int32, dt DateTime, `n.ui8` Array(UInt8), `n.s` Array(String)) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r2\', d, k, 8192)
|
||||
CREATE TABLE test.replicated_alter2 (`d` Date, `k` UInt64, `i32` Int32, `dt` DateTime, `n.ui8` Array(UInt8), `n.s` Array(String)) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r2\', d, k, 8192)
|
||||
2015-01-01 8 40 2012-12-12 12:12:12 [1,2,3] ['12','13','14']
|
||||
2015-01-01 9 41 1992-01-01 08:00:00 [] []
|
||||
2015-01-01 10 42 0000-00-00 00:00:00 [] []
|
||||
@ -43,7 +43,7 @@ dt DateTime
|
||||
n.ui8 Array(UInt8)
|
||||
n.s Array(String)
|
||||
n.d Array(Date)
|
||||
CREATE TABLE test.replicated_alter1 ( d Date, k UInt64, i32 Int32, dt DateTime, `n.ui8` Array(UInt8), `n.s` Array(String), `n.d` Array(Date)) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r1\', d, k, 8192)
|
||||
CREATE TABLE test.replicated_alter1 (`d` Date, `k` UInt64, `i32` Int32, `dt` DateTime, `n.ui8` Array(UInt8), `n.s` Array(String), `n.d` Array(Date)) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r1\', d, k, 8192)
|
||||
d Date
|
||||
k UInt64
|
||||
i32 Int32
|
||||
@ -51,7 +51,7 @@ dt DateTime
|
||||
n.ui8 Array(UInt8)
|
||||
n.s Array(String)
|
||||
n.d Array(Date)
|
||||
CREATE TABLE test.replicated_alter2 ( d Date, k UInt64, i32 Int32, dt DateTime, `n.ui8` Array(UInt8), `n.s` Array(String), `n.d` Array(Date)) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r2\', d, k, 8192)
|
||||
CREATE TABLE test.replicated_alter2 (`d` Date, `k` UInt64, `i32` Int32, `dt` DateTime, `n.ui8` Array(UInt8), `n.s` Array(String), `n.d` Array(Date)) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r2\', d, k, 8192)
|
||||
2015-01-01 7 39 2014-07-14 13:26:50 [10,20,30] ['120','130','140'] ['2000-01-01','2000-01-01','2000-01-03']
|
||||
2015-01-01 8 40 2012-12-12 12:12:12 [1,2,3] ['12','13','14'] ['0000-00-00','0000-00-00','0000-00-00']
|
||||
2015-01-01 9 41 1992-01-01 08:00:00 [] [] []
|
||||
@ -64,7 +64,7 @@ n.ui8 Array(UInt8)
|
||||
n.s Array(String)
|
||||
n.d Array(Date)
|
||||
s String DEFAULT \'0\'
|
||||
CREATE TABLE test.replicated_alter1 ( d Date, k UInt64, i32 Int32, dt DateTime, `n.ui8` Array(UInt8), `n.s` Array(String), `n.d` Array(Date), s String DEFAULT \'0\') ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r1\', d, k, 8192)
|
||||
CREATE TABLE test.replicated_alter1 (`d` Date, `k` UInt64, `i32` Int32, `dt` DateTime, `n.ui8` Array(UInt8), `n.s` Array(String), `n.d` Array(Date), `s` String DEFAULT \'0\') ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r1\', d, k, 8192)
|
||||
d Date
|
||||
k UInt64
|
||||
i32 Int32
|
||||
@ -73,7 +73,7 @@ n.ui8 Array(UInt8)
|
||||
n.s Array(String)
|
||||
n.d Array(Date)
|
||||
s String DEFAULT \'0\'
|
||||
CREATE TABLE test.replicated_alter2 ( d Date, k UInt64, i32 Int32, dt DateTime, `n.ui8` Array(UInt8), `n.s` Array(String), `n.d` Array(Date), s String DEFAULT \'0\') ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r2\', d, k, 8192)
|
||||
CREATE TABLE test.replicated_alter2 (`d` Date, `k` UInt64, `i32` Int32, `dt` DateTime, `n.ui8` Array(UInt8), `n.s` Array(String), `n.d` Array(Date), `s` String DEFAULT \'0\') ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r2\', d, k, 8192)
|
||||
2015-01-01 6 38 2014-07-15 13:26:50 [10,20,30] ['asd','qwe','qwe'] ['2000-01-01','2000-01-01','2000-01-03'] 100500
|
||||
2015-01-01 7 39 2014-07-14 13:26:50 [10,20,30] ['120','130','140'] ['2000-01-01','2000-01-01','2000-01-03'] 0
|
||||
2015-01-01 8 40 2012-12-12 12:12:12 [1,2,3] ['12','13','14'] ['0000-00-00','0000-00-00','0000-00-00'] 0
|
||||
@ -86,7 +86,7 @@ dt DateTime
|
||||
n.ui8 Array(UInt8)
|
||||
n.s Array(String)
|
||||
s Int64
|
||||
CREATE TABLE test.replicated_alter1 ( d Date, k UInt64, i32 Int32, dt DateTime, `n.ui8` Array(UInt8), `n.s` Array(String), s Int64) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r1\', d, k, 8192)
|
||||
CREATE TABLE test.replicated_alter1 (`d` Date, `k` UInt64, `i32` Int32, `dt` DateTime, `n.ui8` Array(UInt8), `n.s` Array(String), `s` Int64) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r1\', d, k, 8192)
|
||||
d Date
|
||||
k UInt64
|
||||
i32 Int32
|
||||
@ -94,7 +94,7 @@ dt DateTime
|
||||
n.ui8 Array(UInt8)
|
||||
n.s Array(String)
|
||||
s Int64
|
||||
CREATE TABLE test.replicated_alter2 ( d Date, k UInt64, i32 Int32, dt DateTime, `n.ui8` Array(UInt8), `n.s` Array(String), s Int64) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r2\', d, k, 8192)
|
||||
CREATE TABLE test.replicated_alter2 (`d` Date, `k` UInt64, `i32` Int32, `dt` DateTime, `n.ui8` Array(UInt8), `n.s` Array(String), `s` Int64) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r2\', d, k, 8192)
|
||||
2015-01-01 6 38 2014-07-15 13:26:50 [10,20,30] ['asd','qwe','qwe'] 100500
|
||||
2015-01-01 7 39 2014-07-14 13:26:50 [10,20,30] ['120','130','140'] 0
|
||||
2015-01-01 8 40 2012-12-12 12:12:12 [1,2,3] ['12','13','14'] 0
|
||||
@ -108,7 +108,7 @@ n.ui8 Array(UInt8)
|
||||
n.s Array(String)
|
||||
s UInt32
|
||||
n.d Array(Date)
|
||||
CREATE TABLE test.replicated_alter1 ( d Date, k UInt64, i32 Int32, dt DateTime, `n.ui8` Array(UInt8), `n.s` Array(String), s UInt32, `n.d` Array(Date)) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r1\', d, k, 8192)
|
||||
CREATE TABLE test.replicated_alter1 (`d` Date, `k` UInt64, `i32` Int32, `dt` DateTime, `n.ui8` Array(UInt8), `n.s` Array(String), `s` UInt32, `n.d` Array(Date)) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r1\', d, k, 8192)
|
||||
d Date
|
||||
k UInt64
|
||||
i32 Int32
|
||||
@ -117,7 +117,7 @@ n.ui8 Array(UInt8)
|
||||
n.s Array(String)
|
||||
s UInt32
|
||||
n.d Array(Date)
|
||||
CREATE TABLE test.replicated_alter2 ( d Date, k UInt64, i32 Int32, dt DateTime, `n.ui8` Array(UInt8), `n.s` Array(String), s UInt32, `n.d` Array(Date)) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r2\', d, k, 8192)
|
||||
CREATE TABLE test.replicated_alter2 (`d` Date, `k` UInt64, `i32` Int32, `dt` DateTime, `n.ui8` Array(UInt8), `n.s` Array(String), `s` UInt32, `n.d` Array(Date)) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r2\', d, k, 8192)
|
||||
2015-01-01 6 38 2014-07-15 13:26:50 [10,20,30] ['asd','qwe','qwe'] 100500 ['0000-00-00','0000-00-00','0000-00-00']
|
||||
2015-01-01 7 39 2014-07-14 13:26:50 [10,20,30] ['120','130','140'] 0 ['0000-00-00','0000-00-00','0000-00-00']
|
||||
2015-01-01 8 40 2012-12-12 12:12:12 [1,2,3] ['12','13','14'] 0 ['0000-00-00','0000-00-00','0000-00-00']
|
||||
@ -129,14 +129,14 @@ i32 Int32
|
||||
dt DateTime
|
||||
n.s Array(String)
|
||||
s UInt32
|
||||
CREATE TABLE test.replicated_alter1 ( d Date, k UInt64, i32 Int32, dt DateTime, `n.s` Array(String), s UInt32) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r1\', d, k, 8192)
|
||||
CREATE TABLE test.replicated_alter1 (`d` Date, `k` UInt64, `i32` Int32, `dt` DateTime, `n.s` Array(String), `s` UInt32) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r1\', d, k, 8192)
|
||||
d Date
|
||||
k UInt64
|
||||
i32 Int32
|
||||
dt DateTime
|
||||
n.s Array(String)
|
||||
s UInt32
|
||||
CREATE TABLE test.replicated_alter2 ( d Date, k UInt64, i32 Int32, dt DateTime, `n.s` Array(String), s UInt32) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r2\', d, k, 8192)
|
||||
CREATE TABLE test.replicated_alter2 (`d` Date, `k` UInt64, `i32` Int32, `dt` DateTime, `n.s` Array(String), `s` UInt32) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r2\', d, k, 8192)
|
||||
2015-01-01 6 38 2014-07-15 13:26:50 ['asd','qwe','qwe'] 100500
|
||||
2015-01-01 7 39 2014-07-14 13:26:50 ['120','130','140'] 0
|
||||
2015-01-01 8 40 2012-12-12 12:12:12 ['12','13','14'] 0
|
||||
@ -147,13 +147,13 @@ k UInt64
|
||||
i32 Int32
|
||||
dt DateTime
|
||||
s UInt32
|
||||
CREATE TABLE test.replicated_alter1 ( d Date, k UInt64, i32 Int32, dt DateTime, s UInt32) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r1\', d, k, 8192)
|
||||
CREATE TABLE test.replicated_alter1 (`d` Date, `k` UInt64, `i32` Int32, `dt` DateTime, `s` UInt32) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r1\', d, k, 8192)
|
||||
d Date
|
||||
k UInt64
|
||||
i32 Int32
|
||||
dt DateTime
|
||||
s UInt32
|
||||
CREATE TABLE test.replicated_alter2 ( d Date, k UInt64, i32 Int32, dt DateTime, s UInt32) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r2\', d, k, 8192)
|
||||
CREATE TABLE test.replicated_alter2 (`d` Date, `k` UInt64, `i32` Int32, `dt` DateTime, `s` UInt32) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r2\', d, k, 8192)
|
||||
2015-01-01 6 38 2014-07-15 13:26:50 100500
|
||||
2015-01-01 7 39 2014-07-14 13:26:50 0
|
||||
2015-01-01 8 40 2012-12-12 12:12:12 0
|
||||
@ -166,7 +166,7 @@ dt DateTime
|
||||
s UInt32
|
||||
n.s Array(String)
|
||||
n.d Array(Date)
|
||||
CREATE TABLE test.replicated_alter1 ( d Date, k UInt64, i32 Int32, dt DateTime, s UInt32, `n.s` Array(String), `n.d` Array(Date)) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r1\', d, k, 8192)
|
||||
CREATE TABLE test.replicated_alter1 (`d` Date, `k` UInt64, `i32` Int32, `dt` DateTime, `s` UInt32, `n.s` Array(String), `n.d` Array(Date)) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r1\', d, k, 8192)
|
||||
d Date
|
||||
k UInt64
|
||||
i32 Int32
|
||||
@ -174,7 +174,7 @@ dt DateTime
|
||||
s UInt32
|
||||
n.s Array(String)
|
||||
n.d Array(Date)
|
||||
CREATE TABLE test.replicated_alter2 ( d Date, k UInt64, i32 Int32, dt DateTime, s UInt32, `n.s` Array(String), `n.d` Array(Date)) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r2\', d, k, 8192)
|
||||
CREATE TABLE test.replicated_alter2 (`d` Date, `k` UInt64, `i32` Int32, `dt` DateTime, `s` UInt32, `n.s` Array(String), `n.d` Array(Date)) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r2\', d, k, 8192)
|
||||
2015-01-01 6 38 2014-07-15 13:26:50 100500 [] []
|
||||
2015-01-01 7 39 2014-07-14 13:26:50 0 [] []
|
||||
2015-01-01 8 40 2012-12-12 12:12:12 0 [] []
|
||||
@ -185,13 +185,13 @@ k UInt64
|
||||
i32 Int32
|
||||
dt DateTime
|
||||
s UInt32
|
||||
CREATE TABLE test.replicated_alter1 ( d Date, k UInt64, i32 Int32, dt DateTime, s UInt32) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r1\', d, k, 8192)
|
||||
CREATE TABLE test.replicated_alter1 (`d` Date, `k` UInt64, `i32` Int32, `dt` DateTime, `s` UInt32) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r1\', d, k, 8192)
|
||||
d Date
|
||||
k UInt64
|
||||
i32 Int32
|
||||
dt DateTime
|
||||
s UInt32
|
||||
CREATE TABLE test.replicated_alter2 ( d Date, k UInt64, i32 Int32, dt DateTime, s UInt32) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r2\', d, k, 8192)
|
||||
CREATE TABLE test.replicated_alter2 (`d` Date, `k` UInt64, `i32` Int32, `dt` DateTime, `s` UInt32) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r2\', d, k, 8192)
|
||||
2015-01-01 6 38 2014-07-15 13:26:50 100500
|
||||
2015-01-01 7 39 2014-07-14 13:26:50 0
|
||||
2015-01-01 8 40 2012-12-12 12:12:12 0
|
||||
@ -202,13 +202,13 @@ k UInt64
|
||||
i32 Int32
|
||||
dt Date
|
||||
s DateTime
|
||||
CREATE TABLE test.replicated_alter1 ( d Date, k UInt64, i32 Int32, dt Date, s DateTime) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r1\', d, k, 8192)
|
||||
CREATE TABLE test.replicated_alter1 (`d` Date, `k` UInt64, `i32` Int32, `dt` Date, `s` DateTime) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r1\', d, k, 8192)
|
||||
d Date
|
||||
k UInt64
|
||||
i32 Int32
|
||||
dt Date
|
||||
s DateTime
|
||||
CREATE TABLE test.replicated_alter2 ( d Date, k UInt64, i32 Int32, dt Date, s DateTime) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r2\', d, k, 8192)
|
||||
CREATE TABLE test.replicated_alter2 (`d` Date, `k` UInt64, `i32` Int32, `dt` Date, `s` DateTime) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r2\', d, k, 8192)
|
||||
2015-01-01 6 38 2014-07-15 1970-01-02 06:55:00
|
||||
2015-01-01 7 39 2014-07-14 0000-00-00 00:00:00
|
||||
2015-01-01 8 40 2012-12-12 0000-00-00 00:00:00
|
||||
|
@ -1,7 +1,7 @@
|
||||
A
|
||||
B
|
||||
A 1 TinyLog CREATE TABLE test_show_tables.A ( A UInt8) ENGINE = TinyLog
|
||||
B 1 TinyLog CREATE TABLE test_show_tables.B ( A UInt8) ENGINE = TinyLog
|
||||
A 1 TinyLog CREATE TABLE test_show_tables.A (`A` UInt8) ENGINE = TinyLog
|
||||
B 1 TinyLog CREATE TABLE test_show_tables.B (`A` UInt8) ENGINE = TinyLog
|
||||
test_temporary_table
|
||||
['test_show_tables'] ['test_materialized']
|
||||
0
|
||||
|
@ -1,4 +1,4 @@
|
||||
1
|
||||
CREATE TEMPORARY TABLE temp_tab ( number UInt64) ENGINE = Memory
|
||||
CREATE TEMPORARY TABLE temp_tab (`number` UInt64) ENGINE = Memory
|
||||
temp_tab
|
||||
0
|
||||
|
@ -1 +1 @@
|
||||
CREATE VIEW test.test_view ( id UInt64) AS SELECT * FROM test.test WHERE id = (SELECT 1)
|
||||
CREATE VIEW test.test_view (`id` UInt64) AS SELECT * FROM test.test WHERE id = (SELECT 1)
|
||||
|
@ -7,7 +7,7 @@ hello
|
||||
hello
|
||||
hello
|
||||
1970-01-01 00:00:01
|
||||
CREATE TABLE test.cast ( x UInt8, e Enum8('hello' = 1, 'world' = 2) DEFAULT CAST(x, 'Enum8(\'hello\' = 1, \'world\' = 2)')) ENGINE = MergeTree ORDER BY e SETTINGS index_granularity = 8192
|
||||
CREATE TABLE test.cast (`x` UInt8, `e` Enum8('hello' = 1, 'world' = 2) DEFAULT CAST(x, 'Enum8(\'hello\' = 1, \'world\' = 2)')) ENGINE = MergeTree ORDER BY e SETTINGS index_granularity = 8192
|
||||
x UInt8
|
||||
e Enum8(\'hello\' = 1, \'world\' = 2) DEFAULT CAST(x, \'Enum8(\\\'hello\\\' = 1, \\\'world\\\' = 2)\')
|
||||
1 hello
|
||||
|
@ -1,4 +1,4 @@
|
||||
CREATE TABLE test.cast1 ( x UInt8, e Enum8('hello' = 1, 'world' = 2) DEFAULT CAST(x, 'Enum8(\'hello\' = 1, \'world\' = 2)')) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_cast', 'r1') ORDER BY e SETTINGS index_granularity = 8192
|
||||
CREATE TABLE test.cast1 (`x` UInt8, `e` Enum8('hello' = 1, 'world' = 2) DEFAULT CAST(x, 'Enum8(\'hello\' = 1, \'world\' = 2)')) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_cast', 'r1') ORDER BY e SETTINGS index_granularity = 8192
|
||||
x UInt8
|
||||
e Enum8(\'hello\' = 1, \'world\' = 2) DEFAULT CAST(x, \'Enum8(\\\'hello\\\' = 1, \\\'world\\\' = 2)\')
|
||||
1 hello
|
||||
|
@ -1,4 +1,4 @@
|
||||
CREATE TABLE test.check_query_comment_column ( first_column UInt8 DEFAULT 1 COMMENT \'comment 1\', second_column UInt8 MATERIALIZED first_column COMMENT \'comment 2\', third_column UInt8 ALIAS second_column COMMENT \'comment 3\', fourth_column UInt8 COMMENT \'comment 4\', fifth_column UInt8) ENGINE = TinyLog
|
||||
CREATE TABLE test.check_query_comment_column (`first_column` UInt8 DEFAULT 1 COMMENT \'comment 1\', `second_column` UInt8 MATERIALIZED first_column COMMENT \'comment 2\', `third_column` UInt8 ALIAS second_column COMMENT \'comment 3\', `fourth_column` UInt8 COMMENT \'comment 4\', `fifth_column` UInt8) ENGINE = TinyLog
|
||||
first_column UInt8 DEFAULT 1 comment 1
|
||||
second_column UInt8 MATERIALIZED first_column comment 2
|
||||
third_column UInt8 ALIAS second_column comment 3
|
||||
@ -11,7 +11,7 @@ fifth_column UInt8
|
||||
│ check_query_comment_column │ fourth_column │ comment 4 │
|
||||
│ check_query_comment_column │ fifth_column │ │
|
||||
└────────────────────────────┴───────────────┴───────────┘
|
||||
CREATE TABLE test.check_query_comment_column ( first_column UInt8 DEFAULT 1 COMMENT \'comment 1_1\', second_column UInt8 MATERIALIZED first_column COMMENT \'comment 2_1\', third_column UInt8 ALIAS second_column COMMENT \'comment 3_1\', fourth_column UInt8 COMMENT \'comment 4_1\', fifth_column UInt8 COMMENT \'comment 5_1\') ENGINE = TinyLog
|
||||
CREATE TABLE test.check_query_comment_column (`first_column` UInt8 DEFAULT 1 COMMENT \'comment 1_1\', `second_column` UInt8 MATERIALIZED first_column COMMENT \'comment 2_1\', `third_column` UInt8 ALIAS second_column COMMENT \'comment 3_1\', `fourth_column` UInt8 COMMENT \'comment 4_1\', `fifth_column` UInt8 COMMENT \'comment 5_1\') ENGINE = TinyLog
|
||||
┌─table──────────────────────┬─name──────────┬─comment─────┐
|
||||
│ check_query_comment_column │ first_column │ comment 1_2 │
|
||||
│ check_query_comment_column │ second_column │ comment 2_2 │
|
||||
@ -19,8 +19,8 @@ CREATE TABLE test.check_query_comment_column ( first_column UInt8 DEFAULT 1 COMM
|
||||
│ check_query_comment_column │ fourth_column │ comment 4_2 │
|
||||
│ check_query_comment_column │ fifth_column │ comment 5_2 │
|
||||
└────────────────────────────┴───────────────┴─────────────┘
|
||||
CREATE TABLE test.check_query_comment_column ( first_column UInt8 DEFAULT 1 COMMENT \'comment 1_2\', second_column UInt8 MATERIALIZED first_column COMMENT \'comment 2_2\', third_column UInt8 ALIAS second_column COMMENT \'comment 3_2\', fourth_column UInt8 COMMENT \'comment 4_2\', fifth_column UInt8 COMMENT \'comment 5_2\') ENGINE = TinyLog
|
||||
CREATE TABLE test.check_query_comment_column ( first_column UInt8 COMMENT \'comment 1\', second_column UInt8 COMMENT \'comment 2\', third_column UInt8 COMMENT \'comment 3\') ENGINE = MergeTree() PARTITION BY second_column ORDER BY first_column SAMPLE BY first_column SETTINGS index_granularity = 8192
|
||||
CREATE TABLE test.check_query_comment_column (`first_column` UInt8 DEFAULT 1 COMMENT \'comment 1_2\', `second_column` UInt8 MATERIALIZED first_column COMMENT \'comment 2_2\', `third_column` UInt8 ALIAS second_column COMMENT \'comment 3_2\', `fourth_column` UInt8 COMMENT \'comment 4_2\', `fifth_column` UInt8 COMMENT \'comment 5_2\') ENGINE = TinyLog
|
||||
CREATE TABLE test.check_query_comment_column (`first_column` UInt8 COMMENT \'comment 1\', `second_column` UInt8 COMMENT \'comment 2\', `third_column` UInt8 COMMENT \'comment 3\') ENGINE = MergeTree() PARTITION BY second_column ORDER BY first_column SAMPLE BY first_column SETTINGS index_granularity = 8192
|
||||
first_column UInt8 comment 1
|
||||
second_column UInt8 comment 2
|
||||
third_column UInt8 comment 3
|
||||
@ -29,8 +29,8 @@ third_column UInt8 comment 3
|
||||
│ check_query_comment_column │ second_column │ comment 2 │
|
||||
│ check_query_comment_column │ third_column │ comment 3 │
|
||||
└────────────────────────────┴───────────────┴───────────┘
|
||||
CREATE TABLE test.check_query_comment_column ( first_column UInt8 COMMENT \'comment 1_2\', second_column UInt8 COMMENT \'comment 2_2\', third_column UInt8 COMMENT \'comment 3_2\') ENGINE = MergeTree() PARTITION BY second_column ORDER BY first_column SAMPLE BY first_column SETTINGS index_granularity = 8192
|
||||
CREATE TABLE test.check_query_comment_column ( first_column UInt8 COMMENT \'comment 1_3\', second_column UInt8 COMMENT \'comment 2_3\', third_column UInt8 COMMENT \'comment 3_3\') ENGINE = MergeTree() PARTITION BY second_column ORDER BY first_column SAMPLE BY first_column SETTINGS index_granularity = 8192
|
||||
CREATE TABLE test.check_query_comment_column (`first_column` UInt8 COMMENT \'comment 1_2\', `second_column` UInt8 COMMENT \'comment 2_2\', `third_column` UInt8 COMMENT \'comment 3_2\') ENGINE = MergeTree() PARTITION BY second_column ORDER BY first_column SAMPLE BY first_column SETTINGS index_granularity = 8192
|
||||
CREATE TABLE test.check_query_comment_column (`first_column` UInt8 COMMENT \'comment 1_3\', `second_column` UInt8 COMMENT \'comment 2_3\', `third_column` UInt8 COMMENT \'comment 3_3\') ENGINE = MergeTree() PARTITION BY second_column ORDER BY first_column SAMPLE BY first_column SETTINGS index_granularity = 8192
|
||||
┌─table──────────────────────┬─name──────────┬─comment─────┐
|
||||
│ check_query_comment_column │ first_column │ comment 1_3 │
|
||||
│ check_query_comment_column │ second_column │ comment 2_3 │
|
||||
|
@ -1,4 +1,4 @@
|
||||
CREATE TABLE test.ipv4_test ( ipv4_ IPv4) ENGINE = Memory
|
||||
CREATE TABLE test.ipv4_test (`ipv4_` IPv4) ENGINE = Memory
|
||||
0.0.0.0 00
|
||||
8.8.8.8 08080808
|
||||
127.0.0.1 7F000001
|
||||
@ -10,7 +10,7 @@ CREATE TABLE test.ipv4_test ( ipv4_ IPv4) ENGINE = Memory
|
||||
> 127.0.0.1 255.255.255.255
|
||||
= 127.0.0.1 127.0.0.1
|
||||
euqality of IPv4-mapped IPv6 value and IPv4 promoted to IPv6 with function: 1
|
||||
CREATE TABLE test.ipv6_test ( ipv6_ IPv6) ENGINE = Memory
|
||||
CREATE TABLE test.ipv6_test (`ipv6_` IPv6) ENGINE = Memory
|
||||
:: 00000000000000000000000000000000
|
||||
:: 00000000000000000000000000000000
|
||||
::ffff:8.8.8.8 00000000000000000000FFFF08080808
|
||||
|
@ -1,4 +1,4 @@
|
||||
CREATE MATERIALIZED VIEW test.t_mv ( date Date, platform Enum8('a' = 0, 'b' = 1), app Enum8('a' = 0, 'b' = 1)) ENGINE = MergeTree ORDER BY date SETTINGS index_granularity = 8192 AS SELECT date, platform, app FROM test.t WHERE (app = (SELECT min(app) FROM test.u )) AND (platform = (SELECT (SELECT min(platform) FROM test.v )))
|
||||
CREATE MATERIALIZED VIEW test.t_mv (`date` Date, `platform` Enum8('a' = 0, 'b' = 1), `app` Enum8('a' = 0, 'b' = 1)) ENGINE = MergeTree ORDER BY date SETTINGS index_granularity = 8192 AS SELECT date, platform, app FROM test.t WHERE (app = (SELECT min(app) FROM test.u )) AND (platform = (SELECT (SELECT min(platform) FROM test.v )))
|
||||
2000-01-01 a a
|
||||
2000-01-02 b b
|
||||
2000-01-03 a a
|
||||
|
@ -1,6 +1,6 @@
|
||||
CREATE TABLE test.check_comments ( column_name1 UInt8 DEFAULT 1 COMMENT \'comment\', column_name2 UInt8 COMMENT \'non default comment\') ENGINE = ReplicatedMergeTree(\'clickhouse/tables/test_comments\', \'r1\') ORDER BY column_name1 SETTINGS index_granularity = 8192
|
||||
CREATE TABLE test.check_comments (`column_name1` UInt8 DEFAULT 1 COMMENT \'comment\', `column_name2` UInt8 COMMENT \'non default comment\') ENGINE = ReplicatedMergeTree(\'clickhouse/tables/test_comments\', \'r1\') ORDER BY column_name1 SETTINGS index_granularity = 8192
|
||||
column_name1 UInt8 DEFAULT 1 comment
|
||||
column_name2 UInt8 non default comment
|
||||
CREATE TABLE test.check_comments ( column_name1 UInt8 DEFAULT 1 COMMENT \'another comment\', column_name2 UInt8 COMMENT \'non default comment\') ENGINE = ReplicatedMergeTree(\'clickhouse/tables/test_comments\', \'r1\') ORDER BY column_name1 SETTINGS index_granularity = 8192
|
||||
CREATE TABLE test.check_comments (`column_name1` UInt8 DEFAULT 1 COMMENT \'another comment\', `column_name2` UInt8 COMMENT \'non default comment\') ENGINE = ReplicatedMergeTree(\'clickhouse/tables/test_comments\', \'r1\') ORDER BY column_name1 SETTINGS index_granularity = 8192
|
||||
column_name1 UInt8 DEFAULT 1 another comment
|
||||
column_name2 UInt8 non default comment
|
||||
|
@ -9,4 +9,4 @@
|
||||
1 2 1 30
|
||||
1 2 4 90
|
||||
*** Check SHOW CREATE TABLE ***
|
||||
CREATE TABLE test.summing ( x UInt32, y UInt32, z UInt32, val UInt32) ENGINE = SummingMergeTree PRIMARY KEY (x, y) ORDER BY (x, y, -z) SETTINGS index_granularity = 8192
|
||||
CREATE TABLE test.summing (`x` UInt32, `y` UInt32, `z` UInt32, `val` UInt32) ENGINE = SummingMergeTree PRIMARY KEY (x, y) ORDER BY (x, y, -z) SETTINGS index_granularity = 8192
|
||||
|
@ -9,6 +9,6 @@
|
||||
1 2 1 30
|
||||
1 2 4 90
|
||||
*** Check SHOW CREATE TABLE ***
|
||||
CREATE TABLE test.summing_r2 ( x UInt32, y UInt32, z UInt32, val UInt32) ENGINE = ReplicatedSummingMergeTree(\'/clickhouse/tables/test/summing\', \'r2\') PRIMARY KEY (x, y) ORDER BY (x, y, -z) SETTINGS index_granularity = 8192
|
||||
CREATE TABLE test.summing_r2 (`x` UInt32, `y` UInt32, `z` UInt32, `val` UInt32) ENGINE = ReplicatedSummingMergeTree(\'/clickhouse/tables/test/summing\', \'r2\') PRIMARY KEY (x, y) ORDER BY (x, y, -z) SETTINGS index_granularity = 8192
|
||||
*** Check SHOW CREATE TABLE after offline ALTER ***
|
||||
CREATE TABLE test.summing_r2 ( x UInt32, y UInt32, z UInt32, t UInt32, val UInt32) ENGINE = ReplicatedSummingMergeTree(\'/clickhouse/tables/test/summing\', \'r2\') PRIMARY KEY (x, y) ORDER BY (x, y, t * t) SETTINGS index_granularity = 8192
|
||||
CREATE TABLE test.summing_r2 (`x` UInt32, `y` UInt32, `z` UInt32, `t` UInt32, `val` UInt32) ENGINE = ReplicatedSummingMergeTree(\'/clickhouse/tables/test/summing\', \'r2\') PRIMARY KEY (x, y) ORDER BY (x, y, t * t) SETTINGS index_granularity = 8192
|
||||
|
@ -9,10 +9,10 @@
|
||||
10003
|
||||
274972506.6
|
||||
9175437371954010821
|
||||
CREATE TABLE test.compression_codec_multiple_more_types ( id Decimal(38, 13) CODEC(ZSTD(1), LZ4, ZSTD(1), ZSTD(1), Delta(2), Delta(4), Delta(1), LZ4HC(0)), data FixedString(12) CODEC(ZSTD(1), ZSTD(1), Delta(1), Delta(1), Delta(1), NONE, NONE, NONE, LZ4HC(0)), `ddd.age` Array(UInt8) CODEC(LZ4, LZ4HC(0), NONE, NONE, NONE, ZSTD(1), Delta(8)), `ddd.Name` Array(String) CODEC(LZ4, LZ4HC(0), NONE, NONE, NONE, ZSTD(1), Delta(8))) ENGINE = MergeTree() ORDER BY tuple() SETTINGS index_granularity = 8192
|
||||
CREATE TABLE test.compression_codec_multiple_more_types (`id` Decimal(38, 13) CODEC(ZSTD(1), LZ4, ZSTD(1), ZSTD(1), Delta(2), Delta(4), Delta(1), LZ4HC(0)), `data` FixedString(12) CODEC(ZSTD(1), ZSTD(1), Delta(1), Delta(1), Delta(1), NONE, NONE, NONE, LZ4HC(0)), `ddd.age` Array(UInt8) CODEC(LZ4, LZ4HC(0), NONE, NONE, NONE, ZSTD(1), Delta(8)), `ddd.Name` Array(String) CODEC(LZ4, LZ4HC(0), NONE, NONE, NONE, ZSTD(1), Delta(8))) ENGINE = MergeTree() ORDER BY tuple() SETTINGS index_granularity = 8192
|
||||
1.5555555555555 hello world! [77] ['John']
|
||||
7.1000000000000 xxxxxxxxxxxx [127] ['Henry']
|
||||
!
|
||||
222
|
||||
!ZSTD
|
||||
CREATE TABLE test.test_default_delta ( id UInt64 CODEC(Delta(8)), data String CODEC(Delta(1)), somedate Date CODEC(Delta(2)), somenum Float64 CODEC(Delta(8)), somestr FixedString(3) CODEC(Delta(1)), othernum Int64 CODEC(Delta(8)), yetothernum Float32 CODEC(Delta(4)), `ddd.age` Array(UInt8) CODEC(Delta(1)), `ddd.Name` Array(String) CODEC(Delta(1)), `ddd.OName` Array(String) CODEC(Delta(1)), `ddd.BName` Array(String) CODEC(Delta(1))) ENGINE = MergeTree() ORDER BY tuple() SETTINGS index_granularity = 8192
|
||||
CREATE TABLE test.test_default_delta (`id` UInt64 CODEC(Delta(8)), `data` String CODEC(Delta(1)), `somedate` Date CODEC(Delta(2)), `somenum` Float64 CODEC(Delta(8)), `somestr` FixedString(3) CODEC(Delta(1)), `othernum` Int64 CODEC(Delta(8)), `yetothernum` Float32 CODEC(Delta(4)), `ddd.age` Array(UInt8) CODEC(Delta(1)), `ddd.Name` Array(String) CODEC(Delta(1)), `ddd.OName` Array(String) CODEC(Delta(1)), `ddd.BName` Array(String) CODEC(Delta(1))) ENGINE = MergeTree() ORDER BY tuple() SETTINGS index_granularity = 8192
|
||||
|
@ -1,9 +1,9 @@
|
||||
CREATE TABLE test.compression_codec_log ( id UInt64 CODEC(LZ4), data String CODEC(ZSTD(1)), ddd Date CODEC(NONE), somenum Float64 CODEC(ZSTD(2)), somestr FixedString(3) CODEC(LZ4HC(7)), othernum Int64 CODEC(Delta(8))) ENGINE = Log()
|
||||
CREATE TABLE test.compression_codec_log (`id` UInt64 CODEC(LZ4), `data` String CODEC(ZSTD(1)), `ddd` Date CODEC(NONE), `somenum` Float64 CODEC(ZSTD(2)), `somestr` FixedString(3) CODEC(LZ4HC(7)), `othernum` Int64 CODEC(Delta(8))) ENGINE = Log()
|
||||
1 hello 2018-12-14 1.1 aaa 5
|
||||
2 world 2018-12-15 2.2 bbb 6
|
||||
3 ! 2018-12-16 3.3 ccc 7
|
||||
2
|
||||
CREATE TABLE test.compression_codec_multiple_log ( id UInt64 CODEC(LZ4, ZSTD(1), NONE, LZ4HC(0), Delta(4)), data String CODEC(ZSTD(2), NONE, Delta(2), LZ4HC(0), LZ4, LZ4, Delta(8)), ddd Date CODEC(NONE, NONE, NONE, Delta(1), LZ4, ZSTD(1), LZ4HC(0), LZ4HC(0)), somenum Float64 CODEC(Delta(4), LZ4, LZ4, ZSTD(2), LZ4HC(5), ZSTD(3), ZSTD(1))) ENGINE = Log()
|
||||
CREATE TABLE test.compression_codec_multiple_log (`id` UInt64 CODEC(LZ4, ZSTD(1), NONE, LZ4HC(0), Delta(4)), `data` String CODEC(ZSTD(2), NONE, Delta(2), LZ4HC(0), LZ4, LZ4, Delta(8)), `ddd` Date CODEC(NONE, NONE, NONE, Delta(1), LZ4, ZSTD(1), LZ4HC(0), LZ4HC(0)), `somenum` Float64 CODEC(Delta(4), LZ4, LZ4, ZSTD(2), LZ4HC(5), ZSTD(3), ZSTD(1))) ENGINE = Log()
|
||||
1 world 2018-10-05 1.1
|
||||
2 hello 2018-10-01 2.2
|
||||
3 buy 2018-10-11 3.3
|
||||
@ -11,12 +11,12 @@ CREATE TABLE test.compression_codec_multiple_log ( id UInt64 CODEC(LZ4, ZSTD(1),
|
||||
10003
|
||||
274972506.6
|
||||
9175437371954010821
|
||||
CREATE TABLE test.compression_codec_tiny_log ( id UInt64 CODEC(LZ4), data String CODEC(ZSTD(1)), ddd Date CODEC(NONE), somenum Float64 CODEC(ZSTD(2)), somestr FixedString(3) CODEC(LZ4HC(7)), othernum Int64 CODEC(Delta(8))) ENGINE = TinyLog()
|
||||
CREATE TABLE test.compression_codec_tiny_log (`id` UInt64 CODEC(LZ4), `data` String CODEC(ZSTD(1)), `ddd` Date CODEC(NONE), `somenum` Float64 CODEC(ZSTD(2)), `somestr` FixedString(3) CODEC(LZ4HC(7)), `othernum` Int64 CODEC(Delta(8))) ENGINE = TinyLog()
|
||||
1 hello 2018-12-14 1.1 aaa 5
|
||||
2 world 2018-12-15 2.2 bbb 6
|
||||
3 ! 2018-12-16 3.3 ccc 7
|
||||
2
|
||||
CREATE TABLE test.compression_codec_multiple_tiny_log ( id UInt64 CODEC(LZ4, ZSTD(1), NONE, LZ4HC(0), Delta(4)), data String CODEC(ZSTD(2), NONE, Delta(2), LZ4HC(0), LZ4, LZ4, Delta(8)), ddd Date CODEC(NONE, NONE, NONE, Delta(1), LZ4, ZSTD(1), LZ4HC(0), LZ4HC(0)), somenum Float64 CODEC(Delta(4), LZ4, LZ4, ZSTD(2), LZ4HC(5), ZSTD(3), ZSTD(1))) ENGINE = TinyLog()
|
||||
CREATE TABLE test.compression_codec_multiple_tiny_log (`id` UInt64 CODEC(LZ4, ZSTD(1), NONE, LZ4HC(0), Delta(4)), `data` String CODEC(ZSTD(2), NONE, Delta(2), LZ4HC(0), LZ4, LZ4, Delta(8)), `ddd` Date CODEC(NONE, NONE, NONE, Delta(1), LZ4, ZSTD(1), LZ4HC(0), LZ4HC(0)), `somenum` Float64 CODEC(Delta(4), LZ4, LZ4, ZSTD(2), LZ4HC(5), ZSTD(3), ZSTD(1))) ENGINE = TinyLog()
|
||||
1 world 2018-10-05 1.1
|
||||
2 hello 2018-10-01 2.2
|
||||
3 buy 2018-10-11 3.3
|
||||
|
@ -1,4 +1,4 @@
|
||||
CREATE TABLE test.minmax_idx ( u64 UInt64, i32 Int32, INDEX idx1 u64 * i32 TYPE minmax GRANULARITY 10, INDEX idx3 u64 - i32 TYPE minmax GRANULARITY 10, INDEX idx2 u64 + i32 TYPE minmax GRANULARITY 10) ENGINE = MergeTree() ORDER BY u64 SETTINGS index_granularity = 8192
|
||||
CREATE TABLE test.minmax_idx (`u64` UInt64, `i32` Int32, INDEX idx1 u64 * i32 TYPE minmax GRANULARITY 10, INDEX idx3 u64 - i32 TYPE minmax GRANULARITY 10, INDEX idx2 u64 + i32 TYPE minmax GRANULARITY 10) ENGINE = MergeTree() ORDER BY u64 SETTINGS index_granularity = 8192
|
||||
1 2
|
||||
1 2
|
||||
1 2
|
||||
@ -6,15 +6,15 @@ CREATE TABLE test.minmax_idx ( u64 UInt64, i32 Int32, INDEX idx1 u64 * i32 TYP
|
||||
1 2
|
||||
1 2
|
||||
1 2
|
||||
CREATE TABLE test.minmax_idx ( u64 UInt64, i32 Int32, INDEX idx3 u64 - i32 TYPE minmax GRANULARITY 10, INDEX idx2 u64 + i32 TYPE minmax GRANULARITY 10) ENGINE = MergeTree() ORDER BY u64 SETTINGS index_granularity = 8192
|
||||
CREATE TABLE test.minmax_idx (`u64` UInt64, `i32` Int32, INDEX idx3 u64 - i32 TYPE minmax GRANULARITY 10, INDEX idx2 u64 + i32 TYPE minmax GRANULARITY 10) ENGINE = MergeTree() ORDER BY u64 SETTINGS index_granularity = 8192
|
||||
1 2
|
||||
1 2
|
||||
1 2
|
||||
1 2
|
||||
1 2
|
||||
1 2
|
||||
CREATE TABLE test.minmax_idx ( u64 UInt64, i32 Int32) ENGINE = MergeTree() ORDER BY u64 SETTINGS index_granularity = 8192
|
||||
CREATE TABLE test.minmax_idx ( u64 UInt64, i32 Int32, INDEX idx1 u64 * i32 TYPE minmax GRANULARITY 10) ENGINE = MergeTree() ORDER BY u64 SETTINGS index_granularity = 8192
|
||||
CREATE TABLE test.minmax_idx (`u64` UInt64, `i32` Int32) ENGINE = MergeTree() ORDER BY u64 SETTINGS index_granularity = 8192
|
||||
CREATE TABLE test.minmax_idx (`u64` UInt64, `i32` Int32, INDEX idx1 u64 * i32 TYPE minmax GRANULARITY 10) ENGINE = MergeTree() ORDER BY u64 SETTINGS index_granularity = 8192
|
||||
1 2
|
||||
1 2
|
||||
1 2
|
||||
@ -23,6 +23,6 @@ CREATE TABLE test.minmax_idx ( u64 UInt64, i32 Int32, INDEX idx1 u64 * i32 TYP
|
||||
1 2
|
||||
1 2
|
||||
1 2
|
||||
CREATE TABLE test.minmax_idx2 ( u64 UInt64, i32 Int32) ENGINE = MergeTree() ORDER BY u64 SETTINGS index_granularity = 8192
|
||||
CREATE TABLE test.minmax_idx2 (`u64` UInt64, `i32` Int32) ENGINE = MergeTree() ORDER BY u64 SETTINGS index_granularity = 8192
|
||||
1 2
|
||||
1 2
|
||||
|
@ -1,5 +1,5 @@
|
||||
CREATE TABLE test.minmax_idx ( u64 UInt64, i32 Int32, INDEX idx1 u64 * i32 TYPE minmax GRANULARITY 10, INDEX idx3 u64 - i32 TYPE minmax GRANULARITY 10, INDEX idx2 u64 + i32 TYPE minmax GRANULARITY 10) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/indices_alter1\', \'r1\') ORDER BY u64 SETTINGS index_granularity = 8192
|
||||
CREATE TABLE test.minmax_idx_r ( u64 UInt64, i32 Int32, INDEX idx1 u64 * i32 TYPE minmax GRANULARITY 10, INDEX idx3 u64 - i32 TYPE minmax GRANULARITY 10, INDEX idx2 u64 + i32 TYPE minmax GRANULARITY 10) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/indices_alter1\', \'r2\') ORDER BY u64 SETTINGS index_granularity = 8192
|
||||
CREATE TABLE test.minmax_idx (`u64` UInt64, `i32` Int32, INDEX idx1 u64 * i32 TYPE minmax GRANULARITY 10, INDEX idx3 u64 - i32 TYPE minmax GRANULARITY 10, INDEX idx2 u64 + i32 TYPE minmax GRANULARITY 10) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/indices_alter1\', \'r1\') ORDER BY u64 SETTINGS index_granularity = 8192
|
||||
CREATE TABLE test.minmax_idx_r (`u64` UInt64, `i32` Int32, INDEX idx1 u64 * i32 TYPE minmax GRANULARITY 10, INDEX idx3 u64 - i32 TYPE minmax GRANULARITY 10, INDEX idx2 u64 + i32 TYPE minmax GRANULARITY 10) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/indices_alter1\', \'r2\') ORDER BY u64 SETTINGS index_granularity = 8192
|
||||
1 2
|
||||
1 2
|
||||
1 2
|
||||
@ -14,8 +14,8 @@ CREATE TABLE test.minmax_idx_r ( u64 UInt64, i32 Int32, INDEX idx1 u64 * i32 T
|
||||
3 2
|
||||
19 9
|
||||
65 75
|
||||
CREATE TABLE test.minmax_idx ( u64 UInt64, i32 Int32, INDEX idx3 u64 - i32 TYPE minmax GRANULARITY 10, INDEX idx2 u64 + i32 TYPE minmax GRANULARITY 10) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/indices_alter1\', \'r1\') ORDER BY u64 SETTINGS index_granularity = 8192
|
||||
CREATE TABLE test.minmax_idx_r ( u64 UInt64, i32 Int32, INDEX idx3 u64 - i32 TYPE minmax GRANULARITY 10, INDEX idx2 u64 + i32 TYPE minmax GRANULARITY 10) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/indices_alter1\', \'r2\') ORDER BY u64 SETTINGS index_granularity = 8192
|
||||
CREATE TABLE test.minmax_idx (`u64` UInt64, `i32` Int32, INDEX idx3 u64 - i32 TYPE minmax GRANULARITY 10, INDEX idx2 u64 + i32 TYPE minmax GRANULARITY 10) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/indices_alter1\', \'r1\') ORDER BY u64 SETTINGS index_granularity = 8192
|
||||
CREATE TABLE test.minmax_idx_r (`u64` UInt64, `i32` Int32, INDEX idx3 u64 - i32 TYPE minmax GRANULARITY 10, INDEX idx2 u64 + i32 TYPE minmax GRANULARITY 10) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/indices_alter1\', \'r2\') ORDER BY u64 SETTINGS index_granularity = 8192
|
||||
1 2
|
||||
1 4
|
||||
1 5
|
||||
@ -28,10 +28,10 @@ CREATE TABLE test.minmax_idx_r ( u64 UInt64, i32 Int32, INDEX idx3 u64 - i32 T
|
||||
3 2
|
||||
19 9
|
||||
65 75
|
||||
CREATE TABLE test.minmax_idx ( u64 UInt64, i32 Int32) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/indices_alter1\', \'r1\') ORDER BY u64 SETTINGS index_granularity = 8192
|
||||
CREATE TABLE test.minmax_idx_r ( u64 UInt64, i32 Int32) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/indices_alter1\', \'r2\') ORDER BY u64 SETTINGS index_granularity = 8192
|
||||
CREATE TABLE test.minmax_idx ( u64 UInt64, i32 Int32, INDEX idx1 u64 * i32 TYPE minmax GRANULARITY 10) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/indices_alter1\', \'r1\') ORDER BY u64 SETTINGS index_granularity = 8192
|
||||
CREATE TABLE test.minmax_idx_r ( u64 UInt64, i32 Int32, INDEX idx1 u64 * i32 TYPE minmax GRANULARITY 10) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/indices_alter1\', \'r2\') ORDER BY u64 SETTINGS index_granularity = 8192
|
||||
CREATE TABLE test.minmax_idx (`u64` UInt64, `i32` Int32) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/indices_alter1\', \'r1\') ORDER BY u64 SETTINGS index_granularity = 8192
|
||||
CREATE TABLE test.minmax_idx_r (`u64` UInt64, `i32` Int32) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/indices_alter1\', \'r2\') ORDER BY u64 SETTINGS index_granularity = 8192
|
||||
CREATE TABLE test.minmax_idx (`u64` UInt64, `i32` Int32, INDEX idx1 u64 * i32 TYPE minmax GRANULARITY 10) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/indices_alter1\', \'r1\') ORDER BY u64 SETTINGS index_granularity = 8192
|
||||
CREATE TABLE test.minmax_idx_r (`u64` UInt64, `i32` Int32, INDEX idx1 u64 * i32 TYPE minmax GRANULARITY 10) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/indices_alter1\', \'r2\') ORDER BY u64 SETTINGS index_granularity = 8192
|
||||
1 2
|
||||
1 4
|
||||
1 5
|
||||
@ -44,14 +44,14 @@ CREATE TABLE test.minmax_idx_r ( u64 UInt64, i32 Int32, INDEX idx1 u64 * i32 T
|
||||
3 2
|
||||
19 9
|
||||
65 75
|
||||
CREATE TABLE test.minmax_idx2 ( u64 UInt64, i32 Int32, INDEX idx1 u64 + i32 TYPE minmax GRANULARITY 10, INDEX idx2 u64 * i32 TYPE minmax GRANULARITY 10) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/indices_alter2\', \'r1\') ORDER BY u64 SETTINGS index_granularity = 8192
|
||||
CREATE TABLE test.minmax_idx2_r ( u64 UInt64, i32 Int32, INDEX idx1 u64 + i32 TYPE minmax GRANULARITY 10, INDEX idx2 u64 * i32 TYPE minmax GRANULARITY 10) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/indices_alter2\', \'r2\') ORDER BY u64 SETTINGS index_granularity = 8192
|
||||
CREATE TABLE test.minmax_idx2 (`u64` UInt64, `i32` Int32, INDEX idx1 u64 + i32 TYPE minmax GRANULARITY 10, INDEX idx2 u64 * i32 TYPE minmax GRANULARITY 10) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/indices_alter2\', \'r1\') ORDER BY u64 SETTINGS index_granularity = 8192
|
||||
CREATE TABLE test.minmax_idx2_r (`u64` UInt64, `i32` Int32, INDEX idx1 u64 + i32 TYPE minmax GRANULARITY 10, INDEX idx2 u64 * i32 TYPE minmax GRANULARITY 10) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/indices_alter2\', \'r2\') ORDER BY u64 SETTINGS index_granularity = 8192
|
||||
1 2
|
||||
1 3
|
||||
1 2
|
||||
1 3
|
||||
CREATE TABLE test.minmax_idx2 ( u64 UInt64, i32 Int32) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/indices_alter2\', \'r1\') ORDER BY u64 SETTINGS index_granularity = 8192
|
||||
CREATE TABLE test.minmax_idx2_r ( u64 UInt64, i32 Int32) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/indices_alter2\', \'r2\') ORDER BY u64 SETTINGS index_granularity = 8192
|
||||
CREATE TABLE test.minmax_idx2 (`u64` UInt64, `i32` Int32) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/indices_alter2\', \'r1\') ORDER BY u64 SETTINGS index_granularity = 8192
|
||||
CREATE TABLE test.minmax_idx2_r (`u64` UInt64, `i32` Int32) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/indices_alter2\', \'r2\') ORDER BY u64 SETTINGS index_granularity = 8192
|
||||
1 2
|
||||
1 3
|
||||
1 2
|
||||
|
@ -0,0 +1,44 @@
|
||||
on
|
||||
l \N \N String Nullable(String)
|
||||
l \N \N String Nullable(String)
|
||||
r \N String Nullable(String)
|
||||
\N r \N Nullable(String) Nullable(String)
|
||||
l \N String Nullable(String)
|
||||
l \N String Nullable(String)
|
||||
r \N String Nullable(String)
|
||||
\N r \N Nullable(String) Nullable(String)
|
||||
\N \N
|
||||
0 \N
|
||||
using
|
||||
l \N String Nullable(String)
|
||||
l \N String Nullable(String)
|
||||
\N String Nullable(String)
|
||||
\N \N Nullable(String) Nullable(String)
|
||||
l \N String Nullable(String)
|
||||
l \N String Nullable(String)
|
||||
\N String Nullable(String)
|
||||
\N \N Nullable(String) Nullable(String)
|
||||
\N \N
|
||||
0 \N
|
||||
on + join_use_nulls
|
||||
l \N \N TODO Nullable(String)
|
||||
l \N \N TODO Nullable(String)
|
||||
r \N TODO Nullable(String)
|
||||
\N r \N Nullable(String) Nullable(String)
|
||||
l \N TODO Nullable(String)
|
||||
l \N TODO Nullable(String)
|
||||
r \N TODO Nullable(String)
|
||||
\N r \N Nullable(String) Nullable(String)
|
||||
\N \N
|
||||
0 \N
|
||||
using + join_use_nulls
|
||||
l \N TODO Nullable(String)
|
||||
l \N TODO Nullable(String)
|
||||
\N TODO Nullable(String)
|
||||
\N \N Nullable(String) Nullable(String)
|
||||
l \N TODO Nullable(String)
|
||||
l \N TODO Nullable(String)
|
||||
\N TODO Nullable(String)
|
||||
\N \N Nullable(String) Nullable(String)
|
||||
\N \N
|
||||
0 \N
|
@ -0,0 +1,69 @@
|
||||
USE test;
|
||||
|
||||
DROP TABLE IF EXISTS t1;
|
||||
DROP TABLE IF EXISTS t2;
|
||||
DROP TABLE IF EXISTS t3;
|
||||
CREATE TABLE t1 ( id String ) ENGINE = Memory;
|
||||
CREATE TABLE t2 ( id Nullable(String) ) ENGINE = Memory;
|
||||
CREATE TABLE t3 ( id Nullable(String), not_id Nullable(String) ) ENGINE = Memory;
|
||||
|
||||
insert into t1 values ('l');
|
||||
insert into t3 (id) values ('r');
|
||||
|
||||
SELECT 'on';
|
||||
|
||||
SELECT *, toTypeName(t1.id), toTypeName(t3.id) FROM t1 ANY LEFT JOIN t3 ON t1.id = t3.id;
|
||||
SELECT *, toTypeName(t1.id), toTypeName(t3.id) FROM t1 ANY FULL JOIN t3 ON t1.id = t3.id;
|
||||
SELECT *, toTypeName(t2.id), toTypeName(t3.id) FROM t2 ANY FULL JOIN t3 ON t2.id = t3.id;
|
||||
|
||||
SELECT *, toTypeName(t1.id), toTypeName(t3.id) FROM t1 LEFT JOIN t3 ON t1.id = t3.id;
|
||||
SELECT *, toTypeName(t1.id), toTypeName(t3.id) FROM t1 FULL JOIN t3 ON t1.id = t3.id;
|
||||
SELECT *, toTypeName(t2.id), toTypeName(t3.id) FROM t2 FULL JOIN t3 ON t2.id = t3.id;
|
||||
|
||||
SELECT t3.id = 'l', t3.not_id = 'l' FROM t1 ANY LEFT JOIN t3 ON t1.id = t3.id;
|
||||
SELECT t3.id = 'l', t3.not_id = 'l' FROM t1 LEFT JOIN t3 ON t1.id = t3.id;
|
||||
|
||||
SELECT 'using';
|
||||
|
||||
SELECT *, toTypeName(t1.id), toTypeName(t3.id) FROM t1 ANY LEFT JOIN t3 USING(id);
|
||||
SELECT *, toTypeName(t1.id), toTypeName(t3.id) FROM t1 ANY FULL JOIN t3 USING(id);
|
||||
SELECT *, toTypeName(t2.id), toTypeName(t3.id) FROM t2 ANY FULL JOIN t3 USING(id);
|
||||
|
||||
SELECT *, toTypeName(t1.id), toTypeName(t3.id) FROM t1 LEFT JOIN t3 USING(id);
|
||||
SELECT *, toTypeName(t1.id), toTypeName(t3.id) FROM t1 FULL JOIN t3 USING(id);
|
||||
SELECT *, toTypeName(t2.id), toTypeName(t3.id) FROM t2 FULL JOIN t3 USING(id);
|
||||
|
||||
SELECT t3.id = 'l', t3.not_id = 'l' FROM t1 ANY LEFT JOIN t3 USING(id);
|
||||
SELECT t3.id = 'l', t3.not_id = 'l' FROM t1 LEFT JOIN t3 USING(id);
|
||||
|
||||
SET join_use_nulls = 1;
|
||||
-- TODO: toTypeName(t1.id) String -> Nullable(String)
|
||||
|
||||
SELECT 'on + join_use_nulls';
|
||||
|
||||
SELECT *, 'TODO', toTypeName(t3.id) FROM t1 ANY LEFT JOIN t3 ON t1.id = t3.id;
|
||||
SELECT *, 'TODO', toTypeName(t3.id) FROM t1 ANY FULL JOIN t3 ON t1.id = t3.id;
|
||||
SELECT *, toTypeName(t2.id), toTypeName(t3.id) FROM t2 ANY FULL JOIN t3 ON t2.id = t3.id;
|
||||
|
||||
SELECT *, 'TODO', toTypeName(t3.id) FROM t1 LEFT JOIN t3 ON t1.id = t3.id;
|
||||
SELECT *, 'TODO', toTypeName(t3.id) FROM t1 FULL JOIN t3 ON t1.id = t3.id;
|
||||
SELECT *, toTypeName(t2.id), toTypeName(t3.id) FROM t2 FULL JOIN t3 ON t2.id = t3.id;
|
||||
|
||||
SELECT t3.id = 'l', t3.not_id = 'l' FROM t1 ANY LEFT JOIN t3 ON t1.id = t3.id;
|
||||
SELECT t3.id = 'l', t3.not_id = 'l' FROM t1 LEFT JOIN t3 ON t1.id = t3.id;
|
||||
|
||||
SELECT 'using + join_use_nulls';
|
||||
|
||||
SELECT *, 'TODO', toTypeName(t3.id) FROM t1 ANY LEFT JOIN t3 USING(id);
|
||||
SELECT *, 'TODO', toTypeName(t3.id) FROM t1 ANY FULL JOIN t3 USING(id);
|
||||
SELECT *, toTypeName(t2.id), toTypeName(t3.id) FROM t2 ANY FULL JOIN t3 USING(id);
|
||||
|
||||
SELECT *, 'TODO', toTypeName(t3.id) FROM t1 LEFT JOIN t3 USING(id);
|
||||
SELECT *, 'TODO', toTypeName(t3.id) FROM t1 FULL JOIN t3 USING(id);
|
||||
SELECT *, toTypeName(t2.id), toTypeName(t3.id) FROM t2 FULL JOIN t3 USING(id);
|
||||
|
||||
SELECT t3.id = 'l', t3.not_id = 'l' FROM t1 ANY LEFT JOIN t3 USING(id);
|
||||
SELECT t3.id = 'l', t3.not_id = 'l' FROM t1 LEFT JOIN t3 USING(id);
|
||||
|
||||
DROP TABLE t1;
|
||||
DROP TABLE t2;
|
@ -0,0 +1,2 @@
|
||||
1 0
|
||||
\N 1
|
19
dbms/tests/queries/0_stateless/00852_any_join_nulls.sql
Normal file
19
dbms/tests/queries/0_stateless/00852_any_join_nulls.sql
Normal file
@ -0,0 +1,19 @@
|
||||
USE test;
|
||||
|
||||
DROP TABLE IF EXISTS table1;
|
||||
DROP TABLE IF EXISTS table2;
|
||||
CREATE TABLE table1 ( id String ) ENGINE = Log;
|
||||
CREATE TABLE table2 ( parent_id String ) ENGINE = Log;
|
||||
|
||||
insert into table1 values ('1');
|
||||
|
||||
SELECT table2.parent_id = '', isNull(table2.parent_id)
|
||||
FROM table1 ANY LEFT JOIN table2 ON table1.id = table2.parent_id;
|
||||
|
||||
SET join_use_nulls = 1;
|
||||
|
||||
SELECT table2.parent_id = '', isNull(table2.parent_id)
|
||||
FROM table1 ANY LEFT JOIN table2 ON table1.id = table2.parent_id;
|
||||
|
||||
DROP TABLE test.table1;
|
||||
DROP TABLE test.table2;
|
@ -1,4 +1,3 @@
|
||||
CREATE DATABASE IF NOT EXISTS test;
|
||||
DROP TABLE IF EXISTS test.defaults;
|
||||
CREATE TABLE IF NOT EXISTS test.defaults
|
||||
(
|
||||
@ -9,7 +8,6 @@ insert into test.defaults values ('ba'), ('aa'), ('ba'), ('b'), ('ba'), ('aa');
|
||||
select val < 1.5 and val > 1.459 from (select entropy(vals) as val from test.defaults);
|
||||
|
||||
|
||||
CREATE DATABASE IF NOT EXISTS test;
|
||||
DROP TABLE IF EXISTS test.defaults;
|
||||
CREATE TABLE IF NOT EXISTS test.defaults
|
||||
(
|
||||
@ -19,7 +17,6 @@ insert into test.defaults values (0), (0), (1), (0), (0), (0), (1), (2), (3), (5
|
||||
select val < 2.4 and val > 2.3393 from (select entropy(vals) as val from test.defaults);
|
||||
|
||||
|
||||
CREATE DATABASE IF NOT EXISTS test;
|
||||
DROP TABLE IF EXISTS test.defaults;
|
||||
CREATE TABLE IF NOT EXISTS test.defaults
|
||||
(
|
||||
@ -29,7 +26,6 @@ insert into test.defaults values (0), (0), (1), (0), (0), (0), (1), (2), (3), (5
|
||||
select val < 2.4 and val > 2.3393 from (select entropy(vals) as val from test.defaults);
|
||||
|
||||
|
||||
CREATE DATABASE IF NOT EXISTS test;
|
||||
DROP TABLE IF EXISTS test.defaults;
|
||||
CREATE TABLE IF NOT EXISTS test.defaults
|
||||
(
|
||||
@ -39,7 +35,6 @@ insert into test.defaults values (0), (0), (-1), (0), (0), (0), (-1), (2), (3),
|
||||
select val < 2.4 and val > 2.3393 from (select entropy(vals) as val from test.defaults);
|
||||
|
||||
|
||||
CREATE DATABASE IF NOT EXISTS test;
|
||||
DROP TABLE IF EXISTS test.defaults;
|
||||
CREATE TABLE IF NOT EXISTS test.defaults
|
||||
(
|
||||
|
@ -20,7 +20,7 @@
|
||||
274972506.6
|
||||
9175437371954010821
|
||||
9175437371954010821
|
||||
CREATE TABLE test.compression_codec_multiple_more_types_replicated ( id Decimal(38, 13) CODEC(ZSTD(1), LZ4, ZSTD(1), ZSTD(1), Delta(2), Delta(4), Delta(1), LZ4HC(0)), data FixedString(12) CODEC(ZSTD(1), ZSTD(1), Delta(1), Delta(1), Delta(1), NONE, NONE, NONE, LZ4HC(0)), `ddd.age` Array(UInt8) CODEC(LZ4, LZ4HC(0), NONE, NONE, NONE, ZSTD(1), Delta(8)), `ddd.Name` Array(String) CODEC(LZ4, LZ4HC(0), NONE, NONE, NONE, ZSTD(1), Delta(8))) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/compression_codec_multiple_more_types_replicated\', \'1\') ORDER BY tuple() SETTINGS index_granularity = 8192
|
||||
CREATE TABLE test.compression_codec_multiple_more_types_replicated (`id` Decimal(38, 13) CODEC(ZSTD(1), LZ4, ZSTD(1), ZSTD(1), Delta(2), Delta(4), Delta(1), LZ4HC(0)), `data` FixedString(12) CODEC(ZSTD(1), ZSTD(1), Delta(1), Delta(1), Delta(1), NONE, NONE, NONE, LZ4HC(0)), `ddd.age` Array(UInt8) CODEC(LZ4, LZ4HC(0), NONE, NONE, NONE, ZSTD(1), Delta(8)), `ddd.Name` Array(String) CODEC(LZ4, LZ4HC(0), NONE, NONE, NONE, ZSTD(1), Delta(8))) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/compression_codec_multiple_more_types_replicated\', \'1\') ORDER BY tuple() SETTINGS index_granularity = 8192
|
||||
1.5555555555555 hello world! [77] ['John']
|
||||
7.1000000000000 xxxxxxxxxxxx [127] ['Henry']
|
||||
!
|
||||
|
@ -1,6 +1,7 @@
|
||||
SET send_logs_level = 'none';
|
||||
|
||||
DROP TABLE IF EXISTS test.compression_codec_replicated;
|
||||
DROP TABLE IF EXISTS test.compression_codec_replicated1;
|
||||
DROP TABLE IF EXISTS test.compression_codec_replicated2;
|
||||
|
||||
CREATE TABLE test.compression_codec_replicated1(
|
||||
id UInt64 CODEC(LZ4),
|
||||
|
@ -1,2 +1,2 @@
|
||||
CREATE VIEW test.t ( number UInt64) AS SELECT number FROM system.numbers
|
||||
CREATE VIEW test.t ( next_number UInt64) AS SELECT number + 1 AS next_number FROM system.numbers
|
||||
CREATE VIEW test.t (`number` UInt64) AS SELECT number FROM system.numbers
|
||||
CREATE VIEW test.t (`next_number` UInt64) AS SELECT number + 1 AS next_number FROM system.numbers
|
||||
|
@ -0,0 +1,9 @@
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
0
|
||||
1
|
||||
1
|
||||
1
|
||||
0
|
@ -0,0 +1,6 @@
|
||||
SET allow_hyperscan = 1;
|
||||
SELECT multiMatchAny(arrayJoin(['hello', 'world', 'hellllllllo', 'wororld', 'abc']), ['hel+o', 'w(or)*ld']);
|
||||
SET allow_hyperscan = 0;
|
||||
SELECT multiMatchAny(arrayJoin(['hello', 'world', 'hellllllllo', 'wororld', 'abc']), ['hel+o', 'w(or)*ld']); -- { serverError 446 }
|
||||
|
||||
SELECT multiSearchAny(arrayJoin(['hello', 'world', 'hello, world', 'abc']), ['hello', 'world']);
|
@ -0,0 +1,3 @@
|
||||
1
|
||||
2
|
||||
[1,8]
|
@ -0,0 +1,3 @@
|
||||
SELECT multiMatchAny('goodbye', ['^hello[, ]+world$', 'go+d *bye', 'w(or)+ld']);
|
||||
SELECT multiMatchAnyIndex('goodbye', ['^hello[, ]+world$', 'go+d *bye', 'w(or)+ld']);
|
||||
SELECT multiSearchAllPositions('hello, world', ['hello', 'world']);
|
@ -34,4 +34,4 @@ RUN apt-get update -y \
|
||||
git \
|
||||
tzdata
|
||||
|
||||
CMD mkdir -p build/build_result && cd build/build_result && cmake .. -DCMAKE_BUILD_TYPE=$BUILD_TYPE -DSANITIZE=$SANITIZER $CMAKE_FLAGS && ninja && mv ./dbms/programs/clickhouse* /output
|
||||
CMD mkdir -p build/build_result && cd build/build_result && cmake .. -DCMAKE_BUILD_TYPE=$BUILD_TYPE -DSANITIZE=$SANITIZER $CMAKE_FLAGS && ninja && mv ./dbms/programs/clickhouse* /output && mv ./dbms/unit_tests_dbms /output
|
||||
|
@ -17,7 +17,8 @@ RUN apt-get update -y \
|
||||
sudo \
|
||||
openssl \
|
||||
netcat-openbsd \
|
||||
telnet
|
||||
telnet \
|
||||
llvm-8
|
||||
|
||||
COPY ./stress /stress
|
||||
COPY log_queries.xml /etc/clickhouse-server/users.d/log_queries.xml
|
||||
@ -28,7 +29,9 @@ CMD dpkg -i package_folder/clickhouse-common-static_*.deb; \
|
||||
dpkg -i package_folder/clickhouse-server_*.deb; \
|
||||
dpkg -i package_folder/clickhouse-client_*.deb; \
|
||||
dpkg -i package_folder/clickhouse-test_*.deb; \
|
||||
ln -s /usr/lib/llvm-8/bin/llvm-symbolizer /usr/bin/llvm-symbolizer; \
|
||||
echo "TSAN_OPTIONS='halt_on_error=1'" >> /etc/environment; \
|
||||
echo "TSAN_SYMBOLIZER_PATH=/usr/lib/llvm-8/bin/llvm-symbolizer" >> /etc/environment; \
|
||||
echo "UBSAN_OPTIONS='print_stacktrace=1'" >> /etc/environment; \
|
||||
echo "ASAN_SYMBOLIZER_PATH=/usr/lib/llvm-6.0/bin/llvm-symbolizer" >> /etc/environment; \
|
||||
echo "UBSAN_SYMBOLIZER_PATH=/usr/lib/llvm-6.0/bin/llvm-symbolizer" >> /etc/environment; \
|
||||
|
@ -25,7 +25,7 @@ SELECT 1 - 0.9
|
||||
|
||||
- The result of the calculation depends on the calculation method (the processor type and architecture of the computer system).
|
||||
- Floating-point calculations might result in numbers such as infinity (`Inf`) and "not-a-number" (`NaN`). This should be taken into account when processing the results of calculations.
|
||||
- When reading floating point numbers from rows, the result might not be the nearest machine-representable number.
|
||||
- When parsing floating point numbers from text, the result might not be the nearest machine-representable number.
|
||||
|
||||
## NaN and Inf
|
||||
|
||||
|
@ -19,6 +19,7 @@ musl/sched_cpucount.c
|
||||
musl/glob.c
|
||||
musl/exp2f.c
|
||||
musl/pwritev.c
|
||||
musl/getentropy.c
|
||||
musl/getrandom.c
|
||||
musl/fcntl.c
|
||||
musl/timespec_get.c
|
||||
|
33
libs/libglibc-compatibility/musl/getentropy.c
Normal file
33
libs/libglibc-compatibility/musl/getentropy.c
Normal file
@ -0,0 +1,33 @@
|
||||
#define _DEFAULT_SOURCE
|
||||
#include <unistd.h>
|
||||
#include <sys/random.h>
|
||||
#include <pthread.h>
|
||||
#include <errno.h>
|
||||
|
||||
int getentropy(void *buffer, size_t len)
|
||||
{
|
||||
int cs, ret = 0;
|
||||
char *pos = buffer;
|
||||
|
||||
if (len > 256) {
|
||||
errno = EIO;
|
||||
return -1;
|
||||
}
|
||||
|
||||
pthread_setcancelstate(PTHREAD_CANCEL_DISABLE, &cs);
|
||||
|
||||
while (len) {
|
||||
ret = getrandom(pos, len, 0);
|
||||
if (ret < 0) {
|
||||
if (errno == EINTR) continue;
|
||||
else break;
|
||||
}
|
||||
pos += ret;
|
||||
len -= ret;
|
||||
ret = 0;
|
||||
}
|
||||
|
||||
pthread_setcancelstate(cs, 0);
|
||||
|
||||
return ret;
|
||||
}
|
Loading…
Reference in New Issue
Block a user