From 24ff635435a2fe15956aa730187847315f632782 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Mon, 3 Feb 2020 11:02:52 +0100 Subject: [PATCH 01/15] draft implementation, wip --- dbms/src/Formats/FormatFactory.h | 6 ++- .../Processors/Formats/IRowOutputFormat.cpp | 4 +- .../Storages/Kafka/KafkaBlockOutputStream.cpp | 4 +- dbms/src/Storages/Kafka/StorageKafka.cpp | 4 +- dbms/src/Storages/Kafka/StorageKafka.h | 2 +- .../Kafka/WriteBufferToKafkaProducer.cpp | 45 +++++++++++++++++-- .../Kafka/WriteBufferToKafkaProducer.h | 10 +++-- 7 files changed, 60 insertions(+), 15 deletions(-) diff --git a/dbms/src/Formats/FormatFactory.h b/dbms/src/Formats/FormatFactory.h index 345ceaee690..7c18971e0eb 100644 --- a/dbms/src/Formats/FormatFactory.h +++ b/dbms/src/Formats/FormatFactory.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include @@ -9,7 +10,6 @@ #include #include - namespace DB { @@ -53,7 +53,9 @@ public: /// This callback allows to perform some additional actions after writing a single row. /// It's initial purpose was to flush Kafka message for each row. - using WriteCallback = std::function; + using WriteCallback = std::function; private: using InputCreator = std::functioncount_row(); }); + child = FormatFactory::instance().getOutput(storage.getFormatName(), *buffer, getHeader(), context, [this](const Columns & columns, size_t row){ buffer->count_row(columns, row); }); } void KafkaBlockOutputStream::write(const Block & block) diff --git a/dbms/src/Storages/Kafka/StorageKafka.cpp b/dbms/src/Storages/Kafka/StorageKafka.cpp index 6b0bab72bb0..450a65081be 100644 --- a/dbms/src/Storages/Kafka/StorageKafka.cpp +++ b/dbms/src/Storages/Kafka/StorageKafka.cpp @@ -231,7 +231,7 @@ ConsumerBufferPtr StorageKafka::popReadBuffer(std::chrono::milliseconds timeout) } -ProducerBufferPtr StorageKafka::createWriteBuffer() +ProducerBufferPtr StorageKafka::createWriteBuffer(Block header) { cppkafka::Configuration conf; conf.set("metadata.broker.list", brokers); @@ -245,7 +245,7 @@ ProducerBufferPtr StorageKafka::createWriteBuffer() size_t poll_timeout = settings.stream_poll_timeout_ms.totalMilliseconds(); return std::make_shared( - producer, topics[0], row_delimiter ? std::optional{row_delimiter} : std::optional(), 1, 1024, std::chrono::milliseconds(poll_timeout)); + producer, topics[0], row_delimiter ? std::optional{row_delimiter} : std::optional(), 1, 1024, std::chrono::milliseconds(poll_timeout), header); } diff --git a/dbms/src/Storages/Kafka/StorageKafka.h b/dbms/src/Storages/Kafka/StorageKafka.h index bf710f58202..6071f7b3f2a 100644 --- a/dbms/src/Storages/Kafka/StorageKafka.h +++ b/dbms/src/Storages/Kafka/StorageKafka.h @@ -53,7 +53,7 @@ public: ConsumerBufferPtr popReadBuffer(); ConsumerBufferPtr popReadBuffer(std::chrono::milliseconds timeout); - ProducerBufferPtr createWriteBuffer(); + ProducerBufferPtr createWriteBuffer(Block header); const auto & getTopics() const { return topics; } const auto & getFormatName() const { return format_name; } diff --git a/dbms/src/Storages/Kafka/WriteBufferToKafkaProducer.cpp b/dbms/src/Storages/Kafka/WriteBufferToKafkaProducer.cpp index f88b7eaad5c..3bfa684bd2d 100644 --- a/dbms/src/Storages/Kafka/WriteBufferToKafkaProducer.cpp +++ b/dbms/src/Storages/Kafka/WriteBufferToKafkaProducer.cpp @@ -1,4 +1,7 @@ #include "WriteBufferToKafkaProducer.h" +#include "Core/Block.h" +#include "Columns/ColumnString.h" +#include "Columns/ColumnsNumber.h" namespace DB { @@ -8,7 +11,9 @@ WriteBufferToKafkaProducer::WriteBufferToKafkaProducer( std::optional delimiter, size_t rows_per_message, size_t chunk_size_, - std::chrono::milliseconds poll_timeout) + std::chrono::milliseconds poll_timeout, + Block header + ) : WriteBuffer(nullptr, 0) , producer(producer_) , topic(topic_) @@ -17,6 +22,20 @@ WriteBufferToKafkaProducer::WriteBufferToKafkaProducer( , chunk_size(chunk_size_) , timeout(poll_timeout) { + for (size_t i = 0; i < header.columns(); ++i) + { + auto column_info = header.getByPosition(i); + + if (column_info.name == "_key" && isString(column_info.type) ) + { + key_column_index = i; + } + else if (column_info.name == "_timestamp" && isDateTime(column_info.type)) { + timestamp_column_index = i; + } + } + + } WriteBufferToKafkaProducer::~WriteBufferToKafkaProducer() @@ -24,8 +43,9 @@ WriteBufferToKafkaProducer::~WriteBufferToKafkaProducer() assert(rows == 0 && chunks.empty()); } -void WriteBufferToKafkaProducer::count_row() +void WriteBufferToKafkaProducer::count_row(const Columns & columns, size_t current_row) { + if (++rows % max_rows == 0) { std::string payload; @@ -35,11 +55,30 @@ void WriteBufferToKafkaProducer::count_row() int trunk_delim = delim && chunks.back()[offset() - 1] == delim ? 1 : 0; payload.append(chunks.back(), 0, offset() - trunk_delim); + cppkafka::MessageBuilder builder(topic); + builder.payload(payload); + + // Note: if it will be few rows per message - it will take the value from last row of block + if (key_column_index) + { + const auto & key_column = assert_cast(*columns[key_column_index.value()]); + const auto key_data = key_column.getDataAt(current_row); + builder.key(cppkafka::Buffer(key_data.data, key_data.size)); + } + + if (timestamp_column_index) + { + const auto & timestamp_column = assert_cast(*columns[timestamp_column_index.value()]); + const auto timestamp = std::chrono::seconds{timestamp_column.getElement(current_row)}; + builder.timestamp(timestamp); + } + while (true) { try { - producer->produce(cppkafka::MessageBuilder(topic).payload(payload)); + + producer->produce(builder); } catch (cppkafka::HandleException & e) { diff --git a/dbms/src/Storages/Kafka/WriteBufferToKafkaProducer.h b/dbms/src/Storages/Kafka/WriteBufferToKafkaProducer.h index b6751551ec7..76486bc44a4 100644 --- a/dbms/src/Storages/Kafka/WriteBufferToKafkaProducer.h +++ b/dbms/src/Storages/Kafka/WriteBufferToKafkaProducer.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include @@ -8,7 +9,7 @@ namespace DB { - +class Block; using ProducerPtr = std::shared_ptr; class WriteBufferToKafkaProducer : public WriteBuffer @@ -20,10 +21,11 @@ public: std::optional delimiter, size_t rows_per_message, size_t chunk_size_, - std::chrono::milliseconds poll_timeout); + std::chrono::milliseconds poll_timeout, + Block header); ~WriteBufferToKafkaProducer() override; - void count_row(); + void count_row(const Columns & columns, size_t row); void flush(); private: @@ -38,6 +40,8 @@ private: size_t rows = 0; std::list chunks; + std::optional key_column_index; + std::optional timestamp_column_index; }; } From 8c736e6c9f94be6d93a4d5257ea19743cf412c13 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Tue, 4 Feb 2020 22:19:34 +0100 Subject: [PATCH 02/15] After review fixes --- dbms/src/Storages/Kafka/StorageKafka.cpp | 4 ++-- dbms/src/Storages/Kafka/StorageKafka.h | 2 +- .../Kafka/WriteBufferToKafkaProducer.cpp | 19 ++++++++++++++----- .../Kafka/WriteBufferToKafkaProducer.h | 2 +- 4 files changed, 18 insertions(+), 9 deletions(-) diff --git a/dbms/src/Storages/Kafka/StorageKafka.cpp b/dbms/src/Storages/Kafka/StorageKafka.cpp index 450a65081be..6e3e63a0e09 100644 --- a/dbms/src/Storages/Kafka/StorageKafka.cpp +++ b/dbms/src/Storages/Kafka/StorageKafka.cpp @@ -231,7 +231,7 @@ ConsumerBufferPtr StorageKafka::popReadBuffer(std::chrono::milliseconds timeout) } -ProducerBufferPtr StorageKafka::createWriteBuffer(Block header) +ProducerBufferPtr StorageKafka::createWriteBuffer(const Block & header) { cppkafka::Configuration conf; conf.set("metadata.broker.list", brokers); @@ -245,7 +245,7 @@ ProducerBufferPtr StorageKafka::createWriteBuffer(Block header) size_t poll_timeout = settings.stream_poll_timeout_ms.totalMilliseconds(); return std::make_shared( - producer, topics[0], row_delimiter ? std::optional{row_delimiter} : std::optional(), 1, 1024, std::chrono::milliseconds(poll_timeout), header); + producer, topics[0], row_delimiter ? std::optional{row_delimiter} : std::nullopt, 1, 1024, std::chrono::milliseconds(poll_timeout), header); } diff --git a/dbms/src/Storages/Kafka/StorageKafka.h b/dbms/src/Storages/Kafka/StorageKafka.h index 6071f7b3f2a..aa35ecc8453 100644 --- a/dbms/src/Storages/Kafka/StorageKafka.h +++ b/dbms/src/Storages/Kafka/StorageKafka.h @@ -53,7 +53,7 @@ public: ConsumerBufferPtr popReadBuffer(); ConsumerBufferPtr popReadBuffer(std::chrono::milliseconds timeout); - ProducerBufferPtr createWriteBuffer(Block header); + ProducerBufferPtr createWriteBuffer(const Block & header); const auto & getTopics() const { return topics; } const auto & getFormatName() const { return format_name; } diff --git a/dbms/src/Storages/Kafka/WriteBufferToKafkaProducer.cpp b/dbms/src/Storages/Kafka/WriteBufferToKafkaProducer.cpp index 3bfa684bd2d..74deda6ddbd 100644 --- a/dbms/src/Storages/Kafka/WriteBufferToKafkaProducer.cpp +++ b/dbms/src/Storages/Kafka/WriteBufferToKafkaProducer.cpp @@ -12,7 +12,7 @@ WriteBufferToKafkaProducer::WriteBufferToKafkaProducer( size_t rows_per_message, size_t chunk_size_, std::chrono::milliseconds poll_timeout, - Block header + const Block & header ) : WriteBuffer(nullptr, 0) , producer(producer_) @@ -48,12 +48,22 @@ void WriteBufferToKafkaProducer::count_row(const Columns & columns, size_t curre if (++rows % max_rows == 0) { + const std::string & last_chunk = chunks.back(); + size_t last_chunk_size = offset(); + + // if last character of last chunk is delimeter - we don't need it + if (delim && last_chunk[last_chunk_size - 1] == delim) + --last_chunk_size; + std::string payload; - payload.reserve((chunks.size() - 1) * chunk_size + offset()); + payload.reserve((chunks.size() - 1) * chunk_size + last_chunk_size); + + // concat all chunks except the last one for (auto i = chunks.begin(), e = --chunks.end(); i != e; ++i) payload.append(*i); - int trunk_delim = delim && chunks.back()[offset() - 1] == delim ? 1 : 0; - payload.append(chunks.back(), 0, offset() - trunk_delim); + + // add last one + payload.append(last_chunk, 0, last_chunk_size); cppkafka::MessageBuilder builder(topic); builder.payload(payload); @@ -77,7 +87,6 @@ void WriteBufferToKafkaProducer::count_row(const Columns & columns, size_t curre { try { - producer->produce(builder); } catch (cppkafka::HandleException & e) diff --git a/dbms/src/Storages/Kafka/WriteBufferToKafkaProducer.h b/dbms/src/Storages/Kafka/WriteBufferToKafkaProducer.h index 76486bc44a4..731dc2dc69f 100644 --- a/dbms/src/Storages/Kafka/WriteBufferToKafkaProducer.h +++ b/dbms/src/Storages/Kafka/WriteBufferToKafkaProducer.h @@ -22,7 +22,7 @@ public: size_t rows_per_message, size_t chunk_size_, std::chrono::milliseconds poll_timeout, - Block header); + const Block & header); ~WriteBufferToKafkaProducer() override; void count_row(const Columns & columns, size_t row); From 9de5dffb5c97eb93545ae25eaf87ec195a590148 Mon Sep 17 00:00:00 2001 From: Andrew Onyshchuk Date: Tue, 4 Feb 2020 16:12:24 -0600 Subject: [PATCH 03/15] Upgrade librdkafka to v1.3.0 Allow bundled librdkafka on Mac --- cmake/find/rdkafka.cmake | 4 ++-- contrib/librdkafka | 2 +- contrib/librdkafka-cmake/CMakeLists.txt | 2 ++ contrib/librdkafka-cmake/config.h | 9 +++++++++ 4 files changed, 14 insertions(+), 3 deletions(-) diff --git a/cmake/find/rdkafka.cmake b/cmake/find/rdkafka.cmake index 491f50b00bc..dfab142a3cd 100644 --- a/cmake/find/rdkafka.cmake +++ b/cmake/find/rdkafka.cmake @@ -1,5 +1,5 @@ # Freebsd: contrib/cppkafka/include/cppkafka/detail/endianness.h:53:23: error: 'betoh16' was not declared in this scope -if (NOT ARCH_ARM AND NOT ARCH_32 AND NOT APPLE AND NOT OS_FREEBSD AND OPENSSL_FOUND) +if (NOT ARCH_ARM AND NOT ARCH_32 AND NOT OS_FREEBSD AND OPENSSL_FOUND) option (ENABLE_RDKAFKA "Enable kafka" ${ENABLE_LIBRARIES}) endif () @@ -10,7 +10,7 @@ endif () if (ENABLE_RDKAFKA) -if (OS_LINUX AND NOT ARCH_ARM AND USE_LIBGSASL) +if (NOT ARCH_ARM AND USE_LIBGSASL) option (USE_INTERNAL_RDKAFKA_LIBRARY "Set to FALSE to use system librdkafka instead of the bundled" ${NOT_UNBUNDLED}) endif () diff --git a/contrib/librdkafka b/contrib/librdkafka index 6160ec275a5..4ffe54b4f59 160000 --- a/contrib/librdkafka +++ b/contrib/librdkafka @@ -1 +1 @@ -Subproject commit 6160ec275a5bb0a4088ede3c5f2afde638bbef65 +Subproject commit 4ffe54b4f59ee5ae3767f9f25dc14651a3384d62 diff --git a/contrib/librdkafka-cmake/CMakeLists.txt b/contrib/librdkafka-cmake/CMakeLists.txt index fc1ba23f7c1..0f7b8e349ff 100644 --- a/contrib/librdkafka-cmake/CMakeLists.txt +++ b/contrib/librdkafka-cmake/CMakeLists.txt @@ -23,6 +23,8 @@ set(SRCS ${RDKAFKA_SOURCE_DIR}/rdkafka_lz4.c ${RDKAFKA_SOURCE_DIR}/rdkafka_metadata.c ${RDKAFKA_SOURCE_DIR}/rdkafka_metadata_cache.c + ${RDKAFKA_SOURCE_DIR}/rdkafka_mock.c + ${RDKAFKA_SOURCE_DIR}/rdkafka_mock_handlers.c ${RDKAFKA_SOURCE_DIR}/rdkafka_msg.c ${RDKAFKA_SOURCE_DIR}/rdkafka_msgset_reader.c ${RDKAFKA_SOURCE_DIR}/rdkafka_msgset_writer.c diff --git a/contrib/librdkafka-cmake/config.h b/contrib/librdkafka-cmake/config.h index 78b9bc613c3..dae2c7424e9 100644 --- a/contrib/librdkafka-cmake/config.h +++ b/contrib/librdkafka-cmake/config.h @@ -75,8 +75,17 @@ #define HAVE_STRNDUP 1 // strerror_r #define HAVE_STRERROR_R 1 + +#ifdef __APPLE__ +#define HAVE_PTHREAD_SETNAME_DARWIN 1 +#if (__ENVIRONMENT_MAC_OS_X_VERSION_MIN_REQUIRED__ <= 101400) +#define _TTHREAD_EMULATE_TIMESPEC_GET_ +#endif + +#else // pthread_setname_gnu #define HAVE_PTHREAD_SETNAME_GNU 1 +#endif // python //#define HAVE_PYTHON 1 // disable C11 threads for compatibility with old libc From cf353924337b52669f58112cc4e2738acd329b54 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Tue, 4 Feb 2020 23:12:26 +0100 Subject: [PATCH 04/15] Test for producer with _timestamp and _key --- .../integration/test_storage_kafka/test.py | 53 ++++++++++++++++++- 1 file changed, 51 insertions(+), 2 deletions(-) diff --git a/dbms/tests/integration/test_storage_kafka/test.py b/dbms/tests/integration/test_storage_kafka/test.py index c37482ae18b..0ca2d57d47c 100644 --- a/dbms/tests/integration/test_storage_kafka/test.py +++ b/dbms/tests/integration/test_storage_kafka/test.py @@ -789,6 +789,55 @@ def test_kafka_virtual_columns2(kafka_cluster): assert TSV(result) == TSV(expected) + +@pytest.mark.timeout(240) +def test_kafka_produce_key_timestamp(kafka_cluster): + instance.query(''' + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.consumer; + CREATE TABLE test.kafka_writer (key UInt64, value UInt64, _key String, _timestamp DateTime) + ENGINE = Kafka + SETTINGS kafka_broker_list = 'kafka1:19092', + kafka_topic_list = 'insert3', + kafka_group_name = 'insert3', + kafka_format = 'TSV', + kafka_row_delimiter = '\\n'; + + CREATE TABLE test.kafka (key UInt64, value UInt64, inserted_key String, inserted_timestamp DateTime) + ENGINE = Kafka + SETTINGS kafka_broker_list = 'kafka1:19092', + kafka_topic_list = 'insert3', + kafka_group_name = 'insert3', + kafka_format = 'TSV', + kafka_row_delimiter = '\\n'; + + CREATE MATERIALIZED VIEW test.view Engine=Log AS + SELECT key, value, inserted_key, toUnixTimestamp(inserted_timestamp), _key, _topic, _partition, _offset, toUnixTimestamp(_timestamp) FROM test.kafka; + ''') + + instance.query("INSERT INTO test.kafka_writer VALUES ({},{},'{}',toDateTime({}))".format(1,1,'k1',1577836801)) + instance.query("INSERT INTO test.kafka_writer VALUES ({},{},'{}',toDateTime({}))".format(2,2,'k2',1577836802)) + instance.query("INSERT INTO test.kafka_writer VALUES ({},{},'{}',toDateTime({})),({},{},'{}',toDateTime({}))".format(3,3,'k3',1577836803,4,4,'k4',1577836804)) + instance.query("INSERT INTO test.kafka_writer VALUES ({},{},'{}',toDateTime({}))".format(5,5,'k5',1577836805)) + + time.sleep(10) + + result = instance.query("SELECT * FROM test.view ORDER BY value", ignore_error=True) + + print(result) + + expected = '''\ +1 1 k1 1577836801 k1 insert3 0 0 1577836801 +2 2 k2 1577836802 k2 insert3 0 1 1577836802 +3 3 k3 1577836803 k3 insert3 0 2 1577836803 +4 4 k4 1577836804 k4 insert3 0 3 1577836804 +5 5 k5 1577836805 k5 insert3 0 4 1577836805 +''' + + assert TSV(result) == TSV(expected) + + + @pytest.mark.timeout(600) def test_kafka_flush_by_time(kafka_cluster): instance.query(''' @@ -876,9 +925,9 @@ def test_kafka_flush_by_block_size(kafka_cluster): time.sleep(1) result = instance.query('SELECT count() FROM test.view') - print(result) + # print(result) - # kafka_cluster.open_bash_shell('instance') + # kafka_cluster.open_bash_shell('instance') instance.query(''' DROP TABLE test.consumer; From e6a536618e08acc2d888835e84852774239a9f2a Mon Sep 17 00:00:00 2001 From: Andrew Onyshchuk Date: Tue, 4 Feb 2020 19:38:00 -0600 Subject: [PATCH 05/15] Allow bundled libgsasl on Mac --- cmake/find/libgsasl.cmake | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/cmake/find/libgsasl.cmake b/cmake/find/libgsasl.cmake index 729401292db..589e965e19b 100644 --- a/cmake/find/libgsasl.cmake +++ b/cmake/find/libgsasl.cmake @@ -1,4 +1,4 @@ -if (NOT APPLE AND NOT ARCH_32) +if (NOT ARCH_32) option (USE_INTERNAL_LIBGSASL_LIBRARY "Set to FALSE to use system libgsasl library instead of bundled" ${NOT_UNBUNDLED}) endif () @@ -16,7 +16,7 @@ if (NOT USE_INTERNAL_LIBGSASL_LIBRARY) endif () if (LIBGSASL_LIBRARY AND LIBGSASL_INCLUDE_DIR) -elseif (NOT MISSING_INTERNAL_LIBGSASL_LIBRARY AND NOT APPLE AND NOT ARCH_32) +elseif (NOT MISSING_INTERNAL_LIBGSASL_LIBRARY AND NOT ARCH_32) set (LIBGSASL_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/contrib/libgsasl/src ${ClickHouse_SOURCE_DIR}/contrib/libgsasl/linux_x86_64/include) set (USE_INTERNAL_LIBGSASL_LIBRARY 1) set (LIBGSASL_LIBRARY libgsasl) From babd3ca34e7ca9684ac8a4c5460ef5785d273ada Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Wed, 5 Feb 2020 08:05:55 +0100 Subject: [PATCH 06/15] Fixing test build, more review fixes --- .../Formats/tests/tab_separated_streams.cpp | 2 +- .../Kafka/WriteBufferToKafkaProducer.cpp | 24 ++++++++++++------- .../integration/test_storage_kafka/test.py | 2 +- 3 files changed, 17 insertions(+), 11 deletions(-) diff --git a/dbms/src/Formats/tests/tab_separated_streams.cpp b/dbms/src/Formats/tests/tab_separated_streams.cpp index 671043b9aac..4071e453502 100644 --- a/dbms/src/Formats/tests/tab_separated_streams.cpp +++ b/dbms/src/Formats/tests/tab_separated_streams.cpp @@ -45,7 +45,7 @@ try BlockInputStreamPtr block_input = std::make_shared(std::move(input_format)); BlockOutputStreamPtr block_output = std::make_shared( - std::make_shared(out_buf, sample, false, false, [] {}, format_settings)); + std::make_shared(out_buf, sample, false, false, [](const Columns & /* columns */, size_t /* row */){}, format_settings)); copyData(*block_input, *block_output); return 0; diff --git a/dbms/src/Storages/Kafka/WriteBufferToKafkaProducer.cpp b/dbms/src/Storages/Kafka/WriteBufferToKafkaProducer.cpp index 74deda6ddbd..8d6c1ae537f 100644 --- a/dbms/src/Storages/Kafka/WriteBufferToKafkaProducer.cpp +++ b/dbms/src/Storages/Kafka/WriteBufferToKafkaProducer.cpp @@ -22,20 +22,26 @@ WriteBufferToKafkaProducer::WriteBufferToKafkaProducer( , chunk_size(chunk_size_) , timeout(poll_timeout) { - for (size_t i = 0; i < header.columns(); ++i) + if (header.has("_key")) { - auto column_info = header.getByPosition(i); - - if (column_info.name == "_key" && isString(column_info.type) ) + auto column_index = header.getPositionByName("_key"); + auto column_info = header.getByPosition(column_index); + if (isString(column_info.type)) { - key_column_index = i; - } - else if (column_info.name == "_timestamp" && isDateTime(column_info.type)) { - timestamp_column_index = i; + key_column_index = column_index; } + // else ? (not sure it's a good place to report smth to user) } - + if (header.has("_timestamp")) + { + auto column_index = header.getPositionByName("_timestamp"); + auto column_info = header.getByPosition(column_index); + if (isDateTime(column_info.type)) + { + timestamp_column_index = column_index; + } + } } WriteBufferToKafkaProducer::~WriteBufferToKafkaProducer() diff --git a/dbms/tests/integration/test_storage_kafka/test.py b/dbms/tests/integration/test_storage_kafka/test.py index 0ca2d57d47c..8c4f2fbc9ef 100644 --- a/dbms/tests/integration/test_storage_kafka/test.py +++ b/dbms/tests/integration/test_storage_kafka/test.py @@ -824,7 +824,7 @@ def test_kafka_produce_key_timestamp(kafka_cluster): result = instance.query("SELECT * FROM test.view ORDER BY value", ignore_error=True) - print(result) + # print(result) expected = '''\ 1 1 k1 1577836801 k1 insert3 0 0 1577836801 From c76e5a470a2a506650994c91c81dcd0af9819ff1 Mon Sep 17 00:00:00 2001 From: Andrew Onyshchuk Date: Wed, 5 Feb 2020 08:12:35 -0600 Subject: [PATCH 07/15] update contrib/libgsasl --- contrib/libgsasl | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/libgsasl b/contrib/libgsasl index 3b8948a4042..42ef2068704 160000 --- a/contrib/libgsasl +++ b/contrib/libgsasl @@ -1 +1 @@ -Subproject commit 3b8948a4042e34fb00b4fb987535dc9e02e39040 +Subproject commit 42ef20687042637252e64df1934b6d47771486d1 From bf23e3918ace3262310477370874efddf7773c3f Mon Sep 17 00:00:00 2001 From: Andrew Onyshchuk Date: Wed, 5 Feb 2020 09:02:39 -0600 Subject: [PATCH 08/15] trigger build --- contrib/librdkafka-cmake/config.h | 1 + 1 file changed, 1 insertion(+) diff --git a/contrib/librdkafka-cmake/config.h b/contrib/librdkafka-cmake/config.h index dae2c7424e9..a4c69cd7578 100644 --- a/contrib/librdkafka-cmake/config.h +++ b/contrib/librdkafka-cmake/config.h @@ -77,6 +77,7 @@ #define HAVE_STRERROR_R 1 #ifdef __APPLE__ +// pthread_setname_np #define HAVE_PTHREAD_SETNAME_DARWIN 1 #if (__ENVIRONMENT_MAC_OS_X_VERSION_MIN_REQUIRED__ <= 101400) #define _TTHREAD_EMULATE_TIMESPEC_GET_ From b9fc9b4569323b9fa4353cd45a4b0e28a443c6b3 Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Sat, 8 Feb 2020 00:07:18 +0300 Subject: [PATCH 09/15] fix order of ranges in MergeTreeDataSelectExecutor --- .../MergeTree/MergeTreeBlockReadUtils.cpp | 5 ++++- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 5 ++--- .../Storages/MergeTree/MergeTreeReadPool.cpp | 21 +++++++------------ .../MergeTree/MergeTreeSelectProcessor.cpp | 7 +------ ...01078_merge_tree_read_one_thread.reference | 1 + .../01078_merge_tree_read_one_thread.sql | 16 ++++++++++++++ 6 files changed, 31 insertions(+), 24 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/01078_merge_tree_read_one_thread.reference create mode 100644 dbms/tests/queries/0_stateless/01078_merge_tree_read_one_thread.sql diff --git a/dbms/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp b/dbms/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp index 920697f3c32..7f71bfc28c1 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp @@ -73,7 +73,10 @@ MergeTreeReadTask::MergeTreeReadTask( : data_part{data_part_}, mark_ranges{mark_ranges_}, part_index_in_query{part_index_in_query_}, ordered_names{ordered_names_}, column_name_set{column_name_set_}, columns{columns_}, pre_columns{pre_columns_}, remove_prewhere_column{remove_prewhere_column_}, should_reorder{should_reorder_}, size_predictor{std::move(size_predictor_)} -{} +{ + /// We need to save marks in reverse order. + std::reverse(mark_ranges.begin(), mark_ranges.end()); +} MergeTreeReadTask::~MergeTreeReadTask() = default; diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 33b5ac4999d..6fc592bdc2c 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -718,9 +718,6 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreams( for (size_t i = 0; i < parts.size(); ++i) { total_rows += parts[i].getRowsCount(); - /// Let the ranges be listed from right to left so that the leftmost range can be dropped using `pop_back()`. - std::reverse(parts[i].ranges.begin(), parts[i].ranges.end()); - sum_marks_in_parts[i] = parts[i].getMarksCount(); sum_marks += sum_marks_in_parts[i]; @@ -897,6 +894,8 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder( } new_ranges.emplace_back(range.begin, range.end); } + + /// Restore left-to-right order. std::reverse(new_ranges.begin(), new_ranges.end()); } diff --git a/dbms/src/Storages/MergeTree/MergeTreeReadPool.cpp b/dbms/src/Storages/MergeTree/MergeTreeReadPool.cpp index a70dfc2d78c..2003b2f0af1 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeReadPool.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeReadPool.cpp @@ -28,11 +28,6 @@ MergeTreeReadPool::MergeTreeReadPool( column_names{column_names_}, do_not_steal_tasks{do_not_steal_tasks_}, predict_block_size_bytes{preferred_block_size_bytes_ > 0}, prewhere_info{prewhere_info_}, parts_ranges{parts_} { - /// reverse from right-to-left to left-to-right - /// because 'reverse' was done in MergeTreeDataSelectExecutor - for (auto & part_ranges : parts_ranges) - std::reverse(std::begin(part_ranges.ranges), std::end(part_ranges.ranges)); - /// parts don't contain duplicate MergeTreeDataPart's. const auto per_part_sum_marks = fillPerPartInfo(parts_, check_columns_); fillPerThreadInfo(threads_, sum_marks_, per_part_sum_marks, parts_, min_marks_for_concurrent_read_); @@ -79,10 +74,9 @@ MergeTreeReadTaskPtr MergeTreeReadPool::getTask(const size_t min_marks_to_read, { const auto marks_to_get_from_range = marks_in_part; - /** Ranges are in right-to-left order, because 'reverse' was done in MergeTreeDataSelectExecutor - * and that order is supported in 'fillPerThreadInfo'. - */ + /// Ranges are in right-to-left order, because 'reverse' was done in 'fillPerThreadInfo'. ranges_to_get_from_part = thread_task.ranges; + std::reverse(ranges_to_get_from_part.begin(), ranges_to_get_from_part.end()); marks_in_part -= marks_to_get_from_range; @@ -113,11 +107,7 @@ MergeTreeReadTaskPtr MergeTreeReadPool::getTask(const size_t min_marks_to_read, marks_in_part -= marks_to_get_from_range; need_marks -= marks_to_get_from_range; } - - /** Change order to right-to-left, for MergeTreeThreadSelectBlockInputStream to get ranges with .pop_back() - * (order was changed to left-to-right due to .pop_back() above). - */ - std::reverse(std::begin(ranges_to_get_from_part), std::end(ranges_to_get_from_part)); + /// Order of ranges was changed to left-to-right due to .pop_back() above. } auto curr_task_size_predictor = !per_part_size_predictor[part_idx] ? nullptr @@ -211,7 +201,6 @@ std::vector MergeTreeReadPool::fillPerPartInfo( /// Read marks for every data part. size_t sum_marks = 0; - /// Ranges are in right-to-left order, due to 'reverse' in MergeTreeDataSelectExecutor. for (const auto & range : part.ranges) sum_marks += range.end - range.begin; @@ -251,6 +240,10 @@ void MergeTreeReadPool::fillPerThreadInfo( { threads_tasks.resize(threads); + /// Let the ranges be listed from right to left so that the leftmost range can be dropped using `pop_back()`. + for (auto & part : parts) + std::reverse(part.ranges.begin(), part.ranges.end()); + const size_t min_marks_per_thread = (sum_marks - 1) / threads + 1; for (size_t i = 0; i < threads && !parts.empty(); ++i) diff --git a/dbms/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp b/dbms/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp index dac42859eef..0f0822cd88c 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp @@ -95,11 +95,6 @@ try task_columns = getReadTaskColumns(storage, data_part, required_columns, prewhere_info, check_columns); - /** @note you could simply swap `reverse` in if and else branches of MergeTreeDataSelectExecutor, - * and remove this reverse. */ - MarkRanges remaining_mark_ranges = all_mark_ranges; - std::reverse(remaining_mark_ranges.begin(), remaining_mark_ranges.end()); - auto size_predictor = (preferred_block_size_bytes == 0) ? nullptr : std::make_unique(data_part, ordered_names, data_part->storage.getSampleBlock()); @@ -109,7 +104,7 @@ try column_name_set = NameSet{column_names.begin(), column_names.end()}; task = std::make_unique( - data_part, remaining_mark_ranges, part_index_in_query, ordered_names, column_name_set, task_columns.columns, + data_part, all_mark_ranges, part_index_in_query, ordered_names, column_name_set, task_columns.columns, task_columns.pre_columns, prewhere_info && prewhere_info->remove_prewhere_column, task_columns.should_reorder, std::move(size_predictor)); diff --git a/dbms/tests/queries/0_stateless/01078_merge_tree_read_one_thread.reference b/dbms/tests/queries/0_stateless/01078_merge_tree_read_one_thread.reference new file mode 100644 index 00000000000..697cb3a26d7 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01078_merge_tree_read_one_thread.reference @@ -0,0 +1 @@ +300 diff --git a/dbms/tests/queries/0_stateless/01078_merge_tree_read_one_thread.sql b/dbms/tests/queries/0_stateless/01078_merge_tree_read_one_thread.sql new file mode 100644 index 00000000000..41fcbe81709 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01078_merge_tree_read_one_thread.sql @@ -0,0 +1,16 @@ +drop table if exists t; + +create table t (a Int, b Int) engine = MergeTree order by (a, b) settings index_granularity = 400; + +insert into t select 0, 0 from numbers(50); +insert into t select 0, 1 from numbers(350); +insert into t select 1, 2 from numbers(400); +insert into t select 2, 2 from numbers(400); +insert into t select 3, 0 from numbers(100); + +select sleep(1) format Null; -- sleep a bit to wait possible merges after insert + +set max_threads = 1; +optimize table t final; + +select sum(a) from t where a in (0, 3) and b = 0; From 852772364e3b202286f0e2669d3fd0dc258ea3bf Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Mon, 10 Feb 2020 15:36:01 +0300 Subject: [PATCH 10/15] replace std::vector to std::deque for MarkRanges --- dbms/src/Storages/MergeTree/MarkRange.h | 4 ++-- .../MergeTree/MergeTreeBlockReadUtils.cpp | 2 -- .../MergeTree/MergeTreeBlockReadUtils.h | 3 +-- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 19 ++++----------- .../MergeTree/MergeTreeRangeReader.cpp | 6 ++--- .../Storages/MergeTree/MergeTreeReadPool.cpp | 24 ++++--------------- dbms/src/Storages/MergeTree/MergeTreeReader.h | 2 +- 7 files changed, 17 insertions(+), 43 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MarkRange.h b/dbms/src/Storages/MergeTree/MarkRange.h index 657ffe32f78..8380914a455 100644 --- a/dbms/src/Storages/MergeTree/MarkRange.h +++ b/dbms/src/Storages/MergeTree/MarkRange.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include namespace DB @@ -19,7 +19,7 @@ struct MarkRange MarkRange(const size_t begin_, const size_t end_) : begin{begin_}, end{end_} {} }; -using MarkRanges = std::vector; +using MarkRanges = std::deque; } diff --git a/dbms/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp b/dbms/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp index 7f71bfc28c1..e1c0305abfd 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp @@ -74,8 +74,6 @@ MergeTreeReadTask::MergeTreeReadTask( ordered_names{ordered_names_}, column_name_set{column_name_set_}, columns{columns_}, pre_columns{pre_columns_}, remove_prewhere_column{remove_prewhere_column_}, should_reorder{should_reorder_}, size_predictor{std::move(size_predictor_)} { - /// We need to save marks in reverse order. - std::reverse(mark_ranges.begin(), mark_ranges.end()); } MergeTreeReadTask::~MergeTreeReadTask() = default; diff --git a/dbms/src/Storages/MergeTree/MergeTreeBlockReadUtils.h b/dbms/src/Storages/MergeTree/MergeTreeBlockReadUtils.h index 19c6adbd9c7..108742e1101 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeBlockReadUtils.h +++ b/dbms/src/Storages/MergeTree/MergeTreeBlockReadUtils.h @@ -30,8 +30,7 @@ struct MergeTreeReadTask { /// data part which should be read while performing this task MergeTreeData::DataPartPtr data_part; - /** Ranges to read from `data_part`. - * Specified in reverse order for MergeTreeThreadSelectBlockInputStream's convenience of calling .pop_back(). */ + /// Ranges to read from `data_part`. MarkRanges mark_ranges; /// for virtual `part_index` virtual column size_t part_index_in_query; diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 6fc592bdc2c..28e8eb486d3 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -823,9 +823,6 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder( sum_marks_in_parts[i] = parts[i].getMarksCount(); sum_marks += sum_marks_in_parts[i]; - /// Let the ranges be listed from right to left so that the leftmost range can be dropped using `pop_back()`. - std::reverse(parts[i].ranges.begin(), parts[i].ranges.end()); - if (parts[i].data_part->index_granularity_info.is_adaptive) adaptive_parts++; } @@ -888,15 +885,12 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder( auto range = *it; while (range.begin + marks_in_range < range.end) { - new_ranges.emplace_back(range.end - marks_in_range, range.end); + new_ranges.emplace_front(range.end - marks_in_range, range.end); range.end -= marks_in_range; marks_in_range = std::min(marks_in_range * 2, max_marks_in_range); } - new_ranges.emplace_back(range.begin, range.end); + new_ranges.emplace_front(range.begin, range.end); } - - /// Restore left-to-right order. - std::reverse(new_ranges.begin(), new_ranges.end()); } return new_ranges; @@ -918,7 +912,7 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder( RangesInDataPart part = parts.back(); parts.pop_back(); - size_t & marks_in_part = sum_marks_in_parts.back(); + size_t & marks_in_part = sum_marks_in_parts.front(); /// We will not take too few rows from a part. if (marks_in_part >= min_marks_for_concurrent_read && @@ -935,9 +929,6 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder( /// We take the whole part if it is small enough. if (marks_in_part <= need_marks) { - /// Restore the order of segments. - std::reverse(part.ranges.begin(), part.ranges.end()); - ranges_to_get_from_part = part.ranges; need_marks -= marks_in_part; @@ -951,7 +942,7 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder( if (part.ranges.empty()) throw Exception("Unexpected end of ranges while spreading marks among streams", ErrorCodes::LOGICAL_ERROR); - MarkRange & range = part.ranges.back(); + MarkRange & range = part.ranges.front(); const size_t marks_in_range = range.end - range.begin; const size_t marks_to_get_from_range = std::min(marks_in_range, need_marks); @@ -961,7 +952,7 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder( marks_in_part -= marks_to_get_from_range; need_marks -= marks_to_get_from_range; if (range.begin == range.end) - part.ranges.pop_back(); + part.ranges.pop_front(); } parts.emplace_back(part); } diff --git a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp index a0bd2567fe5..56e5cfdb3d2 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -690,9 +690,9 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::startReadingChain(size_t if (stream.isFinished()) { result.addRows(stream.finalize(result.columns)); - stream = Stream(ranges.back().begin, ranges.back().end, merge_tree_reader); - result.addRange(ranges.back()); - ranges.pop_back(); + stream = Stream(ranges.front().begin, ranges.front().end, merge_tree_reader); + result.addRange(ranges.front()); + ranges.pop_front(); } auto rows_to_read = std::min(space_left, stream.numPendingRowsInCurrentGranule()); diff --git a/dbms/src/Storages/MergeTree/MergeTreeReadPool.cpp b/dbms/src/Storages/MergeTree/MergeTreeReadPool.cpp index 2003b2f0af1..fc63cca809a 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeReadPool.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeReadPool.cpp @@ -73,10 +73,7 @@ MergeTreeReadTaskPtr MergeTreeReadPool::getTask(const size_t min_marks_to_read, if (marks_in_part <= need_marks) { const auto marks_to_get_from_range = marks_in_part; - - /// Ranges are in right-to-left order, because 'reverse' was done in 'fillPerThreadInfo'. ranges_to_get_from_part = thread_task.ranges; - std::reverse(ranges_to_get_from_part.begin(), ranges_to_get_from_part.end()); marks_in_part -= marks_to_get_from_range; @@ -91,7 +88,7 @@ MergeTreeReadTaskPtr MergeTreeReadPool::getTask(const size_t min_marks_to_read, /// Loop through part ranges. while (need_marks > 0 && !thread_task.ranges.empty()) { - auto & range = thread_task.ranges.back(); + auto & range = thread_task.ranges.front(); const size_t marks_in_range = range.end - range.begin; const size_t marks_to_get_from_range = std::min(marks_in_range, need_marks); @@ -100,14 +97,13 @@ MergeTreeReadTaskPtr MergeTreeReadPool::getTask(const size_t min_marks_to_read, range.begin += marks_to_get_from_range; if (range.begin == range.end) { - std::swap(range, thread_task.ranges.back()); - thread_task.ranges.pop_back(); + std::swap(range, thread_task.ranges.front()); + thread_task.ranges.pop_front(); } marks_in_part -= marks_to_get_from_range; need_marks -= marks_to_get_from_range; } - /// Order of ranges was changed to left-to-right due to .pop_back() above. } auto curr_task_size_predictor = !per_part_size_predictor[part_idx] ? nullptr @@ -240,10 +236,6 @@ void MergeTreeReadPool::fillPerThreadInfo( { threads_tasks.resize(threads); - /// Let the ranges be listed from right to left so that the leftmost range can be dropped using `pop_back()`. - for (auto & part : parts) - std::reverse(part.ranges.begin(), part.ranges.end()); - const size_t min_marks_per_thread = (sum_marks - 1) / threads + 1; for (size_t i = 0; i < threads && !parts.empty(); ++i) @@ -272,7 +264,6 @@ void MergeTreeReadPool::fillPerThreadInfo( /// Get whole part to read if it is small enough. if (marks_in_part <= need_marks) { - /// Leave ranges in right-to-left order for convenience to use .pop_back() in .getTask() ranges_to_get_from_part = part.ranges; marks_in_ranges = marks_in_part; @@ -288,7 +279,7 @@ void MergeTreeReadPool::fillPerThreadInfo( if (part.ranges.empty()) throw Exception("Unexpected end of ranges while spreading marks among threads", ErrorCodes::LOGICAL_ERROR); - MarkRange & range = part.ranges.back(); + MarkRange & range = part.ranges.front(); const size_t marks_in_range = range.end - range.begin; const size_t marks_to_get_from_range = std::min(marks_in_range, need_marks); @@ -298,13 +289,8 @@ void MergeTreeReadPool::fillPerThreadInfo( marks_in_part -= marks_to_get_from_range; need_marks -= marks_to_get_from_range; if (range.begin == range.end) - part.ranges.pop_back(); + part.ranges.pop_front(); } - - /** Change order to right-to-left, for getTask() to get ranges with .pop_back() - * (order was changed to left-to-right due to .pop_back() above). - */ - std::reverse(std::begin(ranges_to_get_from_part), std::end(ranges_to_get_from_part)); } threads_tasks[i].parts_and_ranges.push_back({ part_idx, ranges_to_get_from_part }); diff --git a/dbms/src/Storages/MergeTree/MergeTreeReader.h b/dbms/src/Storages/MergeTree/MergeTreeReader.h index b0642c06108..65485f950c6 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeReader.h +++ b/dbms/src/Storages/MergeTree/MergeTreeReader.h @@ -57,7 +57,7 @@ public: size_t getFirstMarkToRead() const { - return all_mark_ranges.back().begin; + return all_mark_ranges.front().begin; } private: using FileStreams = std::map>; From 90a121cc1e2b52b09e7a36a1769b732405292eb1 Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Mon, 10 Feb 2020 19:34:47 +0300 Subject: [PATCH 11/15] remove useless line --- dbms/src/Storages/MergeTree/MergeTreeReadPool.cpp | 3 --- 1 file changed, 3 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeReadPool.cpp b/dbms/src/Storages/MergeTree/MergeTreeReadPool.cpp index fc63cca809a..15d87e60e24 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeReadPool.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeReadPool.cpp @@ -96,10 +96,7 @@ MergeTreeReadTaskPtr MergeTreeReadPool::getTask(const size_t min_marks_to_read, ranges_to_get_from_part.emplace_back(range.begin, range.begin + marks_to_get_from_range); range.begin += marks_to_get_from_range; if (range.begin == range.end) - { - std::swap(range, thread_task.ranges.front()); thread_task.ranges.pop_front(); - } marks_in_part -= marks_to_get_from_range; need_marks -= marks_to_get_from_range; From ca61b57183953d7c199cc626c31efdb00f0ce842 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Tue, 11 Feb 2020 18:01:16 +0300 Subject: [PATCH 12/15] performance comparison --- docker/test/performance-comparison/compare.sh | 39 +++++++++++++------ docker/test/performance-comparison/perf.py | 6 +++ docker/test/performance-comparison/report.py | 9 +++-- docs/en/development/performance_comparison.md | 39 +++++++++++++++++++ 4 files changed, 78 insertions(+), 15 deletions(-) create mode 100644 docs/en/development/performance_comparison.md diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index 7b6a5beecae..691b59fef4b 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -133,10 +133,9 @@ function run_tests do test_name=$(basename $test ".xml") echo test $test_name - #TIMEFORMAT=$(printf "time\t$test_name\t%%3R\t%%3U\t%%3S\n") TIMEFORMAT=$(printf "$test_name\t%%3R\t%%3U\t%%3S\n") - #time "$script_dir/perf.py" "$test" > >(tee "$test_name-raw.tsv") 2> >(tee "$test_name-err.log") || continue - { time "$script_dir/perf.py" "$test" > "$test_name-raw.tsv" 2> "$test_name-err.log" ; } 2>> "wall-clock-times.tsv" || continue + # the grep is to filter out set -x output and keep only time output + { time "$script_dir/perf.py" "$test" > "$test_name-raw.tsv" 2> "$test_name-err.log" ; } 2>&1 >/dev/null | grep -v ^+ >> "wall-clock-times.tsv" || continue grep ^query "$test_name-raw.tsv" | cut -f2- > "$test_name-queries.tsv" grep ^client-time "$test_name-raw.tsv" | cut -f2- > "$test_name-client-time.tsv" right/clickhouse local --file "$test_name-queries.tsv" --structure 'query text, run int, version UInt32, time float' --query "$(cat $script_dir/eqmed.sql)" > "$test_name-report.tsv" @@ -147,14 +146,19 @@ function run_tests function report { result_structure="left float, right float, diff float, rd Array(float), query text" -rm test-times.tsv test-dump.tsv unstable.tsv changed-perf.tsv unstable-tests.tsv unstable-queries.tsv bad-tests.tsv slow-on-client.tsv ||: +rm test-times.tsv test-dump.tsv unstable.tsv changed-perf.tsv unstable-tests.tsv unstable-queries.tsv bad-tests.tsv slow-on-client.tsv all-queries.tsv ||: right/clickhouse local --query " create table queries engine Memory as select replaceAll(_file, '-report.tsv', '') test, if(abs(diff) < 0.05 and rd[3] > 0.05, 1, 0) unstable, if(abs(diff) > 0.05 and abs(diff) > rd[3], 1, 0) changed, * - from file('*-report.tsv', TSV, 'left float, right float, diff float, rd Array(float), query text'); + from file('*-report.tsv', TSV, 'left float, right float, diff float, rd Array(float), query text') + -- FIXME Comparison mode doesn't make sense for queries that complete + -- immediately, so for now we pretend they don't exist. We don't want to + -- remove them altogether because we want to be able to detect regressions, + -- but the right way to do this is not yet clear. + where left + right > 0.01; create table changed_perf_tsv engine File(TSV, 'changed-perf.tsv') as select left, right, diff, rd, test, query from queries where changed @@ -177,20 +181,31 @@ create table slow_on_client_tsv engine File(TSV, 'slow-on-client.tsv') as from query_time where p > 1.02 order by p desc; create table test_time engine Memory as - select test, floor(sum(client), 3) client, count(*) queries - from query_time group by test; + select test, sum(client) total_client_time, + max(client) query_max, min(client) query_min, count(*) queries + from query_time + -- for consistency, filter out everything we filtered out of queries table + semi join queries using query + group by test; create table test_times_tsv engine File(TSV, 'test-times.tsv') as - select wall_clock.test, real, client, queries, - floor(real / queries, 3) real_per_query - from test_time join wall_clock using test - order by real desc; + select wall_clock.test, real, + floor(total_client_time, 3), + queries, + floor(query_max, 3), + floor(real / queries, 3) avg_real_per_query, + floor(query_min, 3) + from test_time right join wall_clock using test + order by query_max / query_min desc; create table all_queries_tsv engine File(TSV, 'all-queries.tsv') as select left, right, diff, rd, test, query from queries order by rd[3] desc; " -grep Exception:[^:] *-err.log > run-errors.log + +# Remember that grep sets error code when nothing is found, hence the bayan +# operator +grep Exception:[^:] *-err.log > run-errors.log ||: $script_dir/report.py > report.html } diff --git a/docker/test/performance-comparison/perf.py b/docker/test/performance-comparison/perf.py index 4eeb6e76828..1a2021c629a 100755 --- a/docker/test/performance-comparison/perf.py +++ b/docker/test/performance-comparison/perf.py @@ -94,6 +94,12 @@ test_query_templates = [q.text for q in root.findall('query')] test_queries = substitute_parameters(test_query_templates, parameter_combinations) for q in test_queries: + # Warmup: run once on both servers. Helps to bring the data into memory, + # precompile the queries, etc. + for c in connections: + c.execute(q) + + # Now, perform measured runs. # Track the time spent by the client to process this query, so that we can notice # out the queries that take long to process on the client side, e.g. by sending # excessive data. diff --git a/docker/test/performance-comparison/report.py b/docker/test/performance-comparison/report.py index 9f17d1a19bd..21865532268 100755 --- a/docker/test/performance-comparison/report.py +++ b/docker/test/performance-comparison/report.py @@ -144,9 +144,12 @@ params['test_part'] = ( rows = tsv_rows('bad-tests.tsv')) + table_template.format( anchor = nextTableAnchor(), - caption = 'Tests with most unstable queries', - header = table_header(['Test', 'Wall clock time, s', 'Client time, s', - 'Number of queries', 'Wall clock time per all runs of a query, s']), + caption = 'Tests times', + header = table_header(['Test', 'Wall clock time, s', 'Total client time, s', + 'Number of queries', + 'Max client time
(sum for all runs), s', + 'Avg wall clock time
(sum for all runs), s', + 'Min client time
(sum for all runs), s']), rows = tsv_rows('test-times.tsv')) ) print(doc_template.format_map(params)) diff --git a/docs/en/development/performance_comparison.md b/docs/en/development/performance_comparison.md new file mode 100644 index 00000000000..6158ec11859 --- /dev/null +++ b/docs/en/development/performance_comparison.md @@ -0,0 +1,39 @@ +# [draft] Performance comparison test + +This is an experimental mode that compares performance of old and new server +side by side. Both servers are run, and the query is executed on one then another, +measuring the times. This setup should remove much of the variability present in +the current performance tests, which only run the new version and compare with +the old results recorded some time in the past. + +To interpret the observed results, we build randomization distribution for the +observed difference of median times between old and new server, under the null +hypothesis that the performance distribution is the same (for the details of the +method, see [1]). We consider the observed difference in performance significant, +if it is above 5% and above the 95th percentile of the randomization distribution. +We also consider the test to be unstable, if the observed difference is less than +5%, but the 95th percentile is above 5% -- this means that we are likely to observe +performance differences above 5% more often than in 5% runs, so the test is likely +to have false positives. + +### How to read the report + +Should add inline comments there, because who reads the docs anyway. They must +be collapsible and I am afraid of Javascript, so I'm going to do it later. + +### How to run +No convenient way -- run the entire docker container, specifying PR number (0 for master) +and SHA of the commit to test: +``` +docker run --network=host --volume=$(pwd)/workspace:/workspace --volume=$(pwd)/output:/output -e PR_TO_TEST={} -e SHA_TO_TEST={} yandex/clickhouse-performance-comparison +``` +Then see the `report.html` in the `output` directory. + +There are some environment variables that influence what the test does: + * `-e CHCP_RUNS` -- the number of runs; + * `-e CHPC_TEST_GLOB` -- the names of the tests (xml files) to run, interpreted + as a shell glob. + + +### References +1\. Box, Hunter, Hunter "Statictics for exprerimenters", p. 78: "A Randomized Design Used in the Comparison of Standard and Modified Fertilizer Mixtures for Tomato Plants." From 1f4ba36cbc60b629ecd6efbb1c4e26aa98ca41f7 Mon Sep 17 00:00:00 2001 From: Ivan <5627721+abyss7@users.noreply.github.com> Date: Tue, 11 Feb 2020 21:05:08 +0300 Subject: [PATCH 13/15] Fix some tests to support pytest with query tests (#9062) --- ...s_and_insert_without_explicit_database.sql | 40 +- ...46_clear_column_in_partition_zookeeper.sql | 76 +-- .../00508_materialized_view_to.sql | 36 +- ...t_database_when_create_materializ_view.sql | 32 +- ...0578_merge_table_shadow_virtual_column.sql | 16 +- .../tests/queries/0_stateless/00612_count.sql | 30 +- .../queries/0_stateless/00612_shard_count.sql | 12 +- .../00740_database_in_nested_view.sql | 26 +- ...51_default_databasename_for_view.reference | 2 +- .../00751_default_databasename_for_view.sql | 31 +- .../00800_versatile_storage_join.sql | 72 +-- .../queries/0_stateless/00870_t64_codec.sql | 154 +++--- .../0_stateless/00871_t64_codec_signed.sql | 208 ++++---- .../0_stateless/00872_t64_bit_codec.sql | 154 +++--- .../0_stateless/00921_datetime64_basic.sql | 4 +- .../queries/0_stateless/00926_geo_to_h3.sql | 2 - .../0_stateless/00936_crc_functions.sql | 2 - .../00940_order_by_read_in_order.sql | 6 +- .../0_stateless/00945_bloom_filter_index.sql | 448 +++++++++--------- .../00950_test_double_delta_codec.sql | 4 +- .../0_stateless/00950_test_gorilla_codec.sql | 4 +- .../00961_temporary_live_view_watch.sql | 22 +- .../00968_file_engine_in_subquery.sql | 3 + ..._select_format_jsoneachrowwithprogress.sql | 16 +- ...w_watch_format_jsoneachrowwithprogress.sql | 22 +- .../0_stateless/00972_live_view_select_1.sql | 8 +- .../0_stateless/00973_live_view_select.sql | 24 +- .../00973_live_view_with_subquery_select.sql | 25 +- ...73_live_view_with_subquery_select_join.sql | 37 +- ...iew_with_subquery_select_join_no_alias.sql | 37 +- ..._live_view_with_subquery_select_nested.sql | 25 +- ...ubquery_select_nested_with_aggregation.sql | 25 +- ...ct_nested_with_aggregation_table_alias.sql | 25 +- ..._view_with_subquery_select_table_alias.sql | 25 +- ..._with_subquery_select_with_aggregation.sql | 25 +- ...ry_select_with_aggregation_in_subquery.sql | 25 +- ...0974_live_view_select_with_aggregation.sql | 20 +- .../0_stateless/00975_live_view_create.sql | 10 +- .../00975_recursive_materialized_view.sql | 10 +- .../00975_sample_prewhere_distributed.sql | 2 + .../00976_live_view_select_version.sql | 16 +- .../00977_live_view_watch_events.sql | 22 +- .../0_stateless/00978_live_view_watch.sql | 22 +- .../0_stateless/00979_set_index_not.sql | 12 +- ...ptimize_read_in_order_with_aggregation.sql | 2 + .../0_stateless/00997_set_index_array.sql | 11 +- .../01009_insert_select_data_loss.sql | 2 + .../01011_test_create_as_skip_indices.sql | 4 + .../01018_ddl_dictionaries_create.sql | 1 + .../01019_Buffer_and_max_memory_usage.sql | 3 + .../0_stateless/01021_only_tuple_columns.sql | 2 + .../01030_storage_set_supports_read.sql | 1 + .../01045_bloom_filter_null_array.sql | 22 +- ..._simple_aggregate_sizes_of_columns_bug.sql | 2 + ...tal_streaming_from_2_src_with_feedback.sql | 3 +- ...9_materialized_view_alter_target_table.sql | 4 + ...r_target_table_with_default_expression.sql | 4 + .../01073_attach_if_not_exists.sql | 1 + .../01077_yet_another_prewhere_test.sql | 4 +- 59 files changed, 962 insertions(+), 921 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00101_materialized_views_and_insert_without_explicit_database.sql b/dbms/tests/queries/0_stateless/00101_materialized_views_and_insert_without_explicit_database.sql index cedbd296804..69fc61a4968 100644 --- a/dbms/tests/queries/0_stateless/00101_materialized_views_and_insert_without_explicit_database.sql +++ b/dbms/tests/queries/0_stateless/00101_materialized_views_and_insert_without_explicit_database.sql @@ -1,24 +1,31 @@ -DROP TABLE IF EXISTS default.test_table; -DROP TABLE IF EXISTS default.test_view; -DROP TABLE IF EXISTS default.test_view_filtered; +CREATE DATABASE IF NOT EXISTS test_00101_0; -CREATE TABLE default.test_table (EventDate Date, CounterID UInt32, UserID UInt64, EventTime DateTime, UTCEventTime DateTime) ENGINE = MergeTree(EventDate, CounterID, 8192); -CREATE MATERIALIZED VIEW default.test_view (Rows UInt64, MaxHitTime DateTime) ENGINE = Memory AS SELECT count() AS Rows, max(UTCEventTime) AS MaxHitTime FROM default.test_table; -CREATE MATERIALIZED VIEW default.test_view_filtered (EventDate Date, CounterID UInt32) ENGINE = Memory POPULATE AS SELECT CounterID, EventDate FROM default.test_table WHERE EventDate < '2013-01-01'; +USE test_00101_0; -INSERT INTO default.test_table (EventDate, UTCEventTime) VALUES ('2014-01-02', '2014-01-02 03:04:06'); +DROP TABLE IF EXISTS test_table; +DROP TABLE IF EXISTS test_view; +DROP TABLE IF EXISTS test_view_filtered; -SELECT * FROM default.test_table; -SELECT * FROM default.test_view; -SELECT * FROM default.test_view_filtered; +CREATE TABLE test_table (EventDate Date, CounterID UInt32, UserID UInt64, EventTime DateTime, UTCEventTime DateTime) ENGINE = MergeTree(EventDate, CounterID, 8192); +CREATE MATERIALIZED VIEW test_view (Rows UInt64, MaxHitTime DateTime) ENGINE = Memory AS SELECT count() AS Rows, max(UTCEventTime) AS MaxHitTime FROM test_table; +CREATE MATERIALIZED VIEW test_view_filtered (EventDate Date, CounterID UInt32) ENGINE = Memory POPULATE AS SELECT CounterID, EventDate FROM test_table WHERE EventDate < '2013-01-01'; -DROP TABLE default.test_table; -DROP TABLE default.test_view; -DROP TABLE default.test_view_filtered; +INSERT INTO test_table (EventDate, UTCEventTime) VALUES ('2014-01-02', '2014-01-02 03:04:06'); + +SELECT * FROM test_table; +SELECT * FROM test_view; +SELECT * FROM test_view_filtered; + +DROP TABLE test_table; +DROP TABLE test_view; +DROP TABLE test_view_filtered; -- Check only sophisticated constructors and desctructors: -USE test; +CREATE DATABASE IF NOT EXISTS test_00101_1; + +USE test_00101_1; + DROP TABLE IF EXISTS tmp; DROP TABLE IF EXISTS tmp_mv; DROP TABLE IF EXISTS tmp_mv2; @@ -44,3 +51,8 @@ EXISTS TABLE `.inner.tmp_mv`; EXISTS TABLE `.inner.tmp_mv2`; EXISTS TABLE `.inner.tmp_mv3`; EXISTS TABLE `.inner.tmp_mv4`; + +DROP TABLE tmp; + +DROP DATABASE test_00101_0; +DROP DATABASE test_00101_1; diff --git a/dbms/tests/queries/0_stateless/00446_clear_column_in_partition_zookeeper.sql b/dbms/tests/queries/0_stateless/00446_clear_column_in_partition_zookeeper.sql index 72f97d50886..47f644f85a2 100644 --- a/dbms/tests/queries/0_stateless/00446_clear_column_in_partition_zookeeper.sql +++ b/dbms/tests/queries/0_stateless/00446_clear_column_in_partition_zookeeper.sql @@ -1,73 +1,73 @@ SELECT '===Ordinary case==='; -DROP TABLE IF EXISTS test.clear_column; -CREATE TABLE test.clear_column (d Date, num Int64, str String) ENGINE = MergeTree(d, d, 8192); +DROP TABLE IF EXISTS clear_column; +CREATE TABLE clear_column (d Date, num Int64, str String) ENGINE = MergeTree(d, d, 8192); -INSERT INTO test.clear_column VALUES ('2016-12-12', 1, 'a'), ('2016-11-12', 2, 'b'); +INSERT INTO clear_column VALUES ('2016-12-12', 1, 'a'), ('2016-11-12', 2, 'b'); -SELECT data_uncompressed_bytes FROM system.columns WHERE (database = 'test') AND (table = 'clear_column') AND (name = 'num'); +SELECT data_uncompressed_bytes FROM system.columns WHERE (database = currentDatabase()) AND (table = 'clear_column') AND (name = 'num'); -SELECT num, str FROM test.clear_column ORDER BY num; -ALTER TABLE test.clear_column CLEAR COLUMN num IN PARTITION '201612'; -SELECT num, str FROM test.clear_column ORDER BY num; +SELECT num, str FROM clear_column ORDER BY num; +ALTER TABLE clear_column CLEAR COLUMN num IN PARTITION '201612'; +SELECT num, str FROM clear_column ORDER BY num; -SELECT data_uncompressed_bytes FROM system.columns WHERE (database = 'test') AND (table = 'clear_column') AND (name = 'num'); -ALTER TABLE test.clear_column CLEAR COLUMN num IN PARTITION '201611'; -SELECT data_compressed_bytes, data_uncompressed_bytes FROM system.columns WHERE (database = 'test') AND (table = 'clear_column') AND (name = 'num'); +SELECT data_uncompressed_bytes FROM system.columns WHERE (database = currentDatabase()) AND (table = 'clear_column') AND (name = 'num'); +ALTER TABLE clear_column CLEAR COLUMN num IN PARTITION '201611'; +SELECT data_compressed_bytes, data_uncompressed_bytes FROM system.columns WHERE (database = currentDatabase()) AND (table = 'clear_column') AND (name = 'num'); -DROP TABLE test.clear_column; +DROP TABLE clear_column; SELECT '===Replicated case==='; SYSTEM STOP MERGES; -DROP TABLE IF EXISTS test.clear_column1; -DROP TABLE IF EXISTS test.clear_column2; -CREATE TABLE test.clear_column1 (d Date, i Int64) ENGINE = ReplicatedMergeTree('/clickhouse/test/tables/clear_column', '1', d, d, 8192); -CREATE TABLE test.clear_column2 (d Date, i Int64) ENGINE = ReplicatedMergeTree('/clickhouse/test/tables/clear_column', '2', d, d, 8192); +DROP TABLE IF EXISTS clear_column1; +DROP TABLE IF EXISTS clear_column2; +CREATE TABLE clear_column1 (d Date, i Int64) ENGINE = ReplicatedMergeTree('/clickhouse/test/tables/clear_column', '1', d, d, 8192); +CREATE TABLE clear_column2 (d Date, i Int64) ENGINE = ReplicatedMergeTree('/clickhouse/test/tables/clear_column', '2', d, d, 8192); -INSERT INTO test.clear_column1 (d) VALUES ('2000-01-01'), ('2000-02-01'); -SYSTEM SYNC REPLICA test.clear_column2; +INSERT INTO clear_column1 (d) VALUES ('2000-01-01'), ('2000-02-01'); +SYSTEM SYNC REPLICA clear_column2; SET replication_alter_partitions_sync=2; -ALTER TABLE test.clear_column1 ADD COLUMN s String; -ALTER TABLE test.clear_column1 CLEAR COLUMN s IN PARTITION '200001'; +ALTER TABLE clear_column1 ADD COLUMN s String; +ALTER TABLE clear_column1 CLEAR COLUMN s IN PARTITION '200001'; -INSERT INTO test.clear_column1 VALUES ('2000-01-01', 1, 'a'), ('2000-02-01', 2, 'b'); -INSERT INTO test.clear_column1 VALUES ('2000-01-01', 3, 'c'), ('2000-02-01', 4, 'd'); -SYSTEM SYNC REPLICA test.clear_column2; +INSERT INTO clear_column1 VALUES ('2000-01-01', 1, 'a'), ('2000-02-01', 2, 'b'); +INSERT INTO clear_column1 VALUES ('2000-01-01', 3, 'c'), ('2000-02-01', 4, 'd'); +SYSTEM SYNC REPLICA clear_column2; SELECT 'all'; -SELECT * FROM test.clear_column2 ORDER BY d, i, s; +SELECT * FROM clear_column2 ORDER BY d, i, s; SELECT 'w/o i 1'; -ALTER TABLE test.clear_column1 CLEAR COLUMN i IN PARTITION '200001'; -SELECT * FROM test.clear_column2 ORDER BY d, i, s; +ALTER TABLE clear_column1 CLEAR COLUMN i IN PARTITION '200001'; +SELECT * FROM clear_column2 ORDER BY d, i, s; SELECT 'w/o is 1'; -ALTER TABLE test.clear_column1 CLEAR COLUMN s IN PARTITION '200001'; -SELECT * FROM test.clear_column2 ORDER BY d, i, s; +ALTER TABLE clear_column1 CLEAR COLUMN s IN PARTITION '200001'; +SELECT * FROM clear_column2 ORDER BY d, i, s; SELECT 'w/o is 12'; -ALTER TABLE test.clear_column1 CLEAR COLUMN i IN PARTITION '200002'; -ALTER TABLE test.clear_column1 CLEAR COLUMN s IN PARTITION '200002'; -SELECT DISTINCT * FROM test.clear_column2 ORDER BY d, i, s; -SELECT DISTINCT * FROM test.clear_column2 ORDER BY d, i, s; +ALTER TABLE clear_column1 CLEAR COLUMN i IN PARTITION '200002'; +ALTER TABLE clear_column1 CLEAR COLUMN s IN PARTITION '200002'; +SELECT DISTINCT * FROM clear_column2 ORDER BY d, i, s; +SELECT DISTINCT * FROM clear_column2 ORDER BY d, i, s; SELECT 'sizes'; -SELECT sum(data_uncompressed_bytes) FROM system.columns WHERE database='test' AND table LIKE 'clear_column_' AND (name = 'i' OR name = 's') GROUP BY table; +SELECT sum(data_uncompressed_bytes) FROM system.columns WHERE database=currentDatabase() AND table LIKE 'clear_column_' AND (name = 'i' OR name = 's') GROUP BY table; -- double call should be OK -ALTER TABLE test.clear_column1 CLEAR COLUMN s IN PARTITION '200001'; -ALTER TABLE test.clear_column1 CLEAR COLUMN s IN PARTITION '200002'; +ALTER TABLE clear_column1 CLEAR COLUMN s IN PARTITION '200001'; +ALTER TABLE clear_column1 CLEAR COLUMN s IN PARTITION '200002'; -- Merges cannot be blocked after all manipulations SET optimize_throw_if_noop = 1; SYSTEM START MERGES; -OPTIMIZE TABLE test.clear_column1 PARTITION '200001'; -OPTIMIZE TABLE test.clear_column1 PARTITION '200002'; +OPTIMIZE TABLE clear_column1 PARTITION '200001'; +OPTIMIZE TABLE clear_column1 PARTITION '200002'; -- clear column in empty partition should be Ok -ALTER TABLE test.clear_column1 CLEAR COLUMN s IN PARTITION '200012', CLEAR COLUMN i IN PARTITION '200012'; +ALTER TABLE clear_column1 CLEAR COLUMN s IN PARTITION '200012', CLEAR COLUMN i IN PARTITION '200012'; -- Drop empty partition also Ok -ALTER TABLE test.clear_column1 DROP PARTITION '200012', DROP PARTITION '200011'; +ALTER TABLE clear_column1 DROP PARTITION '200012', DROP PARTITION '200011'; diff --git a/dbms/tests/queries/0_stateless/00508_materialized_view_to.sql b/dbms/tests/queries/0_stateless/00508_materialized_view_to.sql index b701247f7ee..7010df090eb 100644 --- a/dbms/tests/queries/0_stateless/00508_materialized_view_to.sql +++ b/dbms/tests/queries/0_stateless/00508_materialized_view_to.sql @@ -1,33 +1,37 @@ -DROP TABLE IF EXISTS test.src; -DROP TABLE IF EXISTS test.dst; -DROP TABLE IF EXISTS test.mv_00508; +CREATE DATABASE test_00508; -CREATE TABLE test.src (x UInt8) ENGINE = Null; -CREATE TABLE test.dst (x UInt8) ENGINE = Memory; +USE test_00508; -USE test; +DROP TABLE IF EXISTS src; +DROP TABLE IF EXISTS dst; +DROP TABLE IF EXISTS mv_00508; -CREATE MATERIALIZED VIEW test.mv_00508 TO dst AS SELECT * FROM src; +CREATE TABLE src (x UInt8) ENGINE = Null; +CREATE TABLE dst (x UInt8) ENGINE = Memory; + +CREATE MATERIALIZED VIEW mv_00508 TO dst AS SELECT * FROM src; INSERT INTO src VALUES (1), (2); -SELECT * FROM test.mv_00508 ORDER BY x; +SELECT * FROM mv_00508 ORDER BY x; -- Detach MV and see if the data is still readable -DETACH TABLE test.mv_00508; +DETACH TABLE mv_00508; SELECT * FROM dst ORDER BY x; USE default; -- Reattach MV (shortcut) -ATTACH TABLE test.mv_00508; +ATTACH TABLE test_00508.mv_00508; -INSERT INTO test.src VALUES (3); +INSERT INTO test_00508.src VALUES (3); -SELECT * FROM test.mv_00508 ORDER BY x; +SELECT * FROM test_00508.mv_00508 ORDER BY x; -- Drop the MV and see if the data is still readable -DROP TABLE test.mv_00508; -SELECT * FROM test.dst ORDER BY x; +DROP TABLE test_00508.mv_00508; +SELECT * FROM test_00508.dst ORDER BY x; -DROP TABLE test.src; -DROP TABLE test.dst; +DROP TABLE test_00508.src; +DROP TABLE test_00508.dst; + +DROP DATABASE test_00508; diff --git a/dbms/tests/queries/0_stateless/00571_non_exist_database_when_create_materializ_view.sql b/dbms/tests/queries/0_stateless/00571_non_exist_database_when_create_materializ_view.sql index d770ec71f0f..1ad261de35c 100644 --- a/dbms/tests/queries/0_stateless/00571_non_exist_database_when_create_materializ_view.sql +++ b/dbms/tests/queries/0_stateless/00571_non_exist_database_when_create_materializ_view.sql @@ -1,25 +1,29 @@ -DROP DATABASE IF EXISTS none; -DROP TABLE IF EXISTS test.test_00571; -DROP TABLE IF EXISTS test.test_materialized_00571; +CREATE DATABASE test_00571; -USE test; +USE test_00571; + +DROP DATABASE IF EXISTS none; +DROP TABLE IF EXISTS test_00571; +DROP TABLE IF EXISTS test_materialized_00571; CREATE DATABASE none; -CREATE TABLE test.test_00571 ( date Date, platform Enum8('a' = 0, 'b' = 1, 'c' = 2), app Enum8('a' = 0, 'b' = 1) ) ENGINE = MergeTree(date, (platform, app), 8192); -CREATE MATERIALIZED VIEW test.test_materialized_00571 ENGINE = MergeTree(date, (platform, app), 8192) POPULATE AS SELECT date, platform, app FROM (SELECT * FROM test_00571); +CREATE TABLE test_00571 ( date Date, platform Enum8('a' = 0, 'b' = 1, 'c' = 2), app Enum8('a' = 0, 'b' = 1) ) ENGINE = MergeTree(date, (platform, app), 8192); +CREATE MATERIALIZED VIEW test_materialized_00571 ENGINE = MergeTree(date, (platform, app), 8192) POPULATE AS SELECT date, platform, app FROM (SELECT * FROM test_00571); USE none; -INSERT INTO test.test_00571 VALUES('2018-02-16', 'a', 'a'); +INSERT INTO test_00571.test_00571 VALUES('2018-02-16', 'a', 'a'); -SELECT * FROM test.test_00571; -SELECT * FROM test.test_materialized_00571; +SELECT * FROM test_00571.test_00571; +SELECT * FROM test_00571.test_materialized_00571; -DETACH TABLE test.test_materialized_00571; -ATTACH TABLE test.test_materialized_00571; +DETACH TABLE test_00571.test_materialized_00571; +ATTACH TABLE test_00571.test_materialized_00571; -SELECT * FROM test.test_materialized_00571; +SELECT * FROM test_00571.test_materialized_00571; DROP DATABASE IF EXISTS none; -DROP TABLE IF EXISTS test.test_00571; -DROP TABLE IF EXISTS test.test_materialized_00571; +DROP TABLE IF EXISTS test_00571.test_00571; +DROP TABLE IF EXISTS test_00571.test_materialized_00571; + +DROP DATABASE test_00571; diff --git a/dbms/tests/queries/0_stateless/00578_merge_table_shadow_virtual_column.sql b/dbms/tests/queries/0_stateless/00578_merge_table_shadow_virtual_column.sql index c9b14fdade4..3071e307517 100644 --- a/dbms/tests/queries/0_stateless/00578_merge_table_shadow_virtual_column.sql +++ b/dbms/tests/queries/0_stateless/00578_merge_table_shadow_virtual_column.sql @@ -1,11 +1,11 @@ -DROP TABLE IF EXISTS test.numbers1; -DROP TABLE IF EXISTS test.numbers2; +DROP TABLE IF EXISTS numbers1; +DROP TABLE IF EXISTS numbers2; -CREATE TABLE test.numbers1 ENGINE = Memory AS SELECT number as _table FROM numbers(1000); -CREATE TABLE test.numbers2 ENGINE = Memory AS SELECT number as _table FROM numbers(1000); +CREATE TABLE numbers1 ENGINE = Memory AS SELECT number as _table FROM numbers(1000); +CREATE TABLE numbers2 ENGINE = Memory AS SELECT number as _table FROM numbers(1000); -SELECT count() FROM merge(test, '^numbers\\d+$') WHERE _table='numbers1'; -- { serverError 43 } -SELECT count() FROM merge(test, '^numbers\\d+$') WHERE _table=1; +SELECT count() FROM merge(currentDatabase(), '^numbers\\d+$') WHERE _table='numbers1'; -- { serverError 43 } +SELECT count() FROM merge(currentDatabase(), '^numbers\\d+$') WHERE _table=1; -DROP TABLE test.numbers1; -DROP TABLE test.numbers2; +DROP TABLE numbers1; +DROP TABLE numbers2; diff --git a/dbms/tests/queries/0_stateless/00612_count.sql b/dbms/tests/queries/0_stateless/00612_count.sql index e701fa1cc73..5dd9c770700 100644 --- a/dbms/tests/queries/0_stateless/00612_count.sql +++ b/dbms/tests/queries/0_stateless/00612_count.sql @@ -1,18 +1,18 @@ -DROP TABLE IF EXISTS test.count; +DROP TABLE IF EXISTS count; -CREATE TABLE test.count (x UInt64) ENGINE = MergeTree ORDER BY tuple(); -INSERT INTO test.count SELECT * FROM numbers(1234567); +CREATE TABLE count (x UInt64) ENGINE = MergeTree ORDER BY tuple(); +INSERT INTO count SELECT * FROM numbers(1234567); -SELECT count() FROM test.count; -SELECT count() * 2 FROM test.count; -SELECT count() FROM (SELECT * FROM test.count UNION ALL SELECT * FROM test.count); -SELECT count() FROM test.count WITH TOTALS; -SELECT arrayJoin([count(), count()]) FROM test.count; -SELECT arrayJoin([count(), count()]) FROM test.count LIMIT 1; -SELECT arrayJoin([count(), count()]) FROM test.count LIMIT 1, 1; -SELECT arrayJoin([count(), count()]) AS x FROM test.count LIMIT 1 BY x; -SELECT arrayJoin([count(), count() + 1]) AS x FROM test.count LIMIT 1 BY x; -SELECT count() FROM test.count HAVING count() = 1234567; -SELECT count() FROM test.count HAVING count() != 1234567; +SELECT count() FROM count; +SELECT count() * 2 FROM count; +SELECT count() FROM (SELECT * FROM count UNION ALL SELECT * FROM count); +SELECT count() FROM count WITH TOTALS; +SELECT arrayJoin([count(), count()]) FROM count; +SELECT arrayJoin([count(), count()]) FROM count LIMIT 1; +SELECT arrayJoin([count(), count()]) FROM count LIMIT 1, 1; +SELECT arrayJoin([count(), count()]) AS x FROM count LIMIT 1 BY x; +SELECT arrayJoin([count(), count() + 1]) AS x FROM count LIMIT 1 BY x; +SELECT count() FROM count HAVING count() = 1234567; +SELECT count() FROM count HAVING count() != 1234567; -DROP TABLE test.count; +DROP TABLE count; diff --git a/dbms/tests/queries/0_stateless/00612_shard_count.sql b/dbms/tests/queries/0_stateless/00612_shard_count.sql index b59ba029351..2a69e9ed9c8 100644 --- a/dbms/tests/queries/0_stateless/00612_shard_count.sql +++ b/dbms/tests/queries/0_stateless/00612_shard_count.sql @@ -1,9 +1,9 @@ -DROP TABLE IF EXISTS test.count; +DROP TABLE IF EXISTS count; -CREATE TABLE test.count (x UInt64) ENGINE = MergeTree ORDER BY tuple(); -INSERT INTO test.count SELECT * FROM numbers(1234567); +CREATE TABLE count (x UInt64) ENGINE = MergeTree ORDER BY tuple(); +INSERT INTO count SELECT * FROM numbers(1234567); -SELECT count() FROM remote('127.0.0.{1,2}', test.count); -SELECT count() / 2 FROM remote('127.0.0.{1,2}', test.count); +SELECT count() FROM remote('127.0.0.{1,2}', currentDatabase(), count); +SELECT count() / 2 FROM remote('127.0.0.{1,2}', currentDatabase(), count); -DROP TABLE test.count; +DROP TABLE count; diff --git a/dbms/tests/queries/0_stateless/00740_database_in_nested_view.sql b/dbms/tests/queries/0_stateless/00740_database_in_nested_view.sql index a40c470e767..0106196279f 100644 --- a/dbms/tests/queries/0_stateless/00740_database_in_nested_view.sql +++ b/dbms/tests/queries/0_stateless/00740_database_in_nested_view.sql @@ -1,9 +1,11 @@ -DROP TABLE IF EXISTS test.test_00740; -DROP TABLE IF EXISTS test.test_view_00740; -DROP TABLE IF EXISTS test.test_nested_view_00740; -DROP TABLE IF EXISTS test.test_joined_view_00740; +CREATE DATABASE test_00740; +USE test_00740; + +DROP TABLE IF EXISTS test_00740; +DROP TABLE IF EXISTS test_view_00740; +DROP TABLE IF EXISTS test_nested_view_00740; +DROP TABLE IF EXISTS test_joined_view_00740; -USE test; CREATE VIEW test_00740 AS SELECT 1 AS N; CREATE VIEW test_view_00740 AS SELECT * FROM test_00740; CREATE VIEW test_nested_view_00740 AS SELECT * FROM (SELECT * FROM test_00740); @@ -14,10 +16,12 @@ SELECT * FROM test_nested_view_00740; SELECT * FROM test_joined_view_00740; USE default; -SELECT * FROM test.test_view_00740; -SELECT * FROM test.test_nested_view_00740; -SELECT * FROM test.test_joined_view_00740; +SELECT * FROM test_00740.test_view_00740; +SELECT * FROM test_00740.test_nested_view_00740; +SELECT * FROM test_00740.test_joined_view_00740; -DROP TABLE IF EXISTS test.test_00740; -DROP TABLE IF EXISTS test.test_view_00740; -DROP TABLE IF EXISTS test.test_nested_view_00740; +DROP TABLE IF EXISTS test_00740.test_00740; +DROP TABLE IF EXISTS test_00740.test_view_00740; +DROP TABLE IF EXISTS test_00740.test_nested_view_00740; + +DROP DATABASE test_00740; diff --git a/dbms/tests/queries/0_stateless/00751_default_databasename_for_view.reference b/dbms/tests/queries/0_stateless/00751_default_databasename_for_view.reference index e45dde1921e..2873fcbee3b 100644 --- a/dbms/tests/queries/0_stateless/00751_default_databasename_for_view.reference +++ b/dbms/tests/queries/0_stateless/00751_default_databasename_for_view.reference @@ -1,4 +1,4 @@ -CREATE MATERIALIZED VIEW test.t_mv_00751 (`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_00751 WHERE (app = (SELECT min(app) FROM test.u_00751)) AND (platform = (SELECT (SELECT min(platform) FROM test.v_00751))) +CREATE MATERIALIZED VIEW test_00751.t_mv_00751 (`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_00751.t_00751 WHERE (app = (SELECT min(app) FROM test_00751.u_00751)) AND (platform = (SELECT (SELECT min(platform) FROM test_00751.v_00751))) 2000-01-01 a a 2000-01-02 b b 2000-01-03 a a diff --git a/dbms/tests/queries/0_stateless/00751_default_databasename_for_view.sql b/dbms/tests/queries/0_stateless/00751_default_databasename_for_view.sql index bb43b99b461..0f962b31196 100644 --- a/dbms/tests/queries/0_stateless/00751_default_databasename_for_view.sql +++ b/dbms/tests/queries/0_stateless/00751_default_databasename_for_view.sql @@ -1,4 +1,5 @@ -USE test; +CREATE DATABASE test_00751; +USE test_00751; DROP TABLE IF EXISTS t_00751; DROP TABLE IF EXISTS t_mv_00751; @@ -22,23 +23,25 @@ CREATE MATERIALIZED VIEW t_mv_00751 ENGINE = MergeTree ORDER BY date AS SELECT date, platform, app FROM t_00751 WHERE app = (SELECT min(app) from u_00751) AND platform = (SELECT (SELECT min(platform) from v_00751)); -SHOW CREATE TABLE test.t_mv_00751 FORMAT TabSeparatedRaw; +SHOW CREATE TABLE test_00751.t_mv_00751 FORMAT TabSeparatedRaw; USE default; -DETACH TABLE test.t_mv_00751; -ATTACH TABLE test.t_mv_00751; +DETACH TABLE test_00751.t_mv_00751; +ATTACH TABLE test_00751.t_mv_00751; -INSERT INTO test.t_00751 VALUES ('2000-01-01', 'a', 'a') ('2000-01-02', 'b', 'b'); +INSERT INTO test_00751.t_00751 VALUES ('2000-01-01', 'a', 'a') ('2000-01-02', 'b', 'b'); -INSERT INTO test.u_00751 VALUES ('a'); -INSERT INTO test.v_00751 VALUES ('a'); +INSERT INTO test_00751.u_00751 VALUES ('a'); +INSERT INTO test_00751.v_00751 VALUES ('a'); -INSERT INTO test.t_00751 VALUES ('2000-01-03', 'a', 'a') ('2000-01-04', 'b', 'b'); +INSERT INTO test_00751.t_00751 VALUES ('2000-01-03', 'a', 'a') ('2000-01-04', 'b', 'b'); -SELECT * FROM test.t_00751 ORDER BY date; -SELECT * FROM test.t_mv_00751 ORDER BY date; +SELECT * FROM test_00751.t_00751 ORDER BY date; +SELECT * FROM test_00751.t_mv_00751 ORDER BY date; -DROP TABLE test.t_00751; -DROP TABLE test.t_mv_00751; -DROP TABLE test.u_00751; -DROP TABLE test.v_00751; +DROP TABLE test_00751.t_00751; +DROP TABLE test_00751.t_mv_00751; +DROP TABLE test_00751.u_00751; +DROP TABLE test_00751.v_00751; + +DROP DATABASE test_00751; diff --git a/dbms/tests/queries/0_stateless/00800_versatile_storage_join.sql b/dbms/tests/queries/0_stateless/00800_versatile_storage_join.sql index 07f596e90bf..1984d6c2a6a 100644 --- a/dbms/tests/queries/0_stateless/00800_versatile_storage_join.sql +++ b/dbms/tests/queries/0_stateless/00800_versatile_storage_join.sql @@ -1,34 +1,36 @@ -DROP TABLE IF EXISTS test.join_any_inner; -DROP TABLE IF EXISTS test.join_any_left; -DROP TABLE IF EXISTS test.join_any_left_null; -DROP TABLE IF EXISTS test.join_all_inner; -DROP TABLE IF EXISTS test.join_all_left; -DROP TABLE IF EXISTS test.join_string_key; +CREATE DATABASE test_00800; -CREATE TABLE test.join_any_inner (s String, x Array(UInt8), k UInt64) ENGINE = Join(ANY, INNER, k); -CREATE TABLE test.join_any_left (s String, x Array(UInt8), k UInt64) ENGINE = Join(ANY, LEFT, k); -CREATE TABLE test.join_all_inner (s String, x Array(UInt8), k UInt64) ENGINE = Join(ALL, INNER, k); -CREATE TABLE test.join_all_left (s String, x Array(UInt8), k UInt64) ENGINE = Join(ALL, LEFT, k); +USE test_00800; -USE test; +DROP TABLE IF EXISTS join_any_inner; +DROP TABLE IF EXISTS join_any_left; +DROP TABLE IF EXISTS join_any_left_null; +DROP TABLE IF EXISTS join_all_inner; +DROP TABLE IF EXISTS join_all_left; +DROP TABLE IF EXISTS join_string_key; -INSERT INTO test.join_any_inner VALUES ('abc', [0], 1), ('def', [1, 2], 2); -INSERT INTO test.join_any_left VALUES ('abc', [0], 1), ('def', [1, 2], 2); -INSERT INTO test.join_all_inner VALUES ('abc', [0], 1), ('def', [1, 2], 2); -INSERT INTO test.join_all_left VALUES ('abc', [0], 1), ('def', [1, 2], 2); +CREATE TABLE join_any_inner (s String, x Array(UInt8), k UInt64) ENGINE = Join(ANY, INNER, k); +CREATE TABLE join_any_left (s String, x Array(UInt8), k UInt64) ENGINE = Join(ANY, LEFT, k); +CREATE TABLE join_all_inner (s String, x Array(UInt8), k UInt64) ENGINE = Join(ALL, INNER, k); +CREATE TABLE join_all_left (s String, x Array(UInt8), k UInt64) ENGINE = Join(ALL, LEFT, k); + +INSERT INTO join_any_inner VALUES ('abc', [0], 1), ('def', [1, 2], 2); +INSERT INTO join_any_left VALUES ('abc', [0], 1), ('def', [1, 2], 2); +INSERT INTO join_all_inner VALUES ('abc', [0], 1), ('def', [1, 2], 2); +INSERT INTO join_all_left VALUES ('abc', [0], 1), ('def', [1, 2], 2); -- read from StorageJoin SELECT '--------read--------'; -SELECT * from test.join_any_inner; -SELECT * from test.join_any_left; -SELECT * from test.join_all_inner; -SELECT * from test.join_all_left; +SELECT * from join_any_inner; +SELECT * from join_any_left; +SELECT * from join_all_inner; +SELECT * from join_all_left; -- create StorageJoin tables with customized settings -CREATE TABLE test.join_any_left_null (s String, k UInt64) ENGINE = Join(ANY, LEFT, k) SETTINGS join_use_nulls = 1; -INSERT INTO test.join_any_left_null VALUES ('abc', 1), ('def', 2); +CREATE TABLE join_any_left_null (s String, k UInt64) ENGINE = Join(ANY, LEFT, k) SETTINGS join_use_nulls = 1; +INSERT INTO join_any_left_null VALUES ('abc', 1), ('def', 2); -- joinGet SELECT '--------joinGet--------'; @@ -41,31 +43,31 @@ SELECT ''; SELECT joinGet(join_any_left, 's', number) FROM numbers(3); SELECT ''; -SELECT joinGet(test.join_any_left_null, 's', number) FROM numbers(3); +SELECT joinGet(join_any_left_null, 's', number) FROM numbers(3); SELECT ''; -CREATE TABLE test.join_string_key (s String, x Array(UInt8), k UInt64) ENGINE = Join(ANY, LEFT, s); -INSERT INTO test.join_string_key VALUES ('abc', [0], 1), ('def', [1, 2], 2); +CREATE TABLE join_string_key (s String, x Array(UInt8), k UInt64) ENGINE = Join(ANY, LEFT, s); +INSERT INTO join_string_key VALUES ('abc', [0], 1), ('def', [1, 2], 2); SELECT joinGet('join_string_key', 'x', 'abc'), joinGet('join_string_key', 'k', 'abc'); USE default; -DROP TABLE test.join_any_inner; -DROP TABLE test.join_any_left; -DROP TABLE test.join_any_left_null; -DROP TABLE test.join_all_inner; -DROP TABLE test.join_all_left; -DROP TABLE test.join_string_key; +DROP TABLE test_00800.join_any_inner; +DROP TABLE test_00800.join_any_left; +DROP TABLE test_00800.join_any_left_null; +DROP TABLE test_00800.join_all_inner; +DROP TABLE test_00800.join_all_left; +DROP TABLE test_00800.join_string_key; -- test provided by Alexander Zaitsev -DROP TABLE IF EXISTS test.join_test; -CREATE TABLE test.join_test (a UInt8, b UInt8) Engine = Join(ANY, LEFT, a); +DROP TABLE IF EXISTS test_00800.join_test; +CREATE TABLE test_00800.join_test (a UInt8, b UInt8) Engine = Join(ANY, LEFT, a); -USE test; +USE test_00800; select joinGet('join_test', 'b', 1); USE system; -SELECT joinGet('test.join_test', 'b', 1); +SELECT joinGet('test_00800.join_test', 'b', 1); USE default; -DROP TABLE test.join_test; +DROP TABLE test_00800.join_test; diff --git a/dbms/tests/queries/0_stateless/00870_t64_codec.sql b/dbms/tests/queries/0_stateless/00870_t64_codec.sql index 654f7fa7818..a8a86b6fd93 100644 --- a/dbms/tests/queries/0_stateless/00870_t64_codec.sql +++ b/dbms/tests/queries/0_stateless/00870_t64_codec.sql @@ -1,6 +1,6 @@ -DROP TABLE IF EXISTS test.t64; +DROP TABLE IF EXISTS t64; -CREATE TABLE test.t64 +CREATE TABLE t64 ( u8 UInt8, t_u8 UInt8 Codec(T64, ZSTD), @@ -12,90 +12,90 @@ CREATE TABLE test.t64 t_u64 UInt64 Codec(T64, ZSTD) ) ENGINE MergeTree() ORDER BY tuple(); -INSERT INTO test.t64 SELECT number AS x, x, x, x, x, x, x, x FROM numbers(1); -INSERT INTO test.t64 SELECT number AS x, x, x, x, x, x, x, x FROM numbers(2); -INSERT INTO test.t64 SELECT 42 AS x, x, x, x, x, x, x, x FROM numbers(4); +INSERT INTO t64 SELECT number AS x, x, x, x, x, x, x, x FROM numbers(1); +INSERT INTO t64 SELECT number AS x, x, x, x, x, x, x, x FROM numbers(2); +INSERT INTO t64 SELECT 42 AS x, x, x, x, x, x, x, x FROM numbers(4); -SELECT * FROM test.t64 ORDER BY u64; +SELECT * FROM t64 ORDER BY u64; -INSERT INTO test.t64 SELECT number AS x, x, x, x, x, x, x, x FROM numbers(intExp2(8)); -INSERT INTO test.t64 SELECT number AS x, x, x, x, x, x, x, x FROM numbers(intExp2(9)); -SELECT * FROM test.t64 WHERE u8 != t_u8; -SELECT * FROM test.t64 WHERE u16 != t_u16; -SELECT * FROM test.t64 WHERE u32 != t_u32; -SELECT * FROM test.t64 WHERE u64 != t_u64; +INSERT INTO t64 SELECT number AS x, x, x, x, x, x, x, x FROM numbers(intExp2(8)); +INSERT INTO t64 SELECT number AS x, x, x, x, x, x, x, x FROM numbers(intExp2(9)); +SELECT * FROM t64 WHERE u8 != t_u8; +SELECT * FROM t64 WHERE u16 != t_u16; +SELECT * FROM t64 WHERE u32 != t_u32; +SELECT * FROM t64 WHERE u64 != t_u64; -INSERT INTO test.t64 SELECT (intExp2(16) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(10); -INSERT INTO test.t64 SELECT (intExp2(16) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(11); -INSERT INTO test.t64 SELECT (intExp2(16) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(64); -INSERT INTO test.t64 SELECT (intExp2(16) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(65); -INSERT INTO test.t64 SELECT (intExp2(16) - 1 + number) AS x, x, x, x, x, x, x, x FROM numbers(65); -SELECT * FROM test.t64 WHERE u8 != t_u8; -SELECT * FROM test.t64 WHERE u16 != t_u16; -SELECT * FROM test.t64 WHERE u32 != t_u32; -SELECT * FROM test.t64 WHERE u64 != t_u64; +INSERT INTO t64 SELECT (intExp2(16) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(10); +INSERT INTO t64 SELECT (intExp2(16) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(11); +INSERT INTO t64 SELECT (intExp2(16) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(64); +INSERT INTO t64 SELECT (intExp2(16) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(65); +INSERT INTO t64 SELECT (intExp2(16) - 1 + number) AS x, x, x, x, x, x, x, x FROM numbers(65); +SELECT * FROM t64 WHERE u8 != t_u8; +SELECT * FROM t64 WHERE u16 != t_u16; +SELECT * FROM t64 WHERE u32 != t_u32; +SELECT * FROM t64 WHERE u64 != t_u64; -INSERT INTO test.t64 SELECT (intExp2(24) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(10); -INSERT INTO test.t64 SELECT (intExp2(24) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(11); -INSERT INTO test.t64 SELECT (intExp2(24) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(128); -INSERT INTO test.t64 SELECT (intExp2(24) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(129); -INSERT INTO test.t64 SELECT (intExp2(24) - 1 + number) AS x, x, x, x, x, x, x, x FROM numbers(129); -SELECT * FROM test.t64 WHERE u8 != t_u8; -SELECT * FROM test.t64 WHERE u16 != t_u16; -SELECT * FROM test.t64 WHERE u32 != t_u32; -SELECT * FROM test.t64 WHERE u64 != t_u64; +INSERT INTO t64 SELECT (intExp2(24) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(10); +INSERT INTO t64 SELECT (intExp2(24) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(11); +INSERT INTO t64 SELECT (intExp2(24) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(128); +INSERT INTO t64 SELECT (intExp2(24) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(129); +INSERT INTO t64 SELECT (intExp2(24) - 1 + number) AS x, x, x, x, x, x, x, x FROM numbers(129); +SELECT * FROM t64 WHERE u8 != t_u8; +SELECT * FROM t64 WHERE u16 != t_u16; +SELECT * FROM t64 WHERE u32 != t_u32; +SELECT * FROM t64 WHERE u64 != t_u64; -INSERT INTO test.t64 SELECT (intExp2(32) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(10); -INSERT INTO test.t64 SELECT (intExp2(32) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(20); -INSERT INTO test.t64 SELECT (intExp2(32) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(256); -INSERT INTO test.t64 SELECT (intExp2(32) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(257); -INSERT INTO test.t64 SELECT (intExp2(32) - 1 + number) AS x, x, x, x, x, x, x, x FROM numbers(257); -SELECT * FROM test.t64 WHERE u8 != t_u8; -SELECT * FROM test.t64 WHERE u16 != t_u16; -SELECT * FROM test.t64 WHERE u32 != t_u32; -SELECT * FROM test.t64 WHERE u64 != t_u64; +INSERT INTO t64 SELECT (intExp2(32) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(10); +INSERT INTO t64 SELECT (intExp2(32) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(20); +INSERT INTO t64 SELECT (intExp2(32) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(256); +INSERT INTO t64 SELECT (intExp2(32) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(257); +INSERT INTO t64 SELECT (intExp2(32) - 1 + number) AS x, x, x, x, x, x, x, x FROM numbers(257); +SELECT * FROM t64 WHERE u8 != t_u8; +SELECT * FROM t64 WHERE u16 != t_u16; +SELECT * FROM t64 WHERE u32 != t_u32; +SELECT * FROM t64 WHERE u64 != t_u64; -INSERT INTO test.t64 SELECT (intExp2(40) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(10); -INSERT INTO test.t64 SELECT (intExp2(40) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(20); -INSERT INTO test.t64 SELECT (intExp2(40) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(512); -INSERT INTO test.t64 SELECT (intExp2(40) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(513); -INSERT INTO test.t64 SELECT (intExp2(40) - 1 + number) AS x, x, x, x, x, x, x, x FROM numbers(513); -SELECT * FROM test.t64 WHERE u8 != t_u8; -SELECT * FROM test.t64 WHERE u16 != t_u16; -SELECT * FROM test.t64 WHERE u32 != t_u32; -SELECT * FROM test.t64 WHERE u64 != t_u64; +INSERT INTO t64 SELECT (intExp2(40) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(10); +INSERT INTO t64 SELECT (intExp2(40) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(20); +INSERT INTO t64 SELECT (intExp2(40) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(512); +INSERT INTO t64 SELECT (intExp2(40) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(513); +INSERT INTO t64 SELECT (intExp2(40) - 1 + number) AS x, x, x, x, x, x, x, x FROM numbers(513); +SELECT * FROM t64 WHERE u8 != t_u8; +SELECT * FROM t64 WHERE u16 != t_u16; +SELECT * FROM t64 WHERE u32 != t_u32; +SELECT * FROM t64 WHERE u64 != t_u64; -INSERT INTO test.t64 SELECT (intExp2(48) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(10); -INSERT INTO test.t64 SELECT (intExp2(48) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(20); -INSERT INTO test.t64 SELECT (intExp2(48) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(1024); -INSERT INTO test.t64 SELECT (intExp2(48) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(1025); -INSERT INTO test.t64 SELECT (intExp2(48) - 1 + number) AS x, x, x, x, x, x, x, x FROM numbers(1025); -SELECT * FROM test.t64 WHERE u8 != t_u8; -SELECT * FROM test.t64 WHERE u16 != t_u16; -SELECT * FROM test.t64 WHERE u32 != t_u32; -SELECT * FROM test.t64 WHERE u64 != t_u64; +INSERT INTO t64 SELECT (intExp2(48) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(10); +INSERT INTO t64 SELECT (intExp2(48) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(20); +INSERT INTO t64 SELECT (intExp2(48) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(1024); +INSERT INTO t64 SELECT (intExp2(48) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(1025); +INSERT INTO t64 SELECT (intExp2(48) - 1 + number) AS x, x, x, x, x, x, x, x FROM numbers(1025); +SELECT * FROM t64 WHERE u8 != t_u8; +SELECT * FROM t64 WHERE u16 != t_u16; +SELECT * FROM t64 WHERE u32 != t_u32; +SELECT * FROM t64 WHERE u64 != t_u64; -INSERT INTO test.t64 SELECT (intExp2(56) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(10); -INSERT INTO test.t64 SELECT (intExp2(56) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(20); -INSERT INTO test.t64 SELECT (intExp2(56) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(2048); -INSERT INTO test.t64 SELECT (intExp2(56) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(2049); -INSERT INTO test.t64 SELECT (intExp2(56) - 1 + number) AS x, x, x, x, x, x, x, x FROM numbers(2049); -SELECT * FROM test.t64 WHERE u8 != t_u8; -SELECT * FROM test.t64 WHERE u16 != t_u16; -SELECT * FROM test.t64 WHERE u32 != t_u32; -SELECT * FROM test.t64 WHERE u64 != t_u64; +INSERT INTO t64 SELECT (intExp2(56) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(10); +INSERT INTO t64 SELECT (intExp2(56) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(20); +INSERT INTO t64 SELECT (intExp2(56) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(2048); +INSERT INTO t64 SELECT (intExp2(56) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(2049); +INSERT INTO t64 SELECT (intExp2(56) - 1 + number) AS x, x, x, x, x, x, x, x FROM numbers(2049); +SELECT * FROM t64 WHERE u8 != t_u8; +SELECT * FROM t64 WHERE u16 != t_u16; +SELECT * FROM t64 WHERE u32 != t_u32; +SELECT * FROM t64 WHERE u64 != t_u64; -INSERT INTO test.t64 SELECT (intExp2(63) + number * intExp2(62)) AS x, x, x, x, x, x, x, x FROM numbers(10); -SELECT * FROM test.t64 WHERE u8 != t_u8; -SELECT * FROM test.t64 WHERE u16 != t_u16; -SELECT * FROM test.t64 WHERE u32 != t_u32; -SELECT * FROM test.t64 WHERE u64 != t_u64; +INSERT INTO t64 SELECT (intExp2(63) + number * intExp2(62)) AS x, x, x, x, x, x, x, x FROM numbers(10); +SELECT * FROM t64 WHERE u8 != t_u8; +SELECT * FROM t64 WHERE u16 != t_u16; +SELECT * FROM t64 WHERE u32 != t_u32; +SELECT * FROM t64 WHERE u64 != t_u64; -OPTIMIZE TABLE test.t64 FINAL; +OPTIMIZE TABLE t64 FINAL; -SELECT * FROM test.t64 WHERE u8 != t_u8; -SELECT * FROM test.t64 WHERE u16 != t_u16; -SELECT * FROM test.t64 WHERE u32 != t_u32; -SELECT * FROM test.t64 WHERE u64 != t_u64; +SELECT * FROM t64 WHERE u8 != t_u8; +SELECT * FROM t64 WHERE u16 != t_u16; +SELECT * FROM t64 WHERE u32 != t_u32; +SELECT * FROM t64 WHERE u64 != t_u64; -DROP TABLE test.t64; +DROP TABLE t64; diff --git a/dbms/tests/queries/0_stateless/00871_t64_codec_signed.sql b/dbms/tests/queries/0_stateless/00871_t64_codec_signed.sql index 5e0afe71a02..f21325c12cd 100644 --- a/dbms/tests/queries/0_stateless/00871_t64_codec_signed.sql +++ b/dbms/tests/queries/0_stateless/00871_t64_codec_signed.sql @@ -1,6 +1,6 @@ -DROP TABLE IF EXISTS test.t64; +DROP TABLE IF EXISTS t64; -CREATE TABLE test.t64 +CREATE TABLE t64 ( i8 Int8, t_i8 Int8 Codec(T64, LZ4), @@ -12,117 +12,117 @@ CREATE TABLE test.t64 t_i64 Int64 Codec(T64, LZ4) ) ENGINE MergeTree() ORDER BY tuple(); -INSERT INTO test.t64 SELECT toInt32(number)-1 AS x, x, x, x, x, x, x, x FROM numbers(2); -INSERT INTO test.t64 SELECT toInt32(number)-1 AS x, x, x, x, x, x, x, x FROM numbers(3); -INSERT INTO test.t64 SELECT 42 AS x, x, x, x, x, x, x, x FROM numbers(4); +INSERT INTO t64 SELECT toInt32(number)-1 AS x, x, x, x, x, x, x, x FROM numbers(2); +INSERT INTO t64 SELECT toInt32(number)-1 AS x, x, x, x, x, x, x, x FROM numbers(3); +INSERT INTO t64 SELECT 42 AS x, x, x, x, x, x, x, x FROM numbers(4); -SELECT * FROM test.t64 ORDER BY i64; +SELECT * FROM t64 ORDER BY i64; -INSERT INTO test.t64 SELECT (intExp2(8) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(20); -SELECT i8, t_i8 FROM test.t64 WHERE i8 != t_i8; +INSERT INTO t64 SELECT (intExp2(8) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(20); +SELECT i8, t_i8 FROM t64 WHERE i8 != t_i8; -INSERT INTO test.t64 SELECT number AS x, x, x, x, x, x, x, x FROM numbers(intExp2(8)); -INSERT INTO test.t64 SELECT number AS x, x, x, x, x, x, x, x FROM numbers(intExp2(9)); -SELECT * FROM test.t64 WHERE i8 != t_i8; -SELECT * FROM test.t64 WHERE i16 != t_i16; -SELECT * FROM test.t64 WHERE i32 != t_i32; -SELECT * FROM test.t64 WHERE i64 != t_i64; +INSERT INTO t64 SELECT number AS x, x, x, x, x, x, x, x FROM numbers(intExp2(8)); +INSERT INTO t64 SELECT number AS x, x, x, x, x, x, x, x FROM numbers(intExp2(9)); +SELECT * FROM t64 WHERE i8 != t_i8; +SELECT * FROM t64 WHERE i16 != t_i16; +SELECT * FROM t64 WHERE i32 != t_i32; +SELECT * FROM t64 WHERE i64 != t_i64; -INSERT INTO test.t64 SELECT (intExp2(16) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(10); -INSERT INTO test.t64 SELECT (intExp2(16) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(11); -INSERT INTO test.t64 SELECT (intExp2(16) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(64); -INSERT INTO test.t64 SELECT (intExp2(16) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(65); -INSERT INTO test.t64 SELECT (intExp2(16) - 1 + number) AS x, x, x, x, x, x, x, x FROM numbers(65); -INSERT INTO test.t64 SELECT (10 - toInt64(intExp2(16)) + number) AS x, x, x, x, x, x, x, x FROM numbers(10); -INSERT INTO test.t64 SELECT (10 - toInt64(intExp2(16)) + number) AS x, x, x, x, x, x, x, x FROM numbers(11); -INSERT INTO test.t64 SELECT (64 - toInt64(intExp2(16)) + number) AS x, x, x, x, x, x, x, x FROM numbers(64); -INSERT INTO test.t64 SELECT (64 - toInt64(intExp2(16)) + number) AS x, x, x, x, x, x, x, x FROM numbers(65); -INSERT INTO test.t64 SELECT (1 - toInt64(intExp2(16)) + number) AS x, x, x, x, x, x, x, x FROM numbers(65); -SELECT * FROM test.t64 WHERE i8 != t_i8; -SELECT * FROM test.t64 WHERE i16 != t_i16; -SELECT * FROM test.t64 WHERE i32 != t_i32; -SELECT * FROM test.t64 WHERE i64 != t_i64; +INSERT INTO t64 SELECT (intExp2(16) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(10); +INSERT INTO t64 SELECT (intExp2(16) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(11); +INSERT INTO t64 SELECT (intExp2(16) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(64); +INSERT INTO t64 SELECT (intExp2(16) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(65); +INSERT INTO t64 SELECT (intExp2(16) - 1 + number) AS x, x, x, x, x, x, x, x FROM numbers(65); +INSERT INTO t64 SELECT (10 - toInt64(intExp2(16)) + number) AS x, x, x, x, x, x, x, x FROM numbers(10); +INSERT INTO t64 SELECT (10 - toInt64(intExp2(16)) + number) AS x, x, x, x, x, x, x, x FROM numbers(11); +INSERT INTO t64 SELECT (64 - toInt64(intExp2(16)) + number) AS x, x, x, x, x, x, x, x FROM numbers(64); +INSERT INTO t64 SELECT (64 - toInt64(intExp2(16)) + number) AS x, x, x, x, x, x, x, x FROM numbers(65); +INSERT INTO t64 SELECT (1 - toInt64(intExp2(16)) + number) AS x, x, x, x, x, x, x, x FROM numbers(65); +SELECT * FROM t64 WHERE i8 != t_i8; +SELECT * FROM t64 WHERE i16 != t_i16; +SELECT * FROM t64 WHERE i32 != t_i32; +SELECT * FROM t64 WHERE i64 != t_i64; -INSERT INTO test.t64 SELECT (intExp2(24) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(10); -INSERT INTO test.t64 SELECT (intExp2(24) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(11); -INSERT INTO test.t64 SELECT (intExp2(24) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(128); -INSERT INTO test.t64 SELECT (intExp2(24) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(129); -INSERT INTO test.t64 SELECT (intExp2(24) - 1 + number) AS x, x, x, x, x, x, x, x FROM numbers(129); -INSERT INTO test.t64 SELECT (10 - toInt64(intExp2(24)) + number) AS x, x, x, x, x, x, x, x FROM numbers(10); -INSERT INTO test.t64 SELECT (10 - toInt64(intExp2(24)) + number) AS x, x, x, x, x, x, x, x FROM numbers(11); -INSERT INTO test.t64 SELECT (64 - toInt64(intExp2(24)) + number) AS x, x, x, x, x, x, x, x FROM numbers(128); -INSERT INTO test.t64 SELECT (64 - toInt64(intExp2(24)) + number) AS x, x, x, x, x, x, x, x FROM numbers(129); -INSERT INTO test.t64 SELECT (1 - toInt64(intExp2(24)) + number) AS x, x, x, x, x, x, x, x FROM numbers(129); -SELECT * FROM test.t64 WHERE i8 != t_i8; -SELECT * FROM test.t64 WHERE i16 != t_i16; -SELECT * FROM test.t64 WHERE i32 != t_i32; -SELECT * FROM test.t64 WHERE i64 != t_i64; +INSERT INTO t64 SELECT (intExp2(24) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(10); +INSERT INTO t64 SELECT (intExp2(24) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(11); +INSERT INTO t64 SELECT (intExp2(24) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(128); +INSERT INTO t64 SELECT (intExp2(24) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(129); +INSERT INTO t64 SELECT (intExp2(24) - 1 + number) AS x, x, x, x, x, x, x, x FROM numbers(129); +INSERT INTO t64 SELECT (10 - toInt64(intExp2(24)) + number) AS x, x, x, x, x, x, x, x FROM numbers(10); +INSERT INTO t64 SELECT (10 - toInt64(intExp2(24)) + number) AS x, x, x, x, x, x, x, x FROM numbers(11); +INSERT INTO t64 SELECT (64 - toInt64(intExp2(24)) + number) AS x, x, x, x, x, x, x, x FROM numbers(128); +INSERT INTO t64 SELECT (64 - toInt64(intExp2(24)) + number) AS x, x, x, x, x, x, x, x FROM numbers(129); +INSERT INTO t64 SELECT (1 - toInt64(intExp2(24)) + number) AS x, x, x, x, x, x, x, x FROM numbers(129); +SELECT * FROM t64 WHERE i8 != t_i8; +SELECT * FROM t64 WHERE i16 != t_i16; +SELECT * FROM t64 WHERE i32 != t_i32; +SELECT * FROM t64 WHERE i64 != t_i64; -INSERT INTO test.t64 SELECT (intExp2(32) - 2 + number) AS x, x, x, x, x, x, x, x FROM numbers(2); -INSERT INTO test.t64 SELECT (intExp2(32) - 2 + number) AS x, x, x, x, x, x, x, x FROM numbers(3); -INSERT INTO test.t64 SELECT (intExp2(32) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(64); -INSERT INTO test.t64 SELECT (intExp2(32) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(65); -INSERT INTO test.t64 SELECT (intExp2(32) - 1 + number) AS x, x, x, x, x, x, x, x FROM numbers(65); -INSERT INTO test.t64 SELECT (10 - toInt64(intExp2(32)) + number) AS x, x, x, x, x, x, x, x FROM numbers(10); -INSERT INTO test.t64 SELECT (10 - toInt64(intExp2(32)) + number) AS x, x, x, x, x, x, x, x FROM numbers(11); -INSERT INTO test.t64 SELECT (64 - toInt64(intExp2(32)) + number) AS x, x, x, x, x, x, x, x FROM numbers(64); -INSERT INTO test.t64 SELECT (64 - toInt64(intExp2(32)) + number) AS x, x, x, x, x, x, x, x FROM numbers(65); -INSERT INTO test.t64 SELECT (1 - toInt64(intExp2(32)) + number) AS x, x, x, x, x, x, x, x FROM numbers(65); -SELECT * FROM test.t64 WHERE i8 != t_i8; -SELECT * FROM test.t64 WHERE i16 != t_i16; -SELECT * FROM test.t64 WHERE i32 != t_i32; -SELECT * FROM test.t64 WHERE i64 != t_i64; +INSERT INTO t64 SELECT (intExp2(32) - 2 + number) AS x, x, x, x, x, x, x, x FROM numbers(2); +INSERT INTO t64 SELECT (intExp2(32) - 2 + number) AS x, x, x, x, x, x, x, x FROM numbers(3); +INSERT INTO t64 SELECT (intExp2(32) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(64); +INSERT INTO t64 SELECT (intExp2(32) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(65); +INSERT INTO t64 SELECT (intExp2(32) - 1 + number) AS x, x, x, x, x, x, x, x FROM numbers(65); +INSERT INTO t64 SELECT (10 - toInt64(intExp2(32)) + number) AS x, x, x, x, x, x, x, x FROM numbers(10); +INSERT INTO t64 SELECT (10 - toInt64(intExp2(32)) + number) AS x, x, x, x, x, x, x, x FROM numbers(11); +INSERT INTO t64 SELECT (64 - toInt64(intExp2(32)) + number) AS x, x, x, x, x, x, x, x FROM numbers(64); +INSERT INTO t64 SELECT (64 - toInt64(intExp2(32)) + number) AS x, x, x, x, x, x, x, x FROM numbers(65); +INSERT INTO t64 SELECT (1 - toInt64(intExp2(32)) + number) AS x, x, x, x, x, x, x, x FROM numbers(65); +SELECT * FROM t64 WHERE i8 != t_i8; +SELECT * FROM t64 WHERE i16 != t_i16; +SELECT * FROM t64 WHERE i32 != t_i32; +SELECT * FROM t64 WHERE i64 != t_i64; -INSERT INTO test.t64 SELECT (intExp2(40) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(10); -INSERT INTO test.t64 SELECT (intExp2(40) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(20); -INSERT INTO test.t64 SELECT (intExp2(40) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(512); -INSERT INTO test.t64 SELECT (intExp2(40) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(513); -INSERT INTO test.t64 SELECT (intExp2(40) - 1 + number) AS x, x, x, x, x, x, x, x FROM numbers(513); -INSERT INTO test.t64 SELECT (10 - toInt64(intExp2(40)) + number) AS x, x, x, x, x, x, x, x FROM numbers(10); -INSERT INTO test.t64 SELECT (10 - toInt64(intExp2(40)) + number) AS x, x, x, x, x, x, x, x FROM numbers(20); -INSERT INTO test.t64 SELECT (64 - toInt64(intExp2(40)) + number) AS x, x, x, x, x, x, x, x FROM numbers(512); -INSERT INTO test.t64 SELECT (64 - toInt64(intExp2(40)) + number) AS x, x, x, x, x, x, x, x FROM numbers(513); -INSERT INTO test.t64 SELECT (1 - toInt64(intExp2(40)) + number) AS x, x, x, x, x, x, x, x FROM numbers(513); -SELECT * FROM test.t64 WHERE i8 != t_i8; -SELECT * FROM test.t64 WHERE i16 != t_i16; -SELECT * FROM test.t64 WHERE i32 != t_i32; -SELECT * FROM test.t64 WHERE i64 != t_i64; +INSERT INTO t64 SELECT (intExp2(40) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(10); +INSERT INTO t64 SELECT (intExp2(40) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(20); +INSERT INTO t64 SELECT (intExp2(40) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(512); +INSERT INTO t64 SELECT (intExp2(40) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(513); +INSERT INTO t64 SELECT (intExp2(40) - 1 + number) AS x, x, x, x, x, x, x, x FROM numbers(513); +INSERT INTO t64 SELECT (10 - toInt64(intExp2(40)) + number) AS x, x, x, x, x, x, x, x FROM numbers(10); +INSERT INTO t64 SELECT (10 - toInt64(intExp2(40)) + number) AS x, x, x, x, x, x, x, x FROM numbers(20); +INSERT INTO t64 SELECT (64 - toInt64(intExp2(40)) + number) AS x, x, x, x, x, x, x, x FROM numbers(512); +INSERT INTO t64 SELECT (64 - toInt64(intExp2(40)) + number) AS x, x, x, x, x, x, x, x FROM numbers(513); +INSERT INTO t64 SELECT (1 - toInt64(intExp2(40)) + number) AS x, x, x, x, x, x, x, x FROM numbers(513); +SELECT * FROM t64 WHERE i8 != t_i8; +SELECT * FROM t64 WHERE i16 != t_i16; +SELECT * FROM t64 WHERE i32 != t_i32; +SELECT * FROM t64 WHERE i64 != t_i64; -INSERT INTO test.t64 SELECT (intExp2(48) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(10); -INSERT INTO test.t64 SELECT (intExp2(48) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(20); -INSERT INTO test.t64 SELECT (intExp2(48) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(1024); -INSERT INTO test.t64 SELECT (intExp2(48) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(1025); -INSERT INTO test.t64 SELECT (intExp2(48) - 1 + number) AS x, x, x, x, x, x, x, x FROM numbers(1025); -INSERT INTO test.t64 SELECT (10 - toInt64(intExp2(48)) + number) AS x, x, x, x, x, x, x, x FROM numbers(10); -INSERT INTO test.t64 SELECT (10 - toInt64(intExp2(48)) + number) AS x, x, x, x, x, x, x, x FROM numbers(20); -INSERT INTO test.t64 SELECT (64 - toInt64(intExp2(48)) + number) AS x, x, x, x, x, x, x, x FROM numbers(1024); -INSERT INTO test.t64 SELECT (64 - toInt64(intExp2(48)) + number) AS x, x, x, x, x, x, x, x FROM numbers(1025); -INSERT INTO test.t64 SELECT (1 - toInt64(intExp2(48)) + number) AS x, x, x, x, x, x, x, x FROM numbers(1025); -SELECT * FROM test.t64 WHERE i8 != t_i8; -SELECT * FROM test.t64 WHERE i16 != t_i16; -SELECT * FROM test.t64 WHERE i32 != t_i32; -SELECT * FROM test.t64 WHERE i64 != t_i64; +INSERT INTO t64 SELECT (intExp2(48) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(10); +INSERT INTO t64 SELECT (intExp2(48) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(20); +INSERT INTO t64 SELECT (intExp2(48) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(1024); +INSERT INTO t64 SELECT (intExp2(48) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(1025); +INSERT INTO t64 SELECT (intExp2(48) - 1 + number) AS x, x, x, x, x, x, x, x FROM numbers(1025); +INSERT INTO t64 SELECT (10 - toInt64(intExp2(48)) + number) AS x, x, x, x, x, x, x, x FROM numbers(10); +INSERT INTO t64 SELECT (10 - toInt64(intExp2(48)) + number) AS x, x, x, x, x, x, x, x FROM numbers(20); +INSERT INTO t64 SELECT (64 - toInt64(intExp2(48)) + number) AS x, x, x, x, x, x, x, x FROM numbers(1024); +INSERT INTO t64 SELECT (64 - toInt64(intExp2(48)) + number) AS x, x, x, x, x, x, x, x FROM numbers(1025); +INSERT INTO t64 SELECT (1 - toInt64(intExp2(48)) + number) AS x, x, x, x, x, x, x, x FROM numbers(1025); +SELECT * FROM t64 WHERE i8 != t_i8; +SELECT * FROM t64 WHERE i16 != t_i16; +SELECT * FROM t64 WHERE i32 != t_i32; +SELECT * FROM t64 WHERE i64 != t_i64; -INSERT INTO test.t64 SELECT (intExp2(56) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(10); -INSERT INTO test.t64 SELECT (intExp2(56) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(20); -INSERT INTO test.t64 SELECT (intExp2(56) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(2048); -INSERT INTO test.t64 SELECT (intExp2(56) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(2049); -INSERT INTO test.t64 SELECT (intExp2(56) - 1 + number) AS x, x, x, x, x, x, x, x FROM numbers(2049); -INSERT INTO test.t64 SELECT (10 - toInt64(intExp2(56)) + number) AS x, x, x, x, x, x, x, x FROM numbers(10); -INSERT INTO test.t64 SELECT (10 - toInt64(intExp2(56)) + number) AS x, x, x, x, x, x, x, x FROM numbers(20); -INSERT INTO test.t64 SELECT (64 - toInt64(intExp2(56)) + number) AS x, x, x, x, x, x, x, x FROM numbers(2048); -INSERT INTO test.t64 SELECT (64 - toInt64(intExp2(56)) + number) AS x, x, x, x, x, x, x, x FROM numbers(2049); -INSERT INTO test.t64 SELECT (1 - toInt64(intExp2(56)) + number) AS x, x, x, x, x, x, x, x FROM numbers(2049); -SELECT * FROM test.t64 WHERE i8 != t_i8; -SELECT * FROM test.t64 WHERE i16 != t_i16; -SELECT * FROM test.t64 WHERE i32 != t_i32; -SELECT * FROM test.t64 WHERE i64 != t_i64; +INSERT INTO t64 SELECT (intExp2(56) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(10); +INSERT INTO t64 SELECT (intExp2(56) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(20); +INSERT INTO t64 SELECT (intExp2(56) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(2048); +INSERT INTO t64 SELECT (intExp2(56) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(2049); +INSERT INTO t64 SELECT (intExp2(56) - 1 + number) AS x, x, x, x, x, x, x, x FROM numbers(2049); +INSERT INTO t64 SELECT (10 - toInt64(intExp2(56)) + number) AS x, x, x, x, x, x, x, x FROM numbers(10); +INSERT INTO t64 SELECT (10 - toInt64(intExp2(56)) + number) AS x, x, x, x, x, x, x, x FROM numbers(20); +INSERT INTO t64 SELECT (64 - toInt64(intExp2(56)) + number) AS x, x, x, x, x, x, x, x FROM numbers(2048); +INSERT INTO t64 SELECT (64 - toInt64(intExp2(56)) + number) AS x, x, x, x, x, x, x, x FROM numbers(2049); +INSERT INTO t64 SELECT (1 - toInt64(intExp2(56)) + number) AS x, x, x, x, x, x, x, x FROM numbers(2049); +SELECT * FROM t64 WHERE i8 != t_i8; +SELECT * FROM t64 WHERE i16 != t_i16; +SELECT * FROM t64 WHERE i32 != t_i32; +SELECT * FROM t64 WHERE i64 != t_i64; -OPTIMIZE TABLE test.t64 FINAL; +OPTIMIZE TABLE t64 FINAL; -SELECT * FROM test.t64 WHERE i8 != t_i8; -SELECT * FROM test.t64 WHERE i16 != t_i16; -SELECT * FROM test.t64 WHERE i32 != t_i32; -SELECT * FROM test.t64 WHERE i64 != t_i64; +SELECT * FROM t64 WHERE i8 != t_i8; +SELECT * FROM t64 WHERE i16 != t_i16; +SELECT * FROM t64 WHERE i32 != t_i32; +SELECT * FROM t64 WHERE i64 != t_i64; -DROP TABLE test.t64; +DROP TABLE t64; diff --git a/dbms/tests/queries/0_stateless/00872_t64_bit_codec.sql b/dbms/tests/queries/0_stateless/00872_t64_bit_codec.sql index d173b84def0..475167b2d29 100644 --- a/dbms/tests/queries/0_stateless/00872_t64_bit_codec.sql +++ b/dbms/tests/queries/0_stateless/00872_t64_bit_codec.sql @@ -1,6 +1,6 @@ -DROP TABLE IF EXISTS test.t64; +DROP TABLE IF EXISTS t64; -CREATE TABLE test.t64 +CREATE TABLE t64 ( u8 UInt8, t_u8 UInt8 Codec(T64('bit'), LZ4), @@ -12,90 +12,90 @@ CREATE TABLE test.t64 t_u64 UInt64 Codec(T64('bit'), LZ4) ) ENGINE MergeTree() ORDER BY tuple(); -INSERT INTO test.t64 SELECT number AS x, x, x, x, x, x, x, x FROM numbers(1); -INSERT INTO test.t64 SELECT number AS x, x, x, x, x, x, x, x FROM numbers(2); -INSERT INTO test.t64 SELECT 42 AS x, x, x, x, x, x, x, x FROM numbers(4); +INSERT INTO t64 SELECT number AS x, x, x, x, x, x, x, x FROM numbers(1); +INSERT INTO t64 SELECT number AS x, x, x, x, x, x, x, x FROM numbers(2); +INSERT INTO t64 SELECT 42 AS x, x, x, x, x, x, x, x FROM numbers(4); -SELECT * FROM test.t64 ORDER BY u64; +SELECT * FROM t64 ORDER BY u64; -INSERT INTO test.t64 SELECT number AS x, x, x, x, x, x, x, x FROM numbers(intExp2(8)); -INSERT INTO test.t64 SELECT number AS x, x, x, x, x, x, x, x FROM numbers(intExp2(9)); -SELECT * FROM test.t64 WHERE u8 != t_u8; -SELECT * FROM test.t64 WHERE u16 != t_u16; -SELECT * FROM test.t64 WHERE u32 != t_u32; -SELECT * FROM test.t64 WHERE u64 != t_u64; +INSERT INTO t64 SELECT number AS x, x, x, x, x, x, x, x FROM numbers(intExp2(8)); +INSERT INTO t64 SELECT number AS x, x, x, x, x, x, x, x FROM numbers(intExp2(9)); +SELECT * FROM t64 WHERE u8 != t_u8; +SELECT * FROM t64 WHERE u16 != t_u16; +SELECT * FROM t64 WHERE u32 != t_u32; +SELECT * FROM t64 WHERE u64 != t_u64; -INSERT INTO test.t64 SELECT (intExp2(16) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(10); -INSERT INTO test.t64 SELECT (intExp2(16) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(11); -INSERT INTO test.t64 SELECT (intExp2(16) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(64); -INSERT INTO test.t64 SELECT (intExp2(16) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(65); -INSERT INTO test.t64 SELECT (intExp2(16) - 1 + number) AS x, x, x, x, x, x, x, x FROM numbers(65); -SELECT * FROM test.t64 WHERE u8 != t_u8; -SELECT * FROM test.t64 WHERE u16 != t_u16; -SELECT * FROM test.t64 WHERE u32 != t_u32; -SELECT * FROM test.t64 WHERE u64 != t_u64; +INSERT INTO t64 SELECT (intExp2(16) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(10); +INSERT INTO t64 SELECT (intExp2(16) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(11); +INSERT INTO t64 SELECT (intExp2(16) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(64); +INSERT INTO t64 SELECT (intExp2(16) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(65); +INSERT INTO t64 SELECT (intExp2(16) - 1 + number) AS x, x, x, x, x, x, x, x FROM numbers(65); +SELECT * FROM t64 WHERE u8 != t_u8; +SELECT * FROM t64 WHERE u16 != t_u16; +SELECT * FROM t64 WHERE u32 != t_u32; +SELECT * FROM t64 WHERE u64 != t_u64; -INSERT INTO test.t64 SELECT (intExp2(24) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(10); -INSERT INTO test.t64 SELECT (intExp2(24) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(11); -INSERT INTO test.t64 SELECT (intExp2(24) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(128); -INSERT INTO test.t64 SELECT (intExp2(24) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(129); -INSERT INTO test.t64 SELECT (intExp2(24) - 1 + number) AS x, x, x, x, x, x, x, x FROM numbers(129); -SELECT * FROM test.t64 WHERE u8 != t_u8; -SELECT * FROM test.t64 WHERE u16 != t_u16; -SELECT * FROM test.t64 WHERE u32 != t_u32; -SELECT * FROM test.t64 WHERE u64 != t_u64; +INSERT INTO t64 SELECT (intExp2(24) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(10); +INSERT INTO t64 SELECT (intExp2(24) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(11); +INSERT INTO t64 SELECT (intExp2(24) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(128); +INSERT INTO t64 SELECT (intExp2(24) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(129); +INSERT INTO t64 SELECT (intExp2(24) - 1 + number) AS x, x, x, x, x, x, x, x FROM numbers(129); +SELECT * FROM t64 WHERE u8 != t_u8; +SELECT * FROM t64 WHERE u16 != t_u16; +SELECT * FROM t64 WHERE u32 != t_u32; +SELECT * FROM t64 WHERE u64 != t_u64; -INSERT INTO test.t64 SELECT (intExp2(32) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(10); -INSERT INTO test.t64 SELECT (intExp2(32) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(20); -INSERT INTO test.t64 SELECT (intExp2(32) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(256); -INSERT INTO test.t64 SELECT (intExp2(32) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(257); -INSERT INTO test.t64 SELECT (intExp2(32) - 1 + number) AS x, x, x, x, x, x, x, x FROM numbers(257); -SELECT * FROM test.t64 WHERE u8 != t_u8; -SELECT * FROM test.t64 WHERE u16 != t_u16; -SELECT * FROM test.t64 WHERE u32 != t_u32; -SELECT * FROM test.t64 WHERE u64 != t_u64; +INSERT INTO t64 SELECT (intExp2(32) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(10); +INSERT INTO t64 SELECT (intExp2(32) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(20); +INSERT INTO t64 SELECT (intExp2(32) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(256); +INSERT INTO t64 SELECT (intExp2(32) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(257); +INSERT INTO t64 SELECT (intExp2(32) - 1 + number) AS x, x, x, x, x, x, x, x FROM numbers(257); +SELECT * FROM t64 WHERE u8 != t_u8; +SELECT * FROM t64 WHERE u16 != t_u16; +SELECT * FROM t64 WHERE u32 != t_u32; +SELECT * FROM t64 WHERE u64 != t_u64; -INSERT INTO test.t64 SELECT (intExp2(40) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(10); -INSERT INTO test.t64 SELECT (intExp2(40) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(20); -INSERT INTO test.t64 SELECT (intExp2(40) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(512); -INSERT INTO test.t64 SELECT (intExp2(40) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(513); -INSERT INTO test.t64 SELECT (intExp2(40) - 1 + number) AS x, x, x, x, x, x, x, x FROM numbers(513); -SELECT * FROM test.t64 WHERE u8 != t_u8; -SELECT * FROM test.t64 WHERE u16 != t_u16; -SELECT * FROM test.t64 WHERE u32 != t_u32; -SELECT * FROM test.t64 WHERE u64 != t_u64; +INSERT INTO t64 SELECT (intExp2(40) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(10); +INSERT INTO t64 SELECT (intExp2(40) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(20); +INSERT INTO t64 SELECT (intExp2(40) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(512); +INSERT INTO t64 SELECT (intExp2(40) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(513); +INSERT INTO t64 SELECT (intExp2(40) - 1 + number) AS x, x, x, x, x, x, x, x FROM numbers(513); +SELECT * FROM t64 WHERE u8 != t_u8; +SELECT * FROM t64 WHERE u16 != t_u16; +SELECT * FROM t64 WHERE u32 != t_u32; +SELECT * FROM t64 WHERE u64 != t_u64; -INSERT INTO test.t64 SELECT (intExp2(48) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(10); -INSERT INTO test.t64 SELECT (intExp2(48) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(20); -INSERT INTO test.t64 SELECT (intExp2(48) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(1024); -INSERT INTO test.t64 SELECT (intExp2(48) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(1025); -INSERT INTO test.t64 SELECT (intExp2(48) - 1 + number) AS x, x, x, x, x, x, x, x FROM numbers(1025); -SELECT * FROM test.t64 WHERE u8 != t_u8; -SELECT * FROM test.t64 WHERE u16 != t_u16; -SELECT * FROM test.t64 WHERE u32 != t_u32; -SELECT * FROM test.t64 WHERE u64 != t_u64; +INSERT INTO t64 SELECT (intExp2(48) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(10); +INSERT INTO t64 SELECT (intExp2(48) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(20); +INSERT INTO t64 SELECT (intExp2(48) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(1024); +INSERT INTO t64 SELECT (intExp2(48) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(1025); +INSERT INTO t64 SELECT (intExp2(48) - 1 + number) AS x, x, x, x, x, x, x, x FROM numbers(1025); +SELECT * FROM t64 WHERE u8 != t_u8; +SELECT * FROM t64 WHERE u16 != t_u16; +SELECT * FROM t64 WHERE u32 != t_u32; +SELECT * FROM t64 WHERE u64 != t_u64; -INSERT INTO test.t64 SELECT (intExp2(56) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(10); -INSERT INTO test.t64 SELECT (intExp2(56) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(20); -INSERT INTO test.t64 SELECT (intExp2(56) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(2048); -INSERT INTO test.t64 SELECT (intExp2(56) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(2049); -INSERT INTO test.t64 SELECT (intExp2(56) - 1 + number) AS x, x, x, x, x, x, x, x FROM numbers(2049); -SELECT * FROM test.t64 WHERE u8 != t_u8; -SELECT * FROM test.t64 WHERE u16 != t_u16; -SELECT * FROM test.t64 WHERE u32 != t_u32; -SELECT * FROM test.t64 WHERE u64 != t_u64; +INSERT INTO t64 SELECT (intExp2(56) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(10); +INSERT INTO t64 SELECT (intExp2(56) - 10 + number) AS x, x, x, x, x, x, x, x FROM numbers(20); +INSERT INTO t64 SELECT (intExp2(56) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(2048); +INSERT INTO t64 SELECT (intExp2(56) - 64 + number) AS x, x, x, x, x, x, x, x FROM numbers(2049); +INSERT INTO t64 SELECT (intExp2(56) - 1 + number) AS x, x, x, x, x, x, x, x FROM numbers(2049); +SELECT * FROM t64 WHERE u8 != t_u8; +SELECT * FROM t64 WHERE u16 != t_u16; +SELECT * FROM t64 WHERE u32 != t_u32; +SELECT * FROM t64 WHERE u64 != t_u64; -INSERT INTO test.t64 SELECT (intExp2(63) + number * intExp2(62)) AS x, x, x, x, x, x, x, x FROM numbers(10); -SELECT * FROM test.t64 WHERE u8 != t_u8; -SELECT * FROM test.t64 WHERE u16 != t_u16; -SELECT * FROM test.t64 WHERE u32 != t_u32; -SELECT * FROM test.t64 WHERE u64 != t_u64; +INSERT INTO t64 SELECT (intExp2(63) + number * intExp2(62)) AS x, x, x, x, x, x, x, x FROM numbers(10); +SELECT * FROM t64 WHERE u8 != t_u8; +SELECT * FROM t64 WHERE u16 != t_u16; +SELECT * FROM t64 WHERE u32 != t_u32; +SELECT * FROM t64 WHERE u64 != t_u64; -OPTIMIZE TABLE test.t64 FINAL; +OPTIMIZE TABLE t64 FINAL; -SELECT * FROM test.t64 WHERE u8 != t_u8; -SELECT * FROM test.t64 WHERE u16 != t_u16; -SELECT * FROM test.t64 WHERE u32 != t_u32; -SELECT * FROM test.t64 WHERE u64 != t_u64; +SELECT * FROM t64 WHERE u8 != t_u8; +SELECT * FROM t64 WHERE u16 != t_u16; +SELECT * FROM t64 WHERE u32 != t_u32; +SELECT * FROM t64 WHERE u64 != t_u64; -DROP TABLE test.t64; +DROP TABLE t64; diff --git a/dbms/tests/queries/0_stateless/00921_datetime64_basic.sql b/dbms/tests/queries/0_stateless/00921_datetime64_basic.sql index 8d46788f1e5..2d7cb975cfc 100644 --- a/dbms/tests/queries/0_stateless/00921_datetime64_basic.sql +++ b/dbms/tests/queries/0_stateless/00921_datetime64_basic.sql @@ -1,5 +1,3 @@ -USE test; - DROP TABLE IF EXISTS A; SELECT CAST(1 as DateTime64('abc')); -- { serverError 43 } # Invalid scale parameter type @@ -25,4 +23,4 @@ SELECT toString(t, 'UTC'), toDate(t), toStartOfDay(t), toStartOfQuarter(t), toTi SELECT toDateTime64('2019-09-16 19:20:11.234', 3, 'Europe/Minsk'); -DROP TABLE A; \ No newline at end of file +DROP TABLE A; diff --git a/dbms/tests/queries/0_stateless/00926_geo_to_h3.sql b/dbms/tests/queries/0_stateless/00926_geo_to_h3.sql index d3ce898c56a..a8315eb7ea6 100644 --- a/dbms/tests/queries/0_stateless/00926_geo_to_h3.sql +++ b/dbms/tests/queries/0_stateless/00926_geo_to_h3.sql @@ -1,5 +1,3 @@ -USE test; - DROP TABLE IF EXISTS table1; CREATE TABLE table1 (lat Float64, lon Float64, resolution UInt8) ENGINE = Memory; diff --git a/dbms/tests/queries/0_stateless/00936_crc_functions.sql b/dbms/tests/queries/0_stateless/00936_crc_functions.sql index fd324ea23fa..4794062b52e 100644 --- a/dbms/tests/queries/0_stateless/00936_crc_functions.sql +++ b/dbms/tests/queries/0_stateless/00936_crc_functions.sql @@ -1,5 +1,3 @@ -USE test; - DROP TABLE IF EXISTS table1; CREATE TABLE table1 (str1 String, str2 String) ENGINE = Memory; diff --git a/dbms/tests/queries/0_stateless/00940_order_by_read_in_order.sql b/dbms/tests/queries/0_stateless/00940_order_by_read_in_order.sql index 34c0d16716e..df872033d70 100644 --- a/dbms/tests/queries/0_stateless/00940_order_by_read_in_order.sql +++ b/dbms/tests/queries/0_stateless/00940_order_by_read_in_order.sql @@ -28,10 +28,10 @@ SELECT a, b, c FROM pk_order ORDER BY a DESC, b DESC, c DESC; DROP TABLE IF EXISTS pk_order; -CREATE TABLE pk_order (d DateTime, a Int32, b Int32) ENGINE = MergeTree ORDER BY (d, a) +CREATE TABLE pk_order (d DateTime, a Int32, b Int32) ENGINE = MergeTree ORDER BY (d, a) PARTITION BY toDate(d) SETTINGS index_granularity=1; -INSERT INTO pk_order +INSERT INTO pk_order SELECT toDateTime('2019-05-05 00:00:00') + INTERVAL number % 10 DAY, number, intHash32(number) from numbers(100); set max_block_size = 1; @@ -52,4 +52,4 @@ CREATE TABLE pk_order (a Int, b Int) ENGINE = MergeTree ORDER BY (a / b); INSERT INTO pk_order SELECT number % 10 + 1, number % 6 + 1 from numbers(100); SELECT * FROM pk_order ORDER BY (a / b), a LIMIT 5; - +DROP TABLE pk_order; diff --git a/dbms/tests/queries/0_stateless/00945_bloom_filter_index.sql b/dbms/tests/queries/0_stateless/00945_bloom_filter_index.sql index b306f2ed7ed..82549a5d395 100755 --- a/dbms/tests/queries/0_stateless/00945_bloom_filter_index.sql +++ b/dbms/tests/queries/0_stateless/00945_bloom_filter_index.sql @@ -1,265 +1,265 @@ -DROP TABLE IF EXISTS test.single_column_bloom_filter; +DROP TABLE IF EXISTS single_column_bloom_filter; -CREATE TABLE test.single_column_bloom_filter (u64 UInt64, i32 Int32, i64 UInt64, INDEX idx (i32) TYPE bloom_filter GRANULARITY 1) ENGINE = MergeTree() ORDER BY u64 SETTINGS index_granularity = 6; +CREATE TABLE single_column_bloom_filter (u64 UInt64, i32 Int32, i64 UInt64, INDEX idx (i32) TYPE bloom_filter GRANULARITY 1) ENGINE = MergeTree() ORDER BY u64 SETTINGS index_granularity = 6; -INSERT INTO test.single_column_bloom_filter SELECT number AS u64, number AS i32, number AS i64 FROM system.numbers LIMIT 100; +INSERT INTO single_column_bloom_filter SELECT number AS u64, number AS i32, number AS i64 FROM system.numbers LIMIT 100; -SELECT COUNT() FROM test.single_column_bloom_filter WHERE i32 = 1 SETTINGS max_rows_to_read = 6; -SELECT COUNT() FROM test.single_column_bloom_filter WHERE (i32, i32) = (1, 2) SETTINGS max_rows_to_read = 6; -SELECT COUNT() FROM test.single_column_bloom_filter WHERE (i32, i64) = (1, 1) SETTINGS max_rows_to_read = 6; -SELECT COUNT() FROM test.single_column_bloom_filter WHERE (i64, (i64, i32)) = (1, (1, 1)) SETTINGS max_rows_to_read = 6; +SELECT COUNT() FROM single_column_bloom_filter WHERE i32 = 1 SETTINGS max_rows_to_read = 6; +SELECT COUNT() FROM single_column_bloom_filter WHERE (i32, i32) = (1, 2) SETTINGS max_rows_to_read = 6; +SELECT COUNT() FROM single_column_bloom_filter WHERE (i32, i64) = (1, 1) SETTINGS max_rows_to_read = 6; +SELECT COUNT() FROM single_column_bloom_filter WHERE (i64, (i64, i32)) = (1, (1, 1)) SETTINGS max_rows_to_read = 6; -SELECT COUNT() FROM test.single_column_bloom_filter WHERE i32 IN (1, 2) SETTINGS max_rows_to_read = 6; -SELECT COUNT() FROM test.single_column_bloom_filter WHERE (i32, i32) IN ((1, 2), (2, 3)) SETTINGS max_rows_to_read = 6; -SELECT COUNT() FROM test.single_column_bloom_filter WHERE (i32, i64) IN ((1, 1), (2, 2)) SETTINGS max_rows_to_read = 6; -SELECT COUNT() FROM test.single_column_bloom_filter WHERE (i64, (i64, i32)) IN ((1, (1, 1)), (2, (2, 2))) SETTINGS max_rows_to_read = 6; -SELECT COUNT() FROM test.single_column_bloom_filter WHERE i32 IN (SELECT arrayJoin([toInt32(1), toInt32(2)])) SETTINGS max_rows_to_read = 6; -SELECT COUNT() FROM test.single_column_bloom_filter WHERE (i32, i32) IN (SELECT arrayJoin([(toInt32(1), toInt32(2)), (toInt32(2), toInt32(3))])) SETTINGS max_rows_to_read = 6; -SELECT COUNT() FROM test.single_column_bloom_filter WHERE (i32, i64) IN (SELECT arrayJoin([(toInt32(1), toUInt64(1)), (toInt32(2), toUInt64(2))])) SETTINGS max_rows_to_read = 6; -SELECT COUNT() FROM test.single_column_bloom_filter WHERE (i64, (i64, i32)) IN (SELECT arrayJoin([(toUInt64(1), (toUInt64(1), toInt32(1))), (toUInt64(2), (toUInt64(2), toInt32(2)))])) SETTINGS max_rows_to_read = 6; -WITH (1, 2) AS liter_prepared_set SELECT COUNT() FROM test.single_column_bloom_filter WHERE i32 IN liter_prepared_set SETTINGS max_rows_to_read = 6; -WITH ((1, 2), (2, 3)) AS liter_prepared_set SELECT COUNT() FROM test.single_column_bloom_filter WHERE (i32, i32) IN liter_prepared_set SETTINGS max_rows_to_read = 6; -WITH ((1, 1), (2, 2)) AS liter_prepared_set SELECT COUNT() FROM test.single_column_bloom_filter WHERE (i32, i64) IN liter_prepared_set SETTINGS max_rows_to_read = 6; -WITH ((1, (1, 1)), (2, (2, 2))) AS liter_prepared_set SELECT COUNT() FROM test.single_column_bloom_filter WHERE (i64, (i64, i32)) IN liter_prepared_set SETTINGS max_rows_to_read = 6; +SELECT COUNT() FROM single_column_bloom_filter WHERE i32 IN (1, 2) SETTINGS max_rows_to_read = 6; +SELECT COUNT() FROM single_column_bloom_filter WHERE (i32, i32) IN ((1, 2), (2, 3)) SETTINGS max_rows_to_read = 6; +SELECT COUNT() FROM single_column_bloom_filter WHERE (i32, i64) IN ((1, 1), (2, 2)) SETTINGS max_rows_to_read = 6; +SELECT COUNT() FROM single_column_bloom_filter WHERE (i64, (i64, i32)) IN ((1, (1, 1)), (2, (2, 2))) SETTINGS max_rows_to_read = 6; +SELECT COUNT() FROM single_column_bloom_filter WHERE i32 IN (SELECT arrayJoin([toInt32(1), toInt32(2)])) SETTINGS max_rows_to_read = 6; +SELECT COUNT() FROM single_column_bloom_filter WHERE (i32, i32) IN (SELECT arrayJoin([(toInt32(1), toInt32(2)), (toInt32(2), toInt32(3))])) SETTINGS max_rows_to_read = 6; +SELECT COUNT() FROM single_column_bloom_filter WHERE (i32, i64) IN (SELECT arrayJoin([(toInt32(1), toUInt64(1)), (toInt32(2), toUInt64(2))])) SETTINGS max_rows_to_read = 6; +SELECT COUNT() FROM single_column_bloom_filter WHERE (i64, (i64, i32)) IN (SELECT arrayJoin([(toUInt64(1), (toUInt64(1), toInt32(1))), (toUInt64(2), (toUInt64(2), toInt32(2)))])) SETTINGS max_rows_to_read = 6; +WITH (1, 2) AS liter_prepared_set SELECT COUNT() FROM single_column_bloom_filter WHERE i32 IN liter_prepared_set SETTINGS max_rows_to_read = 6; +WITH ((1, 2), (2, 3)) AS liter_prepared_set SELECT COUNT() FROM single_column_bloom_filter WHERE (i32, i32) IN liter_prepared_set SETTINGS max_rows_to_read = 6; +WITH ((1, 1), (2, 2)) AS liter_prepared_set SELECT COUNT() FROM single_column_bloom_filter WHERE (i32, i64) IN liter_prepared_set SETTINGS max_rows_to_read = 6; +WITH ((1, (1, 1)), (2, (2, 2))) AS liter_prepared_set SELECT COUNT() FROM single_column_bloom_filter WHERE (i64, (i64, i32)) IN liter_prepared_set SETTINGS max_rows_to_read = 6; -DROP TABLE IF EXISTS test.single_column_bloom_filter; +DROP TABLE IF EXISTS single_column_bloom_filter; -DROP TABLE IF EXISTS test.bloom_filter_types_test; +DROP TABLE IF EXISTS bloom_filter_types_test; -CREATE TABLE test.bloom_filter_types_test (order_key UInt64, i8 Int8, i16 Int16, i32 Int32, i64 Int64, u8 UInt8, u16 UInt16, u32 UInt32, u64 UInt64, f32 Float32, f64 Float64, date Date, date_time DateTime('Europe/Moscow'), str String, fixed_string FixedString(5), INDEX idx (i8, i16, i32, i64, u8, u16, u32, u64, f32, f64, date, date_time, str, fixed_string) TYPE bloom_filter GRANULARITY 1) ENGINE = MergeTree() ORDER BY order_key SETTINGS index_granularity = 6; -INSERT INTO test.bloom_filter_types_test SELECT number AS order_key, toInt8(number) AS i8, toInt16(number) AS i16, toInt32(number) AS i32, toInt64(number) AS i64, toUInt8(number) AS u8, toUInt16(number) AS u16, toUInt32(number) AS u32, toUInt64(number) AS u64, toFloat32(number) AS f32, toFloat64(number) AS f64, toDate(number, 'Europe/Moscow') AS date, toDateTime(number, 'Europe/Moscow') AS date_time, toString(number) AS str, toFixedString(toString(number), 5) AS fixed_string FROM system.numbers LIMIT 100; +CREATE TABLE bloom_filter_types_test (order_key UInt64, i8 Int8, i16 Int16, i32 Int32, i64 Int64, u8 UInt8, u16 UInt16, u32 UInt32, u64 UInt64, f32 Float32, f64 Float64, date Date, date_time DateTime('Europe/Moscow'), str String, fixed_string FixedString(5), INDEX idx (i8, i16, i32, i64, u8, u16, u32, u64, f32, f64, date, date_time, str, fixed_string) TYPE bloom_filter GRANULARITY 1) ENGINE = MergeTree() ORDER BY order_key SETTINGS index_granularity = 6; +INSERT INTO bloom_filter_types_test SELECT number AS order_key, toInt8(number) AS i8, toInt16(number) AS i16, toInt32(number) AS i32, toInt64(number) AS i64, toUInt8(number) AS u8, toUInt16(number) AS u16, toUInt32(number) AS u32, toUInt64(number) AS u64, toFloat32(number) AS f32, toFloat64(number) AS f64, toDate(number, 'Europe/Moscow') AS date, toDateTime(number, 'Europe/Moscow') AS date_time, toString(number) AS str, toFixedString(toString(number), 5) AS fixed_string FROM system.numbers LIMIT 100; -SELECT COUNT() FROM test.bloom_filter_types_test WHERE i8 = 1 SETTINGS max_rows_to_read = 6; -SELECT COUNT() FROM test.bloom_filter_types_test WHERE i16 = 1 SETTINGS max_rows_to_read = 6; -SELECT COUNT() FROM test.bloom_filter_types_test WHERE i32 = 1 SETTINGS max_rows_to_read = 6; -SELECT COUNT() FROM test.bloom_filter_types_test WHERE i64 = 1 SETTINGS max_rows_to_read = 6; -SELECT COUNT() FROM test.bloom_filter_types_test WHERE u8 = 1 SETTINGS max_rows_to_read = 6; -SELECT COUNT() FROM test.bloom_filter_types_test WHERE u16 = 1 SETTINGS max_rows_to_read = 6; -SELECT COUNT() FROM test.bloom_filter_types_test WHERE u32 = 1 SETTINGS max_rows_to_read = 6; -SELECT COUNT() FROM test.bloom_filter_types_test WHERE u64 = 1 SETTINGS max_rows_to_read = 6; -SELECT COUNT() FROM test.bloom_filter_types_test WHERE f32 = 1 SETTINGS max_rows_to_read = 6; -SELECT COUNT() FROM test.bloom_filter_types_test WHERE f64 = 1 SETTINGS max_rows_to_read = 6; -SELECT COUNT() FROM test.bloom_filter_types_test WHERE date = '1970-01-02' SETTINGS max_rows_to_read = 6; -SELECT COUNT() FROM test.bloom_filter_types_test WHERE date_time = toDateTime('1970-01-01 03:00:01', 'Europe/Moscow') SETTINGS max_rows_to_read = 6; -SELECT COUNT() FROM test.bloom_filter_types_test WHERE str = '1' SETTINGS max_rows_to_read = 6; -SELECT COUNT() FROM test.bloom_filter_types_test WHERE fixed_string = toFixedString('1', 5) SETTINGS max_rows_to_read = 12; +SELECT COUNT() FROM bloom_filter_types_test WHERE i8 = 1 SETTINGS max_rows_to_read = 6; +SELECT COUNT() FROM bloom_filter_types_test WHERE i16 = 1 SETTINGS max_rows_to_read = 6; +SELECT COUNT() FROM bloom_filter_types_test WHERE i32 = 1 SETTINGS max_rows_to_read = 6; +SELECT COUNT() FROM bloom_filter_types_test WHERE i64 = 1 SETTINGS max_rows_to_read = 6; +SELECT COUNT() FROM bloom_filter_types_test WHERE u8 = 1 SETTINGS max_rows_to_read = 6; +SELECT COUNT() FROM bloom_filter_types_test WHERE u16 = 1 SETTINGS max_rows_to_read = 6; +SELECT COUNT() FROM bloom_filter_types_test WHERE u32 = 1 SETTINGS max_rows_to_read = 6; +SELECT COUNT() FROM bloom_filter_types_test WHERE u64 = 1 SETTINGS max_rows_to_read = 6; +SELECT COUNT() FROM bloom_filter_types_test WHERE f32 = 1 SETTINGS max_rows_to_read = 6; +SELECT COUNT() FROM bloom_filter_types_test WHERE f64 = 1 SETTINGS max_rows_to_read = 6; +SELECT COUNT() FROM bloom_filter_types_test WHERE date = '1970-01-02' SETTINGS max_rows_to_read = 6; +SELECT COUNT() FROM bloom_filter_types_test WHERE date_time = toDateTime('1970-01-01 03:00:01', 'Europe/Moscow') SETTINGS max_rows_to_read = 6; +SELECT COUNT() FROM bloom_filter_types_test WHERE str = '1' SETTINGS max_rows_to_read = 6; +SELECT COUNT() FROM bloom_filter_types_test WHERE fixed_string = toFixedString('1', 5) SETTINGS max_rows_to_read = 12; -SELECT COUNT() FROM test.bloom_filter_types_test WHERE str IN ( SELECT str FROM test.bloom_filter_types_test); +SELECT COUNT() FROM bloom_filter_types_test WHERE str IN ( SELECT str FROM bloom_filter_types_test); -DROP TABLE IF EXISTS test.bloom_filter_types_test; +DROP TABLE IF EXISTS bloom_filter_types_test; -DROP TABLE IF EXISTS test.bloom_filter_array_types_test; +DROP TABLE IF EXISTS bloom_filter_array_types_test; -CREATE TABLE test.bloom_filter_array_types_test (order_key Array(UInt64), i8 Array(Int8), i16 Array(Int16), i32 Array(Int32), i64 Array(Int64), u8 Array(UInt8), u16 Array(UInt16), u32 Array(UInt32), u64 Array(UInt64), f32 Array(Float32), f64 Array(Float64), date Array(Date), date_time Array(DateTime('Europe/Moscow')), str Array(String), fixed_string Array(FixedString(5)), INDEX idx (i8, i16, i32, i64, u8, u16, u32, u64, f32, f64, date, date_time, str, fixed_string) TYPE bloom_filter GRANULARITY 1) ENGINE = MergeTree() ORDER BY order_key SETTINGS index_granularity = 6; -INSERT INTO test.bloom_filter_array_types_test SELECT groupArray(number) AS order_key, groupArray(toInt8(number)) AS i8, groupArray(toInt16(number)) AS i16, groupArray(toInt32(number)) AS i32, groupArray(toInt64(number)) AS i64, groupArray(toUInt8(number)) AS u8, groupArray(toUInt16(number)) AS u16, groupArray(toUInt32(number)) AS u32, groupArray(toUInt64(number)) AS u64, groupArray(toFloat32(number)) AS f32, groupArray(toFloat64(number)) AS f64, groupArray(toDate(number, 'Europe/Moscow')) AS date, groupArray(toDateTime(number, 'Europe/Moscow')) AS date_time, groupArray(toString(number)) AS str, groupArray(toFixedString(toString(number), 5)) AS fixed_string FROM (SELECT number FROM system.numbers LIMIT 15); -INSERT INTO test.bloom_filter_array_types_test SELECT groupArray(number) AS order_key, groupArray(toInt8(number)) AS i8, groupArray(toInt16(number)) AS i16, groupArray(toInt32(number)) AS i32, groupArray(toInt64(number)) AS i64, groupArray(toUInt8(number)) AS u8, groupArray(toUInt16(number)) AS u16, groupArray(toUInt32(number)) AS u32, groupArray(toUInt64(number)) AS u64, groupArray(toFloat32(number)) AS f32, groupArray(toFloat64(number)) AS f64, groupArray(toDate(number, 'Europe/Moscow')) AS date, groupArray(toDateTime(number, 'Europe/Moscow')) AS date_time, groupArray(toString(number)) AS str, groupArray(toFixedString(toString(number), 5)) AS fixed_string FROM (SELECT number FROM system.numbers WHERE number >= 5 LIMIT 15); -INSERT INTO test.bloom_filter_array_types_test SELECT groupArray(number) AS order_key, groupArray(toInt8(number)) AS i8, groupArray(toInt16(number)) AS i16, groupArray(toInt32(number)) AS i32, groupArray(toInt64(number)) AS i64, groupArray(toUInt8(number)) AS u8, groupArray(toUInt16(number)) AS u16, groupArray(toUInt32(number)) AS u32, groupArray(toUInt64(number)) AS u64, groupArray(toFloat32(number)) AS f32, groupArray(toFloat64(number)) AS f64, groupArray(toDate(number, 'Europe/Moscow')) AS date, groupArray(toDateTime(number, 'Europe/Moscow')) AS date_time, groupArray(toString(number)) AS str, groupArray(toFixedString(toString(number), 5)) AS fixed_string FROM (SELECT number FROM system.numbers WHERE number >= 10 LIMIT 15); +CREATE TABLE bloom_filter_array_types_test (order_key Array(UInt64), i8 Array(Int8), i16 Array(Int16), i32 Array(Int32), i64 Array(Int64), u8 Array(UInt8), u16 Array(UInt16), u32 Array(UInt32), u64 Array(UInt64), f32 Array(Float32), f64 Array(Float64), date Array(Date), date_time Array(DateTime('Europe/Moscow')), str Array(String), fixed_string Array(FixedString(5)), INDEX idx (i8, i16, i32, i64, u8, u16, u32, u64, f32, f64, date, date_time, str, fixed_string) TYPE bloom_filter GRANULARITY 1) ENGINE = MergeTree() ORDER BY order_key SETTINGS index_granularity = 6; +INSERT INTO bloom_filter_array_types_test SELECT groupArray(number) AS order_key, groupArray(toInt8(number)) AS i8, groupArray(toInt16(number)) AS i16, groupArray(toInt32(number)) AS i32, groupArray(toInt64(number)) AS i64, groupArray(toUInt8(number)) AS u8, groupArray(toUInt16(number)) AS u16, groupArray(toUInt32(number)) AS u32, groupArray(toUInt64(number)) AS u64, groupArray(toFloat32(number)) AS f32, groupArray(toFloat64(number)) AS f64, groupArray(toDate(number, 'Europe/Moscow')) AS date, groupArray(toDateTime(number, 'Europe/Moscow')) AS date_time, groupArray(toString(number)) AS str, groupArray(toFixedString(toString(number), 5)) AS fixed_string FROM (SELECT number FROM system.numbers LIMIT 15); +INSERT INTO bloom_filter_array_types_test SELECT groupArray(number) AS order_key, groupArray(toInt8(number)) AS i8, groupArray(toInt16(number)) AS i16, groupArray(toInt32(number)) AS i32, groupArray(toInt64(number)) AS i64, groupArray(toUInt8(number)) AS u8, groupArray(toUInt16(number)) AS u16, groupArray(toUInt32(number)) AS u32, groupArray(toUInt64(number)) AS u64, groupArray(toFloat32(number)) AS f32, groupArray(toFloat64(number)) AS f64, groupArray(toDate(number, 'Europe/Moscow')) AS date, groupArray(toDateTime(number, 'Europe/Moscow')) AS date_time, groupArray(toString(number)) AS str, groupArray(toFixedString(toString(number), 5)) AS fixed_string FROM (SELECT number FROM system.numbers WHERE number >= 5 LIMIT 15); +INSERT INTO bloom_filter_array_types_test SELECT groupArray(number) AS order_key, groupArray(toInt8(number)) AS i8, groupArray(toInt16(number)) AS i16, groupArray(toInt32(number)) AS i32, groupArray(toInt64(number)) AS i64, groupArray(toUInt8(number)) AS u8, groupArray(toUInt16(number)) AS u16, groupArray(toUInt32(number)) AS u32, groupArray(toUInt64(number)) AS u64, groupArray(toFloat32(number)) AS f32, groupArray(toFloat64(number)) AS f64, groupArray(toDate(number, 'Europe/Moscow')) AS date, groupArray(toDateTime(number, 'Europe/Moscow')) AS date_time, groupArray(toString(number)) AS str, groupArray(toFixedString(toString(number), 5)) AS fixed_string FROM (SELECT number FROM system.numbers WHERE number >= 10 LIMIT 15); -SELECT COUNT() FROM test.bloom_filter_array_types_test WHERE has(i8, 1); -SELECT COUNT() FROM test.bloom_filter_array_types_test WHERE has(i16, 1); -SELECT COUNT() FROM test.bloom_filter_array_types_test WHERE has(i32, 1); -SELECT COUNT() FROM test.bloom_filter_array_types_test WHERE has(i64, 1); -SELECT COUNT() FROM test.bloom_filter_array_types_test WHERE has(u8, 1); -SELECT COUNT() FROM test.bloom_filter_array_types_test WHERE has(u16, 1); -SELECT COUNT() FROM test.bloom_filter_array_types_test WHERE has(u32, 1); -SELECT COUNT() FROM test.bloom_filter_array_types_test WHERE has(u64, 1); -SELECT COUNT() FROM test.bloom_filter_array_types_test WHERE has(f32, 1); -SELECT COUNT() FROM test.bloom_filter_array_types_test WHERE has(f64, 1); -SELECT COUNT() FROM test.bloom_filter_array_types_test WHERE has(date, toDate('1970-01-02')); -SELECT COUNT() FROM test.bloom_filter_array_types_test WHERE has(date_time, toDateTime('1970-01-01 03:00:01', 'Europe/Moscow')); -SELECT COUNT() FROM test.bloom_filter_array_types_test WHERE has(str, '1'); -SELECT COUNT() FROM test.bloom_filter_array_types_test WHERE has(fixed_string, toFixedString('1', 5)); +SELECT COUNT() FROM bloom_filter_array_types_test WHERE has(i8, 1); +SELECT COUNT() FROM bloom_filter_array_types_test WHERE has(i16, 1); +SELECT COUNT() FROM bloom_filter_array_types_test WHERE has(i32, 1); +SELECT COUNT() FROM bloom_filter_array_types_test WHERE has(i64, 1); +SELECT COUNT() FROM bloom_filter_array_types_test WHERE has(u8, 1); +SELECT COUNT() FROM bloom_filter_array_types_test WHERE has(u16, 1); +SELECT COUNT() FROM bloom_filter_array_types_test WHERE has(u32, 1); +SELECT COUNT() FROM bloom_filter_array_types_test WHERE has(u64, 1); +SELECT COUNT() FROM bloom_filter_array_types_test WHERE has(f32, 1); +SELECT COUNT() FROM bloom_filter_array_types_test WHERE has(f64, 1); +SELECT COUNT() FROM bloom_filter_array_types_test WHERE has(date, toDate('1970-01-02')); +SELECT COUNT() FROM bloom_filter_array_types_test WHERE has(date_time, toDateTime('1970-01-01 03:00:01', 'Europe/Moscow')); +SELECT COUNT() FROM bloom_filter_array_types_test WHERE has(str, '1'); +SELECT COUNT() FROM bloom_filter_array_types_test WHERE has(fixed_string, toFixedString('1', 5)); -SELECT COUNT() FROM test.bloom_filter_array_types_test WHERE has(i8, 5); -SELECT COUNT() FROM test.bloom_filter_array_types_test WHERE has(i16, 5); -SELECT COUNT() FROM test.bloom_filter_array_types_test WHERE has(i32, 5); -SELECT COUNT() FROM test.bloom_filter_array_types_test WHERE has(i64, 5); -SELECT COUNT() FROM test.bloom_filter_array_types_test WHERE has(u8, 5); -SELECT COUNT() FROM test.bloom_filter_array_types_test WHERE has(u16, 5); -SELECT COUNT() FROM test.bloom_filter_array_types_test WHERE has(u32, 5); -SELECT COUNT() FROM test.bloom_filter_array_types_test WHERE has(u64, 5); -SELECT COUNT() FROM test.bloom_filter_array_types_test WHERE has(f32, 5); -SELECT COUNT() FROM test.bloom_filter_array_types_test WHERE has(f64, 5); -SELECT COUNT() FROM test.bloom_filter_array_types_test WHERE has(date, toDate('1970-01-06')); -SELECT COUNT() FROM test.bloom_filter_array_types_test WHERE has(date_time, toDateTime('1970-01-01 03:00:05', 'Europe/Moscow')); -SELECT COUNT() FROM test.bloom_filter_array_types_test WHERE has(str, '5'); -SELECT COUNT() FROM test.bloom_filter_array_types_test WHERE has(fixed_string, toFixedString('5', 5)); +SELECT COUNT() FROM bloom_filter_array_types_test WHERE has(i8, 5); +SELECT COUNT() FROM bloom_filter_array_types_test WHERE has(i16, 5); +SELECT COUNT() FROM bloom_filter_array_types_test WHERE has(i32, 5); +SELECT COUNT() FROM bloom_filter_array_types_test WHERE has(i64, 5); +SELECT COUNT() FROM bloom_filter_array_types_test WHERE has(u8, 5); +SELECT COUNT() FROM bloom_filter_array_types_test WHERE has(u16, 5); +SELECT COUNT() FROM bloom_filter_array_types_test WHERE has(u32, 5); +SELECT COUNT() FROM bloom_filter_array_types_test WHERE has(u64, 5); +SELECT COUNT() FROM bloom_filter_array_types_test WHERE has(f32, 5); +SELECT COUNT() FROM bloom_filter_array_types_test WHERE has(f64, 5); +SELECT COUNT() FROM bloom_filter_array_types_test WHERE has(date, toDate('1970-01-06')); +SELECT COUNT() FROM bloom_filter_array_types_test WHERE has(date_time, toDateTime('1970-01-01 03:00:05', 'Europe/Moscow')); +SELECT COUNT() FROM bloom_filter_array_types_test WHERE has(str, '5'); +SELECT COUNT() FROM bloom_filter_array_types_test WHERE has(fixed_string, toFixedString('5', 5)); -SELECT COUNT() FROM test.bloom_filter_array_types_test WHERE has(i8, 10); -SELECT COUNT() FROM test.bloom_filter_array_types_test WHERE has(i16, 10); -SELECT COUNT() FROM test.bloom_filter_array_types_test WHERE has(i32, 10); -SELECT COUNT() FROM test.bloom_filter_array_types_test WHERE has(i64, 10); -SELECT COUNT() FROM test.bloom_filter_array_types_test WHERE has(u8, 10); -SELECT COUNT() FROM test.bloom_filter_array_types_test WHERE has(u16, 10); -SELECT COUNT() FROM test.bloom_filter_array_types_test WHERE has(u32, 10); -SELECT COUNT() FROM test.bloom_filter_array_types_test WHERE has(u64, 10); -SELECT COUNT() FROM test.bloom_filter_array_types_test WHERE has(f32, 10); -SELECT COUNT() FROM test.bloom_filter_array_types_test WHERE has(f64, 10); -SELECT COUNT() FROM test.bloom_filter_array_types_test WHERE has(date, toDate('1970-01-11')); -SELECT COUNT() FROM test.bloom_filter_array_types_test WHERE has(date_time, toDateTime('1970-01-01 03:00:10', 'Europe/Moscow')); -SELECT COUNT() FROM test.bloom_filter_array_types_test WHERE has(str, '10'); -SELECT COUNT() FROM test.bloom_filter_array_types_test WHERE has(fixed_string, toFixedString('10', 5)); +SELECT COUNT() FROM bloom_filter_array_types_test WHERE has(i8, 10); +SELECT COUNT() FROM bloom_filter_array_types_test WHERE has(i16, 10); +SELECT COUNT() FROM bloom_filter_array_types_test WHERE has(i32, 10); +SELECT COUNT() FROM bloom_filter_array_types_test WHERE has(i64, 10); +SELECT COUNT() FROM bloom_filter_array_types_test WHERE has(u8, 10); +SELECT COUNT() FROM bloom_filter_array_types_test WHERE has(u16, 10); +SELECT COUNT() FROM bloom_filter_array_types_test WHERE has(u32, 10); +SELECT COUNT() FROM bloom_filter_array_types_test WHERE has(u64, 10); +SELECT COUNT() FROM bloom_filter_array_types_test WHERE has(f32, 10); +SELECT COUNT() FROM bloom_filter_array_types_test WHERE has(f64, 10); +SELECT COUNT() FROM bloom_filter_array_types_test WHERE has(date, toDate('1970-01-11')); +SELECT COUNT() FROM bloom_filter_array_types_test WHERE has(date_time, toDateTime('1970-01-01 03:00:10', 'Europe/Moscow')); +SELECT COUNT() FROM bloom_filter_array_types_test WHERE has(str, '10'); +SELECT COUNT() FROM bloom_filter_array_types_test WHERE has(fixed_string, toFixedString('10', 5)); -DROP TABLE IF EXISTS test.bloom_filter_array_types_test; +DROP TABLE IF EXISTS bloom_filter_array_types_test; -DROP TABLE IF EXISTS test.bloom_filter_null_types_test; +DROP TABLE IF EXISTS bloom_filter_null_types_test; -CREATE TABLE test.bloom_filter_null_types_test (order_key UInt64, i8 Nullable(Int8), i16 Nullable(Int16), i32 Nullable(Int32), i64 Nullable(Int64), u8 Nullable(UInt8), u16 Nullable(UInt16), u32 Nullable(UInt32), u64 Nullable(UInt64), f32 Nullable(Float32), f64 Nullable(Float64), date Nullable(Date), date_time Nullable(DateTime('Europe/Moscow')), str Nullable(String), fixed_string Nullable(FixedString(5)), INDEX idx (i8, i16, i32, i64, u8, u16, u32, u64, f32, f64, date, date_time, str, fixed_string) TYPE bloom_filter GRANULARITY 1) ENGINE = MergeTree() ORDER BY order_key SETTINGS index_granularity = 6; -INSERT INTO test.bloom_filter_null_types_test SELECT number AS order_key, toInt8(number) AS i8, toInt16(number) AS i16, toInt32(number) AS i32, toInt64(number) AS i64, toUInt8(number) AS u8, toUInt16(number) AS u16, toUInt32(number) AS u32, toUInt64(number) AS u64, toFloat32(number) AS f32, toFloat64(number) AS f64, toDate(number, 'Europe/Moscow') AS date, toDateTime(number, 'Europe/Moscow') AS date_time, toString(number) AS str, toFixedString(toString(number), 5) AS fixed_string FROM system.numbers LIMIT 100; -INSERT INTO test.bloom_filter_null_types_test SELECT 0 AS order_key, NULL AS i8, NULL AS i16, NULL AS i32, NULL AS i64, NULL AS u8, NULL AS u16, NULL AS u32, NULL AS u64, NULL AS f32, NULL AS f64, NULL AS date, NULL AS date_time, NULL AS str, NULL AS fixed_string; +CREATE TABLE bloom_filter_null_types_test (order_key UInt64, i8 Nullable(Int8), i16 Nullable(Int16), i32 Nullable(Int32), i64 Nullable(Int64), u8 Nullable(UInt8), u16 Nullable(UInt16), u32 Nullable(UInt32), u64 Nullable(UInt64), f32 Nullable(Float32), f64 Nullable(Float64), date Nullable(Date), date_time Nullable(DateTime('Europe/Moscow')), str Nullable(String), fixed_string Nullable(FixedString(5)), INDEX idx (i8, i16, i32, i64, u8, u16, u32, u64, f32, f64, date, date_time, str, fixed_string) TYPE bloom_filter GRANULARITY 1) ENGINE = MergeTree() ORDER BY order_key SETTINGS index_granularity = 6; +INSERT INTO bloom_filter_null_types_test SELECT number AS order_key, toInt8(number) AS i8, toInt16(number) AS i16, toInt32(number) AS i32, toInt64(number) AS i64, toUInt8(number) AS u8, toUInt16(number) AS u16, toUInt32(number) AS u32, toUInt64(number) AS u64, toFloat32(number) AS f32, toFloat64(number) AS f64, toDate(number, 'Europe/Moscow') AS date, toDateTime(number, 'Europe/Moscow') AS date_time, toString(number) AS str, toFixedString(toString(number), 5) AS fixed_string FROM system.numbers LIMIT 100; +INSERT INTO bloom_filter_null_types_test SELECT 0 AS order_key, NULL AS i8, NULL AS i16, NULL AS i32, NULL AS i64, NULL AS u8, NULL AS u16, NULL AS u32, NULL AS u64, NULL AS f32, NULL AS f64, NULL AS date, NULL AS date_time, NULL AS str, NULL AS fixed_string; -SELECT COUNT() FROM test.bloom_filter_null_types_test WHERE i8 = 1 SETTINGS max_rows_to_read = 6; -SELECT COUNT() FROM test.bloom_filter_null_types_test WHERE i16 = 1 SETTINGS max_rows_to_read = 6; -SELECT COUNT() FROM test.bloom_filter_null_types_test WHERE i32 = 1 SETTINGS max_rows_to_read = 6; -SELECT COUNT() FROM test.bloom_filter_null_types_test WHERE i64 = 1 SETTINGS max_rows_to_read = 6; -SELECT COUNT() FROM test.bloom_filter_null_types_test WHERE u8 = 1 SETTINGS max_rows_to_read = 6; -SELECT COUNT() FROM test.bloom_filter_null_types_test WHERE u16 = 1 SETTINGS max_rows_to_read = 6; -SELECT COUNT() FROM test.bloom_filter_null_types_test WHERE u32 = 1 SETTINGS max_rows_to_read = 6; -SELECT COUNT() FROM test.bloom_filter_null_types_test WHERE u64 = 1 SETTINGS max_rows_to_read = 6; -SELECT COUNT() FROM test.bloom_filter_null_types_test WHERE f32 = 1 SETTINGS max_rows_to_read = 6; -SELECT COUNT() FROM test.bloom_filter_null_types_test WHERE f64 = 1 SETTINGS max_rows_to_read = 6; -SELECT COUNT() FROM test.bloom_filter_null_types_test WHERE date = '1970-01-02' SETTINGS max_rows_to_read = 6; -SELECT COUNT() FROM test.bloom_filter_null_types_test WHERE date_time = toDateTime('1970-01-01 03:00:01', 'Europe/Moscow') SETTINGS max_rows_to_read = 6; -SELECT COUNT() FROM test.bloom_filter_null_types_test WHERE str = '1' SETTINGS max_rows_to_read = 6; -SELECT COUNT() FROM test.bloom_filter_null_types_test WHERE fixed_string = toFixedString('1', 5) SETTINGS max_rows_to_read = 12; +SELECT COUNT() FROM bloom_filter_null_types_test WHERE i8 = 1 SETTINGS max_rows_to_read = 6; +SELECT COUNT() FROM bloom_filter_null_types_test WHERE i16 = 1 SETTINGS max_rows_to_read = 6; +SELECT COUNT() FROM bloom_filter_null_types_test WHERE i32 = 1 SETTINGS max_rows_to_read = 6; +SELECT COUNT() FROM bloom_filter_null_types_test WHERE i64 = 1 SETTINGS max_rows_to_read = 6; +SELECT COUNT() FROM bloom_filter_null_types_test WHERE u8 = 1 SETTINGS max_rows_to_read = 6; +SELECT COUNT() FROM bloom_filter_null_types_test WHERE u16 = 1 SETTINGS max_rows_to_read = 6; +SELECT COUNT() FROM bloom_filter_null_types_test WHERE u32 = 1 SETTINGS max_rows_to_read = 6; +SELECT COUNT() FROM bloom_filter_null_types_test WHERE u64 = 1 SETTINGS max_rows_to_read = 6; +SELECT COUNT() FROM bloom_filter_null_types_test WHERE f32 = 1 SETTINGS max_rows_to_read = 6; +SELECT COUNT() FROM bloom_filter_null_types_test WHERE f64 = 1 SETTINGS max_rows_to_read = 6; +SELECT COUNT() FROM bloom_filter_null_types_test WHERE date = '1970-01-02' SETTINGS max_rows_to_read = 6; +SELECT COUNT() FROM bloom_filter_null_types_test WHERE date_time = toDateTime('1970-01-01 03:00:01', 'Europe/Moscow') SETTINGS max_rows_to_read = 6; +SELECT COUNT() FROM bloom_filter_null_types_test WHERE str = '1' SETTINGS max_rows_to_read = 6; +SELECT COUNT() FROM bloom_filter_null_types_test WHERE fixed_string = toFixedString('1', 5) SETTINGS max_rows_to_read = 12; -SELECT COUNT() FROM test.bloom_filter_null_types_test WHERE isNull(i8); -SELECT COUNT() FROM test.bloom_filter_null_types_test WHERE isNull(i16); -SELECT COUNT() FROM test.bloom_filter_null_types_test WHERE isNull(i32); -SELECT COUNT() FROM test.bloom_filter_null_types_test WHERE isNull(i64); -SELECT COUNT() FROM test.bloom_filter_null_types_test WHERE isNull(u8); -SELECT COUNT() FROM test.bloom_filter_null_types_test WHERE isNull(u16); -SELECT COUNT() FROM test.bloom_filter_null_types_test WHERE isNull(u32); -SELECT COUNT() FROM test.bloom_filter_null_types_test WHERE isNull(u64); -SELECT COUNT() FROM test.bloom_filter_null_types_test WHERE isNull(f32); -SELECT COUNT() FROM test.bloom_filter_null_types_test WHERE isNull(f64); -SELECT COUNT() FROM test.bloom_filter_null_types_test WHERE isNull(date); -SELECT COUNT() FROM test.bloom_filter_null_types_test WHERE isNull(date_time); -SELECT COUNT() FROM test.bloom_filter_null_types_test WHERE isNull(str); -SELECT COUNT() FROM test.bloom_filter_null_types_test WHERE isNull(fixed_string); +SELECT COUNT() FROM bloom_filter_null_types_test WHERE isNull(i8); +SELECT COUNT() FROM bloom_filter_null_types_test WHERE isNull(i16); +SELECT COUNT() FROM bloom_filter_null_types_test WHERE isNull(i32); +SELECT COUNT() FROM bloom_filter_null_types_test WHERE isNull(i64); +SELECT COUNT() FROM bloom_filter_null_types_test WHERE isNull(u8); +SELECT COUNT() FROM bloom_filter_null_types_test WHERE isNull(u16); +SELECT COUNT() FROM bloom_filter_null_types_test WHERE isNull(u32); +SELECT COUNT() FROM bloom_filter_null_types_test WHERE isNull(u64); +SELECT COUNT() FROM bloom_filter_null_types_test WHERE isNull(f32); +SELECT COUNT() FROM bloom_filter_null_types_test WHERE isNull(f64); +SELECT COUNT() FROM bloom_filter_null_types_test WHERE isNull(date); +SELECT COUNT() FROM bloom_filter_null_types_test WHERE isNull(date_time); +SELECT COUNT() FROM bloom_filter_null_types_test WHERE isNull(str); +SELECT COUNT() FROM bloom_filter_null_types_test WHERE isNull(fixed_string); -SELECT COUNT() FROM test.bloom_filter_null_types_test WHERE str IN ( SELECT str FROM test.bloom_filter_null_types_test); +SELECT COUNT() FROM bloom_filter_null_types_test WHERE str IN ( SELECT str FROM bloom_filter_null_types_test); -DROP TABLE IF EXISTS test.bloom_filter_null_types_test; +DROP TABLE IF EXISTS bloom_filter_null_types_test; -DROP TABLE IF EXISTS test.bloom_filter_lc_null_types_test; +DROP TABLE IF EXISTS bloom_filter_lc_null_types_test; -CREATE TABLE test.bloom_filter_lc_null_types_test (order_key UInt64, str LowCardinality(Nullable(String)), fixed_string LowCardinality(Nullable(FixedString(5))), INDEX idx (str, fixed_string) TYPE bloom_filter GRANULARITY 1) ENGINE = MergeTree() ORDER BY order_key SETTINGS index_granularity = 6; -INSERT INTO test.bloom_filter_lc_null_types_test SELECT number AS order_key, toString(number) AS str, toFixedString(toString(number), 5) AS fixed_string FROM system.numbers LIMIT 100; -INSERT INTO test.bloom_filter_lc_null_types_test SELECT 0 AS order_key, NULL AS str, NULL AS fixed_string; +CREATE TABLE bloom_filter_lc_null_types_test (order_key UInt64, str LowCardinality(Nullable(String)), fixed_string LowCardinality(Nullable(FixedString(5))), INDEX idx (str, fixed_string) TYPE bloom_filter GRANULARITY 1) ENGINE = MergeTree() ORDER BY order_key SETTINGS index_granularity = 6; +INSERT INTO bloom_filter_lc_null_types_test SELECT number AS order_key, toString(number) AS str, toFixedString(toString(number), 5) AS fixed_string FROM system.numbers LIMIT 100; +INSERT INTO bloom_filter_lc_null_types_test SELECT 0 AS order_key, NULL AS str, NULL AS fixed_string; -SELECT COUNT() FROM test.bloom_filter_lc_null_types_test WHERE str = '1' SETTINGS max_rows_to_read = 6; -SELECT COUNT() FROM test.bloom_filter_lc_null_types_test WHERE fixed_string = toFixedString('1', 5) SETTINGS max_rows_to_read = 12; +SELECT COUNT() FROM bloom_filter_lc_null_types_test WHERE str = '1' SETTINGS max_rows_to_read = 6; +SELECT COUNT() FROM bloom_filter_lc_null_types_test WHERE fixed_string = toFixedString('1', 5) SETTINGS max_rows_to_read = 12; -SELECT COUNT() FROM test.bloom_filter_lc_null_types_test WHERE isNull(str); -SELECT COUNT() FROM test.bloom_filter_lc_null_types_test WHERE isNull(fixed_string); +SELECT COUNT() FROM bloom_filter_lc_null_types_test WHERE isNull(str); +SELECT COUNT() FROM bloom_filter_lc_null_types_test WHERE isNull(fixed_string); -SELECT COUNT() FROM test.bloom_filter_lc_null_types_test WHERE str IN ( SELECT str FROM test.bloom_filter_lc_null_types_test); +SELECT COUNT() FROM bloom_filter_lc_null_types_test WHERE str IN ( SELECT str FROM bloom_filter_lc_null_types_test); -DROP TABLE IF EXISTS test.bloom_filter_lc_null_types_test; +DROP TABLE IF EXISTS bloom_filter_lc_null_types_test; -DROP TABLE IF EXISTS test.bloom_filter_array_lc_null_types_test; +DROP TABLE IF EXISTS bloom_filter_array_lc_null_types_test; -CREATE TABLE test.bloom_filter_array_lc_null_types_test (order_key Array(LowCardinality(Nullable((UInt64)))), i8 Array(LowCardinality(Nullable((Int8)))), i16 Array(LowCardinality(Nullable((Int16)))), i32 Array(LowCardinality(Nullable((Int32)))), i64 Array(LowCardinality(Nullable((Int64)))), u8 Array(LowCardinality(Nullable((UInt8)))), u16 Array(LowCardinality(Nullable((UInt16)))), u32 Array(LowCardinality(Nullable((UInt32)))), u64 Array(LowCardinality(Nullable((UInt64)))), f32 Array(LowCardinality(Nullable((Float32)))), f64 Array(LowCardinality(Nullable((Float64)))), date Array(LowCardinality(Nullable((Date)))), date_time Array(LowCardinality(Nullable(DateTime('Europe/Moscow')))), str Array(LowCardinality(Nullable((String)))), fixed_string Array(LowCardinality(Nullable(FixedString(5)))), INDEX idx (i8, i16, i32, i64, u8, u16, u32, u64, f32, f64, date, date_time, str, fixed_string) TYPE bloom_filter GRANULARITY 1) ENGINE = MergeTree() ORDER BY order_key SETTINGS index_granularity = 6; -INSERT INTO test.bloom_filter_array_lc_null_types_test SELECT groupArray(number) AS order_key, groupArray(toInt8(number)) AS i8, groupArray(toInt16(number)) AS i16, groupArray(toInt32(number)) AS i32, groupArray(toInt64(number)) AS i64, groupArray(toUInt8(number)) AS u8, groupArray(toUInt16(number)) AS u16, groupArray(toUInt32(number)) AS u32, groupArray(toUInt64(number)) AS u64, groupArray(toFloat32(number)) AS f32, groupArray(toFloat64(number)) AS f64, groupArray(toDate(number, 'Europe/Moscow')) AS date, groupArray(toDateTime(number, 'Europe/Moscow')) AS date_time, groupArray(toString(number)) AS str, groupArray(toFixedString(toString(number), 5)) AS fixed_string FROM (SELECT number FROM system.numbers LIMIT 15); -INSERT INTO test.bloom_filter_array_lc_null_types_test SELECT groupArray(number) AS order_key, groupArray(toInt8(number)) AS i8, groupArray(toInt16(number)) AS i16, groupArray(toInt32(number)) AS i32, groupArray(toInt64(number)) AS i64, groupArray(toUInt8(number)) AS u8, groupArray(toUInt16(number)) AS u16, groupArray(toUInt32(number)) AS u32, groupArray(toUInt64(number)) AS u64, groupArray(toFloat32(number)) AS f32, groupArray(toFloat64(number)) AS f64, groupArray(toDate(number, 'Europe/Moscow')) AS date, groupArray(toDateTime(number, 'Europe/Moscow')) AS date_time, groupArray(toString(number)) AS str, groupArray(toFixedString(toString(number), 5)) AS fixed_string FROM (SELECT number FROM system.numbers WHERE number >= 5 LIMIT 15); -INSERT INTO test.bloom_filter_array_lc_null_types_test SELECT groupArray(number) AS order_key, groupArray(toInt8(number)) AS i8, groupArray(toInt16(number)) AS i16, groupArray(toInt32(number)) AS i32, groupArray(toInt64(number)) AS i64, groupArray(toUInt8(number)) AS u8, groupArray(toUInt16(number)) AS u16, groupArray(toUInt32(number)) AS u32, groupArray(toUInt64(number)) AS u64, groupArray(toFloat32(number)) AS f32, groupArray(toFloat64(number)) AS f64, groupArray(toDate(number, 'Europe/Moscow')) AS date, groupArray(toDateTime(number, 'Europe/Moscow')) AS date_time, groupArray(toString(number)) AS str, groupArray(toFixedString(toString(number), 5)) AS fixed_string FROM (SELECT number FROM system.numbers WHERE number >= 10 LIMIT 15); -INSERT INTO test.bloom_filter_array_lc_null_types_test SELECT n AS order_key, n AS i8, n AS i16, n AS i32, n AS i64, n AS u8, n AS u16, n AS u32, n AS u64, n AS f32, n AS f64, n AS date, n AS date_time, n AS str, n AS fixed_string FROM (SELECT [NULL] AS n); -INSERT INTO test.bloom_filter_array_lc_null_types_test SELECT [NULL, n] AS order_key, [NULL, toInt8(n)] AS i8, [NULL, toInt16(n)] AS i16, [NULL, toInt32(n)] AS i32, [NULL, toInt64(n)] AS i64, [NULL, toUInt8(n)] AS u8, [NULL, toUInt16(n)] AS u16, [NULL, toUInt32(n)] AS u32, [NULL, toUInt64(n)] AS u64, [NULL, toFloat32(n)] AS f32, [NULL, toFloat64(n)] AS f64, [NULL, toDate(n, 'Europe/Moscow')] AS date, [NULL, toDateTime(n, 'Europe/Moscow')] AS date_time, [NULL, toString(n)] AS str, [NULL, toFixedString(toString(n), 5)] AS fixed_string FROM (SELECT 100 as n); +CREATE TABLE bloom_filter_array_lc_null_types_test (order_key Array(LowCardinality(Nullable((UInt64)))), i8 Array(LowCardinality(Nullable((Int8)))), i16 Array(LowCardinality(Nullable((Int16)))), i32 Array(LowCardinality(Nullable((Int32)))), i64 Array(LowCardinality(Nullable((Int64)))), u8 Array(LowCardinality(Nullable((UInt8)))), u16 Array(LowCardinality(Nullable((UInt16)))), u32 Array(LowCardinality(Nullable((UInt32)))), u64 Array(LowCardinality(Nullable((UInt64)))), f32 Array(LowCardinality(Nullable((Float32)))), f64 Array(LowCardinality(Nullable((Float64)))), date Array(LowCardinality(Nullable((Date)))), date_time Array(LowCardinality(Nullable(DateTime('Europe/Moscow')))), str Array(LowCardinality(Nullable((String)))), fixed_string Array(LowCardinality(Nullable(FixedString(5)))), INDEX idx (i8, i16, i32, i64, u8, u16, u32, u64, f32, f64, date, date_time, str, fixed_string) TYPE bloom_filter GRANULARITY 1) ENGINE = MergeTree() ORDER BY order_key SETTINGS index_granularity = 6; +INSERT INTO bloom_filter_array_lc_null_types_test SELECT groupArray(number) AS order_key, groupArray(toInt8(number)) AS i8, groupArray(toInt16(number)) AS i16, groupArray(toInt32(number)) AS i32, groupArray(toInt64(number)) AS i64, groupArray(toUInt8(number)) AS u8, groupArray(toUInt16(number)) AS u16, groupArray(toUInt32(number)) AS u32, groupArray(toUInt64(number)) AS u64, groupArray(toFloat32(number)) AS f32, groupArray(toFloat64(number)) AS f64, groupArray(toDate(number, 'Europe/Moscow')) AS date, groupArray(toDateTime(number, 'Europe/Moscow')) AS date_time, groupArray(toString(number)) AS str, groupArray(toFixedString(toString(number), 5)) AS fixed_string FROM (SELECT number FROM system.numbers LIMIT 15); +INSERT INTO bloom_filter_array_lc_null_types_test SELECT groupArray(number) AS order_key, groupArray(toInt8(number)) AS i8, groupArray(toInt16(number)) AS i16, groupArray(toInt32(number)) AS i32, groupArray(toInt64(number)) AS i64, groupArray(toUInt8(number)) AS u8, groupArray(toUInt16(number)) AS u16, groupArray(toUInt32(number)) AS u32, groupArray(toUInt64(number)) AS u64, groupArray(toFloat32(number)) AS f32, groupArray(toFloat64(number)) AS f64, groupArray(toDate(number, 'Europe/Moscow')) AS date, groupArray(toDateTime(number, 'Europe/Moscow')) AS date_time, groupArray(toString(number)) AS str, groupArray(toFixedString(toString(number), 5)) AS fixed_string FROM (SELECT number FROM system.numbers WHERE number >= 5 LIMIT 15); +INSERT INTO bloom_filter_array_lc_null_types_test SELECT groupArray(number) AS order_key, groupArray(toInt8(number)) AS i8, groupArray(toInt16(number)) AS i16, groupArray(toInt32(number)) AS i32, groupArray(toInt64(number)) AS i64, groupArray(toUInt8(number)) AS u8, groupArray(toUInt16(number)) AS u16, groupArray(toUInt32(number)) AS u32, groupArray(toUInt64(number)) AS u64, groupArray(toFloat32(number)) AS f32, groupArray(toFloat64(number)) AS f64, groupArray(toDate(number, 'Europe/Moscow')) AS date, groupArray(toDateTime(number, 'Europe/Moscow')) AS date_time, groupArray(toString(number)) AS str, groupArray(toFixedString(toString(number), 5)) AS fixed_string FROM (SELECT number FROM system.numbers WHERE number >= 10 LIMIT 15); +INSERT INTO bloom_filter_array_lc_null_types_test SELECT n AS order_key, n AS i8, n AS i16, n AS i32, n AS i64, n AS u8, n AS u16, n AS u32, n AS u64, n AS f32, n AS f64, n AS date, n AS date_time, n AS str, n AS fixed_string FROM (SELECT [NULL] AS n); +INSERT INTO bloom_filter_array_lc_null_types_test SELECT [NULL, n] AS order_key, [NULL, toInt8(n)] AS i8, [NULL, toInt16(n)] AS i16, [NULL, toInt32(n)] AS i32, [NULL, toInt64(n)] AS i64, [NULL, toUInt8(n)] AS u8, [NULL, toUInt16(n)] AS u16, [NULL, toUInt32(n)] AS u32, [NULL, toUInt64(n)] AS u64, [NULL, toFloat32(n)] AS f32, [NULL, toFloat64(n)] AS f64, [NULL, toDate(n, 'Europe/Moscow')] AS date, [NULL, toDateTime(n, 'Europe/Moscow')] AS date_time, [NULL, toString(n)] AS str, [NULL, toFixedString(toString(n), 5)] AS fixed_string FROM (SELECT 100 as n); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(i8, 1); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(i16, 1); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(i32, 1); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(i64, 1); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(u8, 1); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(u16, 1); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(u32, 1); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(u64, 1); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(f32, 1); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(f64, 1); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(date, toDate('1970-01-02')); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(date_time, toDateTime('1970-01-01 03:00:01', 'Europe/Moscow')); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(str, '1'); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(fixed_string, toFixedString('1', 5)); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(i8, 1); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(i16, 1); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(i32, 1); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(i64, 1); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(u8, 1); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(u16, 1); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(u32, 1); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(u64, 1); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(f32, 1); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(f64, 1); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(date, toDate('1970-01-02')); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(date_time, toDateTime('1970-01-01 03:00:01', 'Europe/Moscow')); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(str, '1'); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(fixed_string, toFixedString('1', 5)); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(i8, 5); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(i16, 5); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(i32, 5); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(i64, 5); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(u8, 5); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(u16, 5); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(u32, 5); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(u64, 5); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(f32, 5); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(f64, 5); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(date, toDate('1970-01-06')); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(date_time, toDateTime('1970-01-01 03:00:05', 'Europe/Moscow')); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(str, '5'); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(fixed_string, toFixedString('5', 5)); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(i8, 5); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(i16, 5); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(i32, 5); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(i64, 5); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(u8, 5); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(u16, 5); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(u32, 5); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(u64, 5); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(f32, 5); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(f64, 5); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(date, toDate('1970-01-06')); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(date_time, toDateTime('1970-01-01 03:00:05', 'Europe/Moscow')); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(str, '5'); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(fixed_string, toFixedString('5', 5)); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(i8, 10); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(i16, 10); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(i32, 10); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(i64, 10); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(u8, 10); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(u16, 10); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(u32, 10); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(u64, 10); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(f32, 10); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(f64, 10); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(date, toDate('1970-01-11')); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(date_time, toDateTime('1970-01-01 03:00:10', 'Europe/Moscow')); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(str, '10'); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(fixed_string, toFixedString('10', 5)); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(i8, 10); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(i16, 10); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(i32, 10); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(i64, 10); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(u8, 10); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(u16, 10); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(u32, 10); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(u64, 10); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(f32, 10); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(f64, 10); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(date, toDate('1970-01-11')); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(date_time, toDateTime('1970-01-01 03:00:10', 'Europe/Moscow')); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(str, '10'); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(fixed_string, toFixedString('10', 5)); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(i8, NULL); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(i16, NULL); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(i32, NULL); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(i64, NULL); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(u8, NULL); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(u16, NULL); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(u32, NULL); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(u64, NULL); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(f32, NULL); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(f64, NULL); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(date, NULL); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(date_time, NULL); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(str, NULL); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(fixed_string, NULL); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(i8, NULL); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(i16, NULL); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(i32, NULL); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(i64, NULL); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(u8, NULL); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(u16, NULL); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(u32, NULL); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(u64, NULL); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(f32, NULL); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(f64, NULL); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(date, NULL); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(date_time, NULL); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(str, NULL); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(fixed_string, NULL); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(i8, 100); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(i16, 100); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(i32, 100); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(i64, 100); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(u8, 100); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(u16, 100); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(u32, 100); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(u64, 100); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(f32, 100); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(f64, 100); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(date, toDate('1970-04-11')); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(date_time, toDateTime('1970-01-01 03:01:40', 'Europe/Moscow')); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(str, '100'); -SELECT COUNT() FROM test.bloom_filter_array_lc_null_types_test WHERE has(fixed_string, toFixedString('100', 5)); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(i8, 100); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(i16, 100); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(i32, 100); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(i64, 100); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(u8, 100); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(u16, 100); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(u32, 100); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(u64, 100); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(f32, 100); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(f64, 100); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(date, toDate('1970-04-11')); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(date_time, toDateTime('1970-01-01 03:01:40', 'Europe/Moscow')); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(str, '100'); +SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(fixed_string, toFixedString('100', 5)); -DROP TABLE IF EXISTS test.bloom_filter_array_lc_null_types_test; +DROP TABLE IF EXISTS bloom_filter_array_lc_null_types_test; -DROP TABLE IF EXISTS test.bloom_filter_array_offsets_lc_str; -CREATE TABLE test.bloom_filter_array_offsets_lc_str (order_key int, str Array(LowCardinality((String))), INDEX idx str TYPE bloom_filter(1.01) GRANULARITY 1024) ENGINE = MergeTree() ORDER BY order_key SETTINGS index_granularity = 1024; -INSERT INTO test.bloom_filter_array_offsets_lc_str SELECT number AS i, if(i%2, ['value'], []) FROM system.numbers LIMIT 10000; -SELECT count() FROM test.bloom_filter_array_offsets_lc_str WHERE has(str, 'value'); -DROP TABLE IF EXISTS test.bloom_filter_array_offsets_lc_str; +DROP TABLE IF EXISTS bloom_filter_array_offsets_lc_str; +CREATE TABLE bloom_filter_array_offsets_lc_str (order_key int, str Array(LowCardinality((String))), INDEX idx str TYPE bloom_filter(1.01) GRANULARITY 1024) ENGINE = MergeTree() ORDER BY order_key SETTINGS index_granularity = 1024; +INSERT INTO bloom_filter_array_offsets_lc_str SELECT number AS i, if(i%2, ['value'], []) FROM system.numbers LIMIT 10000; +SELECT count() FROM bloom_filter_array_offsets_lc_str WHERE has(str, 'value'); +DROP TABLE IF EXISTS bloom_filter_array_offsets_lc_str; -DROP TABLE IF EXISTS test.bloom_filter_array_offsets_str; -CREATE TABLE test.bloom_filter_array_offsets_str (order_key int, str Array(String), INDEX idx str TYPE bloom_filter(1.01) GRANULARITY 1024) ENGINE = MergeTree() ORDER BY order_key SETTINGS index_granularity = 1024; -INSERT INTO test.bloom_filter_array_offsets_str SELECT number AS i, if(i%2, ['value'], []) FROM system.numbers LIMIT 10000; -SELECT count() FROM test.bloom_filter_array_offsets_str WHERE has(str, 'value'); -DROP TABLE IF EXISTS test.bloom_filter_array_offsets_str; +DROP TABLE IF EXISTS bloom_filter_array_offsets_str; +CREATE TABLE bloom_filter_array_offsets_str (order_key int, str Array(String), INDEX idx str TYPE bloom_filter(1.01) GRANULARITY 1024) ENGINE = MergeTree() ORDER BY order_key SETTINGS index_granularity = 1024; +INSERT INTO bloom_filter_array_offsets_str SELECT number AS i, if(i%2, ['value'], []) FROM system.numbers LIMIT 10000; +SELECT count() FROM bloom_filter_array_offsets_str WHERE has(str, 'value'); +DROP TABLE IF EXISTS bloom_filter_array_offsets_str; -DROP TABLE IF EXISTS test.bloom_filter_array_offsets_i; -CREATE TABLE test.bloom_filter_array_offsets_i (order_key int, i Array(int), INDEX idx i TYPE bloom_filter(1.01) GRANULARITY 1024) ENGINE = MergeTree() ORDER BY order_key SETTINGS index_granularity = 1024; -INSERT INTO test.bloom_filter_array_offsets_i SELECT number AS i, if(i%2, [99999], []) FROM system.numbers LIMIT 10000; -SELECT count() FROM test.bloom_filter_array_offsets_i WHERE has(i, 99999); -DROP TABLE IF EXISTS test.bloom_filter_array_offsets_i; +DROP TABLE IF EXISTS bloom_filter_array_offsets_i; +CREATE TABLE bloom_filter_array_offsets_i (order_key int, i Array(int), INDEX idx i TYPE bloom_filter(1.01) GRANULARITY 1024) ENGINE = MergeTree() ORDER BY order_key SETTINGS index_granularity = 1024; +INSERT INTO bloom_filter_array_offsets_i SELECT number AS i, if(i%2, [99999], []) FROM system.numbers LIMIT 10000; +SELECT count() FROM bloom_filter_array_offsets_i WHERE has(i, 99999); +DROP TABLE IF EXISTS bloom_filter_array_offsets_i; diff --git a/dbms/tests/queries/0_stateless/00950_test_double_delta_codec.sql b/dbms/tests/queries/0_stateless/00950_test_double_delta_codec.sql index 93c6b0d749b..a71264910ac 100644 --- a/dbms/tests/queries/0_stateless/00950_test_double_delta_codec.sql +++ b/dbms/tests/queries/0_stateless/00950_test_double_delta_codec.sql @@ -1,5 +1,3 @@ -USE test; - DROP TABLE IF EXISTS codecTest; CREATE TABLE codecTest ( @@ -164,4 +162,4 @@ AND ORDER BY table, name, type; -DROP TABLE IF EXISTS codecTest; \ No newline at end of file +DROP TABLE IF EXISTS codecTest; diff --git a/dbms/tests/queries/0_stateless/00950_test_gorilla_codec.sql b/dbms/tests/queries/0_stateless/00950_test_gorilla_codec.sql index c4975cdcb5d..a6e0f1d7b11 100644 --- a/dbms/tests/queries/0_stateless/00950_test_gorilla_codec.sql +++ b/dbms/tests/queries/0_stateless/00950_test_gorilla_codec.sql @@ -1,5 +1,3 @@ -USE test; - DROP TABLE IF EXISTS codecTest; CREATE TABLE codecTest ( @@ -60,4 +58,4 @@ AND c2.key = c1.key - 1 LIMIT 10; -DROP TABLE IF EXISTS codecTest; \ No newline at end of file +DROP TABLE IF EXISTS codecTest; diff --git a/dbms/tests/queries/0_stateless/00961_temporary_live_view_watch.sql b/dbms/tests/queries/0_stateless/00961_temporary_live_view_watch.sql index 7992da92f97..936f45dfced 100644 --- a/dbms/tests/queries/0_stateless/00961_temporary_live_view_watch.sql +++ b/dbms/tests/queries/0_stateless/00961_temporary_live_view_watch.sql @@ -1,20 +1,20 @@ SET allow_experimental_live_view = 1; -DROP TABLE IF EXISTS test.lv; -DROP TABLE IF EXISTS test.mt; +DROP TABLE IF EXISTS lv; +DROP TABLE IF EXISTS mt; -CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); -CREATE TEMPORARY LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt; +CREATE TABLE mt (a Int32) Engine=MergeTree order by tuple(); +CREATE TEMPORARY LIVE VIEW lv AS SELECT sum(a) FROM mt; -WATCH test.lv LIMIT 0; +WATCH lv LIMIT 0; -INSERT INTO test.mt VALUES (1),(2),(3); +INSERT INTO mt VALUES (1),(2),(3); -WATCH test.lv LIMIT 0; +WATCH lv LIMIT 0; -INSERT INTO test.mt VALUES (4),(5),(6); +INSERT INTO mt VALUES (4),(5),(6); -WATCH test.lv LIMIT 0; +WATCH lv LIMIT 0; -DROP TABLE test.lv; -DROP TABLE test.mt; +DROP TABLE lv; +DROP TABLE mt; diff --git a/dbms/tests/queries/0_stateless/00968_file_engine_in_subquery.sql b/dbms/tests/queries/0_stateless/00968_file_engine_in_subquery.sql index 5ed3be77cd6..44df28d5739 100644 --- a/dbms/tests/queries/0_stateless/00968_file_engine_in_subquery.sql +++ b/dbms/tests/queries/0_stateless/00968_file_engine_in_subquery.sql @@ -7,3 +7,6 @@ INSERT INTO tableFile_00968 SELECT number FROM system.numbers LIMIT 10; INSERT INTO tableMergeTree_00968 SELECT number FROM system.numbers LIMIT 100; SELECT id FROM tableMergeTree_00968 WHERE id IN (SELECT number FROM tableFile_00968) ORDER BY id; + +DROP TABLE tableFile_00968; +DROP TABLE tableMergeTree_00968; diff --git a/dbms/tests/queries/0_stateless/00968_live_view_select_format_jsoneachrowwithprogress.sql b/dbms/tests/queries/0_stateless/00968_live_view_select_format_jsoneachrowwithprogress.sql index 1023cdf6b29..eef27ba4999 100644 --- a/dbms/tests/queries/0_stateless/00968_live_view_select_format_jsoneachrowwithprogress.sql +++ b/dbms/tests/queries/0_stateless/00968_live_view_select_format_jsoneachrowwithprogress.sql @@ -1,14 +1,14 @@ SET allow_experimental_live_view = 1; -DROP TABLE IF EXISTS test.lv; -DROP TABLE IF EXISTS test.mt; +DROP TABLE IF EXISTS lv; +DROP TABLE IF EXISTS mt; -CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); -CREATE LIVE VIEW test.lv AS SELECT * FROM test.mt; +CREATE TABLE mt (a Int32) Engine=MergeTree order by tuple(); +CREATE LIVE VIEW lv AS SELECT * FROM mt; -INSERT INTO test.mt VALUES (1),(2),(3); +INSERT INTO mt VALUES (1),(2),(3); -SELECT * FROM test.lv FORMAT JSONEachRowWithProgress; +SELECT * FROM lv FORMAT JSONEachRowWithProgress; -DROP TABLE test.lv; -DROP TABLE test.mt; +DROP TABLE lv; +DROP TABLE mt; diff --git a/dbms/tests/queries/0_stateless/00969_live_view_watch_format_jsoneachrowwithprogress.sql b/dbms/tests/queries/0_stateless/00969_live_view_watch_format_jsoneachrowwithprogress.sql index 3e46d55c014..c890cc28dc6 100644 --- a/dbms/tests/queries/0_stateless/00969_live_view_watch_format_jsoneachrowwithprogress.sql +++ b/dbms/tests/queries/0_stateless/00969_live_view_watch_format_jsoneachrowwithprogress.sql @@ -1,20 +1,20 @@ SET allow_experimental_live_view = 1; -DROP TABLE IF EXISTS test.lv; -DROP TABLE IF EXISTS test.mt; +DROP TABLE IF EXISTS lv; +DROP TABLE IF EXISTS mt; -CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); -CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt; +CREATE TABLE mt (a Int32) Engine=MergeTree order by tuple(); +CREATE LIVE VIEW lv AS SELECT sum(a) FROM mt; -WATCH test.lv LIMIT 0 FORMAT JSONEachRowWithProgress; +WATCH lv LIMIT 0 FORMAT JSONEachRowWithProgress; -INSERT INTO test.mt VALUES (1),(2),(3); +INSERT INTO mt VALUES (1),(2),(3); -WATCH test.lv LIMIT 0 FORMAT JSONEachRowWithProgress; +WATCH lv LIMIT 0 FORMAT JSONEachRowWithProgress; -INSERT INTO test.mt VALUES (4),(5),(6); +INSERT INTO mt VALUES (4),(5),(6); -WATCH test.lv LIMIT 0 FORMAT JSONEachRowWithProgress; +WATCH lv LIMIT 0 FORMAT JSONEachRowWithProgress; -DROP TABLE test.lv; -DROP TABLE test.mt; +DROP TABLE lv; +DROP TABLE mt; diff --git a/dbms/tests/queries/0_stateless/00972_live_view_select_1.sql b/dbms/tests/queries/0_stateless/00972_live_view_select_1.sql index 135516b0cd3..2034d699065 100644 --- a/dbms/tests/queries/0_stateless/00972_live_view_select_1.sql +++ b/dbms/tests/queries/0_stateless/00972_live_view_select_1.sql @@ -1,9 +1,9 @@ SET allow_experimental_live_view = 1; -DROP TABLE IF EXISTS test.lv; +DROP TABLE IF EXISTS lv; -CREATE LIVE VIEW test.lv AS SELECT 1; +CREATE LIVE VIEW lv AS SELECT 1; -SELECT * FROM test.lv; +SELECT * FROM lv; -DROP TABLE test.lv; +DROP TABLE lv; diff --git a/dbms/tests/queries/0_stateless/00973_live_view_select.sql b/dbms/tests/queries/0_stateless/00973_live_view_select.sql index 4b5ca0a2dd7..f51c2071fc7 100644 --- a/dbms/tests/queries/0_stateless/00973_live_view_select.sql +++ b/dbms/tests/queries/0_stateless/00973_live_view_select.sql @@ -1,20 +1,20 @@ SET allow_experimental_live_view = 1; -DROP TABLE IF EXISTS test.lv; -DROP TABLE IF EXISTS test.mt; +DROP TABLE IF EXISTS lv; +DROP TABLE IF EXISTS mt; -CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); -CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt; +CREATE TABLE mt (a Int32) Engine=MergeTree order by tuple(); +CREATE LIVE VIEW lv AS SELECT sum(a) FROM mt; -INSERT INTO test.mt VALUES (1),(2),(3); +INSERT INTO mt VALUES (1),(2),(3); -SELECT *,_version FROM test.lv; -SELECT *,_version FROM test.lv; +SELECT *,_version FROM lv; +SELECT *,_version FROM lv; -INSERT INTO test.mt VALUES (1),(2),(3); +INSERT INTO mt VALUES (1),(2),(3); -SELECT *,_version FROM test.lv; -SELECT *,_version FROM test.lv; +SELECT *,_version FROM lv; +SELECT *,_version FROM lv; -DROP TABLE test.lv; -DROP TABLE test.mt; +DROP TABLE lv; +DROP TABLE mt; diff --git a/dbms/tests/queries/0_stateless/00973_live_view_with_subquery_select.sql b/dbms/tests/queries/0_stateless/00973_live_view_with_subquery_select.sql index 2e256615925..7ec992e63e7 100644 --- a/dbms/tests/queries/0_stateless/00973_live_view_with_subquery_select.sql +++ b/dbms/tests/queries/0_stateless/00973_live_view_with_subquery_select.sql @@ -1,21 +1,20 @@ SET allow_experimental_live_view = 1; -DROP TABLE IF EXISTS test.lv; -DROP TABLE IF EXISTS test.mt; +DROP TABLE IF EXISTS lv; +DROP TABLE IF EXISTS mt; -CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); -CREATE LIVE VIEW test.lv AS SELECT a FROM (SELECT a FROM test.mt); +CREATE TABLE mt (a Int32) Engine=MergeTree order by tuple(); +CREATE LIVE VIEW lv AS SELECT a FROM (SELECT a FROM mt); -INSERT INTO test.mt VALUES (1),(2),(3); +INSERT INTO mt VALUES (1),(2),(3); -SELECT *,_version FROM test.lv; -SELECT *,_version FROM test.lv; +SELECT *,_version FROM lv; +SELECT *,_version FROM lv; -INSERT INTO test.mt VALUES (1),(2),(3); +INSERT INTO mt VALUES (1),(2),(3); -SELECT *,_version FROM test.lv; -SELECT *,_version FROM test.lv; - -DROP TABLE test.lv; -DROP TABLE test.mt; +SELECT *,_version FROM lv; +SELECT *,_version FROM lv; +DROP TABLE lv; +DROP TABLE mt; diff --git a/dbms/tests/queries/0_stateless/00973_live_view_with_subquery_select_join.sql b/dbms/tests/queries/0_stateless/00973_live_view_with_subquery_select_join.sql index f89455803f3..4045f6c8434 100644 --- a/dbms/tests/queries/0_stateless/00973_live_view_with_subquery_select_join.sql +++ b/dbms/tests/queries/0_stateless/00973_live_view_with_subquery_select_join.sql @@ -1,29 +1,28 @@ SET allow_experimental_live_view = 1; -DROP TABLE IF EXISTS test.lv; -DROP TABLE IF EXISTS test.A; -DROP TABLE IF EXISTS test.B; +DROP TABLE IF EXISTS lv; +DROP TABLE IF EXISTS A; +DROP TABLE IF EXISTS B; -CREATE TABLE test.A (id Int32) Engine=Memory; -CREATE TABLE test.B (id Int32, name String) Engine=Memory; +CREATE TABLE A (id Int32) Engine=Memory; +CREATE TABLE B (id Int32, name String) Engine=Memory; -CREATE LIVE VIEW test.lv AS SELECT id, name FROM ( SELECT A.id, B.name FROM test.A as A, test.B as B WHERE A.id = B.id ); +CREATE LIVE VIEW lv AS SELECT id, name FROM ( SELECT A.id, B.name FROM A as A, B as B WHERE A.id = B.id ); -SELECT * FROM test.lv; +SELECT * FROM lv; -INSERT INTO test.A VALUES (1); -INSERT INTO test.B VALUES (1, 'hello'); +INSERT INTO A VALUES (1); +INSERT INTO B VALUES (1, 'hello'); -SELECT *,_version FROM test.lv ORDER BY id; -SELECT *,_version FROM test.lv ORDER BY id; +SELECT *,_version FROM lv ORDER BY id; +SELECT *,_version FROM lv ORDER BY id; -INSERT INTO test.A VALUES (2) -INSERT INTO test.B VALUES (2, 'hello') +INSERT INTO A VALUES (2) +INSERT INTO B VALUES (2, 'hello') -SELECT *,_version FROM test.lv ORDER BY id; -SELECT *,_version FROM test.lv ORDER BY id; - -DROP TABLE test.lv; -DROP TABLE test.A; -DROP TABLE test.B; +SELECT *,_version FROM lv ORDER BY id; +SELECT *,_version FROM lv ORDER BY id; +DROP TABLE lv; +DROP TABLE A; +DROP TABLE B; diff --git a/dbms/tests/queries/0_stateless/00973_live_view_with_subquery_select_join_no_alias.sql b/dbms/tests/queries/0_stateless/00973_live_view_with_subquery_select_join_no_alias.sql index b8eea8c71e5..1c0e779123d 100644 --- a/dbms/tests/queries/0_stateless/00973_live_view_with_subquery_select_join_no_alias.sql +++ b/dbms/tests/queries/0_stateless/00973_live_view_with_subquery_select_join_no_alias.sql @@ -1,29 +1,28 @@ SET allow_experimental_live_view = 1; -DROP TABLE IF EXISTS test.lv; -DROP TABLE IF EXISTS test.A; -DROP TABLE IF EXISTS test.B; +DROP TABLE IF EXISTS lv; +DROP TABLE IF EXISTS A; +DROP TABLE IF EXISTS B; -CREATE TABLE test.A (id Int32) Engine=Memory; -CREATE TABLE test.B (id Int32, name String) Engine=Memory; +CREATE TABLE A (id Int32) Engine=Memory; +CREATE TABLE B (id Int32, name String) Engine=Memory; -CREATE LIVE VIEW test.lv AS SELECT id, name FROM ( SELECT test.A.id, test.B.name FROM test.A, test.B WHERE test.A.id = test.B.id); +CREATE LIVE VIEW lv AS SELECT id, name FROM ( SELECT A.id, B.name FROM A, B WHERE A.id = B.id); -SELECT * FROM test.lv; +SELECT * FROM lv; -INSERT INTO test.A VALUES (1); -INSERT INTO test.B VALUES (1, 'hello'); +INSERT INTO A VALUES (1); +INSERT INTO B VALUES (1, 'hello'); -SELECT *,_version FROM test.lv ORDER BY id; -SELECT *,_version FROM test.lv ORDER BY id; +SELECT *,_version FROM lv ORDER BY id; +SELECT *,_version FROM lv ORDER BY id; -INSERT INTO test.A VALUES (2) -INSERT INTO test.B VALUES (2, 'hello') +INSERT INTO A VALUES (2) +INSERT INTO B VALUES (2, 'hello') -SELECT *,_version FROM test.lv ORDER BY id; -SELECT *,_version FROM test.lv ORDER BY id; - -DROP TABLE test.lv; -DROP TABLE test.A; -DROP TABLE test.B; +SELECT *,_version FROM lv ORDER BY id; +SELECT *,_version FROM lv ORDER BY id; +DROP TABLE lv; +DROP TABLE A; +DROP TABLE B; diff --git a/dbms/tests/queries/0_stateless/00973_live_view_with_subquery_select_nested.sql b/dbms/tests/queries/0_stateless/00973_live_view_with_subquery_select_nested.sql index f2decda148b..75e42f76ac4 100644 --- a/dbms/tests/queries/0_stateless/00973_live_view_with_subquery_select_nested.sql +++ b/dbms/tests/queries/0_stateless/00973_live_view_with_subquery_select_nested.sql @@ -1,21 +1,20 @@ SET allow_experimental_live_view = 1; -DROP TABLE IF EXISTS test.lv; -DROP TABLE IF EXISTS test.mt; +DROP TABLE IF EXISTS lv; +DROP TABLE IF EXISTS mt; -CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); -CREATE LIVE VIEW test.lv AS SELECT a FROM ( SELECT * FROM ( SELECT a FROM (SELECT a FROM test.mt) ) ); +CREATE TABLE mt (a Int32) Engine=MergeTree order by tuple(); +CREATE LIVE VIEW lv AS SELECT a FROM ( SELECT * FROM ( SELECT a FROM (SELECT a FROM mt) ) ); -INSERT INTO test.mt VALUES (1),(2),(3); +INSERT INTO mt VALUES (1),(2),(3); -SELECT *,_version FROM test.lv; -SELECT *,_version FROM test.lv; +SELECT *,_version FROM lv; +SELECT *,_version FROM lv; -INSERT INTO test.mt VALUES (1),(2),(3); +INSERT INTO mt VALUES (1),(2),(3); -SELECT *,_version FROM test.lv; -SELECT *,_version FROM test.lv; - -DROP TABLE test.lv; -DROP TABLE test.mt; +SELECT *,_version FROM lv; +SELECT *,_version FROM lv; +DROP TABLE lv; +DROP TABLE mt; diff --git a/dbms/tests/queries/0_stateless/00973_live_view_with_subquery_select_nested_with_aggregation.sql b/dbms/tests/queries/0_stateless/00973_live_view_with_subquery_select_nested_with_aggregation.sql index 4dc7b02fc51..ea70d8f27b7 100644 --- a/dbms/tests/queries/0_stateless/00973_live_view_with_subquery_select_nested_with_aggregation.sql +++ b/dbms/tests/queries/0_stateless/00973_live_view_with_subquery_select_nested_with_aggregation.sql @@ -1,21 +1,20 @@ SET allow_experimental_live_view = 1; -DROP TABLE IF EXISTS test.lv; -DROP TABLE IF EXISTS test.mt; +DROP TABLE IF EXISTS lv; +DROP TABLE IF EXISTS mt; -CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); -CREATE LIVE VIEW test.lv AS SELECT * FROM ( SELECT sum(a) FROM ( SELECT a FROM (SELECT a FROM test.mt) ) ); +CREATE TABLE mt (a Int32) Engine=MergeTree order by tuple(); +CREATE LIVE VIEW lv AS SELECT * FROM ( SELECT sum(a) FROM ( SELECT a FROM (SELECT a FROM mt) ) ); -INSERT INTO test.mt VALUES (1),(2),(3); +INSERT INTO mt VALUES (1),(2),(3); -SELECT *,_version FROM test.lv; -SELECT *,_version FROM test.lv; +SELECT *,_version FROM lv; +SELECT *,_version FROM lv; -INSERT INTO test.mt VALUES (1),(2),(3); +INSERT INTO mt VALUES (1),(2),(3); -SELECT *,_version FROM test.lv; -SELECT *,_version FROM test.lv; - -DROP TABLE test.lv; -DROP TABLE test.mt; +SELECT *,_version FROM lv; +SELECT *,_version FROM lv; +DROP TABLE lv; +DROP TABLE mt; diff --git a/dbms/tests/queries/0_stateless/00973_live_view_with_subquery_select_nested_with_aggregation_table_alias.sql b/dbms/tests/queries/0_stateless/00973_live_view_with_subquery_select_nested_with_aggregation_table_alias.sql index 2e10eefda49..10a33a18b61 100644 --- a/dbms/tests/queries/0_stateless/00973_live_view_with_subquery_select_nested_with_aggregation_table_alias.sql +++ b/dbms/tests/queries/0_stateless/00973_live_view_with_subquery_select_nested_with_aggregation_table_alias.sql @@ -1,21 +1,20 @@ SET allow_experimental_live_view = 1; -DROP TABLE IF EXISTS test.lv; -DROP TABLE IF EXISTS test.mt; +DROP TABLE IF EXISTS lv; +DROP TABLE IF EXISTS mt; -CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); -CREATE LIVE VIEW test.lv AS SELECT * FROM ( SELECT sum(boo.x) FROM ( SELECT foo.x FROM (SELECT a AS x FROM test.mt) AS foo) AS boo ); +CREATE TABLE mt (a Int32) Engine=MergeTree order by tuple(); +CREATE LIVE VIEW lv AS SELECT * FROM ( SELECT sum(boo.x) FROM ( SELECT foo.x FROM (SELECT a AS x FROM mt) AS foo) AS boo ); -INSERT INTO test.mt VALUES (1),(2),(3); +INSERT INTO mt VALUES (1),(2),(3); -SELECT *,_version FROM test.lv; -SELECT *,_version FROM test.lv; +SELECT *,_version FROM lv; +SELECT *,_version FROM lv; -INSERT INTO test.mt VALUES (1),(2),(3); +INSERT INTO mt VALUES (1),(2),(3); -SELECT *,_version FROM test.lv; -SELECT *,_version FROM test.lv; - -DROP TABLE test.lv; -DROP TABLE test.mt; +SELECT *,_version FROM lv; +SELECT *,_version FROM lv; +DROP TABLE lv; +DROP TABLE mt; diff --git a/dbms/tests/queries/0_stateless/00973_live_view_with_subquery_select_table_alias.sql b/dbms/tests/queries/0_stateless/00973_live_view_with_subquery_select_table_alias.sql index d5da0854899..9094f923350 100644 --- a/dbms/tests/queries/0_stateless/00973_live_view_with_subquery_select_table_alias.sql +++ b/dbms/tests/queries/0_stateless/00973_live_view_with_subquery_select_table_alias.sql @@ -1,21 +1,20 @@ SET allow_experimental_live_view = 1; -DROP TABLE IF EXISTS test.lv; -DROP TABLE IF EXISTS test.mt; +DROP TABLE IF EXISTS lv; +DROP TABLE IF EXISTS mt; -CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); -CREATE LIVE VIEW test.lv AS SELECT foo.x FROM (SELECT a AS x FROM test.mt) AS foo; +CREATE TABLE mt (a Int32) Engine=MergeTree order by tuple(); +CREATE LIVE VIEW lv AS SELECT foo.x FROM (SELECT a AS x FROM mt) AS foo; -INSERT INTO test.mt VALUES (1),(2),(3); +INSERT INTO mt VALUES (1),(2),(3); -SELECT *,_version FROM test.lv; -SELECT *,_version FROM test.lv; +SELECT *,_version FROM lv; +SELECT *,_version FROM lv; -INSERT INTO test.mt VALUES (1),(2),(3); +INSERT INTO mt VALUES (1),(2),(3); -SELECT *,_version FROM test.lv; -SELECT *,_version FROM test.lv; - -DROP TABLE test.lv; -DROP TABLE test.mt; +SELECT *,_version FROM lv; +SELECT *,_version FROM lv; +DROP TABLE lv; +DROP TABLE mt; diff --git a/dbms/tests/queries/0_stateless/00973_live_view_with_subquery_select_with_aggregation.sql b/dbms/tests/queries/0_stateless/00973_live_view_with_subquery_select_with_aggregation.sql index bc15e8a7356..069889f210e 100644 --- a/dbms/tests/queries/0_stateless/00973_live_view_with_subquery_select_with_aggregation.sql +++ b/dbms/tests/queries/0_stateless/00973_live_view_with_subquery_select_with_aggregation.sql @@ -1,21 +1,20 @@ SET allow_experimental_live_view = 1; -DROP TABLE IF EXISTS test.lv; -DROP TABLE IF EXISTS test.mt; +DROP TABLE IF EXISTS lv; +DROP TABLE IF EXISTS mt; -CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); -CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM (SELECT a FROM test.mt); +CREATE TABLE mt (a Int32) Engine=MergeTree order by tuple(); +CREATE LIVE VIEW lv AS SELECT sum(a) FROM (SELECT a FROM mt); -INSERT INTO test.mt VALUES (1),(2),(3); +INSERT INTO mt VALUES (1),(2),(3); -SELECT *,_version FROM test.lv; -SELECT *,_version FROM test.lv; +SELECT *,_version FROM lv; +SELECT *,_version FROM lv; -INSERT INTO test.mt VALUES (1),(2),(3); +INSERT INTO mt VALUES (1),(2),(3); -SELECT *,_version FROM test.lv; -SELECT *,_version FROM test.lv; - -DROP TABLE test.lv; -DROP TABLE test.mt; +SELECT *,_version FROM lv; +SELECT *,_version FROM lv; +DROP TABLE lv; +DROP TABLE mt; diff --git a/dbms/tests/queries/0_stateless/00973_live_view_with_subquery_select_with_aggregation_in_subquery.sql b/dbms/tests/queries/0_stateless/00973_live_view_with_subquery_select_with_aggregation_in_subquery.sql index 4dd7a12b190..286d0db6102 100644 --- a/dbms/tests/queries/0_stateless/00973_live_view_with_subquery_select_with_aggregation_in_subquery.sql +++ b/dbms/tests/queries/0_stateless/00973_live_view_with_subquery_select_with_aggregation_in_subquery.sql @@ -1,21 +1,20 @@ SET allow_experimental_live_view = 1; -DROP TABLE IF EXISTS test.lv; -DROP TABLE IF EXISTS test.mt; +DROP TABLE IF EXISTS lv; +DROP TABLE IF EXISTS mt; -CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); -CREATE LIVE VIEW test.lv AS SELECT * FROM (SELECT sum(a) FROM test.mt); +CREATE TABLE mt (a Int32) Engine=MergeTree order by tuple(); +CREATE LIVE VIEW lv AS SELECT * FROM (SELECT sum(a) FROM mt); -INSERT INTO test.mt VALUES (1),(2),(3); +INSERT INTO mt VALUES (1),(2),(3); -SELECT *,_version FROM test.lv; -SELECT *,_version FROM test.lv; +SELECT *,_version FROM lv; +SELECT *,_version FROM lv; -INSERT INTO test.mt VALUES (1),(2),(3); +INSERT INTO mt VALUES (1),(2),(3); -SELECT *,_version FROM test.lv; -SELECT *,_version FROM test.lv; - -DROP TABLE test.lv; -DROP TABLE test.mt; +SELECT *,_version FROM lv; +SELECT *,_version FROM lv; +DROP TABLE lv; +DROP TABLE mt; diff --git a/dbms/tests/queries/0_stateless/00974_live_view_select_with_aggregation.sql b/dbms/tests/queries/0_stateless/00974_live_view_select_with_aggregation.sql index 3faaec8f623..c4e7829b7dd 100644 --- a/dbms/tests/queries/0_stateless/00974_live_view_select_with_aggregation.sql +++ b/dbms/tests/queries/0_stateless/00974_live_view_select_with_aggregation.sql @@ -1,18 +1,18 @@ SET allow_experimental_live_view = 1; -DROP TABLE IF EXISTS test.lv; -DROP TABLE IF EXISTS test.mt; +DROP TABLE IF EXISTS lv; +DROP TABLE IF EXISTS mt; -CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); -CREATE LIVE VIEW test.lv AS SELECT * FROM test.mt; +CREATE TABLE mt (a Int32) Engine=MergeTree order by tuple(); +CREATE LIVE VIEW lv AS SELECT * FROM mt; -INSERT INTO test.mt VALUES (1),(2),(3); +INSERT INTO mt VALUES (1),(2),(3); -SELECT sum(a) FROM test.lv; +SELECT sum(a) FROM lv; -INSERT INTO test.mt VALUES (4),(5),(6); +INSERT INTO mt VALUES (4),(5),(6); -SELECT sum(a) FROM test.lv; +SELECT sum(a) FROM lv; -DROP TABLE test.lv; -DROP TABLE test.mt; +DROP TABLE lv; +DROP TABLE mt; diff --git a/dbms/tests/queries/0_stateless/00975_live_view_create.sql b/dbms/tests/queries/0_stateless/00975_live_view_create.sql index 02c1644d193..aa567f6a07f 100644 --- a/dbms/tests/queries/0_stateless/00975_live_view_create.sql +++ b/dbms/tests/queries/0_stateless/00975_live_view_create.sql @@ -1,9 +1,9 @@ SET allow_experimental_live_view = 1; -DROP TABLE IF EXISTS test.mt; +DROP TABLE IF EXISTS mt; -CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); -CREATE LIVE VIEW test.lv AS SELECT * FROM test.mt; +CREATE TABLE mt (a Int32) Engine=MergeTree order by tuple(); +CREATE LIVE VIEW lv AS SELECT * FROM mt; -DROP TABLE test.lv; -DROP TABLE test.mt; +DROP TABLE lv; +DROP TABLE mt; diff --git a/dbms/tests/queries/0_stateless/00975_recursive_materialized_view.sql b/dbms/tests/queries/0_stateless/00975_recursive_materialized_view.sql index 4ca7bc436bf..cd53305539b 100644 --- a/dbms/tests/queries/0_stateless/00975_recursive_materialized_view.sql +++ b/dbms/tests/queries/0_stateless/00975_recursive_materialized_view.sql @@ -1,9 +1,7 @@ -DROP TABLE IF EXISTS test.src; -DROP TABLE IF EXISTS test.dst1; -DROP TABLE IF EXISTS test.src_to_dst1; -DROP TABLE IF EXISTS test.dst2; - -USE test; +DROP TABLE IF EXISTS src; +DROP TABLE IF EXISTS dst1; +DROP TABLE IF EXISTS src_to_dst1; +DROP TABLE IF EXISTS dst2; CREATE TABLE src (x UInt8) ENGINE Memory; CREATE TABLE dst1 (x UInt8) ENGINE Memory; diff --git a/dbms/tests/queries/0_stateless/00975_sample_prewhere_distributed.sql b/dbms/tests/queries/0_stateless/00975_sample_prewhere_distributed.sql index a64eb5d0edf..a35260615c1 100644 --- a/dbms/tests/queries/0_stateless/00975_sample_prewhere_distributed.sql +++ b/dbms/tests/queries/0_stateless/00975_sample_prewhere_distributed.sql @@ -5,3 +5,5 @@ insert into sample_prewhere values ('2019-01-01', 1, toDateTime('2019-07-20 00:0 insert into sample_prewhere values ('2019-01-02', 3, toDateTime('2019-07-20 00:00:03')); select id from remote('127.0.0.{1,3}', currentDatabase(), sample_prewhere) SAMPLE 1 where toDateTime(time) = '2019-07-20 00:00:00'; + +drop table sample_prewhere; diff --git a/dbms/tests/queries/0_stateless/00976_live_view_select_version.sql b/dbms/tests/queries/0_stateless/00976_live_view_select_version.sql index ae1c59a92d7..e9538b2e592 100644 --- a/dbms/tests/queries/0_stateless/00976_live_view_select_version.sql +++ b/dbms/tests/queries/0_stateless/00976_live_view_select_version.sql @@ -1,14 +1,14 @@ SET allow_experimental_live_view = 1; -DROP TABLE IF EXISTS test.lv; -DROP TABLE IF EXISTS test.mt; +DROP TABLE IF EXISTS lv; +DROP TABLE IF EXISTS mt; -CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); -CREATE LIVE VIEW test.lv AS SELECT * FROM test.mt; +CREATE TABLE mt (a Int32) Engine=MergeTree order by tuple(); +CREATE LIVE VIEW lv AS SELECT * FROM mt; -INSERT INTO test.mt VALUES (1),(2),(3); +INSERT INTO mt VALUES (1),(2),(3); -SELECT *,_version FROM test.lv; +SELECT *,_version FROM lv; -DROP TABLE test.lv; -DROP TABLE test.mt; +DROP TABLE lv; +DROP TABLE mt; diff --git a/dbms/tests/queries/0_stateless/00977_live_view_watch_events.sql b/dbms/tests/queries/0_stateless/00977_live_view_watch_events.sql index 3e0d066fb8d..926f6637ec5 100644 --- a/dbms/tests/queries/0_stateless/00977_live_view_watch_events.sql +++ b/dbms/tests/queries/0_stateless/00977_live_view_watch_events.sql @@ -1,20 +1,20 @@ SET allow_experimental_live_view = 1; -DROP TABLE IF EXISTS test.lv; -DROP TABLE IF EXISTS test.mt; +DROP TABLE IF EXISTS lv; +DROP TABLE IF EXISTS mt; -CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); -CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt; +CREATE TABLE mt (a Int32) Engine=MergeTree order by tuple(); +CREATE LIVE VIEW lv AS SELECT sum(a) FROM mt; -WATCH test.lv EVENTS LIMIT 0; +WATCH lv EVENTS LIMIT 0; -INSERT INTO test.mt VALUES (1),(2),(3); +INSERT INTO mt VALUES (1),(2),(3); -WATCH test.lv EVENTS LIMIT 0; +WATCH lv EVENTS LIMIT 0; -INSERT INTO test.mt VALUES (4),(5),(6); +INSERT INTO mt VALUES (4),(5),(6); -WATCH test.lv EVENTS LIMIT 0; +WATCH lv EVENTS LIMIT 0; -DROP TABLE test.lv; -DROP TABLE test.mt; +DROP TABLE lv; +DROP TABLE mt; diff --git a/dbms/tests/queries/0_stateless/00978_live_view_watch.sql b/dbms/tests/queries/0_stateless/00978_live_view_watch.sql index b8d0d93ccab..e2367fc8900 100644 --- a/dbms/tests/queries/0_stateless/00978_live_view_watch.sql +++ b/dbms/tests/queries/0_stateless/00978_live_view_watch.sql @@ -1,20 +1,20 @@ SET allow_experimental_live_view = 1; -DROP TABLE IF EXISTS test.lv; -DROP TABLE IF EXISTS test.mt; +DROP TABLE IF EXISTS lv; +DROP TABLE IF EXISTS mt; -CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); -CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt; +CREATE TABLE mt (a Int32) Engine=MergeTree order by tuple(); +CREATE LIVE VIEW lv AS SELECT sum(a) FROM mt; -WATCH test.lv LIMIT 0; +WATCH lv LIMIT 0; -INSERT INTO test.mt VALUES (1),(2),(3); +INSERT INTO mt VALUES (1),(2),(3); -WATCH test.lv LIMIT 0; +WATCH lv LIMIT 0; -INSERT INTO test.mt VALUES (4),(5),(6); +INSERT INTO mt VALUES (4),(5),(6); -WATCH test.lv LIMIT 0; +WATCH lv LIMIT 0; -DROP TABLE test.lv; -DROP TABLE test.mt; +DROP TABLE lv; +DROP TABLE mt; diff --git a/dbms/tests/queries/0_stateless/00979_set_index_not.sql b/dbms/tests/queries/0_stateless/00979_set_index_not.sql index fd8f9ce2f73..2ad27e337f9 100644 --- a/dbms/tests/queries/0_stateless/00979_set_index_not.sql +++ b/dbms/tests/queries/0_stateless/00979_set_index_not.sql @@ -1,15 +1,15 @@ -DROP TABLE IF EXISTS test.set_index_not; +DROP TABLE IF EXISTS set_index_not; -CREATE TABLE test.set_index_not +CREATE TABLE set_index_not ( name String, status Enum8('alive' = 0, 'rip' = 1), INDEX idx_status status TYPE set(2) GRANULARITY 1 ) ENGINE = MergeTree() ORDER BY name SETTINGS index_granularity = 8192; -insert into test.set_index_not values ('Jon','alive'),('Ramsey','rip'); +insert into set_index_not values ('Jon','alive'),('Ramsey','rip'); -select * from test.set_index_not where status!='rip'; -select * from test.set_index_not where NOT (status ='rip'); +select * from set_index_not where status!='rip'; +select * from set_index_not where NOT (status ='rip'); -DROP TABLE test.set_index_not; \ No newline at end of file +DROP TABLE set_index_not; diff --git a/dbms/tests/queries/0_stateless/00995_optimize_read_in_order_with_aggregation.sql b/dbms/tests/queries/0_stateless/00995_optimize_read_in_order_with_aggregation.sql index 93c907811a5..56a7a8c87e8 100644 --- a/dbms/tests/queries/0_stateless/00995_optimize_read_in_order_with_aggregation.sql +++ b/dbms/tests/queries/0_stateless/00995_optimize_read_in_order_with_aggregation.sql @@ -4,3 +4,5 @@ CREATE TABLE order_with_aggr(a Int) ENGINE = MergeTree ORDER BY a; INSERT INTO order_with_aggr SELECT * FROM numbers(100); SELECT sum(a) as s FROM order_with_aggr ORDER BY s; + +DROP TABLE order_with_aggr; diff --git a/dbms/tests/queries/0_stateless/00997_set_index_array.sql b/dbms/tests/queries/0_stateless/00997_set_index_array.sql index 1692bbb2055..a0cb4ec0547 100644 --- a/dbms/tests/queries/0_stateless/00997_set_index_array.sql +++ b/dbms/tests/queries/0_stateless/00997_set_index_array.sql @@ -1,7 +1,6 @@ +DROP TABLE IF EXISTS set_array; -DROP TABLE IF EXISTS test.set_array; - -CREATE TABLE test.set_array +CREATE TABLE set_array ( primary_key String, index_array Array(UInt64), @@ -9,7 +8,7 @@ CREATE TABLE test.set_array ) ENGINE = MergeTree() ORDER BY (primary_key); -INSERT INTO test.set_array +INSERT INTO set_array select toString(intDiv(number, 1000000)) as primary_key, array(number) as index_array @@ -18,6 +17,6 @@ limit 10000000; SET max_rows_to_read = 8192; -select count() from test.set_array where has(index_array, 333); +select count() from set_array where has(index_array, 333); -DROP TABLE test.set_array; \ No newline at end of file +DROP TABLE set_array; diff --git a/dbms/tests/queries/0_stateless/01009_insert_select_data_loss.sql b/dbms/tests/queries/0_stateless/01009_insert_select_data_loss.sql index 7ecffd8653c..472056176c4 100644 --- a/dbms/tests/queries/0_stateless/01009_insert_select_data_loss.sql +++ b/dbms/tests/queries/0_stateless/01009_insert_select_data_loss.sql @@ -3,3 +3,5 @@ create table tab (x UInt64) engine = MergeTree order by tuple(); insert into tab select number as n from numbers(20) semi left join (select number * 10 as n from numbers(2)) using(n) settings max_block_size = 5; select * from tab order by x; + +drop table tab; diff --git a/dbms/tests/queries/0_stateless/01011_test_create_as_skip_indices.sql b/dbms/tests/queries/0_stateless/01011_test_create_as_skip_indices.sql index 9ac9e2d0a70..4ddad963ffd 100644 --- a/dbms/tests/queries/0_stateless/01011_test_create_as_skip_indices.sql +++ b/dbms/tests/queries/0_stateless/01011_test_create_as_skip_indices.sql @@ -2,3 +2,7 @@ CREATE TABLE foo (key int, INDEX i1 key TYPE minmax GRANULARITY 1) Engine=MergeT CREATE TABLE as_foo AS foo; CREATE TABLE dist (key int, INDEX i1 key TYPE minmax GRANULARITY 1) Engine=Distributed(test_shard_localhost, currentDatabase(), 'foo'); -- { serverError 36 } CREATE TABLE dist_as_foo Engine=Distributed(test_shard_localhost, currentDatabase(), 'foo') AS foo; + +DROP TABLE foo; +DROP TABLE as_foo; +DROP TABLE dist_as_foo; diff --git a/dbms/tests/queries/0_stateless/01018_ddl_dictionaries_create.sql b/dbms/tests/queries/0_stateless/01018_ddl_dictionaries_create.sql index 4e151a9f6e6..d420ecdd445 100644 --- a/dbms/tests/queries/0_stateless/01018_ddl_dictionaries_create.sql +++ b/dbms/tests/queries/0_stateless/01018_ddl_dictionaries_create.sql @@ -161,3 +161,4 @@ DROP DATABASE IF EXISTS ordinary_db; DROP TABLE IF EXISTS database_for_dict.table_for_dict; DROP DATABASE IF EXISTS database_for_dict; +DROP DATABASE IF EXISTS memory_db; diff --git a/dbms/tests/queries/0_stateless/01019_Buffer_and_max_memory_usage.sql b/dbms/tests/queries/0_stateless/01019_Buffer_and_max_memory_usage.sql index cbe5f8a8244..246f7be5a33 100644 --- a/dbms/tests/queries/0_stateless/01019_Buffer_and_max_memory_usage.sql +++ b/dbms/tests/queries/0_stateless/01019_Buffer_and_max_memory_usage.sql @@ -35,3 +35,6 @@ SET min_insert_block_size_rows=100e3; INSERT INTO buffer_ SELECT toUInt64(number) FROM system.numbers LIMIT toUInt64(10e6+1); -- Check that 10e6 rows had been flushed from the query, not from the background worker. SELECT count() FROM buffer_; + +DROP TABLE null_; +DROP TABLE buffer_; diff --git a/dbms/tests/queries/0_stateless/01021_only_tuple_columns.sql b/dbms/tests/queries/0_stateless/01021_only_tuple_columns.sql index 460bdae3cf1..27019a67b98 100644 --- a/dbms/tests/queries/0_stateless/01021_only_tuple_columns.sql +++ b/dbms/tests/queries/0_stateless/01021_only_tuple_columns.sql @@ -20,3 +20,5 @@ DETACH TABLE test; ATTACH TABLE test; SELECT COUNT() FROM test; + +DROP TABLE test; diff --git a/dbms/tests/queries/0_stateless/01030_storage_set_supports_read.sql b/dbms/tests/queries/0_stateless/01030_storage_set_supports_read.sql index 8b406cae769..e2823b7253e 100644 --- a/dbms/tests/queries/0_stateless/01030_storage_set_supports_read.sql +++ b/dbms/tests/queries/0_stateless/01030_storage_set_supports_read.sql @@ -19,3 +19,4 @@ SELECT * FROM userid_test WHERE toUInt64(1) IN (userid_set); SELECT * FROM userid_test WHERE userid IN (userid_set); DROP TABLE userid_test; +DROP TABLE userid_set; diff --git a/dbms/tests/queries/0_stateless/01045_bloom_filter_null_array.sql b/dbms/tests/queries/0_stateless/01045_bloom_filter_null_array.sql index bf97e6be838..a021a2b9e28 100644 --- a/dbms/tests/queries/0_stateless/01045_bloom_filter_null_array.sql +++ b/dbms/tests/queries/0_stateless/01045_bloom_filter_null_array.sql @@ -1,18 +1,18 @@ SET allow_experimental_data_skipping_indices = 1; -DROP TABLE IF EXISTS test.bloom_filter_null_array; +DROP TABLE IF EXISTS bloom_filter_null_array; -CREATE TABLE test.bloom_filter_null_array (v Array(LowCardinality(Nullable(String))), INDEX idx v TYPE bloom_filter(0.1) GRANULARITY 1) ENGINE = MergeTree() ORDER BY v; +CREATE TABLE bloom_filter_null_array (v Array(LowCardinality(Nullable(String))), INDEX idx v TYPE bloom_filter(0.1) GRANULARITY 1) ENGINE = MergeTree() ORDER BY v; -INSERT INTO test.bloom_filter_null_array VALUES ([]); -INSERT INTO test.bloom_filter_null_array VALUES (['1', '2']) ([]) ([]); -INSERT INTO test.bloom_filter_null_array VALUES ([]) ([]) (['2', '3']); +INSERT INTO bloom_filter_null_array VALUES ([]); +INSERT INTO bloom_filter_null_array VALUES (['1', '2']) ([]) ([]); +INSERT INTO bloom_filter_null_array VALUES ([]) ([]) (['2', '3']); -SELECT COUNT() FROM test.bloom_filter_null_array; -SELECT COUNT() FROM test.bloom_filter_null_array WHERE has(v, '1'); -SELECT COUNT() FROM test.bloom_filter_null_array WHERE has(v, '2'); -SELECT COUNT() FROM test.bloom_filter_null_array WHERE has(v, '3'); -SELECT COUNT() FROM test.bloom_filter_null_array WHERE has(v, '4'); +SELECT COUNT() FROM bloom_filter_null_array; +SELECT COUNT() FROM bloom_filter_null_array WHERE has(v, '1'); +SELECT COUNT() FROM bloom_filter_null_array WHERE has(v, '2'); +SELECT COUNT() FROM bloom_filter_null_array WHERE has(v, '3'); +SELECT COUNT() FROM bloom_filter_null_array WHERE has(v, '4'); -DROP TABLE IF EXISTS test.bloom_filter_null_array; +DROP TABLE IF EXISTS bloom_filter_null_array; diff --git a/dbms/tests/queries/0_stateless/01047_simple_aggregate_sizes_of_columns_bug.sql b/dbms/tests/queries/0_stateless/01047_simple_aggregate_sizes_of_columns_bug.sql index a84eeb9070f..7213aaa16c7 100644 --- a/dbms/tests/queries/0_stateless/01047_simple_aggregate_sizes_of_columns_bug.sql +++ b/dbms/tests/queries/0_stateless/01047_simple_aggregate_sizes_of_columns_bug.sql @@ -11,3 +11,5 @@ ALTER TABLE column_size_bug DELETE WHERE value=1; SELECT sleep(1); OPTIMIZE TABLE column_size_bug; + +DROP TABLE column_size_bug; diff --git a/dbms/tests/queries/0_stateless/01064_incremental_streaming_from_2_src_with_feedback.sql b/dbms/tests/queries/0_stateless/01064_incremental_streaming_from_2_src_with_feedback.sql index 41888e1baf3..e5b4b9dbe33 100644 --- a/dbms/tests/queries/0_stateless/01064_incremental_streaming_from_2_src_with_feedback.sql +++ b/dbms/tests/queries/0_stateless/01064_incremental_streaming_from_2_src_with_feedback.sql @@ -123,5 +123,6 @@ DROP TABLE IF EXISTS logins; DROP TABLE IF EXISTS mv_logins2target; DROP TABLE IF EXISTS checkouts; DROP TABLE IF EXISTS mv_checkouts2target; +DROP TABLE target_table; -SYSTEM START MERGES; \ No newline at end of file +SYSTEM START MERGES; diff --git a/dbms/tests/queries/0_stateless/01069_materialized_view_alter_target_table.sql b/dbms/tests/queries/0_stateless/01069_materialized_view_alter_target_table.sql index 6be401597f6..b65835ace33 100644 --- a/dbms/tests/queries/0_stateless/01069_materialized_view_alter_target_table.sql +++ b/dbms/tests/queries/0_stateless/01069_materialized_view_alter_target_table.sql @@ -14,3 +14,7 @@ INSERT INTO mv_source VALUES (1),(2),(3); SELECT * FROM mv ORDER BY a; SELECT * FROM mv_target ORDER BY a; + +DROP TABLE mv; +DROP TABLE mv_source; +DROP TABLE mv_target; diff --git a/dbms/tests/queries/0_stateless/01069_materialized_view_alter_target_table_with_default_expression.sql b/dbms/tests/queries/0_stateless/01069_materialized_view_alter_target_table_with_default_expression.sql index dcfba76f9c2..da2078de82a 100644 --- a/dbms/tests/queries/0_stateless/01069_materialized_view_alter_target_table_with_default_expression.sql +++ b/dbms/tests/queries/0_stateless/01069_materialized_view_alter_target_table_with_default_expression.sql @@ -14,3 +14,7 @@ INSERT INTO mv_source VALUES (1),(2),(3); SELECT * FROM mv ORDER BY a; SELECT * FROM mv_target ORDER BY a; + +DROP TABLE mv; +DROP TABLE mv_source; +DROP TABLE mv_target; diff --git a/dbms/tests/queries/0_stateless/01073_attach_if_not_exists.sql b/dbms/tests/queries/0_stateless/01073_attach_if_not_exists.sql index e9411577d56..1c7877a7c73 100644 --- a/dbms/tests/queries/0_stateless/01073_attach_if_not_exists.sql +++ b/dbms/tests/queries/0_stateless/01073_attach_if_not_exists.sql @@ -4,3 +4,4 @@ ATTACH TABLE IF NOT EXISTS t; DETACH TABLE t; ATTACH TABLE IF NOT EXISTS t; EXISTS TABLE t; +DROP TABLE t; diff --git a/dbms/tests/queries/0_stateless/01077_yet_another_prewhere_test.sql b/dbms/tests/queries/0_stateless/01077_yet_another_prewhere_test.sql index d46b289ab77..491b0791ee3 100644 --- a/dbms/tests/queries/0_stateless/01077_yet_another_prewhere_test.sql +++ b/dbms/tests/queries/0_stateless/01077_yet_another_prewhere_test.sql @@ -2,9 +2,11 @@ drop table if exists t50; create table t50 (a Int, b Int, s String) engine = MergeTree order by a settings index_granularity = 50, index_granularity_bytes=1000; --- some magic to satisfy conditions to run optimizations in MergeTreeRangeReader +-- some magic to satisfy conditions to run optimizations in MergeTreeRangeReader insert into t50 select 0, 1, repeat('a', 10000); insert into t50 select number, multiIf(number < 5, 1, number < 50, 0, number < 55, 1, number < 100, 0, number < 105, 1, 0), '' from numbers(150); optimize table t50 final; select a, b from t50 prewhere b = 1 order by a; + +drop table t50; From 0de8e7cf1a8e6a40a204382111a1b96b8cb00958 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Tue, 11 Feb 2020 23:00:53 +0300 Subject: [PATCH 14/15] performance comparison --- docker/test/performance-comparison/compare.sh | 33 +++++++++++-------- docker/test/performance-comparison/perf.py | 7 ++-- docker/test/performance-comparison/report.py | 10 +++--- 3 files changed, 29 insertions(+), 21 deletions(-) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index 691b59fef4b..c8eeec16962 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -129,7 +129,7 @@ function run_tests rm right/performance/{IPv4,IPv6,modulo,parse_engine_file,number_formatting_formats,select_format}.xml ||: # Run the tests - for test in right/performance/${CHPC_TEST_GLOB:-*.xml} + for test in right/performance/${CHPC_TEST_GLOB:-*}.xml do test_name=$(basename $test ".xml") echo test $test_name @@ -150,22 +150,24 @@ rm test-times.tsv test-dump.tsv unstable.tsv changed-perf.tsv unstable-tests.tsv right/clickhouse local --query " create table queries engine Memory as select replaceAll(_file, '-report.tsv', '') test, - if(abs(diff) < 0.05 and rd[3] > 0.05, 1, 0) unstable, - if(abs(diff) > 0.05 and abs(diff) > rd[3], 1, 0) changed, + left + right < 0.01 as short, + -- FIXME Comparison mode doesn't make sense for queries that complete + -- immediately, so for now we pretend they don't exist. We don't want to + -- remove them altogether because we want to be able to detect regressions, + -- but the right way to do this is not yet clear. + not short and abs(diff) < 0.05 and rd[3] > 0.05 as unstable, + not short and abs(diff) > 0.05 and abs(diff) > rd[3] as changed, * - from file('*-report.tsv', TSV, 'left float, right float, diff float, rd Array(float), query text') - -- FIXME Comparison mode doesn't make sense for queries that complete - -- immediately, so for now we pretend they don't exist. We don't want to - -- remove them altogether because we want to be able to detect regressions, - -- but the right way to do this is not yet clear. - where left + right > 0.01; + from file('*-report.tsv', TSV, 'left float, right float, diff float, rd Array(float), query text'); create table changed_perf_tsv engine File(TSV, 'changed-perf.tsv') as select left, right, diff, rd, test, query from queries where changed order by rd[3] desc; + create table unstable_queries_tsv engine File(TSV, 'unstable-queries.tsv') as select left, right, diff, rd, test, query from queries where unstable order by rd[3] desc; + create table unstable_tests_tsv engine File(TSV, 'bad-tests.tsv') as select test, sum(unstable) u, sum(changed) c, u + c s from queries group by test having s > 0 order by s desc; @@ -182,20 +184,23 @@ create table slow_on_client_tsv engine File(TSV, 'slow-on-client.tsv') as create table test_time engine Memory as select test, sum(client) total_client_time, - max(client) query_max, min(client) query_min, count(*) queries - from query_time - -- for consistency, filter out everything we filtered out of queries table - semi join queries using query + maxIf(client, not short) query_max, + minIf(client, not short) query_min, + count(*) queries, + sum(short) short_queries + from query_time, queries + where query_time.query = queries.query group by test; create table test_times_tsv engine File(TSV, 'test-times.tsv') as select wall_clock.test, real, floor(total_client_time, 3), queries, + short_queries, floor(query_max, 3), floor(real / queries, 3) avg_real_per_query, floor(query_min, 3) - from test_time right join wall_clock using test + from test_time join wall_clock using test order by query_max / query_min desc; create table all_queries_tsv engine File(TSV, 'all-queries.tsv') as diff --git a/docker/test/performance-comparison/perf.py b/docker/test/performance-comparison/perf.py index 1a2021c629a..c45b9f93827 100755 --- a/docker/test/performance-comparison/perf.py +++ b/docker/test/performance-comparison/perf.py @@ -94,10 +94,11 @@ test_query_templates = [q.text for q in root.findall('query')] test_queries = substitute_parameters(test_query_templates, parameter_combinations) for q in test_queries: - # Warmup: run once on both servers. Helps to bring the data into memory, + # Prewarm: run once on both servers. Helps to bring the data into memory, # precompile the queries, etc. - for c in connections: - c.execute(q) + for conn_index, c in enumerate(connections): + res = c.execute(q) + print('prewarm\t' + tsv_escape(q) + '\t' + str(conn_index) + '\t' + str(c.last_query.elapsed)) # Now, perform measured runs. # Track the time spent by the client to process this query, so that we can notice diff --git a/docker/test/performance-comparison/report.py b/docker/test/performance-comparison/report.py index 21865532268..4d36be3d8f8 100755 --- a/docker/test/performance-comparison/report.py +++ b/docker/test/performance-comparison/report.py @@ -81,7 +81,8 @@ def nextRowAnchor(): def tr(x): a = nextRowAnchor() - return '{x}'.format(a=a, x=str(x)) + #return '{x}'.format(a=a, x=str(x)) + return '{x}'.format(a=a, x=str(x)) def td(x): return '' + str(x) + '' @@ -140,16 +141,17 @@ params['test_part'] = ( table_template.format( anchor = nextTableAnchor(), caption = 'Tests with most unstable queries', - header = table_header(['Test', 'Unstable', 'Changed perf', 'Total']), + header = table_header(['Test', 'Unstable', 'Changed perf', 'Total not OK']), rows = tsv_rows('bad-tests.tsv')) + table_template.format( anchor = nextTableAnchor(), caption = 'Tests times', header = table_header(['Test', 'Wall clock time, s', 'Total client time, s', 'Number of queries', - 'Max client time
(sum for all runs), s', + 'Number of short queries', + 'Longest query
(sum for all runs), s', 'Avg wall clock time
(sum for all runs), s', - 'Min client time
(sum for all runs), s']), + 'Shortest query
(sum for all runs), s']), rows = tsv_rows('test-times.tsv')) ) print(doc_template.format_map(params)) From aa44ba844dff903295d8ec4fb33bde84fa6f3590 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Tue, 11 Feb 2020 19:10:29 -0500 Subject: [PATCH 15/15] Update README.md --- README.md | 4 ---- 1 file changed, 4 deletions(-) diff --git a/README.md b/README.md index 5e6f504d223..a545c91886f 100644 --- a/README.md +++ b/README.md @@ -11,7 +11,3 @@ ClickHouse is an open-source column-oriented database management system that all * [Blog](https://clickhouse.yandex/blog/en/) contains various ClickHouse-related articles, as well as announces and reports about events. * [Contacts](https://clickhouse.yandex/#contacts) can help to get your questions answered if there are any. * You can also [fill this form](https://forms.yandex.com/surveys/meet-yandex-clickhouse-team/) to meet Yandex ClickHouse team in person. - -## Upcoming Events - -* [ClickHouse Meetup in New York](https://www.meetup.com/Uber-Engineering-Events-New-York/events/268328663/) on February 11.